[
https://issues.apache.org/jira/browse/HADOOP-15229?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Steve Loughran updated HADOOP-15229:
------------------------------------
Status: Patch Available (was: Open)
HADOOP-15229 Patch 010
Adds passdown of options through MR jobs to the record readers, which, once
they switch to the new builder API, can pick up the options.
Two new option prefixes are added
{{mapreduce.job.input.file.option.}} : prefix for all optional builder
settings
{{mapreduce.job.input.file.must.}} : prefix for all mandatory settings.
There's a class FutureIOHelper to take a job conf and an FSBuilder, extract and
apply all the settings. This would work for output stream configuration too.
For adoption: requires the RecordReader to switch to the API: LineRecordReader,
FixedLengthRecordReader and NLineInputFormat do this. Although the open is now
async, they're just blocking for it to open, extracting and rethrowing IOEs.
Again, a helper method does this.
Not supported: SequenceFileRecordReader, because the read is kicked off
SequenceFile and it'd need changes.
One finding of this work is that the proposed option naming (as we do for the
output file builder), doesn't map well to names in configuraiton files.
Specifically, schema naming "s3a:sql" is better replaced with "s3a.sql',
"s3a:fadvise", and for core FS features, "fs.", e.g. "fs.seek.policy". This
current revision doesn't do that (it maps the first "." to ":", but thats
contrived, ugly and brittle.
I do propose moving to dotted names; this will make it different from the
fs.create() option. Which we may want to revisit before it gets used much.
Test coverage currently looks at LineRecordReader, verifying that an S3 select
can be passed in and you get the filtered record set back. I'd thought about
doing a real MR job, but I'd rather do the test coverage of the other readers
(verify that if you open a file with an unknown mapreduce.job.input.file.must.
option then the open will fail.
Excluding the schema/dotted issue, this is now at the stage where people
downstream can consider playing with.
> Add FileSystem builder-based openFile() API to match createFile()
> -----------------------------------------------------------------
>
> Key: HADOOP-15229
> URL: https://issues.apache.org/jira/browse/HADOOP-15229
> Project: Hadoop Common
> Issue Type: New Feature
> Components: fs, fs/azure, fs/s3
> Affects Versions: 3.0.0
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Major
> Attachments: HADOOP-15229-001.patch, HADOOP-15229-002.patch,
> HADOOP-15229-003.patch, HADOOP-15229-004.patch, HADOOP-15229-004.patch,
> HADOOP-15229-005.patch, HADOOP-15229-006.patch, HADOOP-15229-007.patch,
> HADOOP-15229-009.patch, HADOOP-15229-010.patch
>
>
> Replicate HDFS-1170 and HADOOP-14365 with an API to open files.
> A key requirement of this is not HDFS, it's to put in the fadvise policy for
> working with object stores, where getting the decision to do a full GET and
> TCP abort on seek vs smaller GETs is fundamentally different: the wrong
> option can cost you minutes. S3A and Azure both have adaptive policies now
> (first backward seek), but they still don't do it that well.
> Columnar formats (ORC, Parquet) should be able to say "fs.input.fadvise"
> "random" as an option when they open files; I can imagine other options too.
> The Builder model of [~eddyxu] is the one to mimic, method for method.
> Ideally with as much code reuse as possible
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]