Skip to content
378 changes: 178 additions & 200 deletions native/Cargo.lock

Large diffs are not rendered by default.

6 changes: 3 additions & 3 deletions native/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,11 @@ edition = "2021"
rust-version = "1.85"

[workspace.dependencies]
arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] }
arrow = { version = "55.1.0", features = ["prettyprint", "ffi", "chrono-tz"] }
async-trait = { version = "0.1" }
bytes = { version = "1.10.0" }
parquet = { version = "55.0.0", default-features = false, features = ["experimental"] }
datafusion = { version = "47.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] }
parquet = { version = "55.1.0", default-features = false, features = ["experimental"] }
datafusion = { git = "https://github.com/apache/datafusion", rev = "efd9587", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] }
datafusion-comet-spark-expr = { path = "spark-expr" }
datafusion-comet-proto = { path = "proto" }
chrono = { version = "0.4", default-features = false, features = ["clock"] }
Expand Down
2 changes: 1 addition & 1 deletion native/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ jni = { version = "0.21", features = ["invocation"] }
lazy_static = "1.4"
assertables = "9"
hex = "0.4.3"
datafusion-functions-nested = { version = "47.0.0" }
datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "efd9587" }

[features]
default = []
Expand Down
4 changes: 2 additions & 2 deletions native/core/src/execution/operators/copy.rs
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,8 @@ impl ExecutionPlan for CopyExec {
)))
}

fn statistics(&self) -> DataFusionResult<Statistics> {
self.input.statistics()
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
self.input.partition_statistics(partition)
}

fn properties(&self) -> &PlanProperties {
Expand Down
2 changes: 1 addition & 1 deletion native/core/src/execution/operators/filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ impl FilterExec {
predicate: &Arc<dyn PhysicalExpr>,
default_selectivity: u8,
) -> Result<Statistics> {
let input_stats = input.statistics()?;
let input_stats = input.partition_statistics(None)?;
let schema = input.schema();
if !check_support(predicate, &schema) {
let selectivity = default_selectivity as f64 / 100.0;
Expand Down
24 changes: 14 additions & 10 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ use datafusion::common::{
use datafusion::datasource::listing::PartitionedFile;
use datafusion::logical_expr::type_coercion::other::get_coerce_type_for_case_expression;
use datafusion::logical_expr::{
AggregateUDF, ReturnTypeArgs, WindowFrame, WindowFrameBound, WindowFrameUnits,
AggregateUDF, ReturnFieldArgs, WindowFrame, WindowFrameBound, WindowFrameUnits,
WindowFunctionDefinition,
};
use datafusion::physical_expr::expressions::{Literal, StatsType};
Expand Down Expand Up @@ -886,7 +886,7 @@ impl PhysicalPlanner {
func_name,
fun_expr,
vec![left, right],
data_type,
Field::new(func_name, data_type, true),
)))
}
_ => Ok(Arc::new(BinaryExpr::new(left, op, right))),
Expand Down Expand Up @@ -2191,6 +2191,12 @@ impl PhysicalPlanner {
.coerce_types(&input_expr_types)
.unwrap_or_else(|_| input_expr_types.clone());

let arg_fields = coerced_types
.iter()
.enumerate()
.map(|(i, dt)| Field::new(format!("arg{i}"), dt.clone(), true))
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We now have to provide Field instead of DataType

.collect::<Vec<_>>();

// TODO this should try and find scalar
let arguments = args
.iter()
Expand All @@ -2202,18 +2208,16 @@ impl PhysicalPlanner {
})
.collect::<Vec<_>>();

let nullables = arguments.iter().map(|_| true).collect::<Vec<_>>();

let args = ReturnTypeArgs {
arg_types: &coerced_types,
let args = ReturnFieldArgs {
arg_fields: &arg_fields,
scalar_arguments: &arguments,
nullables: &nullables,
};

let data_type = func
.inner()
.return_type_from_args(args)?
.return_type()
.return_field_from_args(args)?
.clone()
.data_type()
.clone();

(data_type, coerced_types)
Expand Down Expand Up @@ -2246,7 +2250,7 @@ impl PhysicalPlanner {
fun_name,
fun_expr,
args.to_vec(),
data_type,
Field::new(fun_name, data_type, true),
));

Ok(scalar_expr)
Expand Down
2 changes: 1 addition & 1 deletion native/core/src/execution/shuffle/shuffle_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ impl ExecutionPlan for ShuffleWriterExec {
}

fn statistics(&self) -> Result<Statistics> {
self.input.statistics()
self.input.partition_statistics(None)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rename to partition_statistics ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes

}

fn properties(&self) -> &PlanProperties {
Expand Down
4 changes: 2 additions & 2 deletions native/core/src/parquet/parquet_exec.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,8 +78,8 @@ pub(crate) fn init_datasource_exec(
))
});

