-
Notifications
You must be signed in to change notification settings - Fork 867
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
3 changed files
with
229 additions
and
12 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,224 @@ | ||
//! Generates a large parquet file containing dictionary encoded data and demonstrates how | ||
//! the page index, and the record skipping API can dramatically improve performance | ||
use arrow::array::{ | ||
Array, ArrayRef, Float64Builder, Int32Builder, StringBuilder, StringDictionaryBuilder, | ||
}; | ||
use arrow::compute::SlicesIterator; | ||
use arrow::datatypes::{DataType, Field, Schema}; | ||
use arrow::record_batch::RecordBatch; | ||
use bytes::Bytes; | ||
use parquet::arrow::arrow_reader::{ArrowReaderOptions, RowSelection}; | ||
use parquet::arrow::{ArrowReader, ArrowWriter, ParquetFileArrowReader, ProjectionMask}; | ||
use parquet::file::properties::{WriterProperties, WriterVersion}; | ||
use rand::{thread_rng, Rng}; | ||
use std::cmp::Ordering; | ||
use std::ops::Range; | ||
use std::sync::Arc; | ||
use std::time::Instant; | ||
|
||
const NUM_ROW_GROUPS: usize = 2; | ||
const ROWS_PER_ROW_GROUP: usize = 100_000; | ||
const ROWS_PER_FILE: usize = ROWS_PER_ROW_GROUP * NUM_ROW_GROUPS; | ||
|
||
fn generate_batch() -> RecordBatch { | ||
let string_dict_t = | ||
DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); | ||
|
||
let schema = Arc::new(Schema::new(vec![ | ||
Field::new("dict1", string_dict_t.clone(), true), | ||
Field::new("dict2", string_dict_t, true), | ||
Field::new("f64_values", DataType::Float64, true), | ||
])); | ||
|
||
let mut dict1 = StringDictionaryBuilder::new( | ||
Int32Builder::new(ROWS_PER_FILE), | ||
StringBuilder::new(1024), | ||
); | ||
let mut dict2 = StringDictionaryBuilder::new( | ||
Int32Builder::new(ROWS_PER_FILE), | ||
StringBuilder::new(1024), | ||
); | ||
let mut values = Float64Builder::new(ROWS_PER_FILE); | ||
|
||
let dict = &["key0", "key1", "key2", "key3", "key4", "key5"]; | ||
let mut rng = thread_rng(); | ||
for _ in 0..ROWS_PER_FILE { | ||
match rng.gen_bool(0.5) { | ||
true => { | ||
dict1.append(&dict[rng.gen_range(0..dict.len())]).unwrap(); | ||
} | ||
false => dict1.append_null(), | ||
} | ||
|
||
match rng.gen_bool(0.75) { | ||
true => { | ||
dict2.append(&dict[rng.gen_range(0..dict.len())]).unwrap(); | ||
} | ||
false => dict2.append_null(), | ||
} | ||
|
||
values.append_value(rng.gen()); | ||
} | ||
|
||
RecordBatch::try_new( | ||
schema, | ||
vec![ | ||
Arc::new(dict1.finish()), | ||
Arc::new(dict2.finish()), | ||
Arc::new(values.finish()), | ||
], | ||
) | ||
.unwrap() | ||
} | ||
|
||
fn generate_parquet() -> Vec<u8> { | ||
let mut out = Vec::with_capacity(1024); | ||
|
||
let data = generate_batch(); | ||
|
||
let props = WriterProperties::builder() | ||
.set_writer_version(WriterVersion::PARQUET_2_0) | ||
.set_max_row_group_size(ROWS_PER_ROW_GROUP) | ||
.build(); | ||
let mut writer = ArrowWriter::try_new(&mut out, data.schema(), Some(props)).unwrap(); | ||
|
||
writer.write(&data).unwrap(); | ||
|
||
let metadata = writer.close().unwrap(); | ||
assert_eq!(metadata.row_groups.len(), 2); | ||
assert!(metadata.row_groups[0].columns[0] | ||
.column_index_length | ||
.is_some()); | ||
out | ||
} | ||
|
||
fn evaluate_basic(file: Bytes) -> ArrayRef { | ||
let mut reader = ParquetFileArrowReader::try_new(file).unwrap(); | ||
|
||
let batches: Vec<_> = reader | ||
.get_record_reader(1024) | ||
.unwrap() | ||
.map(|result| { | ||
let batch = result.unwrap(); | ||
|
||
let filter_a = | ||
arrow::compute::eq_dyn_utf8_scalar(&batch.columns()[0], "key0").unwrap(); | ||
let filter_b = | ||
arrow::compute::eq_dyn_utf8_scalar(&batch.columns()[1], "key1").unwrap(); | ||
|
||
let combined = arrow::compute::and(&filter_a, &filter_b).unwrap(); | ||
arrow::compute::filter(&batch.column(2), &combined).unwrap() | ||
}) | ||
.collect(); | ||
|
||
let arrays: Vec<_> = batches.iter().map(|x| x.as_ref()).collect(); | ||
arrow::compute::concat(&arrays).unwrap() | ||
} | ||
|
||
fn selection_from_ranges( | ||
ranges: Vec<Range<usize>>, | ||
total_rows: usize, | ||
) -> Vec<RowSelection> { | ||
let mut selection: Vec<RowSelection> = Vec::with_capacity(ranges.len() * 2); | ||
let mut last_end = 0; | ||
for range in ranges { | ||
let len = range.end - range.start; | ||
|
||
match range.start.cmp(&last_end) { | ||
Ordering::Equal => match selection.last_mut() { | ||
Some(last) => last.row_count += len, | ||
None => selection.push(RowSelection::select(len)), | ||
}, | ||
Ordering::Greater => { | ||
selection.push(RowSelection::skip(range.start - last_end)); | ||
selection.push(RowSelection::select(len)) | ||
} | ||
Ordering::Less => panic!("out of order"), | ||
} | ||
last_end = range.end; | ||
} | ||
|
||
if last_end != total_rows { | ||
selection.push(RowSelection::skip(total_rows - last_end)) | ||
} | ||
|
||
selection | ||
} | ||
|
||
fn evaluate_selection( | ||
mut reader: ParquetFileArrowReader, | ||
column: usize, | ||
key: &str, | ||
) -> Vec<RowSelection> { | ||
let mask = ProjectionMask::leaves(reader.parquet_schema(), [column]); | ||
|
||
let mut range_offset = 0; | ||
let mut ranges = vec![]; | ||
for result in reader.get_record_reader_by_columns(mask, 1024).unwrap() { | ||
let batch = result.unwrap(); | ||
let filter = | ||
arrow::compute::eq_dyn_utf8_scalar(&batch.columns()[0], key).unwrap(); | ||
|
||
let valid = SlicesIterator::new(&filter) | ||
.map(|(start, end)| start + range_offset..end + range_offset); | ||
ranges.extend(valid); | ||
range_offset += batch.num_rows(); | ||
} | ||
|
||
let selection = selection_from_ranges(ranges, range_offset); | ||
|
||
// Sanity check | ||
let total = selection.iter().map(|x| x.row_count).sum::<usize>(); | ||
assert_eq!(total, range_offset); | ||
|
||
selection | ||
} | ||
|
||
fn evaluate_pushdown(file: Bytes) -> ArrayRef { | ||
// TODO: This could also make use of the page index | ||
|
||
let reader = ParquetFileArrowReader::try_new(file.clone()).unwrap(); | ||
let selection = evaluate_selection(reader, 0, "key0"); | ||
|
||
// Perhaps we need a way to keep the provide a selection to an existing reader? | ||
let options = ArrowReaderOptions::default().with_row_selection(selection); | ||
let reader = | ||
ParquetFileArrowReader::try_new_with_options(file.clone(), options).unwrap(); | ||
let selection = evaluate_selection(reader, 1, "key1"); | ||
|
||
let total_rows = selection | ||
.iter() | ||
.filter_map(|x| (!x.skip).then(|| x.row_count)) | ||
.sum::<usize>(); | ||
|
||
let options = ArrowReaderOptions::default().with_row_selection(selection); | ||
let mut reader = ParquetFileArrowReader::try_new_with_options(file, options).unwrap(); | ||
let mask = ProjectionMask::leaves(reader.parquet_schema(), [2]); | ||
|
||
let batch = reader | ||
.get_record_reader_by_columns(mask, total_rows) | ||
.unwrap() | ||
.next() | ||
.unwrap() | ||
.unwrap(); | ||
|
||
batch.column(0).clone() | ||
} | ||
|
||
fn main() { | ||
let data: Bytes = generate_parquet().into(); | ||
|
||
let t0 = Instant::now(); | ||
let basic = evaluate_basic(data.clone()); | ||
let t1 = Instant::now(); | ||
let complex = evaluate_pushdown(data); | ||
let t2 = Instant::now(); | ||
|
||
assert_eq!(basic.data(), complex.data()); | ||
println!( | ||
"Simple strategy took {}s vs {}s", | ||
(t1 - t0).as_secs_f64(), | ||
(t2 - t1).as_secs_f64() | ||
); | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters