Return-Path: X-Original-To: apmail-crunch-user-archive@www.apache.org Delivered-To: apmail-crunch-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7CADA1026E for ; Thu, 4 Dec 2014 22:24:11 +0000 (UTC) Received: (qmail 4826 invoked by uid 500); 4 Dec 2014 22:24:11 -0000 Delivered-To: apmail-crunch-user-archive@crunch.apache.org Received: (qmail 4784 invoked by uid 500); 4 Dec 2014 22:24:11 -0000 Mailing-List: contact user-help@crunch.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@crunch.apache.org Delivered-To: mailing list user@crunch.apache.org Received: (qmail 4774 invoked by uid 99); 4 Dec 2014 22:24:11 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 04 Dec 2014 22:24:11 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of jwills@cloudera.com designates 209.85.214.175 as permitted sender) Received: from [209.85.214.175] (HELO mail-ob0-f175.google.com) (209.85.214.175) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 04 Dec 2014 22:24:06 +0000 Received: by mail-ob0-f175.google.com with SMTP id wp4so3598167obc.20 for ; Thu, 04 Dec 2014 14:22:15 -0800 (PST) X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20130820; h=x-gm-message-state:mime-version:in-reply-to:references:from:date :message-id:subject:to:content-type; bh=iVyrV60w0w4pKlywHMVgbVxu2d2mivw8qLYNw+xQp9A=; b=X9QJ4pRfIrf9qh77/zjm8tz7DkxSIzM0ItNlw4c1uY7t/0yIbwy2QV9PDhoOfnT3b9 CAjLxjcwoHS3pgGepmN//jG+PtgKfOGOBsJWGvQa1bSs/TdoQJqutH8FoIvbjMJeQ7nR EH47J1psjg7A+l7QHg/u70V+XPENstnAycnccUIyGMw6606QS80sC3tpN4CiBzrFzRFA Is+y9xZCGD7yqC8EDToLuWcPxyptrfH6JRfkdqZlN2OvwgOarhBOtE1pxstq9lU0mSMR 14r1C+yKP0+aJSQVZzTwTuwgSGy14uUeZi/QXXXcwYnL7MevdhzA5aveV7tpiMPG+pdi Lejg== X-Gm-Message-State: ALoCoQn64LVeGpvG2FJEvq8TeEsk0bEfrUIPhsYR0HVhRairyheaS73r+Yau4QIFpjf/nRumS0vu X-Received: by 10.202.168.204 with SMTP id r195mr8126436oie.72.1417731735686; Thu, 04 Dec 2014 14:22:15 -0800 (PST) MIME-Version: 1.0 Received: by 10.76.144.65 with HTTP; Thu, 4 Dec 2014 14:21:55 -0800 (PST) In-Reply-To: References: From: Josh Wills Date: Thu, 4 Dec 2014 14:21:55 -0800 Message-ID: Subject: Re: Multiple Reduces in a Single Crunch Job To: user@crunch.apache.org Content-Type: multipart/alternative; boundary=001a113cd8767e310c05096b64ed X-Virus-Checked: Checked by ClamAV on apache.org --001a113cd8767e310c05096b64ed Content-Type: text/plain; charset=UTF-8 Danny, Spent a couple of hours today banging on this by hacking on some integration tests but couldn't replicate it. However, I just took a closer look at the plan you posted, and I noticed that all of the files you are writing out are prefixed w/ "hdfs:/" except for the /tmp/crunch-* file that Crunch is creating; is it possible that Crunch is creating the temp file locally on your client machine for some reason? I can't think of why that would happen off the top of my head, but if that is the problem, I'll at least be able to figure out where to look. Josh On Tue, Nov 25, 2014 at 6:30 PM, Danny Morgan wrote: > No problem, Happy Thanksgiving! > > Gobble Gobble... > > ------------------------------ > From: jwills@cloudera.com > Date: Tue, 25 Nov 2014 18:23:14 -0800 > > Subject: Re: Multiple Reduces in a Single Crunch Job > To: user@crunch.apache.org > > Very useful-- thank you. Will dig into it and report back, although I'm > heading out for the holiday so it likely won't be until early next week. > > J > > On Tue, Nov 25, 2014 at 6:16 PM, Danny Morgan > wrote: > > Having a single pipeline in the application didn't fix it. Sticking a > pipeline.run() in the middle also didn't matter either, the plan appears > such that the planner is completely ignoring the second the run() I added. > > However what DOES WORK is if I do: > > collection = secondarySort() > pipeline.cache(collection) > pipeline.run() > newcollection = collection.groupByKey() > > If I try adding the cache() without calling run() in between it doesn't > work. Hope that's enough info for you to fix the possible planner bug. > > Thanks for the help Josh! > > ------------------------------ > From: unluckyboy@hotmail.com > To: user@crunch.apache.org > Subject: RE: Multiple Reduces in a Single Crunch Job > Date: Wed, 26 Nov 2014 01:58:11 +0000 > > > I tried doing a Sample() instead of identity function, but that got fused > into the reduce as well and didn't work. > > First thing I tried was sticking a pipeline.run() in between there and I > was surprised but it didn't work either, same error. I'll rerun that config > now and try to get the dot files for the plan. > > Not sure if this is affecting it but in the same crunch application I have > a completely independent pipeline the runs before this one executes. I'll > turn that off as well and see if it's causing the issue. > > ------------------------------ > From: jwills@cloudera.com > Date: Tue, 25 Nov 2014 17:43:52 -0800 > Subject: Re: Multiple Reduces in a Single Crunch Job > To: user@crunch.apache.org > > Drat, I was hoping it was something simple. You could manually fix it by > injecting a pipeline.run() call between the secondarySort and the > groupByKey(), but of course, we'd like to handle this situation correctly > by default. > > J > > On Tue, Nov 25, 2014 at 5:39 PM, Danny Morgan > wrote: > > I did a parallelDo with the IdentityFn of the output of the secondarySort > and the IdentityFn was just fused into the reduce phase of the > secondarySort and I got the same error message. > > I think you want me to somehow force a map phase in between the two > reduces? > > -Danny > > ------------------------------ > From: josh.wills@gmail.com > Date: Tue, 25 Nov 2014 17:23:29 -0800 > > Subject: Re: Multiple Reduces in a Single Crunch Job > To: user@crunch.apache.org > > Oh, dumb question-- if you put like a dummy function between the > secondarySort and the groupByKey, like an IdentityFn or something, do > things work again? That would help w/diagnosing the problem. > > On Tue, Nov 25, 2014 at 5:15 PM, Josh Wills wrote: > > So if you're getting it quickly, it might be b/c the job isn't recognizing > the dependency between the two separate phases of the job for some reason > (e.g., it's not realizing that one job has to be run before the other one.) > That's an odd situation, but we have had bugs like that in the past; let me > see if I can re-create the situation in an integration test. Which version > of Crunch? > > J > > On Tue, Nov 25, 2014 at 4:40 PM, Danny Morgan > wrote: > > No that's definitely not it. I get this issue if I write to a single > output as well. > > If I remove the groupByKey().combineValues() line and just write out the > output from the SecondarySort it works. Seems to only complain about the > temp path not existing when I have multiple reduce phases in the pipeline. > Also the error seems to happen immediately during the setup or planning > phase, I assume this because the yarn jobs get created but they don't do > anything, and instead of FAILED the error message is "Application killed by > user." > > -Danny > > ------------------------------ > From: jwills@cloudera.com > Date: Tue, 25 Nov 2014 16:30:58 -0800 > > Subject: Re: Multiple Reduces in a Single Crunch Job > To: user@crunch.apache.org > > Ack, sorry-- it's this: https://issues.apache.org/jira/browse/CRUNCH-481 > > > On Tue, Nov 25, 2014 at 4:24 PM, Danny Morgan > wrote: > > Hello Again Josh, > > The link to the Jira issue you sent out seems to be cut off, could you > please resend it? > > I deleted the line where I write the collection to a text file, and > retried it but it didn't work either. Also tried writing the collection out > as Avro instead of Parquet, but got the same error. > > Here's the rest of the stracktrace: > > org.apache.hadoop.mapreduce.lib.input.InvalidInputException: Input path > does not exist: hdfs:///tmp/crunch-2008950085/p1 > at > org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:285) > at > org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.getSplits(CombineFileInputFormat.java:217) > at > org.apache.crunch.impl.mr.run.CrunchInputFormat.getSplits(CrunchInputFormat.java:65) > at > org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:491) > at > org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:508) > at > org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:392) > at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1268) > at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1265) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1528) > at org.apache.hadoop.mapreduce.Job.submit(Job.java:1265) > at > org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchControlledJob.submit(CrunchControlledJob.java:340) > at > org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchJobControl.startReadyJobs(CrunchJobControl.java:277) > at > org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchJobControl.pollJobStatusAndStartNewOnes(CrunchJobControl.java:316) > at > org.apache.crunch.impl.mr.exec.MRExecutor.monitorLoop(MRExecutor.java:113) > at > org.apache.crunch.impl.mr.exec.MRExecutor.access$000(MRExecutor.java:55) > at > org.apache.crunch.impl.mr.exec.MRExecutor$1.run(MRExecutor.java:84) > at java.lang.Thread.run(Thread.java:744) > > Thanks Josh! > > ------------------------------ > From: jwills@cloudera.com > Date: Tue, 25 Nov 2014 16:10:33 -0800 > Subject: Re: Multiple Reduces in a Single Crunch Job > To: user@crunch.apache.org > > > Hey Danny, > > I'm wondering if this is caused by > https://issues.apache.org/jira/browse/CRUNCH-481-- I think we use > different output committers for text files vs. parquet files, so at least > one of the outputs won't be written properly-- does that make sense? > > Josh > > On Tue, Nov 25, 2014 at 4:07 PM, Danny Morgan > wrote: > > Hi Crunchers, > > I've attached a pdf of what my plan looks like. I've run into this problem > before where I have multiple reduce steps chained together in a single > pipeline and always get the same error. > > In the case of the attached pdf the error is "org.apache.hadoop.mapreduce.lib.input.InvalidInputException: > Input path does not exist: hdfs:///tmp/crunch-1279941375/p1" > > That's the temp directory the crunch planner set up for the first reduce > phase. > > Can I run multiple chained reduces within the same pipeline? Do I have to > manually write out the output from the first reduce? > > Here's what the code looks like: > > // Simple mapper > PTable> first = Danny.filterForDanny(logs); > // Secondary sort happens here > PTable second = Danny.extractDannys(first); > // Regular group by > PTable third = > second.groupByKey().combineValues(Aggregators.SUM_LONGS()); > // simple function that populates some fields in the Danny object > with the aggregate results > PCollection> done = Danny.finalize(third); > Pair, PCollection> splits = > Channels.split(done); > splits.second().write(To.textFile(mypath, WriteMode.OVERWRITE); > Target pq_danny = new AvroParquetFileTarget(pqPath)); > splits.first().write(pq_danny, WriteMode.OVERWRITE) > > Thanks! > > -Danny > > > > > -- > Director of Data Science > Cloudera > Twitter: @josh_wills > > > > > -- > Director of Data Science > Cloudera > Twitter: @josh_wills > > > > > -- > Director of Data Science > Cloudera > Twitter: @josh_wills > > > > > > -- > Director of Data Science > Cloudera > Twitter: @josh_wills > > > > > -- > Director of Data Science > Cloudera > Twitter: @josh_wills > -- Director of Data Science Cloudera Twitter: @josh_wills --001a113cd8767e310c05096b64ed Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Danny,

