If dictionary encoded data is specifically a concern, we've added new experimental APIs that should be in the next release that allows for retrieving dictionary data as indexes + dictionaries (ReadBatchWithDictionary) instead of denormalizing them as ReadBatch does.
-Micah On Wed, Jul 21, 2021 at 8:02 AM Adam Hooper <a...@adamhooper.com> wrote: > Hi Micah, > > Thank you for this wonderful description. You've solved my problem exactly. > > Responses inline: > > > "ReadBatchSpaced() in a loop isfaster than reading an entire record >> > batch." >> >> Could you elaborate on this? What code path were you using for reading >> record batches that was slower? > > > I'll elaborate based on my (iffy) memory: > > The slow path, as I recall, is converting from dictionary-encoded string > to string. This decoding is fast in batch, slow otherwise. > > With Arrow 0.15/0.16, in prototype phase, I converted Parquet to Arrow > column chunks before I even began streaming. (I cast DictionaryArray to > StringArray in this step.) Speed was decent, RAM usage wasn't. > > When I upgraded to Arrow 1.0, I tried *not* casting DictionaryArray to > StringArray. RAM usage improved; but testing with a dictionary-heavy file, > I saw a 5x slowdown. > > Then I discovered ReadBatchSpaced(). I love it (and ReadBatch()) because > it skips Arrow entirely. In my benchmarks, batch-reading just 30 values at > a time made my whole program 2x faster than the Arrow 0.16 version, on a > typical 70MB Parquet file. I could trade RAM vs speed by increasing batch > size; speed was optimal at size 1,000. > > Today I don't have time to benchmark any more approaches -- or even > benchmark that the sentences I wrote above are 100% correct. > > Did you try adjusting the batch size with >> ArrowReaderProperties [1] to be ~1000 rows also (by default it is 64 K so >> I >> would imagine a higher memory overhead). There could also be some other >> places where memory efficiency could be improved. >> > > I didn't test this. I'm not keen to benchmark Parquet => Arrow => CSV > because I'm already directly converting Parquet => CSV. I imagine there's > no win for me to find here. > > There are several potential options for the CSV use-case: >> 1. The stream-reader API ( >> >> https://github.com/apache/arrow/blob/8e43f23dcc6a9e630516228f110c48b64d13cec6/cpp/src/parquet/stream_reader.h >> ) >> > > This looks like a beautiful API. I won't try it because I expect > dictionary decoding to be slow. > > >> 2. Using ReadBatch. The logic of determining nulls for non-nested data >> is >> trivial. You simply need to compare definition levels returned to the max >> definition level ( >> >> https://github.com/apache/arrow/blob/d0de88d8384c7593fac1b1e82b276d4a0d364767/cpp/src/parquet/schema.h#L368 >> ). >> Any definition level less than the max indicates a null. This also has >> the >> nice side effect of requiring less memory for when data is null. >> > > This is perfect for me. Thank you -- I will use this approach. > > >> 3. Using a record batch reader ( >> >> https://github.com/apache/arrow/blob/master/cpp/src/parquet/arrow/reader.h#L179 >> ) >> and the Arrow to CSV writer ( >> https://github.com/apache/arrow/blob/master/cpp/src/arrow/csv/writer.h). >> The CSV writer code doesn't support all types yet, they require having a >> cast to string kernel available. If extreme memory efficiency is your >> aim, this is probably not the best option. Speed wise it is probably >> going >> to be pretty competitive and will likely see the most improvements for >> "free" in the long run. > > > Ooh, lovely. Yes, I imagine this can be fastest; but it's not ideal for > streaming because it's high-RAM and high time-to-first-byte. > > Thank you again for your advice. You've been more than helpful. > > Enjoy life, > Adam > > -- > Adam Hooper > +1-514-882-9694 > http://adamhooper.com >