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

Sameer Choudhary commented on HADOOP-15229:
-------------------------------------------

[~ste...@apache.org] Thanks for providing the inputs on 
https://issues.apache.org/jira/browse/HADOOP-15364. Will use this Jira for all 
future feedback and discussion related to S3 Select support for S3A.

 
{quote}SelectInputStream uses Invoker.once() to translate exceptions raised in 
the read() operation of the wrapped stream. Because now its not just an HTTP 
connection, it's a chain of paged results from the S3 select call, and it can 
fail in new ways...even socket exceptions get translated into AWS SDK 
exceptions. These are now converted back to some form of IOE.
{quote}
This is partially correct. As I mentioned before in HADOOP-15364, client 
receives 200 HTTP Status Code at the start of streaming. Not all exceptions 
will be reported by the SDK as EventStreamingException. For SDK to return 
EventStreamingException, it must receive a "Request Level Error Message" from 
the service. The specifics of the protocol are documented at 
[https://docs.aws.amazon.com/AmazonS3/latest/API/RESTObjectSELECTContent.html.] 
There could be cases where the service doesn't send this Message to the client 
(in case of an unknown service side bug that results in early termination of 
some query) or any other network issues. The client in that case will not get 
any exception and will have partial output which could be treated as OK 
response, which would be incorrect. Thus assertion on EndEvent is required. 
Adding the link to sample code showing proper handling again for reference 
[https://docs.aws.amazon.com/AmazonS3/latest/dev/SelectObjectContentUsingJava.html.]
 
{quote}S3A Invoker is used to wrap IO

the initial [select 
call|https://github.com/steveloughran/hadoop/blob/filesystem/HADOOP-15229-openfile/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L3527]
 is retried with the [normal retry 
policy|https://github.com/steveloughran/hadoop/blob/filesystem/HADOOP-15229-openfile/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java].
 If there are specific errors which need extra retry/fast fail, that'd be handy.

There's no attempt to recover from failures in a read() itself, the SDK 
exceptions are simply mapped/wrapped to IOEs.
[https://github.com/steveloughran/hadoop/blob/802fcc9f4d80f6d11582c175efa57ed580d0b25d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java#L210]

looking @ that code, I think we should be collecting stats on failures there
{quote}
Could we raise IOE from the Filesystem in this case along with collecting the 
stats? This would require additional code to track the EndEvent in addition to 
wrapping SDK exceptions.
{quote}always. See 
[https://github.com/steveloughran/hadoop/blob/filesystem/HADOOP-15229-openfile/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md]

The ITest* Tests do this, [in this 
package|https://github.com/steveloughran/hadoop/tree/filesystem/HADOOP-15229-openfile/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select]

Test policy is covered in [S3a test 
policy|https://hadoop.apache.org/docs/r3.1.1/hadoop-aws/tools/hadoop-aws/testing.html].
{quote}
  Thanks for sharing the pointers. Will take a look.
{quote}Get set up for test runs first and then think about what more can go in.
{quote}
 Sure, will review current test suite and provide feedback. 
{quote}For s3 select it'd be good to have some other standard source files to 
test against; landsat.csv.gz only stresses the gzip and csv logic, something 
with timestamps which can be parsed and in bzip2 would be nice. These should be 
public & free: our developers don't have the time nor necessarily the funding 
to create/store these themselves.

One thing which isn't tested yet is handling of bad data: what if the CSV has 
inconsistent numbers of rows, mixes tabs and spaces, datatypes inconsistent, 
not all datetimes parseable etc. Someone should really write those, if just to 
have more error messages and stack traces for the docs.
{quote}
I agree, having authoritative test datasets to assert implementation 
correctness for projects would be great. Let me discuss this internally with 
the team and get back to you. Can't provide an ETA on this though at this point 
in time.

> 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