Hi Patrick,

Thanks for reporting the issue!

Let me try to answer your question in short.

1. In your case, the good data is dictionary-encoded [1] and the size of
the dictionary is 1.
2. The RLE encoding [2] you have observed from the good data applies to the
only indices after dictionary encoding, not the original data.
3. If dictionary encoding is in effect, the first page must be a dictionary
page followed by a set of data pages that are only indices of the
dictionary.
4. Dictionary does not have any null values.
5. By default, the parquet-mr implementation has to decide the encoding of
a page when it reaches 20000 records.
6. In the bad data, the first 20000 records are all null values, so the
dictionary is empty and uses the fallback encoding (which is PLAIN encoding
in parquet format v1).

If at least one record in the beginning 20000 rows is not null, then the
encoded size will be much better.

[1]
https://github.com/apache/parquet-format/blob/master/Encodings.md#dictionary-encoding-plain_dictionary--2-and-rle_dictionary--8
[2]
https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3

Best,
Gang

On Tue, Feb 28, 2023 at 11:29 PM Patrick Hansert <
hans...@informatik.uni-kl.de> wrote:

> Hello everyone!
>
> First of all, I hope I'm in the right place. The contribution guidelines
> directed me here after I discovered the registration in the Jira tracker is
> closed. I'm a Ph.D. student at RPTU Kaiserslautern-Landau, and my current
> research revolves around sorting-based improvements to RLE compression for
> Dremel-encoded data. In the process, I've discovered the following issue in
> Parquet:
>
> Parquet-MR seems to choose the encoding based on the values on the first
> page. If all values on that first page are null, it falls back to plain
> encoding. This is problematic if the dataset is (initially) very sparse or
> has been sorted. Note that sort operations in Spark sort nulls first by
> default, so it is easy to trigger this issue without noticing it. Even
> then, correlations in the dataset may prevent a user from finding a sort
> order which does not trigger the bug. Also note that it can be triggered by
> sorting by a correlated attribute, e.g., an event log stored in
> chronological order with an attribute that has only been added after some
> time. Thus, this issue has an impact beyond academia.
>
> To reproduce the issue, create two data frames, one of which starts with
> more nulls than fit on a page, and write them to disk as Parquet
> val good = Iterator.fill(25000)(5) ++ Iterator.fill(25000)(null) // nulls
> last
> val bad = Iterator.fill(25000)(null) ++ Iterator.fill(25000)(5) // nulls
> first
> // do spark stuff for creating DFs here, I have attached the complete
> Scala code
> goodDf.write.parquet("good")
> badDf.write.parquet("bad")
>
> As can be seen, both Dataframes contain a long run of "5" and a long run
> of nulls. We would expect both to compress very well, but if we check the
> resulting files, we find that the "bad" file is an order of magnitude
> larger. If we inspect them with parquet-cli pages, the "good" file uses
> dictionary and RLE compression as expected:
>
> Column: foo
>
> --------------------------------------------------------------------------------
>   page   type  enc   count   avg size   size       rows     nulls   min /
> max
>   0-D    dict  S _   1       4.00 B     4 B
>   0-1    data  S R   20000   0.00 B     12 B
>   0-2    data  S R   20000   0.00 B     14 B
>   0-3    data  S R   10000   0.00 B     9 B
>
> The "bad" file, on the other hand, does not have a dictionary and is not
> RLE compressed:
>
> Column: foo
>
> --------------------------------------------------------------------------------
>   page   type  enc  count   avg size   size       rows     nulls   min /
> max
>   0-0    data  S _  20000   0.00 B     8 B
>   0-1    data  S _  20000   3.00 B     58.604 kB
>   0-2    data  S _  10000   4.00 B     39.070 kB
>
> I am using Spark 3.2.0, which comes with Parquet 1.12.1. Unfortunately, I
> am not familiar enough with the Parquet codebase to submit a fix myself,
> but I'm happy to help if further questions arise.
>
> Kind Regards
>
> Patrick
>

Reply via email to