alamb commented on issue #8000:
URL: https://github.com/apache/arrow-rs/issues/8000#issuecomment-3160161293

   > I'm watching this. In java, the big speedups we've got in reading data 
from cloud storage come from parallel GETs of rowgroups, either explicitly or 
based on inference of read patterns and knowledge of file structure. We've seen 
speedups of about 30% in read-heavy TPC queries -the more data read, the better 
the improvement.
   
   Thank you for the information @steveloughran 
   
   > These parallel reads are critical to compensating for the latency of cloud 
storage reads.
   > 
   > This means what is being designed has to be able to support this. Ideally 
the different rowgroups should be specified up front and whichever range comes 
in first is pushed to the processing queue, even if is not the first in the 
list. We don't have that in the parquet-java stack, which still has to wait for 
all the ranges.
   
   This is roughly how DataFusion parallelizes reads to a single file -- it 
divides the row groups of single large files across multiple cores and then 
each subset of row groups is parallelized 
   
   Files themselves are prefetched, but the row groups are not, largely because 
that information is not known to anything but the reader
   
   > What I'd recommend then is
   > 
   > 1. determine ranges to retrieve, with as much filtering as is possible; 
ignoring LIMITs on result sizes
   > 2. Request initial set of rowgroup
   > 3. Out of Order processing of results
   > 4. keep scheduling more ranges, unless processing unit cancels read 
process due to satisifed results (LIMIT/SAMPLE etc) or some error.
   
   This makes a lot of sense to me for the overall processing of files from 
object store and is very aligned with where I am trying to go. I think the 
details of scheduling and prefetching, however, likely varies substantially 
from system to system (e.g. when reading from Object Storage vs local SSDs)
   
   Thus, what I am hoping to do in the arrow-rs parquet reader is to get the 
code so that many different IO patterns can be supported. We can provide 
examples how prefetching might work, but prefetching itself would NOT be part 
of the parquet-rs reader given the patterns vary so much. 
   
   I am actually pretty happy with what I have now in 
https://github.com/apache/arrow-rs/pull/7997. A sneak peek:
   
   ```rust
    let mut decoder = get_decoder();
    loop {
       match decoder.try_decode().unwrap() {
          DecodeResult::NeedsData(ranges) => {
            // The decoder needs more data. Fetch the data for the given ranges
            // call decoder.push_ranges(ranges, data) and call again
            push_data(&mut decoder, ranges);
          }
          DecodeResult::Data(batch) => {
            // Successfully decoded the next batch of data
            println!("Got batch with {} rows", batch.num_rows());
          }
          DecodeResult::Finished => {
            // The decoder has finished decoding all data
            break;
          }
       }
    }
   ```
   
   Once we have that pattern in place, I imagine prefetching to look something 
like
   
   ```rust
    let mut decoder = get_decoder();
    loop {
       // PREFETCH: new API in the decoder to ask what ranges would be needed 
next
       let mut next_ranges_needed = decoder.peek_ranges() ;
       while have_capacity_to_prefetch() { 
          schedule_prefetch(next_ranges_needed.next())
        }
     
       // proceed to decoding as normal
       match decoder.try_decode().unwrap() {
          DecodeResult::NeedsData(ranges) => {
            push_data(&mut decoder, ranges);
          }
          DecodeResult::Data(batch) => { 
            // .. process data ... 
          }
          DecodeResult::Finished => {
            break;
          }
       }
    }
   ```
   
   > oh, and collect lots of metrics. data read, data discarded, pipeline 
stalls waiting for new data in particular.
   
   Yes, this is important. Interestingly, @mapleFU brought up the same thing 
yesterday (see 
https://github.com/apache/arrow-rs/issues/7983#issuecomment-3155563684)
   


-- 
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: github-unsubscr...@arrow.apache.org

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

Reply via email to