Spent a couple of hours today ba= nging on this by hacking on some integration tests but couldn't replica= te it. However, I just took a closer look at the plan you posted, and I not= iced that all of the files you are writing out are prefixed w/ "hdfs:/= " except for the /tmp/crunch-* file that Crunch is creating; is it pos= sible that Crunch is creating the temp file locally on your client machine = for some reason? I can't think of why that would happen off the top of = my head, but if that is the problem, I'll at least be able to figure ou= t where to look.

Josh


On Tue, Nov 25, 201= 4 at 6:30 PM, Danny Morgan <unluckyboy@hotmail.com> wro= te:
No problem, Happy Thanksgiving!

Go= bble Gobble...


From: jwills@cloudera.com
Date: Tue, 25 Nov 2014 18:23:= 14 -0800

Subject: Re: Multiple Reduces in a Singl= e Crunch Job
To: user@crunch.apache.org

Very useful-- thank= you. Will dig into it and report back, although I'm heading out for th= e holiday so it likely won't be until early next week.

J

On Tue, Nov 25, 2014 at 6:16 PM, Danny Morgan= <unluckyboy@hotmail.com> wrote:
Having a single pipeline in the application didn'= t fix it. Sticking a pipeline.run() in the middle also didn't matter ei= ther, the plan appears such that the planner is completely ignoring the sec= ond the run() I added.

