Return-Path: X-Original-To: apmail-hadoop-mapreduce-user-archive@minotaur.apache.org Delivered-To: apmail-hadoop-mapreduce-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 6237010354 for ; Sat, 23 Nov 2013 03:31:15 +0000 (UTC) Received: (qmail 91541 invoked by uid 500); 23 Nov 2013 03:30:58 -0000 Delivered-To: apmail-hadoop-mapreduce-user-archive@hadoop.apache.org Received: (qmail 91461 invoked by uid 500); 23 Nov 2013 03:30:57 -0000 Mailing-List: contact user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hadoop.apache.org Delivered-To: mailing list user@hadoop.apache.org Received: (qmail 91454 invoked by uid 99); 23 Nov 2013 03:30:55 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 23 Nov 2013 03:30:55 +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 (nike.apache.org: domain of azuryyyu@gmail.com designates 209.85.212.45 as permitted sender) Received: from [209.85.212.45] (HELO mail-vb0-f45.google.com) (209.85.212.45) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 23 Nov 2013 03:30:49 +0000 Received: by mail-vb0-f45.google.com with SMTP id p14so1467329vbm.32 for ; Fri, 22 Nov 2013 19:30:28 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=SAsth/mVzvI/qBNFa/WK7YUPq2tv/kZhNRC82QpvCgI=; b=jWojCdLKAgFle6LheS9pq5/mUVcOaw5JSqHvoFHWp0344MxAkcbnDVOFqSOV2Zy67c v6LHk1+AehsZT1IvpVvpjGOVEaMh8dT+cc5WkopdLsURVxNxyDr3Cu9JJVuR5WKVsSWe 0Ylj39XQ56Q1SY3F9m4AqKKF9ttQdrfYvheTCkoFttpJv+kdSfWgQLqH8YR3t2PvVfgP fVjnAQqrd9HDf999o85dAavfYhycJbTsnDj0K1TeQiPZpQp+816Lqp+i0cWBMswnJ4EB i1rxR8L9HozuhJShLLygoPM7VIHGkDumEjRyEjQo33YEx5DRyYMn9kcJetU2Fk21DkMo MN4A== MIME-Version: 1.0 X-Received: by 10.52.230.102 with SMTP id sx6mr12193387vdc.15.1385177428240; Fri, 22 Nov 2013 19:30:28 -0800 (PST) Received: by 10.220.163.201 with HTTP; Fri, 22 Nov 2013 19:30:28 -0800 (PST) Received: by 10.220.163.201 with HTTP; Fri, 22 Nov 2013 19:30:28 -0800 (PST) In-Reply-To: References: Date: Sat, 23 Nov 2013 11:30:28 +0800 Message-ID: Subject: Re: Missing records from HDFS From: Azuryy Yu To: user@hadoop.apache.org Content-Type: multipart/alternative; boundary=089e0111ae348faf6604ebcfc0e0 X-Virus-Checked: Checked by ClamAV on apache.org --089e0111ae348faf6604ebcfc0e0 Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable There is problem in the 'initialize', generally, we cannot think split.start as the real start, because FileSplit cannot split on the end of the line accurately, so you need to adjust the start in the 'initialize' to the start of one line if start is not equal to '0'. also, end =3D start + split.length, this is not a real end, because it mayb= e not locate the end of the line. so the Reader MUST adjust the real start and the end in the 'initialize'. otherwise, maybe miss some records. Sure, our FileInputFormat implementation: public class CVSInputFormat extends FileInputFormat { /* * (non-Javadoc) * * @see * org.apache.hadoop.mapreduce.InputFormat#createRecordReader(org.apach= e * .hadoop.mapreduce.InputSplit, * org.apache.hadoop.mapreduce.TaskAttemptContext) */ @Override public RecordReader createRecordReader( InputSplit split, TaskAttemptContext context) { String delimiter =3D context.getConfiguration().get( "textinputformat.record.delimiter"); byte[] recordDelimiterBytes =3D null; if (null !=3D delimiter) recordDelimiterBytes =3D delimiter.getBytes(); return new CVSLineRecordReader(recordDelimiterBytes); } /* * (non-Javadoc) * * @see * org.apache.hadoop.mapreduce.lib.input.FileInputFormat#isSplitable(or= g * .apache.hadoop.mapreduce.JobContext, org.apache.hadoop.fs.Path) */ @Override protected boolean isSplitable(JobContext context, Path file) { CompressionCodec codec =3D new CompressionCodecFactory( context.getConfiguration()).getCodec(file); return codec =3D=3D null; } } the recordReader: public class CVSLineRecordReader extends RecordReader { private static final Log LOG =3D LogFactory.getLog(CVSLineRecordReader.class); public static final String CVS_FIRST_LINE =3D "file.first.line"; private long start; private long pos; private long end; private LineReader in; private int maxLineLength; private FileValidatorDescriptor key =3D null; private Text value =3D null; private Text data =3D null; private byte[] recordDelimiterBytes; public CVSLineRecordReader(byte[] recordDelimiter) { this.recordDelimiterBytes =3D recordDelimiter; } @Override public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException { Properties properties =3D new Properties(); Configuration configuration =3D context.getConfiguration(); Path[] cacheFiles =3D DistributedCache.getLocalCacheFiles(context .getConfiguration()); for (Path cacheFile : cacheFiles) { if (cacheFile.toString().endsWith( context.getConfiguration().get(VALIDATOR_CONF_PATH))) { properties.load(new FileReader(cacheFile.toString())); } } FileSplit split =3D (FileSplit) genericSplit; Configuration job =3D context.getConfiguration(); this.maxLineLength =3D job.getInt("mapred.linerecordreader.maxlengt= h", Integer.MAX_VALUE); start =3D split.getStart(); end =3D start + split.getLength(); pos =3D start; final Path file =3D split.getPath(); // open the file and seek to the start of the split FileSystem fs =3D file.getFileSystem(job); FSDataInputStream fileIn =3D fs.open(split.getPath()); this.in =3D generateReader(fileIn, job); // if CVS_FIRST_LINE does not exist in conf then the csv file first line // is the header if (properties.containsKey(CVS_FIRST_LINE)) { configuration.set(CVS_FIRST_LINE, properties.get(CVS_FIRST_LINE= ) .toString()); } else { readData(); configuration.set(CVS_FIRST_LINE, data.toString()); if (start !=3D 0) { fileIn.seek(start); in =3D generateReader(fileIn, job); pos =3D start; } } key =3D new FileValidatorDescriptor(); key.setFileName(split.getPath().getName()); context.getConfiguration().set("file.name", key.getFileName()); } @Override public boolean nextKeyValue() throws IOException { int newSize =3D readData(); if (newSize =3D=3D 0) { key =3D null; value =3D null; return false; } else { key.setOffset(this.pos); value =3D data; return true; } } private LineReader generateReader(FSDataInputStream fileIn, Configuration job) throws IOException { if (null =3D=3D this.recordDelimiterBytes) { return new LineReader(fileIn, job); } else { return new LineReader(fileIn, job, this.recordDelimiterBytes); } } private int readData() throws IOException { if (data =3D=3D null) { data =3D new Text(); } int newSize =3D 0; while (pos < end) { newSize =3D in.readLine(data, maxLineLength, Math.max((int) Math.min(Integer.MAX_VALUE, end - pos), maxLineLength)); if (newSize =3D=3D 0) { break; } pos +=3D newSize; if (newSize < maxLineLength) { break; } // line too long. try again LOG.info("Skipped line of size " + newSize + " at pos " + (pos - newSize)); } return newSize; } @Override public FileValidatorDescriptor getCurrentKey() { return key; } @Override public Text getCurrentValue() { return value; } @Override public float getProgress() { if (start =3D=3D end) { return 0.0f; } else { return Math.min(1.0f, (pos - start) / (float) (end - start)); } } @Override public synchronized void close() throws IOException { if (in !=3D null) { in.close(); } } } Thanks. De: Azuryy Yu Responder a: "user@hadoop.apache.org" Fecha: viernes, 22 de noviembre de 2013 12:19 Para: "user@hadoop.apache.org" Asunto: Re: Missing records from HDFS I do think this is because of your RecorderReader, can you paste your code here? and give a piece of data example. please use pastebin if you want. On Fri, Nov 22, 2013 at 7:16 PM, ZORAIDA HIDALGO SANCHEZ wr= ote: > One more thing, > > if we split the files then all the records are processed. Files are > of 70,5MB. > > Thanks, > > Zoraida.- > > De: zoraida > Fecha: viernes, 22 de noviembre de 2013 08:59 > > Para: "user@hadoop.apache.org" > Asunto: Re: Missing records from HDFS > > Thanks for your response Azuryy. > > My hadoop version: 2.0.0-cdh4.3.0 > InputFormat: a custom class that extends from FileInputFormat(csv input > format) > These fiels are under the same directory, different files. > My input path is configured using oozie throughout the propertie > mapred.input.dir. > > > Same code and input running on Hadoop 2.0.0-cdh4.2.1 works fine. Does > not discard any record. > > Thanks. > > De: Azuryy Yu > Responder a: "user@hadoop.apache.org" > Fecha: jueves, 21 de noviembre de 2013 07:31 > Para: "user@hadoop.apache.org" > Asunto: Re: Missing records from HDFS > > what's your hadoop version? and which InputFormat are you used? > > these files under one directory or there are lots of subdirectory? how > ddi you configure input path in your main? > > > > On Thu, Nov 21, 2013 at 12:25 AM, ZORAIDA HIDALGO SANCHEZ wrote: > >> Hi all, >> >> my job is not reading all the input records. In the input directory I >> have a set of files containing a total of 6000000 records but only 59970= 00 >> are processed. The Map Input Records counter says 5997000. >> I have tried downloading the files with a getmerge to check how many >> records would return but the correct number is returned(6000000). >> >> Do you have any suggestion? >> >> Thanks. >> >> ------------------------------ >> >> Este mensaje se dirige exclusivamente a su destinatario. Puede consultar >> nuestra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en e= l enlace >> situado m=E1s abajo. >> This message is intended exclusively for its addressee. We only send and >> receive email on the basis of the terms set out at: >> http://www.tid.es/ES/PAGINAS/disclaimer.aspx >> > > > ------------------------------ > > Este mensaje se dirige exclusivamente a su destinatario. Puede consultar > nuestra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en el= enlace > situado m=E1s abajo. > This message is intended exclusively for its addressee. We only send and > receive email on the basis of the terms set out at: > http://www.tid.es/ES/PAGINAS/disclaimer.aspx > ------------------------------ Este mensaje se dirige exclusivamente a su destinatario. Puede consultar nuestra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en el e= nlace situado m=E1s abajo. This message is intended exclusively for its addressee. We only send and receive email on the basis of the terms set out at: http://www.tid.es/ES/PAGINAS/disclaimer.aspx --089e0111ae348faf6604ebcfc0e0 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable

There is problem in the 'initialize', generally, we = cannot think split.start as the real start, because FileSplit cannot split = on the end of the line accurately, so=A0 you need to adjust the start in th= e 'initialize' to the start of one line if start is not equal to &#= 39;0'.

also, end =3D start + split.length, this is not a real end, = because it maybe not locate the end of the line.

so the Reader MUST adjust the real start and the end in the = 'initialize'. otherwise, maybe miss some records.

Sure,

our FileInputFormat implementation:

public class CVSInputFormat extends

=A0 =A0 =A0 =A0 F= ileInputFormat<FileValidatorDescriptor, Text> {

<= br>

=A0 =A0 /*

=A0=A0 =A0 * (non-Javadoc)=

=A0=A0 =A0 *=A0

=A0=A0 =A0 * @see

=A0=A0 =A0 * org.apache.hadoop.mapreduce.InputFormat#createRecordReader(org= .apache

=A0=A0 =A0 * .hadoop.mapreduce.InputSplit,

=A0=A0 =A0 * org.apache.hadoop.mapreduce.TaskAttemptContext)

=A0=A0 =A0 */

=A0 =A0 @Override

=A0 =A0 public RecordReader<FileValidatorDescriptor, Text> createRecor= dReader(

=A0 =A0 =A0 =A0 = =A0 =A0 InputSplit split, TaskAttemptContext context) {

=A0 =A0 =A0 =A0 S= tring delimiter =3D context.getConfiguration().get(

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 "textinputformat.record.delimiter");

=A0 =A0 =A0 =A0 <= span style=3D"color:#931a68"> byte[] recordDelimiterBytes =3D null;

=A0 =A0 =A0 =A0 <= span style=3D"color:#931a68"> if (null !=3D delimiter)

=A0 =A0 =A0 =A0 = =A0 =A0 recordDelimiterBytes =3D delimiter.getBytes();

=A0 =A0 =A0 =A0 <= span style=3D"color:#931a68"> return new CVSLineRecordReader(= recordDelimiterBytes);

=A0 =A0 }

<= br>

=A0 =A0 /*

=A0=A0 =A0 * (non-Javadoc)=

=A0=A0 =A0 *=A0

=A0=A0 =A0 * @see

=A0=A0 =A0 * org.apache.hadoop.mapreduce.lib.input.FileInputFormat#isSplita= ble(org

=A0=A0 =A0 * .apache.hadoop.mapreduce.JobContext, org.apache.hadoop.fs.Path= )

=A0=A0 =A0 */

=A0 =A0 @Override

=A0 =A0 protected boolean isSplitable(J= obContext context, Path file) {

=A0 =A0 =A0 =A0 C= ompressionCodec codec =3D new CompressionCodecFactory(

=A0 =A0 =A0 =A0 = =A0 =A0 =A0 =A0 context.getConfiguration()).getCodec(file);

=A0 =A0 =A0 =A0 <= span style=3D"color:#931a68"> return codec =3D=3D null;

=A0 =A0 }

}


the recordReader:=



public class CVSLineRecordReader extends

=A0 =A0 =A0 =A0 RecordReader<FileValidatorDescri= ptor, Text> {

=A0 =A0 private static final Log LOG =3D LogFactory= .getLog(CVSLineRecordReader.class);


=A0 =A0 public static final String CVS_FIRST_LINE = =3D "file.first.line";


=A0 =A0 private long start;

=A0 =A0 private long pos;

=A0 =A0 private long end;

=A0 =A0 private LineReader in;

=A0 =A0 private int maxLineLength;

=A0 =A0 private FileValidatorDescriptor key =3D nul= l;

=A0 =A0 private Text value =3D null;

=A0 =A0 private Text data =3D null;

=A0 =A0 private byte[] recordDelimiterBytes;


=A0 =A0 public CVSLineRecordReader(byte[] recordDel= imiter) {

=A0 =A0 =A0 =A0 this.recordDelimiterBytes =3D recor= dDelimiter;

=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public void initialize(InputSplit genericSp= lit, TaskAttemptContext context)

=A0 =A0 =A0 =A0 =A0 =A0 throws IOException {

=A0 =A0 =A0 =A0 Properties properties =3D new Prope= rties();

=A0 =A0 =A0 =A0 Configuration configuration =3D con= text.getConfiguration();


=A0 =A0 =A0 =A0 Path[] cacheFiles =3D DistributedCa= che.getLocalCacheFiles(context

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 .getConfiguration()= );

=A0 =A0 =A0 =A0 for (Path cacheFile : cacheFiles) {=

=A0 =A0 =A0 =A0 =A0 =A0 if (cacheFile.toString().en= dsWith(

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 context.get= Configuration().get(VALIDATOR_CONF_PATH))) {

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 properties.load(new= FileReader(cacheFile.toString()));

=A0 =A0 =A0 =A0 =A0 =A0 }

=A0 =A0 =A0 =A0 }


=A0 =A0 =A0 =A0 FileSplit split =3D (FileSplit) gen= ericSplit;

=A0 =A0 =A0 =A0 Configuration job =3D context.getCo= nfiguration();

=A0 =A0 =A0 =A0 this.maxLineLength =3D job.getInt(&= quot;mapred.linerecordreader.maxlength",

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 Integer.MAX_VALUE);=

=A0 =A0 =A0 =A0 start =3D split.getStart();

=A0 =A0 =A0 =A0 end =3D start + split.getLength();<= /p>

=A0 =A0 =A0 =A0 pos =3D start;

=A0 =A0 =A0 =A0 final Path file =3D split.getPath()= ;


=A0 =A0 =A0 =A0 // open the file and seek to the st= art of the split

=A0 =A0 =A0 =A0 FileSystem fs =3D file.getFileSyste= m(job);

=A0 =A0 =A0 =A0 FSDataInputStream fileIn =3D fs.ope= n(split.getPath());


=A0 =A0 =A0 =A0 this.in =3D generateReader(fileIn, job);


=A0 =A0 =A0 =A0 // if CVS_FIRST_LINE does not exist= in conf then the csv file first line

=A0 =A0 =A0 =A0 // is the header

=A0 =A0 =A0 =A0 if (properties.containsKey(CVS_FIRS= T_LINE)) {

=A0 =A0 =A0 =A0 =A0 =A0 configuration.set(CVS_FIRST= _LINE, properties.get(CVS_FIRST_LINE)

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 .toString()= );

=A0 =A0 =A0 =A0 } else {

=A0 =A0 =A0 =A0 =A0 =A0 readData();

=A0 =A0 =A0 =A0 =A0 =A0 configuration.set(CVS_FIRST= _LINE, data.toString());

=A0 =A0 =A0 =A0 =A0 =A0 if (start !=3D 0) {

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 fileIn.seek(start);=

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 in =3D generateRead= er(fileIn, job);

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 pos =3D start;

=A0 =A0 =A0 =A0 =A0 =A0 }

=A0 =A0 =A0 =A0 }


=A0 =A0 =A0 =A0 key =3D new FileValidatorDescriptor= ();

=A0 =A0 =A0 =A0 key.setFileName(split.getPath().get= Name());

=A0 =A0 =A0 =A0 context.getConfiguration().set(&quo= t;file.name", key.g= etFileName());


=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public boolean nextKeyValue() throws IOExce= ption {

=A0 =A0 =A0 =A0 int newSize =3D readData();

=A0 =A0 =A0 =A0 if (newSize =3D=3D 0) {

=A0 =A0 =A0 =A0 =A0 =A0 key =3D null;

=A0 =A0 =A0 =A0 =A0 =A0 value =3D null;

=A0 =A0 =A0 =A0 =A0 =A0 return false;

=A0 =A0 =A0 =A0 } else {

=A0 =A0 =A0 =A0 =A0 =A0 key.setOffset(this.pos);

=A0 =A0 =A0 =A0 =A0 =A0 value =3D data;

=A0 =A0 =A0 =A0 =A0 =A0 return true;

=A0 =A0 =A0 =A0 }

=A0 =A0 }


=A0 =A0 private LineReader generateReader(FSDataInp= utStream fileIn,

=A0 =A0 =A0 =A0 =A0 =A0 Configuration job) throws I= OException {

=A0 =A0 =A0 =A0 if (null =3D=3D this.recordDelimite= rBytes) {

=A0 =A0 =A0 =A0 =A0 =A0 return new LineReader(fileI= n, job);

=A0 =A0 =A0 =A0 } else {

=A0 =A0 =A0 =A0 =A0 =A0 return new LineReader(fileI= n, job, this.recordDelimiterBytes);

=A0 =A0 =A0 =A0 }


=A0 =A0 }


=A0 =A0 private int readData() throws IOException {=

=A0 =A0 =A0 =A0 if (data =3D=3D null) {

=A0 =A0 =A0 =A0 =A0 =A0 data =3D new Text();

=A0 =A0 =A0 =A0 }

=A0 =A0 =A0 =A0 int newSize =3D 0;

=A0 =A0 =A0 =A0 while (pos < end) {

=A0 =A0 =A0 =A0 =A0 =A0 newSize =3D in.readLine(dat= a, maxLineLength,

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 Math.max((i= nt) Math.min(Integer.MAX_VALUE, end - pos),

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0= =A0 maxLineLength));

=A0 =A0 =A0 =A0 =A0 =A0 if (newSize =3D=3D 0) {

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 break;

=A0 =A0 =A0 =A0 =A0 =A0 }

=A0 =A0 =A0 =A0 =A0 =A0 pos +=3D newSize;

=A0 =A0 =A0 =A0 =A0 =A0 if (newSize < maxLineLen= gth) {

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 break;

=A0 =A0 =A0 =A0 =A0 =A0 }


=A0 =A0 =A0 =A0 =A0 =A0 // line too long. try again=

=A0 =A0 =A0 =A0 =A0 =A0 LOG.info("Skipped line= of size " + newSize + " at pos "

=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 + (pos - ne= wSize));

=A0 =A0 =A0 =A0 }

=A0 =A0 =A0 =A0 return newSize;

=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public FileValidatorDescriptor getCurrentKe= y() {

=A0 =A0 =A0 =A0 return key;

=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public Text getCurrentValue() {

=A0 =A0 =A0 =A0 return value;

=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public float getProgress() {

=A0 =A0 =A0 =A0 if (start =3D=3D end) {

=A0 =A0 =A0 =A0 =A0 =A0 return 0.0f;

=A0 =A0 =A0 =A0 } else {

=A0 =A0 =A0 =A0 =A0 =A0 return Math.min(1.0f, (pos = - start) / (float) (end - start));

=A0 =A0 =A0 =A0 }

=A0 =A0 }


=A0 =A0 @Override

=A0 =A0 public synchronized void close() throws IOE= xception {

=A0 =A0 =A0 =A0 if (in !=3D null) {

=A0 =A0 =A0 =A0 =A0 =A0 in.close();

=A0 =A0 =A0 =A0 }

=A0 =A0 }

}


Thanks.


De: Azuryy Yu <azuryyyu@gmail.com>
Responder a: "user@hadoop.apache.org"= ; <user@hado= op.apache.org>
Fecha: viernes, 22 de noviembre de = 2013 12:19
Para: "user@hadoop.apache.org" <<= a href=3D"mailto:user@hadoop.apache.org" target=3D"_blank">user@hadoop.apac= he.org>
Asunto: Re: Missing records from HD= FS

I do think this is because of your RecorderReader, can you= paste your code here? and give a piece of data example.

please use pastebin if you want.=A0


On Fri, Nov 22, 2013 at 7:16 PM, ZORAIDA HIDALGO= SANCHEZ <z= oraida@tid.es> wrote:
One more thing,

if we split the files then all the records are processed. Files are of= =A070,5MB.

Thanks,

Zoraida.-

De: zoraida <zoraida@tid.es>
Fecha: viernes, 22 de noviembre de = 2013 08:59

Para: "user@hadoop.apache.org" <<= a href=3D"mailto:user@hadoop.apache.org" target=3D"_blank">user@hadoop.apac= he.org>
Asunto: Re: Missing records from HD= FS

Thanks for you= r response Azuryy.

My hadoop vers= ion:=A02.0.0-cdh4.3.0
InputFormat: a= custom class that extends from=A0FileInputFormat(csv input format)
These fiels are under the same director= y, different files.
My input path is configured using oozie=A0throug= hout=A0the=A0propertie=A0mapred.input.dir.=A0


Same code and input running on=A0Hadoop 2.0.0-cdh4.2.1 works fine. Does not discard any record.

Thanks.

De: Azuryy Yu <azuryyyu@gmail.com>
Responder a: "user@hadoop.apache.org"= ; <user@hado= op.apache.org>
Fecha: jueves, 21 de noviembre de 2= 013 07:31
Para: "user@hadoop.apache.org" <<= a href=3D"mailto:user@hadoop.apache.org" target=3D"_blank">user@hadoop.apac= he.org>
Asunto: Re: Missing records from HD= FS

what's your hadoop version? and which InputFormat are = you used?

these files under one directory or there are lots of subdirectory? how= ddi you configure input path in your main?



On Thu, Nov 21, 2013 at 12:25 AM, ZORAIDA HIDALG= O SANCHEZ <z= oraida@tid.es> wrote:
Hi all,

my job is not reading all the input records. In the input directory I = have a set of files containing a total of 6000000 records but only 5997000 = are processed. The=A0Map Input Records counter says=A05997000.
I have tried downloading the files with a getmerge to check how many r= ecords would return but the correct number is returned(6000000).

Do you have any suggestion?=A0

Thanks.=A0



Este mensaje se dirige exclusivamente a su destinatario. Puede consultar nu= estra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en el enl= ace situado m=E1s abajo.
This message is intended exclusively for its addressee. We only send and re= ceive email on the basis of the terms set out at:
= http://www.tid.es/ES/PAGINAS/disclaimer.aspx




Este mensaje se dirige exclusivamente a su destinatario. Puede consultar nu= estra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en el enl= ace situado m=E1s abajo.
This message is intended exclusively for its addressee. We only send and re= ceive email on the basis of the terms set out at:
= http://www.tid.es/ES/PAGINAS/disclaimer.aspx




Este mensaje se dirige exclusivamente a su destinatario. Puede consultar nu= estra pol=EDtica de env=EDo y recepci=F3n de correo electr=F3nico en el enl= ace situado m=E1s abajo.
This message is intended exclusively for its addressee. We only send and re= ceive email on the basis of the terms set out at:
= http://www.tid.es/ES/PAGINAS/disclaimer.aspx
--089e0111ae348faf6604ebcfc0e0--