-
Notifications
You must be signed in to change notification settings - Fork 303
Expand file tree
/
Copy pathparquet_exec.rs
More file actions
192 lines (178 loc) · 7.16 KB
/
parquet_exec.rs
File metadata and controls
192 lines (178 loc) · 7.16 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
use crate::execution::operators::ExecutionError;
use crate::parquet::encryption_support::{CometEncryptionConfig, ENCRYPTION_FACTORY_ID};
use crate::parquet::parquet_support::SparkParquetOptions;
use crate::parquet::schema_adapter::SparkSchemaAdapterFactory;
use arrow::datatypes::{Field, SchemaRef};
use datafusion::config::TableParquetOptions;
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::physical_plan::{
FileGroup, FileScanConfigBuilder, FileSource, ParquetSource,
};
use datafusion::datasource::source::DataSourceExec;
use datafusion::execution::object_store::ObjectStoreUrl;
use datafusion::physical_expr::expressions::BinaryExpr;
use datafusion::physical_expr::PhysicalExpr;
use datafusion::prelude::SessionContext;
use datafusion::scalar::ScalarValue;
use datafusion_comet_spark_expr::EvalMode;
use itertools::Itertools;
use std::collections::HashMap;
use std::sync::Arc;
/// Initializes a DataSourceExec plan with a ParquetSource. This may be used by either the
/// `native_datafusion` scan or the `native_iceberg_compat` scan.
///
/// `required_schema`: Schema to be projected by the scan.
///
/// `data_schema`: Schema of the underlying data. It is optional and, if provided, is used
/// instead of `required_schema` to initialize the file scan
///
/// `partition_schema` and `partition_fields` are optional. If `partition_schema` is specified,
/// then `partition_fields` must also be specified
///
/// `object_store_url`: Url to read data from
///
/// `file_groups`: A collection of groups of `PartitionedFiles` that are to be read by the scan
///
/// `projection_vector`: A vector of the indexes in the schema of the fields to be projected
///
/// `data_filters`: Any predicate that must be applied to the data returned by the scan. If
/// specified, then `data_schema` must also be specified.
#[allow(clippy::too_many_arguments)]
pub(crate) fn init_datasource_exec(
required_schema: SchemaRef,
data_schema: Option<SchemaRef>,
partition_schema: Option<SchemaRef>,
partition_fields: Option<Vec<Field>>,
object_store_url: ObjectStoreUrl,
file_groups: Vec<Vec<PartitionedFile>>,
projection_vector: Option<Vec<usize>>,
data_filters: Option<Vec<Arc<dyn PhysicalExpr>>>,
default_values: Option<HashMap<usize, ScalarValue>>,
session_timezone: &str,
case_sensitive: bool,
session_ctx: &Arc<SessionContext>,
encryption_enabled: bool,
) -> Result<Arc<DataSourceExec>, ExecutionError> {
let (table_parquet_options, spark_parquet_options) = get_options(
session_timezone,
case_sensitive,
&object_store_url,
encryption_enabled,
);
let mut parquet_source = ParquetSource::new(table_parquet_options);
// Create a conjunctive form of the vector because ParquetExecBuilder takes
// a single expression
if let Some(data_filters) = data_filters {
let cnf_data_filters = data_filters.clone().into_iter().reduce(|left, right| {
Arc::new(BinaryExpr::new(
left,
datafusion::logical_expr::Operator::And,
right,
))
});
if let Some(filter) = cnf_data_filters {
parquet_source = parquet_source.with_predicate(filter);
}
}
if encryption_enabled {
parquet_source = parquet_source.with_encryption_factory(
session_ctx
.runtime_env()
.parquet_encryption_factory(ENCRYPTION_FACTORY_ID)?,
);
}
let file_source = parquet_source.with_schema_adapter_factory(Arc::new(
SparkSchemaAdapterFactory::new(spark_parquet_options, default_values),
))?;
let file_groups = file_groups
.iter()
.map(|files| FileGroup::new(files.clone()))
.collect();
let file_scan_config = match (data_schema, projection_vector, partition_fields) {
(Some(data_schema), Some(projection_vector), Some(partition_fields)) => {
get_file_config_builder(
data_schema,
partition_schema,
file_groups,
object_store_url,
file_source,
)
.with_projection_indices(Some(projection_vector))
.with_table_partition_cols(partition_fields)
.build()
}
_ => get_file_config_builder(
required_schema,
partition_schema,
file_groups,
object_store_url,
file_source,
)
.build(),
};
Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config))))
}
fn get_options(
session_timezone: &str,
case_sensitive: bool,
object_store_url: &ObjectStoreUrl,
encryption_enabled: bool,
) -> (TableParquetOptions, SparkParquetOptions) {
let mut table_parquet_options = TableParquetOptions::new();
table_parquet_options.global.pushdown_filters = true;
table_parquet_options.global.reorder_filters = true;
table_parquet_options.global.coerce_int96 = Some("us".to_string());
let mut spark_parquet_options =
SparkParquetOptions::new(EvalMode::Legacy, session_timezone, false);
spark_parquet_options.allow_cast_unsigned_ints = true;
spark_parquet_options.case_sensitive = case_sensitive;
if encryption_enabled {
table_parquet_options.crypto.configure_factory(
ENCRYPTION_FACTORY_ID,
&CometEncryptionConfig {
uri_base: object_store_url.to_string(),
},
);
}
(table_parquet_options, spark_parquet_options)
}
fn get_file_config_builder(
schema: SchemaRef,
partition_schema: Option<SchemaRef>,
file_groups: Vec<FileGroup>,
object_store_url: ObjectStoreUrl,
file_source: Arc<dyn FileSource>,
) -> FileScanConfigBuilder {
match partition_schema {
Some(partition_schema) => {
let partition_fields: Vec<Field> = partition_schema
.fields()
.iter()
.map(|field| {
Field::new(field.name(), field.data_type().clone(), field.is_nullable())
})
.collect_vec();
FileScanConfigBuilder::new(object_store_url, Arc::clone(&schema), file_source)
.with_file_groups(file_groups)
.with_table_partition_cols(partition_fields)
}
_ => FileScanConfigBuilder::new(object_store_url, Arc::clone(&schema), file_source)
.with_file_groups(file_groups),
}
}