However what DOES WORK is i= f I do:

collection =3D secondarySort()
p= ipeline.cache(collection)
pipeline.run()
newcollection = =3D collection.groupByKey()

If I try adding the ca= che() without calling run() in between it doesn't work. Hope that's= enough info for you to fix the possible planner bug.

<= div>Thanks for the help Josh!


From: unluckyboy@hotmail.com=
To: user@cr= unch.apache.org
Subject: RE: Multiple Reduces in a Single Crunch Job=
Date: Wed, 26 Nov 2014 01:58:11 +0000


I tried doing a Sample() instead of identity function, but= that got fused into the reduce as well and didn't work.

=
First thing I tried was sticking a pipeline.run() in between there and= I was surprised but it didn't work either, same error. I'll rerun = that config now and try to get the dot files for the plan.

Not sure if this is affecting it but in the same crunch applicatio= n I have a completely independent pipeline the runs before this one execute= s. I'll turn that off as well and see if it's causing the issue.

From: jwills@cloudera.com
Date: Tue, 25 Nov 2014 17:43:52 -0800
Subje= ct: Re: Multiple Reduces in a Single Crunch Job
To: user@crunch.apache.org

<= div dir=3D"ltr">Drat, I was hoping it was something simple. You could manua= lly fix it by injecting a pipeline.run() call between the secondarySort and= the groupByKey(), but of course, we'd like to handle this situation co= rrectly by default.

