Skip to content

Commit

Permalink
feat/copy-to-parquet-parameter: Commit Message:
Browse files Browse the repository at this point in the history
Enhance Parquet File Format Handling for Specific Data Types

Summary:

 • Added ConcreteDataType import to support specific data type handling.
  • Loading branch information
v0y4g3r committed Jul 9, 2024
1 parent 3935479 commit e3db2c7
Showing 1 changed file with 5 additions and 2 deletions.
7 changes: 5 additions & 2 deletions src/common/datasource/src/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ use datafusion::parquet::file::metadata::ParquetMetaData;
use datafusion::parquet::format::FileMetaData;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion::physical_plan::SendableRecordBatchStream;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::SchemaRef;
use futures::future::BoxFuture;
use futures::StreamExt;
Expand Down Expand Up @@ -231,10 +232,12 @@ fn column_wise_config(
props = props.set_column_dictionary_enabled(path, false);
}

// Use delta binary packed encoding for all numeric columns.
// Use delta binary packed encoding for int32/int64 columns.
for column in schema.column_schemas() {
let data_type = &column.data_type;
if data_type.is_numeric() {
if data_type == &ConcreteDataType::int64_datatype()
|| data_type == &ConcreteDataType::int32_datatype()
{
let path = ColumnPath::new(vec![column.name.clone()]);
props = props.set_column_encoding(path, Encoding::DELTA_BINARY_PACKED);
}
Expand Down

0 comments on commit e3db2c7

Please sign in to comment.