if let (Some(filter), Some(data_schema)) = (cnf_data_filters, &data_schema) {
parquet_source = parquet_source.with_predicate(Arc::clone(data_schema), filter);
if let Some(filter) = cnf_data_filters {
parquet_source = parquet_source.with_predicate(filter);
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There were changes to the predicate push-down API

}
}

Expand Down
8 changes: 8 additions & 0 deletions native/core/src/parquet/schema_adapter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
use crate::parquet::parquet_support::{spark_parquet_convert, SparkParquetOptions};
use arrow::array::{new_null_array, RecordBatch, RecordBatchOptions};
use arrow::datatypes::{Schema, SchemaRef};
use datafusion::common::ColumnStatistics;
use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper};
use datafusion::physical_plan::ColumnarValue;
use std::sync::Arc;
Expand Down Expand Up @@ -226,6 +227,13 @@ impl SchemaMapper for SchemaMapping {
let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?;
Ok(record_batch)
}

fn map_column_statistics(
&self,
_file_col_statistics: &[ColumnStatistics],
) -> datafusion::common::Result<Vec<ColumnStatistics>> {
Ok(vec![])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this unused or just placeholder...?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Column statistics are not used in Comet

}
}

#[cfg(test)]
Expand Down
4 changes: 2 additions & 2 deletions native/spark-expr/src/conversion_funcs/cast.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ use arrow::{
record_batch::RecordBatch,
util::display::FormatOptions,
};
use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike};
use chrono::{DateTime, NaiveDate, TimeZone, Timelike};
use datafusion::common::{
cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue,
};
Expand Down Expand Up @@ -2108,7 +2108,7 @@ fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult<Option<i32>>
) {
Some(date) => {
let duration_since_epoch = date
.signed_duration_since(NaiveDateTime::UNIX_EPOCH.date())
.signed_duration_since(DateTime::UNIX_EPOCH.naive_utc().date())
.num_days();
Ok(Some(duration_since_epoch.to_i32().unwrap()))
}
Expand Down
6 changes: 4 additions & 2 deletions native/spark-expr/src/hash_funcs/sha2.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

use crate::math_funcs::hex::hex_strings;
use arrow::array::{Array, StringArray};
use arrow::datatypes::DataType;
use arrow::datatypes::{DataType, Field};
use datafusion::common::cast::as_binary_array;
use datafusion::common::{exec_err, DataFusionError, ScalarValue};
use datafusion::functions::crypto::{sha224, sha256, sha384, sha512};
Expand Down Expand Up @@ -55,10 +55,12 @@ fn wrap_digest_result_as_hex_string(
ColumnarValue::Array(array) => array.len(),
ColumnarValue::Scalar(_) => 1,
};
let return_field = Field::new("foo", DataType::Utf8, false);
let value = digest.invoke_with_args(ScalarFunctionArgs {
args: args.into(),
arg_fields: vec![],
number_rows: row_count,
return_type: &DataType::Utf8,
return_field: &return_field,
})?;
match value {
ColumnarValue::Array(array) => {
Expand Down
Loading