alamb commented on code in PR #7401: URL: https://github.com/apache/arrow-rs/pull/7401#discussion_r2039337556
########## parquet/benches/arrow_reader_row_filter.rs: ########## @@ -0,0 +1,325 @@ +// 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. + +//! Benchmark for evaluating row filters and projections on a Parquet file. +//! +//! This benchmark creates a Parquet file in memory with 100K rows and four columns: +//! - int64: sequential integers +//! - float64: floating-point values (derived from the integers) +//! - utf8View: string values where about half are non-empty, +//! and a few rows (every 10Kth row) are the constant "const" +//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! +//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! +//! Filters tested: +//! - A string filter: `utf8View <> ''` (non-empty) +//! - A string filter: `utf8View = 'const'` (selective) +//! - An integer non-selective filter (e.g. even numbers) +//! - An integer selective filter (e.g. `int64 = 0`) +//! - A timestamp filter (e.g. `ts > threshold`) +//! +//! Projections tested: +//! - All 4 columns. +//! - All columns except the one used for the filter. +//! +//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +use arrow::array::{ + ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; +use arrow_array::builder::StringViewBuilder; +use arrow_array::{Array, StringViewArray}; +use criterion::async_executor::FuturesExecutor; +use futures::TryStreamExt; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::properties::WriterProperties; +use tokio::fs::File; +use tokio::runtime::Runtime; + +/// Create a RecordBatch with 100K rows and four columns. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + + // int64 column: sequential numbers 0..num_rows + let int_values: Vec<i64> = (0..num_rows as i64).collect(); Review Comment: I think it is more common to use fixed seeded random values to create values to avoid artifacts that such regular patterns may introduce There are some good examples here: https://github.com/apache/arrow-rs/blob/d0260fcffa07a4cb8650cc290ab29027a3a8e65c/parquet/benches/arrow_writer.rs#L101-L100 ########## parquet/benches/arrow_reader_row_filter.rs: ########## @@ -0,0 +1,325 @@ +// 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. + +//! Benchmark for evaluating row filters and projections on a Parquet file. +//! +//! This benchmark creates a Parquet file in memory with 100K rows and four columns: +//! - int64: sequential integers +//! - float64: floating-point values (derived from the integers) +//! - utf8View: string values where about half are non-empty, +//! and a few rows (every 10Kth row) are the constant "const" +//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! +//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! +//! Filters tested: +//! - A string filter: `utf8View <> ''` (non-empty) +//! - A string filter: `utf8View = 'const'` (selective) +//! - An integer non-selective filter (e.g. even numbers) +//! - An integer selective filter (e.g. `int64 = 0`) +//! - A timestamp filter (e.g. `ts > threshold`) +//! +//! Projections tested: +//! - All 4 columns. +//! - All columns except the one used for the filter. +//! +//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +use arrow::array::{ + ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; +use arrow_array::builder::StringViewBuilder; +use arrow_array::{Array, StringViewArray}; +use criterion::async_executor::FuturesExecutor; +use futures::TryStreamExt; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::properties::WriterProperties; +use tokio::fs::File; +use tokio::runtime::Runtime; + +/// Create a RecordBatch with 100K rows and four columns. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + + // int64 column: sequential numbers 0..num_rows + let int_values: Vec<i64> = (0..num_rows as i64).collect(); + let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; + + // float64 column: derived from int64 (e.g., multiplied by 0.1) + let float_values: Vec<f64> = (0..num_rows).map(|i| i as f64 * 0.1).collect(); + let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; + + // utf8View column: even rows get non-empty strings; odd rows get an empty string; + // every 10Kth even row is "const" to be selective. + let mut string_view_builder = StringViewBuilder::with_capacity(100_000); + for i in 0..num_rows { + if i % 2 == 0 { + if i % 10_000 == 0 { + string_view_builder.append_value("const"); + } else { + string_view_builder.append_value("nonempty"); + } + } else { + string_view_builder.append_value(""); + } + } + let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; + + // Timestamp column: using milliseconds from an epoch (simply using the row index) + let ts_values: Vec<i64> = (0..num_rows as i64).collect(); + let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; + + let schema = Arc::new(Schema::new(vec![ + Field::new("int64", DataType::Int64, false), + Field::new("float64", DataType::Float64, false), + Field::new("utf8View", DataType::Utf8View, false), + Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + ])); + + RecordBatch::try_new( + schema, + vec![int_array, float_array, utf8_view_array, ts_array], + ) + .unwrap() +} + +/// Writes the record batch to a temporary Parquet file. +fn write_parquet_file() -> NamedTempFile { + let batch = make_record_batch(); + let schema = batch.schema(); + let props = WriterProperties::builder().build(); + + let file = tempfile::Builder::new() + .suffix(".parquet") + .tempfile() + .unwrap(); + { + let file_reopen = file.reopen().unwrap(); + let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); + // Write the entire batch as a single row group. + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + file +} + +/// Filter function: returns a BooleanArray with true when utf8View <> "". +fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("utf8View").unwrap()) + .as_any() + .downcast_ref::<StringViewArray>() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = !array.value(i).is_empty(); + builder.append_value(keep); + } + builder.finish() +} + +/// Filter function: returns a BooleanArray with true when utf8View == "const". +fn filter_utf8_view_const(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("utf8View").unwrap()) + .as_any() + .downcast_ref::<StringViewArray>() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) == "const"; + builder.append_value(keep); + } + builder.finish() +} + +/// Integer non-selective filter: returns true for even numbers. +fn filter_int64_even(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("int64").unwrap()) + .as_any() + .downcast_ref::<Int64Array>() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) % 2 == 0; + builder.append_value(keep); + } + builder.finish() +} + +/// Integer selective filter: returns true only when int64 equals 0. +fn filter_int64_eq_zero(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("int64").unwrap()) + .as_any() + .downcast_ref::<Int64Array>() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) == 0; + builder.append_value(keep); + } + builder.finish() +} + +/// Timestamp filter: returns true when ts > threshold (using 50_000 as example threshold). +fn filter_timestamp_gt(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("ts").unwrap()) + .as_any() + .downcast_ref::<TimestampMillisecondArray>() + .unwrap(); + let threshold = 50_000; + let mut builder = BooleanBuilder::with_capacity(array.len()); + for i in 0..array.len() { + let keep = array.value(i) > threshold; + builder.append_value(keep); + } + builder.finish() +} + +#[derive(Clone)] +enum FilterType { Review Comment: I think having the background about why these particular filters are chosen is important. I realize i did not do a good job of describing them on the ticket, but I will now work on some diagrams and descriptions to explain it better ########## parquet/benches/arrow_reader_row_filter.rs: ########## @@ -0,0 +1,325 @@ +// 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. + +//! Benchmark for evaluating row filters and projections on a Parquet file. +//! +//! This benchmark creates a Parquet file in memory with 100K rows and four columns: +//! - int64: sequential integers +//! - float64: floating-point values (derived from the integers) +//! - utf8View: string values where about half are non-empty, +//! and a few rows (every 10Kth row) are the constant "const" +//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! +//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! +//! Filters tested: +//! - A string filter: `utf8View <> ''` (non-empty) +//! - A string filter: `utf8View = 'const'` (selective) +//! - An integer non-selective filter (e.g. even numbers) +//! - An integer selective filter (e.g. `int64 = 0`) +//! - A timestamp filter (e.g. `ts > threshold`) +//! +//! Projections tested: +//! - All 4 columns. +//! - All columns except the one used for the filter. +//! +//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +use arrow::array::{ + ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; +use arrow_array::builder::StringViewBuilder; +use arrow_array::{Array, StringViewArray}; +use criterion::async_executor::FuturesExecutor; +use futures::TryStreamExt; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::properties::WriterProperties; +use tokio::fs::File; +use tokio::runtime::Runtime; + +/// Create a RecordBatch with 100K rows and four columns. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + + // int64 column: sequential numbers 0..num_rows + let int_values: Vec<i64> = (0..num_rows as i64).collect(); Review Comment: Specifically, I suggest: 1. Random ints 2. Random floats 3. ordered timestamps (as you have them, as that is quite common) 4. Random strings For string view, it is also important to include strings that are more than 12 bytes long (strings less than this are entirely inlined into the view) I printed out the data it is quite regular: ``` Running benches/arrow_reader_row_filter.rs (target/debug/deps/arrow_reader_row_filter-6404dc89531cf7fd) Gnuplot not found, using plotters backend Batch created with 100000 rows First 100 rows +-------+---------------------+----------+-------------------------+ | int64 | float64 | utf8View | ts | +-------+---------------------+----------+-------------------------+ | 0 | 0.0 | const | 1970-01-01T00:00:00 | | 1 | 0.1 | | 1970-01-01T00:00:00.001 | | 2 | 0.2 | nonempty | 1970-01-01T00:00:00.002 | | 3 | 0.30000000000000004 | | 1970-01-01T00:00:00.003 | | 4 | 0.4 | nonempty | 1970-01-01T00:00:00.004 | | 5 | 0.5 | | 1970-01-01T00:00:00.005 | | 6 | 0.6000000000000001 | nonempty | 1970-01-01T00:00:00.006 | | 7 | 0.7000000000000001 | | 1970-01-01T00:00:00.007 | | 8 | 0.8 | nonempty | 1970-01-01T00:00:00.008 | | 9 | 0.9 | | 1970-01-01T00:00:00.009 | | 10 | 1.0 | nonempty | 1970-01-01T00:00:00.010 | | 11 | 1.1 | | 1970-01-01T00:00:00.011 | | 12 | 1.2000000000000002 | nonempty | 1970-01-01T00:00:00.012 | | 13 | 1.3 | | 1970-01-01T00:00:00.013 | | 14 | 1.4000000000000001 | nonempty | 1970-01-01T00:00:00.014 | | 15 | 1.5 | | 1970-01-01T00:00:00.015 | | 16 | 1.6 | nonempty | 1970-01-01T00:00:00.016 | | 17 | 1.7000000000000002 | | 1970-01-01T00:00:00.017 | | 18 | 1.8 | nonempty | 1970-01-01T00:00:00.018 | | 19 | 1.9000000000000001 | | 1970-01-01T00:00:00.019 | | 20 | 2.0 | nonempty | 1970-01-01T00:00:00.020 | | 21 | 2.1 | | 1970-01-01T00:00:00.021 | | 22 | 2.2 | nonempty | 1970-01-01T00:00:00.022 | | 23 | 2.3000000000000003 | | 1970-01-01T00:00:00.023 | | 24 | 2.4000000000000004 | nonempty | 1970-01-01T00:00:00.024 | | 25 | 2.5 | | 1970-01-01T00:00:00.025 | | 26 | 2.6 | nonempty | 1970-01-01T00:00:00.026 | | 27 | 2.7 | | 1970-01-01T00:00:00.027 | | 28 | 2.8000000000000003 | nonempty | 1970-01-01T00:00:00.028 | | 29 | 2.9000000000000004 | | 1970-01-01T00:00:00.029 | | 30 | 3.0 | nonempty | 1970-01-01T00:00:00.030 | | 31 | 3.1 | | 1970-01-01T00:00:00.031 | | 32 | 3.2 | nonempty | 1970-01-01T00:00:00.032 | | 33 | 3.3000000000000003 | | 1970-01-01T00:00:00.033 | | 34 | 3.4000000000000004 | nonempty | 1970-01-01T00:00:00.034 | | 35 | 3.5 | | 1970-01-01T00:00:00.035 | ``` I printed this using ```diff diff --git a/parquet/benches/arrow_reader_row_filter.rs b/parquet/benches/arrow_reader_row_filter.rs index af07636e49..87e353f2c0 100644 --- a/parquet/benches/arrow_reader_row_filter.rs +++ b/parquet/benches/arrow_reader_row_filter.rs @@ -57,6 +57,7 @@ use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMas use parquet::file::properties::WriterProperties; use tokio::fs::File; use tokio::runtime::Runtime; +use arrow_cast::pretty::pretty_format_batches; /// Create a RecordBatch with 100K rows and four columns. fn make_record_batch() -> RecordBatch { @@ -101,11 +102,17 @@ fn make_record_batch() -> RecordBatch { ), ])); - RecordBatch::try_new( + let batch = RecordBatch::try_new( schema, vec![int_array, float_array, utf8_view_array, ts_array], ) - .unwrap() + .unwrap(); + + // Verify the batch was created correctly + println!("Batch created with {} rows", num_rows); + println!("First 100 rows"); + println!("{}", pretty_format_batches(&[batch.clone().slice(0, 100)]).unwrap()); + batch } ``` ########## parquet/benches/arrow_reader_row_filter.rs: ########## @@ -0,0 +1,325 @@ +// 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. + +//! Benchmark for evaluating row filters and projections on a Parquet file. +//! +//! This benchmark creates a Parquet file in memory with 100K rows and four columns: +//! - int64: sequential integers +//! - float64: floating-point values (derived from the integers) +//! - utf8View: string values where about half are non-empty, +//! and a few rows (every 10Kth row) are the constant "const" +//! - ts: timestamp values (using, e.g., a millisecond epoch) +//! +//! It then applies several filter functions and projections, benchmarking the read-back speed. +//! +//! Filters tested: +//! - A string filter: `utf8View <> ''` (non-empty) +//! - A string filter: `utf8View = 'const'` (selective) +//! - An integer non-selective filter (e.g. even numbers) +//! - An integer selective filter (e.g. `int64 = 0`) +//! - A timestamp filter (e.g. `ts > threshold`) +//! +//! Projections tested: +//! - All 4 columns. +//! - All columns except the one used for the filter. +//! +//! To run the benchmark, use `cargo bench --bench bench_filter_projection`. + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +use arrow::array::{ + ArrayRef, BooleanArray, BooleanBuilder, Float64Array, Int64Array, TimestampMillisecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use arrow::record_batch::RecordBatch; +use arrow_array::builder::StringViewBuilder; +use arrow_array::{Array, StringViewArray}; +use criterion::async_executor::FuturesExecutor; +use futures::TryStreamExt; +use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter}; +use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::properties::WriterProperties; +use tokio::fs::File; +use tokio::runtime::Runtime; + +/// Create a RecordBatch with 100K rows and four columns. +fn make_record_batch() -> RecordBatch { + let num_rows = 100_000; + + // int64 column: sequential numbers 0..num_rows + let int_values: Vec<i64> = (0..num_rows as i64).collect(); + let int_array = Arc::new(Int64Array::from(int_values)) as ArrayRef; + + // float64 column: derived from int64 (e.g., multiplied by 0.1) + let float_values: Vec<f64> = (0..num_rows).map(|i| i as f64 * 0.1).collect(); + let float_array = Arc::new(Float64Array::from(float_values)) as ArrayRef; + + // utf8View column: even rows get non-empty strings; odd rows get an empty string; + // every 10Kth even row is "const" to be selective. + let mut string_view_builder = StringViewBuilder::with_capacity(100_000); + for i in 0..num_rows { + if i % 2 == 0 { + if i % 10_000 == 0 { + string_view_builder.append_value("const"); + } else { + string_view_builder.append_value("nonempty"); + } + } else { + string_view_builder.append_value(""); + } + } + let utf8_view_array = Arc::new(string_view_builder.finish()) as ArrayRef; + + // Timestamp column: using milliseconds from an epoch (simply using the row index) + let ts_values: Vec<i64> = (0..num_rows as i64).collect(); + let ts_array = Arc::new(TimestampMillisecondArray::from(ts_values)) as ArrayRef; + + let schema = Arc::new(Schema::new(vec![ + Field::new("int64", DataType::Int64, false), + Field::new("float64", DataType::Float64, false), + Field::new("utf8View", DataType::Utf8View, false), + Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + ])); + + RecordBatch::try_new( + schema, + vec![int_array, float_array, utf8_view_array, ts_array], + ) + .unwrap() +} + +/// Writes the record batch to a temporary Parquet file. +fn write_parquet_file() -> NamedTempFile { + let batch = make_record_batch(); + let schema = batch.schema(); + let props = WriterProperties::builder().build(); + + let file = tempfile::Builder::new() + .suffix(".parquet") + .tempfile() + .unwrap(); + { + let file_reopen = file.reopen().unwrap(); + let mut writer = ArrowWriter::try_new(file_reopen, schema.clone(), Some(props)).unwrap(); + // Write the entire batch as a single row group. + writer.write(&batch).unwrap(); + writer.close().unwrap(); + } + file +} + +/// Filter function: returns a BooleanArray with true when utf8View <> "". +fn filter_utf8_view_nonempty(batch: &RecordBatch) -> BooleanArray { + let array = batch + .column(batch.schema().index_of("utf8View").unwrap()) + .as_any() + .downcast_ref::<StringViewArray>() + .unwrap(); + let mut builder = BooleanBuilder::with_capacity(array.len()); Review Comment: I think we should use the arrows here, like: https://docs.rs/arrow/latest/arrow/compute/kernels/cmp/fn.eq.html That will: 1. Better model what real systems do 2. Not be a bottleneck for evaluation -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
