lidavidm commented on a change in pull request #9656:
URL: https://github.com/apache/arrow/pull/9656#discussion_r606274857
##########
File path: cpp/src/arrow/ipc/reader.cc
##########
@@ -1022,7 +1111,24 @@ class RecordBatchFileReaderImpl : public
RecordBatchFileReader {
ReadStats stats() const override { return stats_; }
+ Result<AsyncGenerator<std::shared_ptr<RecordBatch>>> GetRecordBatchGenerator(
+ int readahead_messages, const io::IOContext& io_context,
+ arrow::internal::Executor* executor) override {
+ auto state =
std::dynamic_pointer_cast<RecordBatchFileReaderImpl>(shared_from_this());
+ auto message_generator = MakeMessageGenerator(state, io_context);
+ if (readahead_messages > 0) {
+ message_generator = MakeReadaheadGenerator(message_generator,
readahead_messages);
+ }
+ return IpcFileRecordBatchGenerator(
+ std::move(state), std::move(message_generator),
+ executor ? executor : arrow::internal::GetCpuThreadPool());
Review comment:
Avoiding the transfer helps the microbenchmarks:
<details>
```
-------------------------------------------------------------------------------------------------
Benchmark Time CPU
Iterations UserCounters...
-------------------------------------------------------------------------------------------------
ReadFile/1/real_time 8155 ns 8155 ns
83360 bytes_per_second=119.75G/s
ReadFile/4/real_time 10869 ns 10868 ns
64480 bytes_per_second=89.8522G/s
ReadFile/16/real_time 22341 ns 22341 ns
31440 bytes_per_second=43.7117G/s
ReadFile/64/real_time 67939 ns 67936 ns
10268 bytes_per_second=14.3741G/s
ReadFile/256/real_time 276022 ns 276015 ns
2566 bytes_per_second=3.53799G/s
ReadFile/1024/real_time 1071829 ns 1071812 ns
649 bytes_per_second=932.984M/s
ReadFile/4096/real_time 4325412 ns 4325390 ns
164 bytes_per_second=231.192M/s
ReadFile/8192/real_time 8346134 ns 8345807 ns
85 bytes_per_second=119.816M/s
ReadFileAsync/1/real_time 12351 ns 12351 ns
57407 bytes_per_second=79.0687G/s
ReadFileAsync/4/real_time 15216 ns 15216 ns
46298 bytes_per_second=64.1783G/s
ReadFileAsync/16/real_time 26723 ns 26723 ns
26312 bytes_per_second=36.5443G/s
ReadFileAsync/64/real_time 73425 ns 73424 ns
9481 bytes_per_second=13.3001G/s
ReadFileAsync/256/real_time 282676 ns 282672 ns
2469 bytes_per_second=3.45471G/s
ReadFileAsync/1024/real_time 1092103 ns 1092063 ns
641 bytes_per_second=915.665M/s
ReadFileAsync/4096/real_time 4361873 ns 4361774 ns
162 bytes_per_second=229.259M/s
ReadFileAsync/8192/real_time 8240870 ns 8240636 ns
85 bytes_per_second=121.346M/s
ReadTempFile/1/real_time 74378 ns 74338 ns
9365 bytes_per_second=210.075G/s
ReadTempFile/4/real_time 76803 ns 76752 ns
9024 bytes_per_second=203.444G/s
ReadTempFile/16/real_time 87857 ns 87825 ns
7955 bytes_per_second=177.846G/s
ReadTempFile/64/real_time 136872 ns 136849 ns
5178 bytes_per_second=114.158G/s
ReadTempFile/256/real_time 342410 ns 342399 ns
2062 bytes_per_second=45.6325G/s
ReadTempFile/1024/real_time 1162314 ns 1162314 ns
602 bytes_per_second=13.443G/s
ReadTempFile/4096/real_time 4419789 ns 4419708 ns
158 bytes_per_second=3.53524G/s
ReadTempFile/8192/real_time 8430516 ns 8430390 ns
83 bytes_per_second=1.85339G/s
ReadTempFileAsync/1/real_time 87235 ns 14569 ns
7997 bytes_per_second=179.113G/s
ReadTempFileAsync/4/real_time 90525 ns 15981 ns
7699 bytes_per_second=172.604G/s
ReadTempFileAsync/16/real_time 103748 ns 21630 ns
6684 bytes_per_second=150.606G/s
ReadTempFileAsync/64/real_time 156567 ns 41886 ns
4311 bytes_per_second=99.7974G/s
ReadTempFileAsync/256/real_time 410901 ns 142398 ns
1792 bytes_per_second=38.0261G/s
ReadTempFileAsync/1024/real_time 1298398 ns 495178 ns
524 bytes_per_second=12.0341G/s
ReadTempFileAsync/4096/real_time 5173738 ns 2166030 ns
100 bytes_per_second=3.02006G/s
ReadTempFileAsync/8192/real_time 9034225 ns 3925295 ns
77 bytes_per_second=1.72953G/s
ReadMmapFile/1/real_time 20204 ns 20156 ns
34579 bytes_per_second=773.351G/s
ReadMmapFile/4/real_time 23204 ns 23159 ns
30345 bytes_per_second=673.382G/s
ReadMmapFile/16/real_time 35476 ns 35436 ns
19728 bytes_per_second=440.444G/s
ReadMmapFile/64/real_time 83301 ns 83278 ns
8457 bytes_per_second=187.572G/s
ReadMmapFile/256/real_time 293505 ns 293499 ns
2380 bytes_per_second=53.2359G/s
ReadMmapFile/1024/real_time 1109479 ns 1109445 ns
631 bytes_per_second=14.0832G/s
ReadMmapFile/4096/real_time 4356631 ns 4356514 ns
161 bytes_per_second=3.58649G/s
ReadMmapFile/8192/real_time 8409587 ns 8409269 ns
84 bytes_per_second=1.858G/s
ReadMmapFileAsync/1/real_time 27636 ns 27584 ns
25754 bytes_per_second=565.384G/s
ReadMmapFileAsync/4/real_time 30530 ns 30488 ns
22957 bytes_per_second=511.796G/s
ReadMmapFileAsync/16/real_time 42464 ns 42427 ns
16453 bytes_per_second=367.958G/s
ReadMmapFileAsync/64/real_time 90109 ns 90087 ns
7737 bytes_per_second=173.401G/s
ReadMmapFileAsync/256/real_time 302359 ns 302334 ns
2298 bytes_per_second=51.6771G/s
ReadMmapFileAsync/1024/real_time 1144287 ns 1143989 ns
622 bytes_per_second=13.6548G/s
ReadMmapFileAsync/4096/real_time 4388511 ns 4388344 ns
159 bytes_per_second=3.56043G/s
ReadMmapFileAsync/8192/real_time 8481302 ns 8481271 ns
83 bytes_per_second=1.84229G/s
ReadCompressedFile/1/real_time 30774624 ns 1439507 ns
23 bytes_per_second=519.909M/s
ReadCompressedFile/4/real_time 9331553 ns 592019 ns
66 bytes_per_second=1.67443G/s
ReadCompressedFile/16/real_time 5767370 ns 1871589 ns
122 bytes_per_second=2.70921G/s
ReadCompressedFile/64/real_time 8557480 ns 6710354 ns
84 bytes_per_second=1.82589G/s
ReadCompressedFile/256/real_time 22625476 ns 21524626 ns
31 bytes_per_second=707.167M/s
ReadCompressedFile/1024/real_time 84407402 ns 81015774 ns
8 bytes_per_second=189.557M/s
ReadCompressedFile/4096/real_time 360272288 ns 341619192 ns
2 bytes_per_second=44.4109M/s
ReadCompressedFile/8192/real_time 675102220 ns 649042983 ns
1 bytes_per_second=23.7001M/s
ReadCompressedFileAsync/1/real_time 62111198 ns 2438692 ns
10 bytes_per_second=257.603M/s
ReadCompressedFileAsync/4/real_time 10011855 ns 898512 ns
76 bytes_per_second=1.56065G/s
ReadCompressedFileAsync/16/real_time 6296336 ns 1995273 ns
109 bytes_per_second=2.4816G/s
ReadCompressedFileAsync/64/real_time 8822786 ns 6852773 ns
76 bytes_per_second=1.77098G/s
ReadCompressedFileAsync/256/real_time 23069353 ns 21996264 ns
29 bytes_per_second=693.561M/s
ReadCompressedFileAsync/1024/real_time 84233734 ns 80703416 ns
8 bytes_per_second=189.948M/s
ReadCompressedFileAsync/4096/real_time 329932800 ns 317996694 ns
2 bytes_per_second=48.4947M/s
ReadCompressedFileAsync/8192/real_time 648313919 ns 625274147 ns
1 bytes_per_second=24.6794M/s
```
</details>
It also improves S3 quite a bit, but doesn't really affect local files.
(This is combined with reducing # of I/O calls, so probably that's what's
actually improving the S3 case.)
<details>


</details>
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]