hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Steve Loughran (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-15229) Add FileSystem builder-based openFile() API to match createFile()
Date Thu, 03 Jan 2019 18:08:00 GMT

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

Steve Loughran commented on HADOOP-15229:
-----------------------------------------

h2. Aaron's comments

thanks for these: will address all the little changes.

bq. Exceptions in the openFile vs Future.get. 

I'm leaving the choice open; some will fail fast (link resolution failure in FileContext,
bad must() option.). Raising path handle support as unsupported
falls out of how the base implementation works...subclasses may want to fail fast.
It's a tradeoff, amplified by how IOExceptions can't be directly thrown in a future or followon
l-exp. Delaying the IO in the base class gets client code used to the fact that even FNFEs
don't surface until later (so helps line people up for slower object store IO). But you end
up in games with remapping exceptions -games you were going to have to end up working with
anyway.

I did consider having the future actually return the IOE as part of the signature, e.g. a
result of type

{code}
CompletableFuture<Pair<FSDataInputStream, IOException>> 
{code}

or some special tuple of IO, 
{code}
final class Result<T> {
  final Optional<T> result;
  final IOException exception;
  private Result(T t, IOException e) { result = Optional.of(t); exception = e;}
  // get or throw
  public T get() throws IOException { if (exception!=null) throw exception; return result.get();}
  public Optional<T> result() { return result;}
  public IOException exception() { return exception;}
  public static<T> Result<T> of(T t) { return new Result(t, null);}
  public static<T> Result<T> thrown(IOException e) { return new Result(Optional.empty(),
e);}
  ...etc  
}
{code}

This would be guaranteed to either have a result, or an exception. But I couldn't see how
easily this would chain together across futures; you'd still be catching and setting everywhere:
I need more practise working with the completable future stuff before thinking about what
would make a good model here. I don't want to do some grand project which is unusable. Plus
there are some other libraries which may do it better, and I haven't done that research. After
all, if I did try to do this "properly", I would want it standalone anyway.

bq. 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?

you can't do a seek to an offset in the file, because the results are coming in dynamically
from a POST; there's no GET for a content length. Which brings it down to: do you skip() or
read-and-discard. The trouble with skip is I'm not sure about all its failure modes here.
{{skip(count)}} can return a value < {{count}} and you are left wondering what to do? Keep
retrying until total == count? Now I know of a way to check for end of stream/errors in the
select, that may be possible.

Seek() will occur in real life, as TextFileFormat will always split if there's no codec. If
you bind to the no-op codec (which you need for gz & bz2 files anyway), then the split
doesn't happen. Really though you'd need a way to turn text file splits off entirely even
if there is no codec. It's wasteful to split an s3 select dataset.

Note: latest patch brings positioned read() up in sync with seek, so that you can do a forward
read(position), just not a backwards one —and it updates the getPos() value
after. This will support any code which does positionedRead() with skips, as long as it never
goes backwards or expects the seek position to be unchanged after.

BTW, I don't see any tests in {{AbstractContractSeekTest}} which do read fully backwards after
a previous seek/read or a previous positioned read. Will add that
as the contract test and then copy into a select test where we do expect it to fail with a
path IOE. Because right now, those contract tests won't find the failure condition I'm creating
here. 


h3. failures in {{ITestS3ATemporaryCredentials}}

HADOOP-14556 does a lot there with setting up STS; don't worry about it here. Do check then
though. I think the builder 


h2. Sameer Choudhary's comments

good point about malformed CSV, will need to create a test there.

Latest iteration will see a read() raise EOFException and then only map to -1 if the end stream
event was returned.
{code}
    try {
      byteRead = once("read()", uri, () -> wrappedStream.read());
    } catch (EOFException e) {
      // this could be one of: end of file, some IO failure
      if (completedSuccessfully.get()) {
        // read was successful
        return -1;
      } else {
        // the stream closed prematurely
        LOG.info("Reading of S3 Select data from {} failed before all results "
            + " were generated.", uri);
        streamStatistics.readException();
        throw new PathIOException(uri,
            "Read of S3 Select data did not complete");
      }
    }
{code}

Tests seem happy; added some to verify a 0 byte file is handled too. 

TODO: test for CSV file with inconsistent #of columns

I worry that some test may only show up if the file is above a certain size, as it will come
in on a later page of responses, won't it?


h2. Yuzhou Sun's comments. 

Well spotted about the LineReader. Actually, we could have the codec return MAX_INT and say
"you must use this codec"; you should be using it anyway to turn off automatic gzip decompression;
use it for text to turn off splitting and have percentage underestimate how much is left -at
least it will always be < 100 %

h2. Other changes

Moved the {{newSelectRequest()}} method into {{WriteOperationsHelper}} and SelectBinding is
passed that in on startup -not the owning FS. That's a hint of what we'd need to do for a
proper M/V split: WriteOperationsHelper represents a fair amount of the underlying model,
or at least lower-level store ops.

With that, all the work to build up and execute the request is moved into the SelectBinding;
the base FS is handing off to it.



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


Mime
View raw message