You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/08/27 15:57:47 UTC

[GitHub] [hadoop] steveloughran commented on pull request #2975: HADOOP-17682. ABFS: Support FileStatus input to OpenFileWithOptions() via OpenFileParameters

steveloughran commented on pull request #2975:
URL: https://github.com/apache/hadoop/pull/2975#issuecomment-907304674


   
   > eTag is a parameter that is sent with every read request to server today. Server ensures that the file is indeed of the expected eTag before proceeding with the read operation. If the file was recreated during the timespan inputStream was created and read is issued, eTag check will fail and so will the read, else read will end up wrongly getting data from the new file. This is the reason we are mandating eTag to be present and in else case fall back to triggering a GetFileStatus. We will changing the log type to debug as per your comment in the else case.
   
   I do know about etags. The S3A connector picks it up on the HEAD or first GET and caches it, to at least ensure that things are unchanged for the duration of the stream. 
   
   
   > Noticed an example where S3 eTag is passed as an optional key (referring to link , for the Hive case where FileStatus is wrapped, is this how eTag gets passed ? And was also wondering how we can get various Hadoop workloads to send it without being aware of which is the current active FileSystem considering the key name is specific to it.
   
   not in use at all by hive, even in PoCs. I'm not worrying about changes between the list and the open, just about consistency once a file is opened. If a file was changed between hive query planning and work execution then as it is invariably of a different length the workers will notice soon enough. Anyway, HDFS doesn't worry about this so why should the other stores?
   
   When you skip the HEAD, the first GET will return the etag. All you have to do is skip setting the If-Match header on the first request, pick the etag from the response and use it after.
   
   
   > I did have a look at #2584 which provides the means to pass mandatory and optional keys through OpenFileParameters. 
   
   This was *already* in the openFile() API call from the outset. Indeed, abfs already supports one, `fs.azure.buffered.pread.disable`.
   
   What the new PR adds is
   * standard options for read policies, split start/end, all of which can be used to optimize reads.
   * tuning of the `withFileStatus()` based on initial use in format libraries, attempts to use in hive, etc.
   * tests for this stuff
   
   No API change, other than `withFileStatus(null)` required to be a no-op, and requirements in the API spec relaxed to say "only check path.getName()" for a match.
   
   
   > Also wanted to check if it would be right to add eTag as a field into the base FileStatus class. Probably as a sub structure within FileStatus so that it can be expanded in future to hold properties that Cloud storage systems commonly depend o
   
   way too late for that. used everywhere, gets serialized etc. What you could do is add some interface `EtagSource` with a `getEtag()` method, and implement that interface for the ABFS and s3a connectors. It's still going to be lost when things like hive replace the custom FileStatus with the standard one.
   
   Where this could be useful is in distcp applications where the etag of an uploaded file could be cached and compared when rescanning a directory. IF a store implements the getFileChecksum() method you can serve the etag that way but (a) distcp can't handle it and (b) if you could get it on every file from a list() call, you save a lot of IO.
   
   So please, have someone implement this in hadoop-common, with specification, tests etc.
   
   
   > OpenFile() change is one of the key performance improvement that we are looking at adopting to. The aspect of FileStatus being passed down in itself reduces HEAD request count by half and look forward to adopting to the new set of read policies too. We will work on understanding how to map various read policies to the current optimizations that the driver has for different read patterns. I think that would translate to a change equivalent to PrepareToOpenFile for ABFS driver.
   
   I've just tried to let apps declare how they want to read things; there's generally a straightforward map to sequential vs random vs adaptive...but I hope that the orc and parquet options would provide even more opportunity to tweak behaviour, e.g .by knowing that there will be an initial footer read sequence, then stripes will be read.
   
   > Would it be ok if we make this change once #2584 checks in ?
   
   yes, but I do want to at least have the azure impl not reimplement the same brittleness around `withFileStatus()` as s3a before it goes into 3.3. That way, consistency.
   
   > Currently we are in a bit of tight schedule and short staffed as we aim to complete on the feature work tracked in HADOOP-17853 and another customer requirement that we are in feasibility analysis stage.
   
   I really want to get a hadoop 3.3.2 out the door with this API in it. It's going to happen before '17853 as that is a big piece of work, and which is going to need lots of review time by others, myself included.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org