Skip to content

Commit 80c1978

Browse files
authored
Merge 38e2225 into aac1844
2 parents aac1844 + 38e2225 commit 80c1978

File tree

18 files changed

+570
-58
lines changed

18 files changed

+570
-58
lines changed

.github/workflows/rust.yml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ jobs:
111111
cargo test
112112
cd arrow
113113
# re-run tests on arrow workspace with additional features
114-
cargo test --features=prettyprint
114+
cargo test --features=prettyprint --features=async
115115
# run test on arrow with minimal set of features
116116
cargo test --no-default-features
117117
cargo run --example builders
@@ -238,7 +238,7 @@ jobs:
238238
run: |
239239
export CARGO_HOME="/github/home/.cargo"
240240
export CARGO_TARGET_DIR="/github/home/target"
241-
cargo clippy --features test_common --all-targets --workspace -- -D warnings -A clippy::redundant_field_names
241+
cargo clippy --features test_common --features prettyprint --features=async --all-targets --workspace -- -D warnings -A clippy::redundant_field_names
242242
243243
check_benches:
244244
name: Check Benchmarks (but don't run them)

arrow/src/util/pretty.rs

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ fn create_table(results: &[RecordBatch]) -> Result<Table> {
7474
let mut cells = Vec::new();
7575
for col in 0..batch.num_columns() {
7676
let column = batch.column(col);
77-
cells.push(Cell::new(&array_value_to_string(&column, row)?));
77+
cells.push(Cell::new(&array_value_to_string(column, row)?));
7878
}
7979
table.add_row(cells);
8080
}
@@ -96,7 +96,7 @@ fn create_column(field: &str, columns: &[ArrayRef]) -> Result<Table> {
9696

9797
for col in columns {
9898
for row in 0..col.len() {
99-
let cells = vec![Cell::new(&array_value_to_string(&col, row)?)];
99+
let cells = vec![Cell::new(&array_value_to_string(col, row)?)];
100100
table.add_row(cells);
101101
}
102102
}
@@ -320,7 +320,7 @@ mod tests {
320320
let mut builder = FixedSizeBinaryBuilder::new(3, 3);
321321

322322
builder.append_value(&[1, 2, 3]).unwrap();
323-
builder.append_null();
323+
builder.append_null().unwrap();
324324
builder.append_value(&[7, 8, 9]).unwrap();
325325

326326
let array = Arc::new(builder.finish());
@@ -677,7 +677,7 @@ mod tests {
677677
)?;
678678

679679
let mut buf = String::new();
680-
write!(&mut buf, "{}", pretty_format_batches(&[batch])?.to_string()).unwrap();
680+
write!(&mut buf, "{}", pretty_format_batches(&[batch])?).unwrap();
681681

682682
let s = vec![
683683
"+---+-----+",
@@ -689,7 +689,7 @@ mod tests {
689689
"| d | 100 |",
690690
"+---+-----+",
691691
];
692-
let expected = String::from(s.join("\n"));
692+
let expected = s.join("\n");
693693
assert_eq!(expected, buf);
694694

695695
Ok(())

parquet/Cargo.toml

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ description = "Apache Parquet implementation in Rust"
2323
homepage = "https://github.com/apache/arrow-rs"
2424
repository = "https://github.com/apache/arrow-rs"
2525
authors = ["Apache Arrow <[email protected]>"]
26-
keywords = [ "arrow", "parquet", "hadoop" ]
26+
keywords = ["arrow", "parquet", "hadoop"]
2727
readme = "README.md"
2828
build = "build.rs"
2929
edition = "2021"
@@ -45,6 +45,8 @@ base64 = { version = "0.13", optional = true }
4545
clap = { version = "2.33.3", optional = true }
4646
serde_json = { version = "1.0", features = ["preserve_order"], optional = true }
4747
rand = "0.8"
48+
futures = { version = "0.3", optional = true }
49+
tokio = { version = "1.0", optional = true, default-features = false, features = ["macros", "fs", "rt", "io-util"] }
4850

4951
[dev-dependencies]
5052
criterion = "0.3"
@@ -55,24 +57,26 @@ brotli = "3.3"
5557
flate2 = "1.0"
5658
lz4 = "1.23"
5759
serde_json = { version = "1.0", features = ["preserve_order"] }
58-
arrow = { path = "../arrow", version = "8.0.0", default-features = false, features = ["test_utils"] }
60+
arrow = { path = "../arrow", version = "8.0.0", default-features = false, features = ["test_utils", "prettyprint"] }
5961

6062
[features]
6163
default = ["arrow", "snap", "brotli", "flate2", "lz4", "zstd", "base64"]
6264
cli = ["serde_json", "base64", "clap"]
6365
test_common = []
6466
# Experimental, unstable functionality primarily used for testing
6567
experimental = []
68+
# Enable async API
69+
async = ["futures", "tokio"]
6670

67-
[[ bin ]]
71+
[[bin]]
6872
name = "parquet-read"
6973
required-features = ["cli"]
7074

71-
[[ bin ]]
75+
[[bin]]
7276
name = "parquet-schema"
7377
required-features = ["cli"]
7478

75-
[[ bin ]]
79+
[[bin]]
7680
name = "parquet-rowcount"
7781
required-features = ["cli"]
7882

parquet/src/arrow/array_reader.rs

Lines changed: 38 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ use arrow::datatypes::{
4242
Float32Type as ArrowFloat32Type, Float64Type as ArrowFloat64Type,
4343
Int16Type as ArrowInt16Type, Int32Type as ArrowInt32Type,
4444
Int64Type as ArrowInt64Type, Int8Type as ArrowInt8Type, IntervalUnit, Schema,
45-
Time32MillisecondType as ArrowTime32MillisecondType,
45+
SchemaRef, Time32MillisecondType as ArrowTime32MillisecondType,
4646
Time32SecondType as ArrowTime32SecondType,
4747
Time64MicrosecondType as ArrowTime64MicrosecondType,
4848
Time64NanosecondType as ArrowTime64NanosecondType, TimeUnit as ArrowTimeUnit,
@@ -91,7 +91,7 @@ pub use byte_array::make_byte_array_reader;
9191
pub use byte_array_dictionary::make_byte_array_dictionary_reader;
9292

9393
/// Array reader reads parquet data into arrow array.
94-
pub trait ArrayReader {
94+
pub trait ArrayReader: Send {
9595
fn as_any(&self) -> &dyn Any;
9696

9797
/// Returns the arrow type of this array reader.
@@ -117,6 +117,26 @@ pub trait ArrayReader {
117117
fn get_rep_levels(&self) -> Option<&[i16]>;
118118
}
119119

120+
/// A collection of row groups
121+
pub trait RowGroupCollection {
122+
/// Get schema of parquet file.
123+
fn schema(&self) -> Result<SchemaDescPtr>;
124+
125+
/// Returns an iterator over the column chunks for particular column
126+
fn column_chunks(&self, i: usize) -> Result<Box<dyn PageIterator>>;
127+
}
128+
129+
impl RowGroupCollection for Arc<dyn FileReader> {
130+
fn schema(&self) -> Result<SchemaDescPtr> {
131+
Ok(self.metadata().file_metadata().schema_descr_ptr())
132+
}
133+
134+
fn column_chunks(&self, column_index: usize) -> Result<Box<dyn PageIterator>> {
135+
let iterator = FilePageIterator::new(column_index, Arc::clone(self))?;
136+
Ok(Box::new(iterator))
137+
}
138+
}
139+
120140
/// Uses `record_reader` to read up to `batch_size` records from `pages`
121141
///
122142
/// Returns the number of records read, which can be less than batch_size if
@@ -482,7 +502,7 @@ where
482502
impl<T, C> ArrayReader for ComplexObjectArrayReader<T, C>
483503
where
484504
T: DataType,
485-
C: Converter<Vec<Option<T::T>>, ArrayRef> + 'static,
505+
C: Converter<Vec<Option<T::T>>, ArrayRef> + Send + 'static,
486506
{
487507
fn as_any(&self) -> &dyn Any {
488508
self
@@ -1315,9 +1335,9 @@ impl ArrayReader for StructArrayReader {
13151335
/// Create array reader from parquet schema, column indices, and parquet file reader.
13161336
pub fn build_array_reader<T>(
13171337
parquet_schema: SchemaDescPtr,
1318-
arrow_schema: Schema,
1338+
arrow_schema: SchemaRef,
13191339
column_indices: T,
1320-
file_reader: Arc<dyn FileReader>,
1340+
row_groups: Box<dyn RowGroupCollection>,
13211341
) -> Result<Box<dyn ArrayReader>>
13221342
where
13231343
T: IntoIterator<Item = usize>,
@@ -1355,13 +1375,8 @@ where
13551375
fields: filtered_root_fields,
13561376
};
13571377

1358-
ArrayReaderBuilder::new(
1359-
Arc::new(proj),
1360-
Arc::new(arrow_schema),
1361-
Arc::new(leaves),
1362-
file_reader,
1363-
)
1364-
.build_array_reader()
1378+
ArrayReaderBuilder::new(Arc::new(proj), arrow_schema, Arc::new(leaves), row_groups)
1379+
.build_array_reader()
13651380
}
13661381

13671382
/// Used to build array reader.
@@ -1371,7 +1386,7 @@ struct ArrayReaderBuilder {
13711386
// Key: columns that need to be included in final array builder
13721387
// Value: column index in schema
13731388
columns_included: Arc<HashMap<*const Type, usize>>,
1374-
file_reader: Arc<dyn FileReader>,
1389+
row_groups: Box<dyn RowGroupCollection>,
13751390
}
13761391

13771392
/// Used in type visitor.
@@ -1671,13 +1686,13 @@ impl<'a> ArrayReaderBuilder {
16711686
root_schema: TypePtr,
16721687
arrow_schema: Arc<Schema>,
16731688
columns_included: Arc<HashMap<*const Type, usize>>,
1674-
file_reader: Arc<dyn FileReader>,
1689+
file_reader: Box<dyn RowGroupCollection>,
16751690
) -> Self {
16761691
Self {
16771692
root_schema,
16781693
arrow_schema,
16791694
columns_included,
1680-
file_reader,
1695+
row_groups: file_reader,
16811696
}
16821697
}
16831698

@@ -1711,10 +1726,10 @@ impl<'a> ArrayReaderBuilder {
17111726
context.rep_level,
17121727
context.path.clone(),
17131728
));
1714-
let page_iterator = Box::new(FilePageIterator::new(
1715-
self.columns_included[&(cur_type.as_ref() as *const Type)],
1716-
self.file_reader.clone(),
1717-
)?);
1729+
1730+
let page_iterator = self
1731+
.row_groups
1732+
.column_chunks(self.columns_included[&(cur_type.as_ref() as *const Type)])?;
17181733

17191734
let arrow_type: Option<ArrowType> = self
17201735
.get_arrow_field(&cur_type, context)
@@ -2827,7 +2842,8 @@ mod tests {
28272842
#[test]
28282843
fn test_create_array_reader() {
28292844
let file = get_test_file("nulls.snappy.parquet");
2830-
let file_reader = Arc::new(SerializedFileReader::new(file).unwrap());
2845+
let file_reader: Arc<dyn FileReader> =
2846+
Arc::new(SerializedFileReader::new(file).unwrap());
28312847

28322848
let file_metadata = file_reader.metadata().file_metadata();
28332849
let arrow_schema = parquet_to_arrow_schema(
@@ -2838,9 +2854,9 @@ mod tests {
28382854

28392855
let array_reader = build_array_reader(
28402856
file_reader.metadata().file_metadata().schema_descr_ptr(),
2841-
arrow_schema,
2857+
Arc::new(arrow_schema),
28422858
vec![0usize].into_iter(),
2843-
file_reader,
2859+
Box::new(file_reader),
28442860
)
28452861
.unwrap();
28462862

parquet/src/arrow/arrow_reader.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -144,9 +144,9 @@ impl ArrowReader for ParquetFileArrowReader {
144144
.metadata()
145145
.file_metadata()
146146
.schema_descr_ptr(),
147-
self.get_schema()?,
147+
Arc::new(self.get_schema()?),
148148
column_indices,
149-
self.file_reader.clone(),
149+
Box::new(self.file_reader.clone()),
150150
)?;
151151

152152
ParquetRecordBatchReader::try_new(batch_size, array_reader)

0 commit comments

Comments
 (0)