Hi Wes, others,
 
Thanks for your reply and the nice test case. Based on you experiment I decided 
to dig a little deeper
to further my understanding. I build the master branch, with the tools. 
I used parquet-scan to do some test. (my understanding is that this is an easy 
way
to look at reading performance, please correct me if I’m wrong)


> On May 9, 2020, at 5:49 PM, Wes McKinney <wesmck...@gmail.com> wrote:
> 
> hi Maarten,
> 
> I added dev@parquet.apache.org to this (if you are not subscribed to
> this list you may want to)
> 
> I made a quick notebook to help illustrate:
> 
> https://gist.github.com/wesm/cabf684db3ce8fdd6df27cf782f7226e
> 
> Summary:
> 
> * Files with 1000+ columns can see the metadata-to-data ratio exceed
> 10% (in the example I made it's 15-20%).
> * The time to deserialize whole files with many columns starts to
> balloon superlinearly with extremely wide files


The meta data (at the column level, as reported by parquet-dump-schema) is 
indeed significant for shorter columns.
Part of this might be the effect of the dictionary encoding? It seems to have 2 
pages per column (dictionary & data?)

Doing some experiments, I noticed 3 things,

1) using —columns=0 to just read one column with parquet-scan, reading the file 
meta-data scales linearly with
number of columns. This time is substantial (0.4s for 100k columns)

2) specifying sets of columns. The reading time per column is constant. On my 
system about 0.12s per
additional 5k columns. (this is with 5k rows)

3) I can actually read/scan 20k columns out of 100k in 0.95s. This is really 
good. 

Doing this from python these numbers are very different. 
The one column case goes to 3.5s and 0.22s per additional 5k columns.

Is it well understood that using python is that much slower? Are my 
observations correct?


> On Sat, May 9, 2020 at 4:28 PM Maarten Ballintijn <maart...@xs4all.nl> wrote:
>> 
>> Wes,
>> 
>> "Users would be well advised to not write columns with large numbers (> 
>> 1000) of columns"
>> You've mentioned this before and as this is in my experience not an uncommon 
>> use-case can you maybe expand a bit on the following related questions. 
>> (use-cases include daily or minute data for a few 10's of thousands items 
>> like stocks or other financial instruments, IoT sensors, etc).
>> 
>> Parquet Standard - Is the issue intrinsic to the Parquet standard you think? 
>> The ability to read a sub-set of the columns and/or row-groups, compact 
>> storage through the use of RLE, categoricals etc, all seem to point to the 
>> format being well suited for these use-cases
> 
> Parquet files by design are pretty heavy on metadata -- which is fine
> when the number of columns is small. When files have many columns, the
> costs associated with dealing with the file metadata really add up
> because the ratio of metadata to data in the file becomes skewed.
> Also, the common FileMetaData must be entirely parsed even when you
> only want to read one column.

I see about 20MB for the 100k columns file. So 200 bytes per column seems a 
lot, but this only becomes an issue when you have many columns (and 100k a 
factor two to there then our use-case). How much of this is Arrow overhead? 


>> Parquet-C++ implementation - Is the issue with current Parquet-C++ 
>> implementation, or any of the dependencies? Is it something which could be 
>> fixed? Would a specialized implementation help? Is the problem related to 
>> going from Parquet -> Arrow -> Python/Pandas? E.g. would a Parquet -> numpy 
>> reader work better?
> 
> No, it's not an issue specific to the C++ implementation.

Yes, the parquet-scan tests indicate that as well.


>> Alternatives - What would you recommend as a superior solution? Store this 
>> data tall i.s.o wide? Use another storage format?
> 
> It really depends on your particular use case. You can try other
> solutions (e.g. Arrow IPC / Feather files, or row-oriented data
> formats) and see what works best

Oh not, not row oriented data-formats :-)  We do like (need?) the ability to 
read a subset of the columns.
If anyone else has other suggestions then I’d definitely love to hear about it.

I’m going to redo some of these experiments with real data to see if it 
translates.

Cheers,
Maarten.


Reply via email to