Doug,

Thanks for your response, I will open a Jira for adding a tell() method
with a description of how it would be used by a Record Reader to start the
discussion. With Christmas vacation I won't be able to provide a patch
until the beginning of January.

   Bob


Robert Goodman
 IBM Big Sheets

Doug Cutting <cutt...@apache.org> wrote on 12/08/2009 04:15:53 PM:

> Doug Cutting <cutt...@apache.org>
> 12/08/2009 04:15 PM
>
> Please respond to
> avro-dev@hadoop.apache.org
>
> To
>
> avro-dev@hadoop.apache.org
>
> cc
>
> Subject
>
> Re: Enhancing DataFileReader.java to support a Hadoop Import
> Formatter and RecordReader
>
> Robert Goodman wrote:
> >      In a Avro formatted file the metadata is at the end of the file
and
> >      must be read to get the schema, codec, etc. If this is done by
every
> >      RecordReader when processing a file split, it will be an expensive
> >      operation. The RecordReader would have to access data which not
> >      likely be local to the node. As an optimization the Input
Formatter
> >      can open the file and read the metadata when determining the file
> >      splits. The metadata information can be put in the InputSplit
which
> >      is passed to the Avro RecordReader.
> >
> >      In processing a file split a Record Reader must sync to the start
of
> >      a block and read past the end of the file split until it reaches
the
> >      end of the current block. The record reader needs to know when it
has
> >      reached a block boundary.
>
> Note that Hadoop's SequenceFile currently reads a small amount of
> metadata from the start of files but that this does not seem to affect
> performance much.
>
> You are right to note that Avro's data file currently requires metadata
> to be also read from the end of the file.  But there's currently a
> proposal to change Avro's data file format:
>
>    https://issues.apache.org/jira/browse/AVRO-160
>
> In short, the plan is to put metadata at the front of each block in the
> file.  Mapreduce applications would still need to read the sync marker
> from the head of the file before processing a split, but would no longer
> need to also read metadata from the end of the file.
>
> Your proposal to read sync markers and/or metadata when constructing
> splits has the downside that it could serialize something that's
> otherwise done in parallel.  For example, let's assume your jobs has
> 1000 input files each with 10 splits on a 100 node cluster.  With your
> proposal you'd need to open and read the headers of 1000 files in the
> client at job submit time.  Having map tasks read these instead would
> result in 10,000 reads of metadata, but they'd happen in parallel, 100
> or more at a time, and all but the first for each file would probably
> not require a seek.  The job client could be written to do these in
> parallel using a thread pool, but I doubt there would be much net job
> speedup, since the amount of metadata is small and fits in a packet or
two.
>
> >      It is possible for an error to occur when processing an Avro
record.
> >      A RecordReader should never just stop on an error and throw an
> >      exception. When an error occurs the Avro RecordReader should sync
to
> >      the next block and continue reading.
>
> I agree that ignoring errors should be possible, but it isn't always
best.
>
> >      The RecordReader needs to be able to get the current position in
the
> >      file to determine when it has reached the end of file split.
>
> Yes, DataFileReader() should probably have a tell() method.
>
> > Some re-factoring of DataFileReader.java would allow the class to be
used
> > by an Avro RecordReader. The following enhancements are being proposed.
> >
> >      Add a constructor which allows the metadata to be passed as a
> >      parameter. The new proposed constructor will not sync to the end
of
> >      the file and read the metadata.
>
> I think AVRO-160 will mostly obviate the need for this.
>
> >      Add a getBlockCount() method which returns the blockCount. This
will
> >      allow the a RecordReader to determine when to stop reading by
> >      checking if the blockCount is zero after reading pass the end of
the
> >      file split.
>
> Won't a tell() method be sufficient for this?  With block-based i/o and
> compression, tell() will generally return the position of the beginning
> of the current block, i.e., will not be incremented except when block
> boundaries are crossed.
>
> >      Add a syncReset() method which will sync to the next block marker
and
> >      reset the blockCount. This method is used by the record reader to
> >      move to the next block when an error occurs and continue reading.
>
> Wouldn't sync(tell()) implement this?
>
> >      Add a tell() method which will return the current position in the
> >      file. This is used by the record reader to determine when it has
read
> >      past the end of a file split.
>
> +1 This is certainly needed.
>
> > If I make these changes, is there interest in have these changes
> > contributed back to Avro.
>
> Please file a Jira issue for each that you intend to implement and
> discussion can continue there.
>
> Cheers,
>
> Doug
>

Reply via email to