J

On Tue, N= ov 25, 2014 at 5:39 PM, Danny Morgan <unluckyboy@hotmail.com><= /span> wrote:
I did a parallelDo with the IdentityFn of the output = of the secondarySort and the IdentityFn was just fused into the reduce phas= e of the secondarySort and I got the same error message.

I think you want me to somehow force a map phase in between the two reduce= s?

-Danny


From: josh.wills@gmail.com
Date: T= ue, 25 Nov 2014 17:23:29 -0800

Subject: Re: Multiple Reduces i= n a Single Crunch Job
To: user@crunch.apache.org

Oh, dumb q= uestion-- if you put like a dummy function between the secondarySort and th= e groupByKey, like an IdentityFn or something, do things work again? That w= ould help w/diagnosing the problem.

On Tue, Nov 25, 2014= at 5:15 PM, Josh Wills <jwills@cloudera.com> wrote:
So if you're getting it quickly, it might be b/c the job isn'= t recognizing the dependency between the two separate phases of the job for= some reason (e.g., it's not realizing that one job has to be run befor= e the other one.) That's an odd situation, but we have had bugs like th= at in the past; let me see if I can re-create the situation in an integrati= on test. Which version of Crunch?

J

On Tue, Nov 25, = 2014 at 4:40 PM, Danny Morgan <unluckyboy@hotmail.com> = wrote:
No that's definitely not it. I get this issue if = I write to a single output as well.

If I remove the=C2= =A0groupByKey().combineValues() line and just write out the output from the= SecondarySort it works. Seems to only complain about the temp path not exi= sting when I have multiple reduce phases in the pipeline. Also the error se= ems to happen immediately during the setup or planning phase, I assume this= because the yarn jobs get created but they don't do anything, and inst= ead of FAILED the error message is "Application killed by user."<= /div>

-Danny


From: jwills@cloudera.com
Date: Tue, 2= 5 Nov 2014 16:30:58 -0800

Subject: Re: Multiple Reduces in a S= ingle Crunch Job
To: user@crunch.apache.org

=

On Tue, Nov 25, 2014 at 4:24 PM, Danny Morgan <unluckyb= oy@hotmail.com> wrote:
Hello Again Josh,

The link to the = Jira issue you sent out seems to be cut off, could you please resend it?

I deleted the line where I write the collection to a= text file, and retried it but it didn't work either. Also tried writin= g the collection out as Avro instead of Parquet, but got the same error.

Here's the rest of the stracktrace:
org.apache.hadoop.mapreduce.lib.input.InvalidInputExcepti= on: Input path does not exist: hdfs:///tmp/crunch-2008950085/p1
= =C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.hadoop.mapreduce.lib.input.FileIn= putFormat.listStatus(FileInputFormat.java:285)
=C2=A0 =C2=A0 =C2= =A0 =C2=A0 at org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.= getSplits(CombineFileInputFormat.java:217)
=C2=A0 =C2=A0 =C2=A0 = =C2=A0 at org.apache.crunch.impl.mr.run.CrunchInputFormat.getSplits(CrunchI= nputFormat.java:65)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.had= oop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:491)
= =C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.hadoop.mapreduce.JobSubmitter.wri= teSplits(JobSubmitter.java:508)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at or= g.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:= 392)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.hadoop.mapreduce.J= ob$10.run(Job.java:1268)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apach= e.hadoop.mapreduce.Job$10.run(Job.java:1265)
=C2=A0 =C2=A0 =C2=A0= =C2=A0 at java.security.AccessController.doPrivileged(Native Method)
=
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at javax.security.auth.Subject.doAs(Subjec= t.java:415)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.hadoop.secu= rity.UserGroupInformation.doAs(UserGroupInformation.java:1528)
= =C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.hadoop.mapreduce.Job.submit(Job.j= ava:1265)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.crunch.hadoop= .mapreduce.lib.jobcontrol.CrunchControlledJob.submit(CrunchControlledJob.ja= va:340)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.crunch.hadoop.m= apreduce.lib.jobcontrol.CrunchJobControl.startReadyJobs(CrunchJobControl.ja= va:277)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.crunch.hadoop.m= apreduce.lib.jobcontrol.CrunchJobControl.pollJobStatusAndStartNewOnes(Crunc= hJobControl.java:316)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.c= runch.impl.mr.exec.MRExecutor.monitorLoop(MRExecutor.java:113)
= =C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.apache.crunch.impl.mr.exec.MRExecutor.ac= cess$000(MRExecutor.java:55)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 at org.a= pache.crunch.impl.mr.exec.MRExecutor$1.run(MRExecutor.java:84)
= =C2=A0 =C2=A0 =C2=A0 =C2=A0 at java.lang.Thread.run(Thread.java:744)
<= div>
Thanks Josh!


