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

Aaron Fabbri commented on HADOOP-15229:
---------------------------------------

Nice work here, as usual [~ste...@apache.org].

{noformat}
+ * the actual outcome i in the returned {@code CompletableFuture}.
{noformat}

/i/is/  This typo is pasted a couple of times.

{noformat}
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   * @throws UnsupportedOperationException PathHandles are not supported.
+   */
+  protected CompletableFuture<FSDataInputStream> openFileWithOptions(
{noformat}
Do you want to mention that all except the UnsupportedOp exceptions are not 
thrown here but deferred to the future's get() call?

Also, are these methods still "throws IOException"?

You can elide the whitespace change to HarFileSystem.java,

Glad S3AOpContext is working out as a place to stash per-op stuff.

{noformat}

+      // method not allowed; seen on S3 Select.
+      // treated as a bad request
+      case 405:
+        ioe = new AWSBadRequestException(message, s3Exception);
+        break;
{noformat}
Good catch.

{noformat}
+    case SelectTool.NAME:
+      // the select tool is not technically a S3Guard tool, but it's on the CLI
+      // because this is the defacto S3 CLI.
{noformat}
Indeed. 

{noformat}
 /**
+   * Closed bit. Volatile so reads are non-blocking.
+   * Updates must be in a synchronized block to guarantee an atomic check and
+   * set
+   */
{noformat}
Out of date comment (it is now an atomic). Volatile mentioned again in another 
comment. Minor nits here.

I'm not finding bugs in the actual code but here's one in the doc's example 
code ;-)

{noformat}

+try (FSDataInputStream select = future.get()) {
+  // process the output
+  stream.read();
+}
{noformat}
/stream/select/

On the current seek() implementation of the Select input stream, what are the 
next enhancements you think we will need?  Can you elaborate a bit on the need 
for single-byte reads as a seek implementation? Is it a limitation of the 
underlying AWS stream or SELECT rest API?

I got to around line 6000 in the diff and am out of time for now. Will follow 
up with comments on the rest soon.









> 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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to