Skip to content

Commit 6a2e631

Browse files
committed
Merge remote-tracking branch 'apache/main' into alamb/update_arrow_57.2.0
2 parents b29b662 + 566bcde commit 6a2e631

File tree

13 files changed

+1156
-93
lines changed

13 files changed

+1156
-93
lines changed

Cargo.lock

Lines changed: 3 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

datafusion/core/tests/user_defined/user_defined_async_scalar_functions.rs

Lines changed: 36 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,16 +21,14 @@ use arrow::array::{Int32Array, RecordBatch, StringArray};
2121
use arrow::datatypes::{DataType, Field, Schema};
2222
use async_trait::async_trait;
2323
use datafusion::prelude::*;
24+
use datafusion_common::test_util::format_batches;
2425
use datafusion_common::{Result, assert_batches_eq};
2526
use datafusion_expr::async_udf::{AsyncScalarUDF, AsyncScalarUDFImpl};
2627
use datafusion_expr::{
2728
ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility,
2829
};
2930

30-
// This test checks the case where batch_size doesn't evenly divide
31-
// the number of rows.
32-
#[tokio::test]
33-
async fn test_async_udf_with_non_modular_batch_size() -> Result<()> {
31+
fn register_table_and_udf() -> Result<SessionContext> {
3432
let num_rows = 3;
3533
let batch_size = 2;
3634

@@ -59,6 +57,15 @@ async fn test_async_udf_with_non_modular_batch_size() -> Result<()> {
5957
.into_scalar_udf(),
6058
);
6159

60+
Ok(ctx)
61+
}
62+
63+
// This test checks the case where batch_size doesn't evenly divide
64+
// the number of rows.
65+
#[tokio::test]
66+
async fn test_async_udf_with_non_modular_batch_size() -> Result<()> {
67+
let ctx = register_table_and_udf()?;
68+
6269
let df = ctx
6370
.sql("SELECT id, test_async_udf(prompt) as result FROM test_table")
6471
.await?;
@@ -81,6 +88,31 @@ async fn test_async_udf_with_non_modular_batch_size() -> Result<()> {
8188
Ok(())
8289
}
8390

91+
// This test checks if metrics are printed for `AsyncFuncExec`
92+
#[tokio::test]
93+
async fn test_async_udf_metrics() -> Result<()> {
94+
let ctx = register_table_and_udf()?;
95+
96+
let df = ctx
97+
.sql(
98+
"EXPLAIN ANALYZE SELECT id, test_async_udf(prompt) as result FROM test_table",
99+
)
100+
.await?;
101+
102+
let result = df.collect().await?;
103+
104+
let explain_analyze_str = format_batches(&result)?.to_string();
105+
let async_func_exec_without_metrics =
106+
explain_analyze_str.split("\n").any(|metric_line| {
107+
metric_line.contains("AsyncFuncExec")
108+
&& !metric_line.contains("output_rows=3")
109+
});
110+
111+
assert!(!async_func_exec_without_metrics);
112+
113+
Ok(())
114+
}
115+
84116
#[derive(Debug, PartialEq, Eq, Hash, Clone)]
85117
struct TestAsyncUDFImpl {
86118
batch_size: usize,

datafusion/datasource-parquet/Cargo.toml

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,9 @@ tokio = { workspace = true }
5656

5757
[dev-dependencies]
5858
chrono = { workspace = true }
59+
criterion = { workspace = true }
60+
datafusion-functions-nested = { workspace = true }
61+
tempfile = { workspace = true }
5962

6063
# Note: add additional linter rules in lib.rs.
6164
# Rust does not support workspace + new linter rules in subcrates yet
@@ -73,3 +76,7 @@ parquet_encryption = [
7376
"datafusion-common/parquet_encryption",
7477
"datafusion-execution/parquet_encryption",
7578
]
79+
80+
[[bench]]
81+
name = "parquet_nested_filter_pushdown"
82+
harness = false
Lines changed: 238 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,238 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
use std::path::{Path, PathBuf};
19+
use std::sync::{Arc, LazyLock};
20+
21+
use arrow::array::{
22+
BinaryBuilder, BooleanArray, ListBuilder, RecordBatch, StringBuilder,
23+
};
24+
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
25+
use criterion::{Criterion, Throughput, criterion_group, criterion_main};
26+
use datafusion_common::ScalarValue;
27+
use datafusion_datasource_parquet::{ParquetFileMetrics, build_row_filter};
28+
use datafusion_expr::{Expr, col};
29+
use datafusion_functions_nested::expr_fn::array_has;
30+
use datafusion_physical_expr::planner::logical2physical;
31+
use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet;
32+
use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder;
33+
use parquet::arrow::{ArrowWriter, ProjectionMask};
34+
use parquet::file::properties::WriterProperties;
35+
use tempfile::TempDir;
36+
37+
const ROW_GROUP_SIZE: usize = 10_000;
38+
const TOTAL_ROW_GROUPS: usize = 10;
39+
const TOTAL_ROWS: usize = ROW_GROUP_SIZE * TOTAL_ROW_GROUPS;
40+
const TARGET_VALUE: &str = "target_value";
41+
const COLUMN_NAME: &str = "list_col";
42+
const PAYLOAD_COLUMN_NAME: &str = "payload";
43+
// Large binary payload to emphasize decoding overhead when pushdown is disabled.
44+
const PAYLOAD_BYTES: usize = 8 * 1024;
45+
46+
struct BenchmarkDataset {
47+
_tempdir: TempDir,
48+
file_path: PathBuf,
49+
}
50+
51+
impl BenchmarkDataset {
52+
fn path(&self) -> &Path {
53+
&self.file_path
54+
}
55+
}
56+
57+
static DATASET: LazyLock<BenchmarkDataset> = LazyLock::new(|| {
58+
create_dataset().expect("failed to prepare parquet benchmark dataset")
59+
});
60+
61+
fn parquet_nested_filter_pushdown(c: &mut Criterion) {
62+
let dataset_path = DATASET.path().to_owned();
63+
let mut group = c.benchmark_group("parquet_nested_filter_pushdown");
64+
group.throughput(Throughput::Elements(TOTAL_ROWS as u64));
65+
66+
group.bench_function("no_pushdown", |b| {
67+
let file_schema = setup_reader(&dataset_path);
68+
let predicate = logical2physical(&create_predicate(), &file_schema);
69+
b.iter(|| {
70+
let matched = scan_with_predicate(&dataset_path, &predicate, false)
71+
.expect("baseline parquet scan with filter succeeded");
72+
assert_eq!(matched, ROW_GROUP_SIZE);
73+
});
74+
});
75+
76+
group.bench_function("with_pushdown", |b| {
77+
let file_schema = setup_reader(&dataset_path);
78+
let predicate = logical2physical(&create_predicate(), &file_schema);
79+
b.iter(|| {
80+
let matched = scan_with_predicate(&dataset_path, &predicate, true)
81+
.expect("pushdown parquet scan with filter succeeded");
82+
assert_eq!(matched, ROW_GROUP_SIZE);
83+
});
84+
});
85+
86+
group.finish();
87+
}
88+
89+
fn setup_reader(path: &Path) -> SchemaRef {
90+
let file = std::fs::File::open(path).expect("failed to open file");
91+
let builder =
92+
ParquetRecordBatchReaderBuilder::try_new(file).expect("failed to build reader");
93+
Arc::clone(builder.schema())
94+
}
95+
96+
fn create_predicate() -> Expr {
97+
array_has(
98+
col(COLUMN_NAME),
99+
Expr::Literal(ScalarValue::Utf8(Some(TARGET_VALUE.to_string())), None),
100+
)
101+
}
102+
103+
fn scan_with_predicate(
104+
path: &Path,
105+
predicate: &Arc<dyn datafusion_physical_expr::PhysicalExpr>,
106+
pushdown: bool,
107+
) -> datafusion_common::Result<usize> {
108+
let file = std::fs::File::open(path)?;
109+
let builder = ParquetRecordBatchReaderBuilder::try_new(file)?;
110+
let metadata = builder.metadata().clone();
111+
let file_schema = builder.schema();
112+
let projection = ProjectionMask::all();
113+
114+
let metrics = ExecutionPlanMetricsSet::new();
115+
let file_metrics = ParquetFileMetrics::new(0, &path.display().to_string(), &metrics);
116+
117+
let builder = if pushdown {
118+
if let Some(row_filter) =
119+
build_row_filter(predicate, file_schema, &metadata, false, &file_metrics)?
120+
{
121+
builder.with_row_filter(row_filter)
122+
} else {
123+
builder
124+
}
125+
} else {
126+
builder
127+
};
128+
129+
let reader = builder.with_projection(projection).build()?;
130+
131+
let mut matched_rows = 0usize;
132+
for batch in reader {
133+
let batch = batch?;
134+
matched_rows += count_matches(predicate, &batch)?;
135+
}
136+
137+
if pushdown {
138+
let pruned_rows = file_metrics.pushdown_rows_pruned.value();
139+
assert_eq!(
140+
pruned_rows,
141+
TOTAL_ROWS - matched_rows,
142+
"row-level pushdown should prune 90% of rows"
143+
);
144+
}
145+
146+
Ok(matched_rows)
147+
}
148+
149+
fn count_matches(
150+
expr: &Arc<dyn datafusion_physical_expr::PhysicalExpr>,
151+
batch: &RecordBatch,
152+
) -> datafusion_common::Result<usize> {
153+
let values = expr.evaluate(batch)?.into_array(batch.num_rows())?;
154+
let bools = values
155+
.as_any()
156+
.downcast_ref::<BooleanArray>()
157+
.expect("boolean filter result");
158+
159+
Ok(bools.iter().filter(|v| matches!(v, Some(true))).count())
160+
}
161+
162+
fn create_dataset() -> datafusion_common::Result<BenchmarkDataset> {
163+
let tempdir = TempDir::new()?;
164+
let file_path = tempdir.path().join("nested_lists.parquet");
165+
166+
let field = Arc::new(Field::new("item", DataType::Utf8, true));
167+
let schema = Arc::new(Schema::new(vec![
168+
Field::new(COLUMN_NAME, DataType::List(field), false),
169+
Field::new(PAYLOAD_COLUMN_NAME, DataType::Binary, false),
170+
]));
171+
172+
let writer_props = WriterProperties::builder()
173+
.set_max_row_group_size(ROW_GROUP_SIZE)
174+
.build();
175+
176+
let mut writer = ArrowWriter::try_new(
177+
std::fs::File::create(&file_path)?,
178+
Arc::clone(&schema),
179+
Some(writer_props),
180+
)?;
181+
182+
// Create sorted row groups with distinct values so that min/max statistics
183+
// allow skipping most groups when applying a selective predicate.
184+
let sorted_values = [
185+
"alpha",
186+
"bravo",
187+
"charlie",
188+
"delta",
189+
"echo",
190+
"foxtrot",
191+
"golf",
192+
"hotel",
193+
"india",
194+
TARGET_VALUE,
195+
];
196+
197+
for value in sorted_values {
198+
let batch = build_list_batch(&schema, value, ROW_GROUP_SIZE)?;
199+
writer.write(&batch)?;
200+
}
201+
202+
writer.close()?;
203+
204+
// Ensure the writer respected the requested row group size
205+
let reader =
206+
ParquetRecordBatchReaderBuilder::try_new(std::fs::File::open(&file_path)?)?;
207+
assert_eq!(reader.metadata().row_groups().len(), TOTAL_ROW_GROUPS);
208+
209+
Ok(BenchmarkDataset {
210+
_tempdir: tempdir,
211+
file_path,
212+
})
213+
}
214+
215+
fn build_list_batch(
216+
schema: &SchemaRef,
217+
value: &str,
218+
len: usize,
219+
) -> datafusion_common::Result<RecordBatch> {
220+
let mut builder = ListBuilder::new(StringBuilder::new());
221+
let mut payload_builder = BinaryBuilder::new();
222+
let payload = vec![1u8; PAYLOAD_BYTES];
223+
for _ in 0..len {
224+
builder.values().append_value(value);
225+
builder.append(true);
226+
payload_builder.append_value(&payload);
227+
}
228+
229+
let array = builder.finish();
230+
let payload_array = payload_builder.finish();
231+
Ok(RecordBatch::try_new(
232+
Arc::clone(schema),
233+
vec![Arc::new(array), Arc::new(payload_array)],
234+
)?)
235+
}
236+
237+
criterion_group!(benches, parquet_nested_filter_pushdown);
238+
criterion_main!(benches);

datafusion/datasource-parquet/src/mod.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ mod row_filter;
3232
mod row_group_filter;
3333
mod sort;
3434
pub mod source;
35+
mod supported_predicates;
3536
mod writer;
3637

3738
pub use access_plan::{ParquetAccessPlan, RowGroupAccess};

0 commit comments

Comments
 (0)