[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1551181414 > FWIW the hadoop 3.3.5 vector io changes might make this PR redundant. on those stores which do it well (s3a, native filesystem); until gcs and abfs add it they'll benefit from it, as will others. FWIW i'll be at berlin buzzwords in june and 1 of my 2 talks will be on this...if anyone from the parquet dev team is around, it'd be great to have you in the session. -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1505269164 @hazelnutsgz hadoop 3.3.5 supports vector IO on an s3 stream; async parallel fetch of blocks, which also works on local fs (and with gcs, abfs TODO items). we see significant performance increases there. There's a PR for it, though as it is 3.3.5+ only, not merged in to asf parquet branches unless the move or we finish a shim library to offer (serialized) support for the api on older releases. -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1153924743 (i could of course add those probes into the shim class, so at least that access of internals was in one place) -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1153923501 bq. perhaps check if the ByteBufferReadable interface is implemented in the stream? The requirement for the `hasCapability("in:readbytebuffer")` to return true postdates the API; there's no way to be confident that if the probe returns false (or hasPathCapability() isn't available) that the method *isn't actually there* see #951 for a design which will trust a `true` response, falling back to looking at the wrapped stream. Note that as it calls getWrapped() it is calling methods tagged LimitedPrivate. it should really do that...at the very least hadoop needs a PR saying "we need to do this because..." and that tag can be changed -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1151417126 I've started work on a fs-api-shim library, with the goal of "apps compile against hadoop 3.2.0 can get access to the 3.3 and 3.4 APIs when available either with transparent fallback (openFile()) or ability to probe the API before trying to invoke https://github.com/steveloughran/fs-api-shim openfile takes the seek & status params, falls back to open() : https://github.com/steveloughran/fs-api-shim/blob/main/fs-api-shim-library/src/main/java/org/apache/hadoop/fs/shim/FileSystemShim.java#L87 ByteBufferReadable will raise UnsupportedException if not found, there is a check for it https://github.com/steveloughran/fs-api-shim/blob/main/fs-api-shim-library/src/main/java/org/apache/hadoop/fs/shim/FSDataInputStreamShim.java Vector IO SHALL be available the same way Adopt 3.2.0 then and we will help give the library the ability to use the newer api calls, even stuff not yet shipped in apache releases. (I want to release this as an asf artifact with oversight by hadoop project. lets us maintain it) -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1136465506 > At this point the bottlenecks in parquet begin to move towards decompression and decoding but IO remains the slowest link in the chain. Latency is the killer; in an HTTP request you want read enough but not discard data or break an http connection if the client suddenly does a seek() or readFully() somewhere else. file listings, existence checks etc. > One thing we get with my PR is that the ParquetFileReader had assumptions built in that all data must be read before downstream can proceed. Some of my changes are related to removing these assumptions and ensuring that downstream processing does not block until an entire column is read so we get efficient pipelining. That'd be great. now, if you could also handle requesting different columns in parallel and processing them out of order. > What does the 128 MB block mean? Is this the amount prefetched for a stream? The read API does not block until the entire block is filled, I presume. this was the abfs client set to do four GET requests of 128MB each. this would be awful for columns stores where smaller ranges are often requested/processed before another seek is made, but quite often parquet does do more back to back reads than just one read/readFully request > With my PR, parquet IO is reading 8MB at a time (default) and downstream is processing 1MB at a time (default) and several such streams (one per column) are in progress at the same time. Hopefully, this read pattern would work with the prefetch. be good to think about vectored IO. and yes, updating parquet dependencies would be good, hadoop 3.3.0 should be the baseline. just sketched out my thoughts on this. I've played with some of this in my own branch. I think the next step would be for me to look at the benchmark code to make it targetable elsewhere. https://docs.google.com/document/d/1y9oOSYbI6fFt547zcQJ0BD8VgvJWdyHBveaiCHzk79k/ -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1135585289 > I was working with s3a > Spark 3.2.1 > Hadoop (Hadoop-aws) 3.3.2 > AWS SDK 1.11.655 thanks., that means you are current with all shipping improvments. the main one extra is to use openFile(), passing in length and requesting randomio. this guarantees ranged GET requests and cuts the initial HEAD probe for existence/size of file. >> have you benchmarked this change with abfs or google gcs connectors to see what difference it makes there? > No I have not. Would love help from anyone in the community with access to these. I only have access to S3. that I have. FWIW, with the right tuning of abfs prefetch (4 threads, 128 MB blocks) i can get full FTTH link rate from a remote store; 700 mbit/s . that's to the base station. once you add wifi the bottlenecks move. -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [parquet-mr] steveloughran commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
steveloughran commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1134843705 1. whose s3 client was used for testing here -if the s3a one, which hadoop release? 2. the azure abfs and gcs connectors do async prefetching of the next block, but are simply assuming that code will read sequentially; if there is another seek/readFully to a new location, those prefetches will be abandoned. there is work in s3a to do prefetching here with caching, so as to reduce the penalty of backwards seeks. https://issues.apache.org/jira/browse/HADOOP-18028 hadoop is adding a vectored IO api intended for libraries like orc and parquet to be able to use, where the application provides an unordered list of ranges, a bytebuffer supplier and gets back a list of futures to wait for. the base implementation simply reads using readFully APi. s3a (and later abfs) will do full async retrieval itself, using the http connection pool. https://issues.apache.org/jira/browse/HADOOP-18103 both vectored io and s3a prefetching will ship this summer in hadoop 3.4.0. i don't see this change conflicting with this, though they may obsolete a lot of it. have you benchmarked this change with abfs or google gcs connectors to see what difference it makes there? -- 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: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org