From: jwills@cloudera.com
Date: Tue,= 25 Nov 2014 16:10:33 -0800
Subject: Re: Multiple Reduces in a Single Cr= unch Job
To: user@crunch.apache.org


Hey Danny,
I'm wondering if this is caused by=C2=A0https://= issues.apache.org/jira/browse/CRUNCH-481-- I think we use different out= put committers for text files vs. parquet files, so at least one of the out= puts won't be written properly-- does that make sense?

Josh

On Tue, Nov 25, 2014 at 4:07 PM, Dan= ny Morgan <unluckyboy@hotmail.com> wrote:
Hi Crunchers,

I've attached a = pdf of what my plan looks like. I've run into this problem before where= I have multiple reduce steps chained together in a single pipeline and alw= ays get the same error.

In the case of the attached pdf the error is "org.apache= .hadoop.mapreduce.lib.input.InvalidInputException: Input path does not exis= t: hdfs:///tmp/crunch-1279941375/p1"

That's = the temp directory the crunch planner set up for the first reduce phase.
Can I run multiple chained reduces within the same pipeline? Do I have t= o manually write out the output from the first reduce?
<= div>
Here's what= the code looks like:

=C2=A0 =C2=A0 =C2=A0 // Simple mapper
=C2=A0 =C2=A0 =C2=A0 PTable<String, Pair<Long, Log>> fi= rst =3D Danny.filterForDanny(logs);
=C2= =A0 =C2=A0 =C2=A0 // Secondary sort happens here
=C2=A0 =C2=A0 =C2=A0 PTable<Danny, Long> second =3D Danny.e= xtractDannys(first);
=C2=A0 =C2=A0 =C2= =A0 // Regular group by
=C2=A0 =C2=A0 = =C2=A0 PTable<Danny, Long> third =3D second.groupByKey().combineValue= s(Aggregators.SUM_LONGS());
=C2=A0 =C2= =A0 =C2=A0 // simple function that populates some fields in the Danny objec= t with the aggregate results
=C2=A0 =C2= =A0 =C2=A0 PCollection<Pair<Danny, String>> done =3D Danny.fina= lize(third);
=C2=A0 =C2=A0 =C2=A0 Pair&= lt;PCollection<Danny>, PCollection<String>> splits =3D Chann= els.split(done);
=C2=A0 =C2=A0 =C2=A0 s= plits.second().write(To.textFile(mypath, WriteMode.OVERWRITE);
=C2=A0 =C2=A0 =C2=A0 Target pq_danny =3D new AvroPa= rquetFileTarget(pqPath));
=C2=A0 =C2=A0= =C2=A0 splits.first().write(pq_danny, WriteMode.OVERWRITE)

Than= ks!

-Danny
=



--
Di= rector of Data Science
Twitter: @josh_wills



--
Di= rector of Data Science
Twitter: @josh_wills



--
Di= rector of Data Science
Twitter: @josh_wills

=



--
Di= rector of Data Science
Twitter: @josh_wills



--
Di= rector of Data Science
Twitter: @josh_wills



--
Director of Data Science
Twitter: @josh_wills
--001a113cd8767e310c05096b64ed--