[ 
https://issues.apache.org/jira/browse/HADOOP-15229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16728194#comment-16728194
 ] 

Yuzhou Sun commented on HADOOP-15229:
-------------------------------------

Hey, sorry about the late reply, was on vacation.

About LineRecordReader (and maybe other RecordReaders), reusing original 
RecordReaders for S3 Select instead of writing a new RecordReader reduced code 
duplication. But this makes it tricky to handle some S3 Select specific issues:
 * In next(), it only reads when the position is less or equal to end of the 
split:

{noformat}
while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
{noformat}
If file is compressed, S3 select may return more bytes than the file size (not 
a good use case though), records will be lost in this case.
 * Also, getProgress() uses file position to calculate the result:

{noformat}
Math.min(1.0f, (getFilePosition() - start) / (float)(end - start));
{noformat}
 

+1 for a followup patch or Jira "ideally with some $0 to use, stable, test 
datasets for all supported formats." This patch becomes much more about just 
"Add FileSystem builder-based openFile() API"

 

And some trivial comments:
 * Java doc of AbstractFSBuilderImpl: maybe move the example of opt() and 
must() to FSBuilder?

 * extra empty lines in FutureIOSupport.java

 * SelectBinding#expandBackslashChars: How about using 
org.apache.commons.lang3.StringEscapeUtils.unescapeJava?

> 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, HADOOP-15229-011.patch, 
> HADOOP-15229-012.patch, HADOOP-15229-013.patch, HADOOP-15229-014.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]

Reply via email to