Skip to content

Commit

Permalink
Change default Parquet writer settings to match arrow-rs (except for …
Browse files Browse the repository at this point in the history
…compression & statistics) (#11558)

* feat(11367): define the default session's writer props, both with and without the parquet feature enabled

* feat(11367): map the defaults that already align between datafusion and arrow-rs

* feat(11367): move datafusion to use extern parquet defaults for most writer settings

* chore(11367): denote what are the remaining misaligned defaults, for further discussion

* chore: fix doc links

* chore: update the readme demonstrating the default config setting

* refactor(11367): remove layer of indirection using the conditional parquet feature, instead rely upon regression testing
  • Loading branch information
wiedld authored Jul 23, 2024
1 parent 65dd364 commit 9f74dcc
Show file tree
Hide file tree
Showing 4 changed files with 23 additions and 92 deletions.
15 changes: 9 additions & 6 deletions datafusion/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -374,18 +374,21 @@ config_namespace! {

/// (writing) Sets parquet writer version
/// valid values are "1.0" and "2.0"
pub writer_version: String, default = "1.0".into()
pub writer_version: String, default = "1.0".to_string()

/// (writing) Sets default parquet compression codec.
/// Valid values are: uncompressed, snappy, gzip(level),
/// lzo, brotli(level), lz4, zstd(level), and lz4_raw.
/// These values are not case sensitive. If NULL, uses
/// default parquet writer setting
///
/// Note that this default setting is not the same as
/// the default parquet writer setting.
pub compression: Option<String>, default = Some("zstd(3)".into())

/// (writing) Sets if dictionary encoding is enabled. If NULL, uses
/// default parquet writer setting
pub dictionary_enabled: Option<bool>, default = None
pub dictionary_enabled: Option<bool>, default = Some(true)

/// (writing) Sets best effort maximum dictionary page size, in bytes
pub dictionary_page_size_limit: usize, default = 1024 * 1024
Expand All @@ -398,21 +401,21 @@ config_namespace! {

/// (writing) Sets max statistics size for any column. If NULL, uses
/// default parquet writer setting
pub max_statistics_size: Option<usize>, default = None
pub max_statistics_size: Option<usize>, default = Some(4096)

/// (writing) Target maximum number of rows in each row group (defaults to 1M
/// rows). Writing larger row groups requires more memory to write, but
/// can get better compression and be faster to read.
pub max_row_group_size: usize, default = 1024 * 1024
pub max_row_group_size: usize, default = 1024 * 1024

/// (writing) Sets "created by" property
pub created_by: String, default = concat!("datafusion version ", env!("CARGO_PKG_VERSION")).into()

/// (writing) Sets column index truncate length
pub column_index_truncate_length: Option<usize>, default = None
pub column_index_truncate_length: Option<usize>, default = Some(64)

/// (writing) Sets best effort maximum number of rows in data page
pub data_page_row_count_limit: usize, default = usize::MAX
pub data_page_row_count_limit: usize, default = 20_000

/// (writing) Sets default encoding for any column.
/// Valid values are: plain, plain_dictionary, rle,
Expand Down
72 changes: 0 additions & 72 deletions datafusion/common/src/file_options/parquet_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -644,56 +644,6 @@ mod tests {
"datafusion's default is zstd"
);

// TODO: data_page_row_count_limit defaults do not match
// refer to https://github.com/apache/datafusion/issues/11367
assert_eq!(
default_writer_props.data_page_row_count_limit(),
20_000,
"extern parquet's default data_page_row_count_limit is 20_000"
);
assert_eq!(
from_datafusion_defaults.data_page_row_count_limit(),
usize::MAX,
"datafusion's default is usize::MAX"
);

// TODO: column_index_truncate_length do not match
// refer to https://github.com/apache/datafusion/issues/11367
assert_eq!(
default_writer_props.column_index_truncate_length(),
Some(64),
"extern parquet's default is 64"
);
assert_eq!(
from_datafusion_defaults.column_index_truncate_length(),
None,
"datafusion's default is None"
);

// The next few examples are where datafusion's default is None.
// But once datafusion's TableParquetOptions are converted to a WriterProperties,
// then we get the extern parquet's defaults.
//
// In other words, we do not get indeterminate behavior in the output writer props.
// But this is only because we use the extern parquet's defaults when we leave
// the datafusion setting as None.

// datafusion's `None` for Option<bool> => becomes parquet's true
// TODO: should this be changed?
// refer to https://github.com/apache/datafusion/issues/11367
assert!(
default_writer_props.dictionary_enabled(&"default".into()),
"extern parquet's default is true"
);
assert_eq!(
default_table_writer_opts.global.dictionary_enabled, None,
"datafusion's has no default"
);
assert!(
from_datafusion_defaults.dictionary_enabled(&"default".into()),
"should see the extern parquet's default over-riding datafusion's None",
);

// datafusion's `None` for Option<String> => becomes parquet's EnabledStatistics::Page
// TODO: should this be changed?
// refer to https://github.com/apache/datafusion/issues/11367
Expand All @@ -712,35 +662,13 @@ mod tests {
"should see the extern parquet's default over-riding datafusion's None",
);

// datafusion's `None` for Option<usize> => becomes parquet's 4096
// TODO: should this be changed?
// refer to https://github.com/apache/datafusion/issues/11367
assert_eq!(
default_writer_props.max_statistics_size(&"default".into()),
4096,
"extern parquet's default is 4096"
);
assert_eq!(
default_table_writer_opts.global.max_statistics_size, None,
"datafusion's has no default"
);
assert_eq!(
default_writer_props.max_statistics_size(&"default".into()),
4096,
"should see the extern parquet's default over-riding datafusion's None",
);

// Confirm all other settings are equal.
// First resolve the known discrepancies, (set as the same).
// TODO: once we fix the above mis-matches, we should be able to remove this.
let mut from_extern_parquet =
session_config_from_writer_props(&default_writer_props);
from_extern_parquet.global.compression = Some("zstd(3)".into());
from_extern_parquet.global.data_page_row_count_limit = usize::MAX;
from_extern_parquet.global.column_index_truncate_length = None;
from_extern_parquet.global.dictionary_enabled = None;
from_extern_parquet.global.statistics_enabled = None;
from_extern_parquet.global.max_statistics_size = None;

// Expected: the remaining should match
let same_created_by = default_table_writer_opts.global.created_by.clone(); // we expect these to be different
Expand Down
18 changes: 9 additions & 9 deletions datafusion/sqllogictest/test_files/information_schema.slt
Original file line number Diff line number Diff line change
Expand Up @@ -184,17 +184,17 @@ datafusion.execution.parquet.bloom_filter_fpp NULL
datafusion.execution.parquet.bloom_filter_ndv NULL
datafusion.execution.parquet.bloom_filter_on_read true
datafusion.execution.parquet.bloom_filter_on_write false
datafusion.execution.parquet.column_index_truncate_length NULL
datafusion.execution.parquet.column_index_truncate_length 64
datafusion.execution.parquet.compression zstd(3)
datafusion.execution.parquet.created_by datafusion
datafusion.execution.parquet.data_page_row_count_limit 18446744073709551615
datafusion.execution.parquet.data_page_row_count_limit 20000
datafusion.execution.parquet.data_pagesize_limit 1048576
datafusion.execution.parquet.dictionary_enabled NULL
datafusion.execution.parquet.dictionary_enabled true
datafusion.execution.parquet.dictionary_page_size_limit 1048576
datafusion.execution.parquet.enable_page_index true
datafusion.execution.parquet.encoding NULL
datafusion.execution.parquet.max_row_group_size 1048576
datafusion.execution.parquet.max_statistics_size NULL
datafusion.execution.parquet.max_statistics_size 4096
datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2
datafusion.execution.parquet.maximum_parallel_row_group_writers 1
datafusion.execution.parquet.metadata_size_hint NULL
Expand Down Expand Up @@ -269,17 +269,17 @@ datafusion.execution.parquet.bloom_filter_fpp NULL (writing) Sets bloom filter f
datafusion.execution.parquet.bloom_filter_ndv NULL (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting
datafusion.execution.parquet.bloom_filter_on_read true (writing) Use any available bloom filters when reading parquet files
datafusion.execution.parquet.bloom_filter_on_write false (writing) Write bloom filters for all columns when creating parquet files
datafusion.execution.parquet.column_index_truncate_length NULL (writing) Sets column index truncate length
datafusion.execution.parquet.compression zstd(3) (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting
datafusion.execution.parquet.column_index_truncate_length 64 (writing) Sets column index truncate length
datafusion.execution.parquet.compression zstd(3) (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting.
datafusion.execution.parquet.created_by datafusion (writing) Sets "created by" property
datafusion.execution.parquet.data_page_row_count_limit 18446744073709551615 (writing) Sets best effort maximum number of rows in data page
datafusion.execution.parquet.data_page_row_count_limit 20000 (writing) Sets best effort maximum number of rows in data page
datafusion.execution.parquet.data_pagesize_limit 1048576 (writing) Sets best effort maximum size of data page in bytes
datafusion.execution.parquet.dictionary_enabled NULL (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting
datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting
datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes
datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded.
datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting
datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read.
datafusion.execution.parquet.max_statistics_size NULL (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting
datafusion.execution.parquet.max_statistics_size 4096 (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting
datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame.
datafusion.execution.parquet.maximum_parallel_row_group_writers 1 (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame.
datafusion.execution.parquet.metadata_size_hint NULL (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer
Expand Down
Loading

0 comments on commit 9f74dcc

Please sign in to comment.