[
https://issues.apache.org/jira/browse/MAPREDUCE-1176?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780225#action_12780225
]
Todd Lipcon commented on MAPREDUCE-1176:
----------------------------------------
Hi,
Just had a chance to look at your patch in more detail now that the initial
formatting issues have been straightened out:
- Please provide a static setter for the
mapreduce.input.fixedlengthinputformat.record.length configuration property,
rather than having users have to specify this property manually. An example of
this is in o.a.h.mapreduce.lib.input.NLineInputFormat.setNumLinesPerSplit. Also
update the javadoc and tests to refer to this new method
- The following code makes me nervous:
{code}
+ long splitSize =
+ ((long)(Math.floor((double)defaultSize /
+ (double)recordLength))) * recordLength;
{code}
Why can't you just keep defaultSize and recordLength as longs? Division of
longs will give you floor-like behavior and you won't have to worry about
floating point inaccuracies.
- In isSplitable, you catch the exception generated by getRecordLength and turn
off splitting. If there is no record length specified doesn't that mean the
input format won't work at all?
- FixedLengthRecordReader: "This record reader does not support compressed
files." Is this true? Or does it just not support *splitting* compressed files?
I see that you've explicitly disallowed it, but I don't understand this
decision.
- Throughout, you've still got 4-space indentation in the method bodies.
Indentation should be by 2.
- In FixedLengthRecordReader, you hard code a 64KB buffer. Why's this? You
should let the filesystem use its default.
- In your read loop, you're not accounting for the case of read returning 0 or
-1, which I believe can happen at EOF, right? Consider using
o.a.h.io.IOUtils.readFully() to replace this loop.
As a general note, I'm not sure I agree with the design here. Rather than
forcing the split to lie on record boundaries, I think it would be simpler to
simply let FileInputFormat compute its own splits, and then when you first open
the record reader, skip forward to the next record boundary and begin reading
from there. Then for the last record of the file, "over read" your split into
the beginning of the next one. This is the strategy that other input formats
take, and should be compatible with the splittable compression codecs (see
TextInputFormat for example).
I don't want to harp too much on the compression thing, in my experience the
sorts of datasets that have these fixed-length records are very highly
compressible - lots and lots of numeric fields/UPCs/zipcodes/etc.
> Contribution: FixedLengthInputFormat and FixedLengthRecordReader
> ----------------------------------------------------------------
>
> Key: MAPREDUCE-1176
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-1176
> Project: Hadoop Map/Reduce
> Issue Type: New Feature
> Affects Versions: 0.20.1, 0.20.2
> Environment: Any
> Reporter: BitsOfInfo
> Priority: Minor
> Attachments: MAPREDUCE-1176-v1.patch, MAPREDUCE-1176-v2.patch
>
>
> Hello,
> I would like to contribute the following two classes for incorporation into
> the mapreduce.lib.input package. These two classes can be used when you need
> to read data from files containing fixed length (fixed width) records. Such
> files have no CR/LF (or any combination thereof), no delimiters etc, but each
> record is a fixed length, and extra data is padded with spaces. The data is
> one gigantic line within a file.
> Provided are two classes first is the FixedLengthInputFormat and its
> corresponding FixedLengthRecordReader. When creating a job that specifies
> this input format, the job must have the
> "mapreduce.input.fixedlengthinputformat.record.length" property set as follows
> myJobConf.setInt("mapreduce.input.fixedlengthinputformat.record.length",[myFixedRecordLength]);
> OR
> myJobConf.setInt(FixedLengthInputFormat.FIXED_RECORD_LENGTH,
> [myFixedRecordLength]);
> This input format overrides computeSplitSize() in order to ensure that
> InputSplits do not contain any partial records since with fixed records there
> is no way to determine where a record begins if that were to occur. Each
> InputSplit passed to the FixedLengthRecordReader will start at the beginning
> of a record, and the last byte in the InputSplit will be the last byte of a
> record. The override of computeSplitSize() delegates to FileInputFormat's
> compute method, and then adjusts the returned split size by doing the
> following: (Math.floor(fileInputFormatsComputedSplitSize / fixedRecordLength)
> * fixedRecordLength)
> This suite of fixed length input format classes, does not support compressed
> files.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.