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 B42191884F for ; Wed, 29 Jul 2015 03:45:28 +0000 (UTC) Received: (qmail 9041 invoked by uid 500); 29 Jul 2015 03:45:28 -0000 Delivered-To: apmail-crunch-user-archive@crunch.apache.org Received: (qmail 9002 invoked by uid 500); 29 Jul 2015 03:45:28 -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 8992 invoked by uid 99); 29 Jul 2015 03:45:28 -0000 Received: from Unknown (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 29 Jul 2015 03:45:28 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 1AE42C0337 for ; Wed, 29 Jul 2015 03:45:28 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 3 X-Spam-Level: *** X-Spam-Status: No, score=3 tagged_above=-999 required=6.31 tests=[HTML_MESSAGE=3, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id b6BYusXyrhDQ for ; Wed, 29 Jul 2015 03:45:17 +0000 (UTC) Received: from mail-yk0-f171.google.com (mail-yk0-f171.google.com [209.85.160.171]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with ESMTPS id 5827820775 for ; Wed, 29 Jul 2015 03:45:16 +0000 (UTC) Received: by ykax123 with SMTP id x123so113364265yka.1 for ; Tue, 28 Jul 2015 20:45:09 -0700 (PDT) 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=9oXjcIM80mpO5AsH/WW+4PDEtWpgWi4VW/AesJDUkO8=; b=ZomA2NvXlRaVatoREg/s2U+fxA3ziNOZqxOGnoT0WAoUqGYk/M2lFJk3giHNt1A3UE dhSSmaYIrlaCy2pgU7J/rM8YfgDU3x7+SPSUA89c7ItJDP5tFW7yUBlCfpWN3dK1+i8n 2VWQCGUAirv4RADEwfY9+YdE+rbpd63kBSfaBBsHGVlThJVC9tOHcMUV8dTyHEdB2NyS 1cF8VQAkyV/q0QfjDgshZUoee6leO2zk8JFA9hc6VT+y1J3P+IVBabSNiW6vs8QXh8+C ekrw4UCpEE1TUHCiiT0FY08J1QJ7MiUwgcpllUnA2WzRfeC00ngdXGZiDWAI5DdqkshR Y3vw== X-Gm-Message-State: ALoCoQlsTCNFroVwyxolHuxT2TFmE3C+yPHah7fFHqhh73T/Svmi4OAcUYNX0+cUrYnkfVmilMpZ X-Received: by 10.13.249.198 with SMTP id j189mr41045568ywf.170.1438141509534; Tue, 28 Jul 2015 20:45:09 -0700 (PDT) MIME-Version: 1.0 Received: by 10.13.196.197 with HTTP; Tue, 28 Jul 2015 20:44:49 -0700 (PDT) In-Reply-To: References: From: Josh Wills Date: Tue, 28 Jul 2015 20:44:49 -0700 Message-ID: Subject: Re: Non Deterministic Record Drops To: user@crunch.apache.org Content-Type: multipart/alternative; boundary=94eb2c080b96d03f0a051bfb6957 --94eb2c080b96d03f0a051bfb6957 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable Hey Everett, The bug was specific to the situation where the input was a PTable (never a PCollection) that was being processed multiple times in a pure map-only job. The reason is that internal to the MR planner, an input PTable is really just a thin wrapper around an input PCollection> (at least from a job configuration perspective.) The planner keeps track of which input PCollections it has seen before as it walks the DAG with a Visitor object that knows how to update an internal graph based on the type of PCollection visited (input, DoFn, union, GBK, etc.) Before I added the fix to the BaseInputTable to distinguish it (in an equals(Object) sense) from a BaseInputCollection, it was possible for the planner to get confused and assign multiple Vertices in the DAG to the same underlying input (one for the PCollection version, and one for the PTable version). Some of the outputs would go to the PCollection, some would go to the PTable, and unless there was a GBK operation that had both "versions" as parents, it was possible for the planner to essentially lose either the PTable or the PCollection vertex when it went to finish the job, which meant that none of those inputs would get read. The order in which the DAG is walked isn't deterministic for outputs that are on the same "level" (i.e., all of the outputs from a map-only job), so the inputs that would get processed in your jobs would change from run to run depending on the order in which they showed up in the graph, as you saw. The change I made ensures that all of the inputs are tracked to the same Vertex in the graph (the one based on the underlying InputCollection that is wrapped by the InputPTable) by the planner, so now no inputs get lost. I hope that helps a little bit. J On Tue, Jul 28, 2015 at 10:36 AM, David Ortiz wrote: > For what it's worth, the optimizer may still read the file more than > once even if there's only one read in your code. All depends on what els= e > is being done. > > *Sent from my Verizon Wireless 4G LTE DROID* > On Jul 28, 2015 1:34 PM, Everett Anderson wrote: > Thanks, Josh!! > > I'm curious about the fix and didn't fully understand from the > description. > > What's interesting about the test is that there's only one Pipeline > read(), but then multiple parallelDo()s on the resulting table, yet you > still hit the issue. We'd thought it must be due to the multiple reads of > the same file. > > Would this have happened in other places where multiple operations were > performed on the same PTable or PCollection, or is it specific to the > operations performed on objects created directly from a read()? > > > > On Mon, Jul 27, 2015 at 6:49 PM, Josh Wills wrote: > >> That was a deeply satisfying bug. Fix is up here: >> https://issues.apache.org/jira/browse/CRUNCH-553 >> >> On Mon, Jul 27, 2015 at 6:29 PM, Jeff Quinn wrote: >> >>> Wow, thanks so much for looking into it. That minimal example >>> seems accurate. Previously when we dug deeper into which records were >>> dropped it appeared entire files were being dropped, not just parts of = one >>> file, so that sounds consistent with what you are seeing. >>> >>> On Monday, July 27, 2015, Josh Wills wrote: >>> >>>> Hey Jeff, >>>> >>>> Okay cool-- I think I've managed to create a simple test that >>>> replicates the behavior you're seeing. I can run this test a few diffe= rent >>>> times, and sometimes I'll get the correct output, but other times I'll= get >>>> an error b/c no records are processed. I'm going to investigate furthe= r and >>>> see if I can identify the source of the randomness. >>>> >>>> public class RecordDropIT { >>>> @Rule >>>> public TemporaryPath tmpDir =3D TemporaryPaths.create(); >>>> >>>> @Test >>>> public void testMultiReadCount() throws Exception { >>>> int numReads =3D 2; >>>> MRPipeline p =3D new MRPipeline(RecordDropIT.class, tmpDir.getDefa= ultConfiguration()); >>>> Path shakes =3D tmpDir.copyResourcePath("shakes.txt"); >>>> TableSource src =3D From.formattedFile(shakes,= TextInputFormat.class, LongWritable.class, Text.class); >>>> List> values =3D Lists.newArrayList(); >>>> for (int i =3D 0; i < numReads; i++) { >>>> PCollection cnt =3D p.read(src).parallelDo(new LineCoun= tFn>(), Writables.ints()); >>>> values.add(cnt.materialize()); >>>> } >>>> for (Iterable iter : values) { >>>> System.out.println(Iterables.getOnlyElement(iter)); >>>> } >>>> p.done(); >>>> } >>>> >>>> public static class LineCountFn extends DoFn { >>>> >>>> private int count =3D 0; >>>> >>>> @Override >>>> public void process(T input, Emitter emitter) { >>>> count++; >>>> } >>>> >>>> @Override >>>> public void cleanup(Emitter emitter) { >>>> emitter.emit(count); >>>> } >>>> } >>>> } >>>> >>>> >>>> On Mon, Jul 27, 2015 at 6:11 PM, Jeff Quinn wrote: >>>> >>>>> Hi Josh, >>>>> >>>>> Thanks so much for your suggestions. >>>>> >>>>> The counts are determined with two methods, I am using a simple pig >>>>> script to count records, and I am also tabulating up the size in byte= s of >>>>> all hdfs output files. Both measures show dropped records / fewer tha= n >>>>> expected output bytes. >>>>> >>>>> To your second point I will go back and do a sweep for that, but I >>>>> am fairly sure no DoFns are making use of intermediate state values w= ithout >>>>> getDetachedValue. Our team is aware of the getDetachedValue gotchas a= s I >>>>> think it has bitten us before. >>>>> >>>>> Thanks ! >>>>> >>>>> Jeff >>>>> >>>>> >>>>> On Monday, July 27, 2015, Josh Wills wrote: >>>>> >>>>>> One more thought-- are any of these DoFns keeping records around as >>>>>> intermediate state values w/o using PType.getDetachedValue to make c= opies >>>>>> of them? >>>>>> >>>>>> J >>>>>> >>>>>> On Mon, Jul 27, 2015 at 5:47 PM, Josh Wills >>>>>> wrote: >>>>>> >>>>>>> Hey Jeff, >>>>>>> >>>>>>> Are the counts determined by Counters? Or is it the length of the >>>>>>> output files? Or both? >>>>>>> >>>>>>> J >>>>>>> >>>>>>> On Mon, Jul 27, 2015 at 5:29 PM, David Ortiz >>>>>>> wrote: >>>>>>> >>>>>>>> Out of curiosity, any reason you went with multiple reads as >>>>>>>> opposed to just performing multiple operations on the same PTable? >>>>>>>> parallelDo returns a new object rather than modifying the initial = one, so a >>>>>>>> single collection can start multiple execution flows. >>>>>>>> >>>>>>>> On Mon, Jul 27, 2015, 8:11 PM Jeff Quinn wrote: >>>>>>>> >>>>>>>>> Hello, >>>>>>>>> >>>>>>>>> We have observed and replicated strange behavior with our crunch >>>>>>>>> application while running on MapReduce via the AWS ElasticMapRedu= ce >>>>>>>>> service. Running a very simple job which is mostly map only, we s= ee that an >>>>>>>>> undetermined subset of records are getting dropped. Specifically,= we >>>>>>>>> expect 30,136,686 output records and have seen output on differen= t trials >>>>>>>>> (running over the same data with the same binary): >>>>>>>>> >>>>>>>>> 22,177,119 records >>>>>>>>> 26,435,670 records >>>>>>>>> 22,362,986 records >>>>>>>>> 29,798,528 records >>>>>>>>> >>>>>>>>> These are all the things about our application which might be >>>>>>>>> unusual and relevant: >>>>>>>>> >>>>>>>>> - We use a custom file input format, via From.formattedFile. It >>>>>>>>> looks like this (basically a carbon copy >>>>>>>>> of org.apache.hadoop.mapreduce.lib.input.TextInputFormat): >>>>>>>>> >>>>>>>>> import org.apache.hadoop.io.LongWritable; >>>>>>>>> import org.apache.hadoop.io.Text; >>>>>>>>> import org.apache.hadoop.mapreduce.InputSplit; >>>>>>>>> import org.apache.hadoop.mapreduce.RecordReader; >>>>>>>>> import org.apache.hadoop.mapreduce.TaskAttemptContext; >>>>>>>>> import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; >>>>>>>>> import org.apache.hadoop.mapreduce.lib.input.LineRecordReader; >>>>>>>>> >>>>>>>>> import java.io.IOException; >>>>>>>>> >>>>>>>>> public class ByteOffsetInputFormat extends FileInputFormat { >>>>>>>>> >>>>>>>>> @Override >>>>>>>>> public RecordReader createRecordReader( >>>>>>>>> InputSplit split, TaskAttemptContext context) throws IOExce= ption, >>>>>>>>> InterruptedException { >>>>>>>>> return new LineRecordReader(); >>>>>>>>> } >>>>>>>>> } >>>>>>>>> >>>>>>>>> - We call org.apache.crunch.Pipeline#read using this InputFormat = many times, for the job in question it is called ~160 times as the input is= ~100 different files. Each file ranges in size from 100MB-8GB. Our job onl= y uses this input format for all input files. >>>>>>>>> >>>>>>>>> - For some files org.apache.crunch.Pipeline#read is called twice = one the same file, and the resulting PTables are processed in different way= s. >>>>>>>>> >>>>>>>>> - It is only the data from these files which org.apache.crunch.Pi= peline#read has been called on more than once during a job that have droppe= d records, all other files consistently do not have dropped records >>>>>>>>> >>>>>>>>> Curious if any Crunch users have experienced similar behavior bef= ore, or if any of these details about my job raise any red flags. >>>>>>>>> >>>>>>>>> Thanks! >>>>>>>>> >>>>>>>>> Jeff Quinn >>>>>>>>> >>>>>>>>> Data Engineer >>>>>>>>> >>>>>>>>> Nuna >>>>>>>>> >>>>>>>>> >>>>>>>>> *DISCLAIMER:* The contents of this email, including any >>>>>>>>> attachments, may contain information that is confidential, propri= etary in >>>>>>>>> nature, protected health information (PHI), or otherwise protecte= d by law >>>>>>>>> from disclosure, and is solely for the use of the intended recipi= ent(s). If >>>>>>>>> you are not the intended recipient, you are hereby notified that = any use, >>>>>>>>> disclosure or copying of this email, including any attachments, i= s >>>>>>>>> unauthorized and strictly prohibited. If you have received this e= mail in >>>>>>>>> error, please notify the sender of this email. Please delete this= and all >>>>>>>>> copies of this email from your system. Any opinions either expres= sed or >>>>>>>>> implied in this email and all attachments, are those of its autho= r only, >>>>>>>>> and do not necessarily reflect those of Nuna Health, Inc. >>>>>>>> >>>>>>>> >>>>>>> >>>>>>> >>>>>>> -- >>>>>>> Director of Data Science >>>>>>> Cloudera >>>>>>> Twitter: @josh_wills >>>>>>> >>>>>> >>>>>> >>>>>> >>>>>> -- >>>>>> Director of Data Science >>>>>> Cloudera >>>>>> Twitter: @josh_wills >>>>>> >>>>> >>>>> *DISCLAIMER:* The contents of this email, including any attachments, >>>>> may contain information that is confidential, proprietary in nature, >>>>> protected health information (PHI), or otherwise protected by law fro= m >>>>> disclosure, and is solely for the use of the intended recipient(s). I= f you >>>>> are not the intended recipient, you are hereby notified that any use, >>>>> disclosure or copying of this email, including any attachments, is >>>>> unauthorized and strictly prohibited. If you have received this email= in >>>>> error, please notify the sender of this email. Please delete this and= all >>>>> copies of this email from your system. Any opinions either expressed = or >>>>> implied in this email and all attachments, are those of its author on= ly, >>>>> and do not necessarily reflect those of Nuna Health, Inc. >>>>> >>>> >>>> >>>> >>>> -- >>>> Director of Data Science >>>> Cloudera >>>> Twitter: @josh_wills >>>> >>> >>> *DISCLAIMER:* The contents of this email, including any attachments, >>> may contain information that is confidential, proprietary in nature, >>> protected health information (PHI), or otherwise protected by law from >>> disclosure, and is solely for the use of the intended recipient(s). If = you >>> are not the intended recipient, you are hereby notified that any use, >>> disclosure or copying of this email, including any attachments, is >>> unauthorized and strictly prohibited. If you have received this email i= n >>> error, please notify the sender of this email. Please delete this and a= ll >>> copies of this email from your system. Any opinions either expressed or >>> implied in this email and all attachments, are those of its author only= , >>> and do not necessarily reflect those of Nuna Health, Inc. >>> >> >> >> >> -- >> Director of Data Science >> Cloudera >> Twitter: @josh_wills >> > > > *DISCLAIMER:* The contents of this email, including any attachments, may > contain information that is confidential, proprietary in nature, protecte= d > health information (PHI), or otherwise protected by law from disclosure, > and is solely for the use of the intended recipient(s). If you are not th= e > intended recipient, you are hereby notified that any use, disclosure or > copying of this email, including any attachments, is unauthorized and > strictly prohibited. If you have received this email in error, please > notify the sender of this email. Please delete this and all copies of thi= s > email from your system. Any opinions either expressed or implied in this > email and all attachments, are those of its author only, and do not > necessarily reflect those of Nuna Health, Inc. > *This email is intended only for the use of the individual(s) to whom it > is addressed. If you have received this communication in error, please > immediately notify the sender and delete the original email.* > --=20 Director of Data Science Cloudera Twitter: @josh_wills --94eb2c080b96d03f0a051bfb6957 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Hey Everett,

The bug was specific to th= e situation where the input was a PTable (never a PCollection) that was bei= ng processed multiple times in a pure map-only job. The reason is that inte= rnal to the MR planner, an input PTable is really just a thin wrapper aroun= d an input PCollection<Pair<K, V>> (at least from a job configu= ration perspective.) The planner keeps track of which input PCollections it= has seen before as it walks the DAG with a Visitor object that knows how t= o update an internal graph based on the type of PCollection visited (input,= DoFn, union, GBK, etc.)

Before I added the fix to= the BaseInputTable to distinguish it (in an equals(Object) sense) from a B= aseInputCollection, it was possible for the planner to get confused and ass= ign multiple Vertices in the DAG to the same underlying input (one for the = PCollection version, and one for the PTable version). Some of the outputs w= ould go to the PCollection, some would go to the PTable, and unless there w= as a GBK operation that had both "versions" as parents, it was po= ssible for the planner to essentially lose either the PTable or the PCollec= tion vertex when it went to finish the job, which meant that none of those = inputs would get read. The order in which the DAG is walked isn't deter= ministic for outputs that are on the same "level" (i.e., all of t= he outputs from a map-only job), so the inputs that would get processed in = your jobs would change from run to run depending on the order in which they= showed up in the graph, as you saw.

The change I = made ensures that all of the inputs are tracked to the same Vertex in the g= raph (the one based on the underlying InputCollection that is wrapped by th= e InputPTable) by the planner, so now no inputs get lost. I hope that helps= a little bit.

J


On Tue, Jul 28, 2015 at = 10:36 AM, David Ortiz <dortiz@videologygroup.com> wr= ote:
For what it's worth, the optimizer may still read the file more th= an once even if there's only one read in your code.=C2=A0 All depends o= n what else is being done.

Sent from my Verizon Wireless 4G LTE = DROID
On Jul 28, 2015 1:34 PM, Everett Anderson <everett@nuna.com> wrote:
Thanks, Josh!!

I'm curious about the fix and didn't fully understand from the= description.

What's interesting about the test is that there's only one Pip= eline read(), but then multiple parallelDo()s on the resulting table, yet y= ou still hit the issue. We'd thought it must be due to the multiple rea= ds of the same file.

Would this have happened in other places where multiple operations wer= e performed on the same PTable or PCollection, or is it specific to the ope= rations performed on objects created directly from a read()?



On Mon, Jul 27, 2015 at 6:49 PM, Josh Wills <jwills@clouder= a.com> wrote:
That was a deeply satisfying bug. Fix is up here:=C2=A0https://issues.apache.org/jira/browse/CRUNCH-553

On Mon, Jul 27, 2015 at 6:29 PM, Jeff Quinn <jeff@nuna.com>= ; wrote:
Wow, thanks so much for looking into it.=C2=A0That minimal example seems=C2= =A0accurate. Previously when we dug deeper into which records were dropped = it appeared entire files=C2=A0were being dropped, not just parts of one fil= e, so that sounds consistent with=C2=A0what you are seeing.

On Monday, July 27, 2015, Josh Wills <jwills@cloudera.com> wrote:
Hey Jeff,

Okay cool-- I think I've managed to create a simple test that repl= icates the behavior you're seeing. I can run this test a few different = times, and sometimes I'll get the correct output, but other times I'= ;ll get an error b/c no records are processed. I'm going to investigate further and see if I can identify the source of the r= andomness.

public class RecordDropIT {
= @Rule
public Te= mporaryPath tmpDir = =3D TemporaryPaths.create();
@Test
public void testMultiReadCount() th= rows Exception {
int numReads =3D 2;
M= RPipeline p =3D new M= RPipeline(RecordDropIT.class= , tmpDir.getDe= faultConfiguration());
Path shakes =3D tmpDir.copyResourcePath("shakes.txt");
TableSource<= LongWritable, Text> src =3D From.forma= ttedFile(shakes, TextInputFormat.class, LongWritable.class, Text.class);
List<Iterable<Integer>> values =3D List= s.newArrayList();
for (int i =3D 0; i < numReads; i++) {
PCollection<Integer> cnt =3D p.r= ead(src).parallelDo(new LineCountFn<Pair<LongWritable, Text>>(), Writables.ints());
values.add(cnt.materializ= e());
}
for (Iterable<Integer> iter : values) {
System.out.println(It= erables.getOnlyElement(iter));
= }
p.done();
}

public static class LineCountFn<T> extends= DoFn<T, Integer> {
private int count
=3D 0;

@Overrid= e
public void process(T input, Emitter<Integer> emitter) {
count++;
}

<= span style=3D"color:#808000">@Override
public void <= /span>cleanup(Emitter<Integer> emitter) {
emitter.emit(count
);
}
}}

On Mon, Jul 27, 2015 at 6:11 PM, Jeff Quin= n <jeff@nuna.com> wrote:
Hi Josh,

Thanks so much=C2=A0for your suggestions.

The counts are determined with two methods, I am using a simple pig sc= ript to count records, and=C2=A0I am also tabulating up the size in bytes o= f all hdfs output files. Both measures show dropped records / fewer than ex= pected output bytes.

To your second point I will go back and do a sweep for that,=C2=A0but = I am fairly sure no=C2=A0DoFns are making use of intermediate state values = without getDetachedValue. Our team is=C2=A0aware of the getDetachedValue go= tchas as I think it has bitten us before.

Thanks !

Jeff


On Monday, July 27, 2015, Josh Wills <jwills@cloudera.com> wro= te:
One more thought-- are any of these DoFns keeping records = around as intermediate state values w/o using PType.getDetachedValue to mak= e copies of them?

J

On Mon, Jul 27, 2015 at 5:47 PM, Josh Will= s <jwills@cloudera.com> wrote:
Hey Jeff,

Are the counts determined by Counters? Or is it the length of the outp= ut files? Or both?

J

On Mon, Jul 27, 2015 at 5:29 PM, David Ort= iz <dpo5003@gmail.com> wrote:

Out of curiosity, any reason you went with multiple reads as= opposed to just performing multiple operations on the same PTable? paralle= lDo returns a new object rather than modifying the initial one, so a single= collection can start multiple execution flows.=C2=A0=C2=A0


On Mon, Jul 27, 2015, 8:11 PM=C2=A0Jeff Quinn <jeff@= nuna.com> wrote:
Hello,

We have observed and replicated= strange behavior with our crunch application while running on MapReduce vi= a the AWS ElasticMapReduce service. Running a very simple job which is most= ly map only, we see that an undetermined subset of records are getting dropped. Specifically, we expect=C2=A030,136= ,686 output records and have seen output on different trials (running over = the same data with the same binary):

22,177,119 records
26,435,670 records
22,362,986 records
29,798,528 records

These are all the things about = our application which might be unusual and relevant:

- We use a custom file input fo= rmat, via From.formattedFile. It looks like this (basically a carbon copy o= f=C2=A0org.apache.hadoop.mapreduce.lib.input.TextInputFormat):

import org.apache.had=
oop.io.LongWritable;
import org= .apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptCo= ntext;
import org.apache.hadoop= .mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
import java.io= .IOException;
public cla=
ss ByteOffsetInputFormat extends FileInputFormat<LongWritable, Text> {

= @Override
public RecordReader<LongWritable, Text> createRecordRea= der(
InputSplit split, TaskAttemptContext context) throws IOException,
In= terruptedException {
return new LineRecordReader();
}
}
- We call org.apache.crunch.Pipeline#read using this InputFormat many times, for =
the job in question it is called ~160 times as the input is ~100 different =
files. Each file ranges in size from 100MB-8GB. Our job only uses this inpu=
t format for all input files.
- For some files=C2=A0org.apache.crunch.Pipeline#read is called twice one the same file, and=
 the resulting PTables are processed in different ways.
- It is only the data from these files which org.apac=
he.crunch.Pipeline#read has been called on more than once during a job that=
 have dropped records, all other files consistently do not have dropped rec=
ords
Curious if any Crunch users have experienced similar =
behavior before, or if any of these details about my job raise any red flag=
s.
Thanks!
Jeff Quinn
Data Engineer
Nuna

DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that is confidential, proprietary in nature, prote= cted health information (PHI), or otherwise protected by law from disclosur= e, and is solely for the use of the intended recipient(s). If you are not t= he intended recipient, you are hereby notified that any use, disclosure or copying of this email, including any = attachments, is unauthorized and strictly prohibited. If you have received = this email in error, please notify the sender of this email. Please delete = this and all copies of this email from your system. Any opinions either expressed or implied in this email a= nd all attachments, are those of its author only, and do not necessarily re= flect those of Nuna Health, Inc.



--
Director of Data Science



--
Director of Data Science

DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that is confidential, proprietary in nature, prote= cted health information (PHI), or otherwise protected by law from disclosur= e, and is solely for the use of the intended recipient(s). If you are not t= he intended recipient, you are hereby notified that any use, disclosure or copying of this email, including any = attachments, is unauthorized and strictly prohibited. If you have received = this email in error, please notify the sender of this email. Please delete = this and all copies of this email from your system. Any opinions either expressed or implied in this email a= nd all attachments, are those of its author only, and do not necessarily re= flect those of Nuna Health, Inc.



--
Director of Data Science

DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that is confidential, proprietary in nature, prote= cted health information (PHI), or otherwise protected by law from disclosur= e, and is solely for the use of the intended recipient(s). If you are not t= he intended recipient, you are hereby notified that any use, disclosure or copying of this email, including any = attachments, is unauthorized and strictly prohibited. If you have received = this email in error, please notify the sender of this email. Please delete = this and all copies of this email from your system. Any opinions either expressed or implied in this email a= nd all attachments, are those of its author only, and do not necessarily re= flect those of Nuna Health, Inc.



--
Director of Data Science


DISCLAIMER:=C2=A0The conten= ts of this email, including any attachments, may contain information that is confidential, proprietary in nature, prote= cted health information (PHI), or otherwise protected by law from disclosur= e, and is solely for the use of the intended recipient(s). If you are not t= he intended recipient, you are hereby notified that any use, disclosure or copying of this email, including any = attachments, is unauthorized and strictly prohibited. If you have received = this email in error, please notify the sender of this email. Please delete = this and all copies of this email from your system. Any opinions either expressed or implied in this email a= nd all attachments, are those of its author only, and do not necessarily re= flect those of Nuna Health, Inc.
This email is intended only for the use of the individual(s)= to whom it is addressed. If you have received this communication in error,= please immediately notify the sender and delete the original email.



--
Director of Data Science
Twitter: @josh_wills
--94eb2c080b96d03f0a051bfb6957--