You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "tustvold (via GitHub)" <gi...@apache.org> on 2023/06/01 09:16:20 UTC

[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4332: feat: add read parquet by custom rowgroup examples

tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212846460


##########
parquet/Cargo.toml:
##########
@@ -29,10 +29,14 @@ edition = { workspace = true }
 rust-version = { workspace = true }
 
 [target.'cfg(target_arch = "wasm32")'.dependencies]
-ahash = { version = "0.8", default-features = false, features = ["compile-time-rng"] }
+ahash = { version = "0.8", default-features = false, features = [

Review Comment:
   Could we not reformat these, I find it much harder to read like this



##########
parquet/examples/read_with_rowgroup.rs:
##########
@@ -0,0 +1,220 @@
+// 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.
+
+use arrow::util::pretty::print_batches;
+use bytes::{Buf, Bytes};
+use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, RowGroups, RowSelection};
+use parquet::arrow::async_reader::AsyncFileReader;
+use parquet::arrow::{parquet_to_arrow_field_levels, ProjectionMask};
+use parquet::column::page::{PageIterator, PageReader};
+use parquet::errors::{ParquetError, Result};
+use parquet::file::metadata::RowGroupMetaData;
+use parquet::file::reader::{ChunkReader, Length};
+use parquet::file::serialized_reader::SerializedPageReader;
+use std::io::{Read, Seek};
+use std::sync::Arc;
+use tokio::fs::File;
+
+#[tokio::main(flavor = "current_thread")]

Review Comment:
   Do we even need tokio for this example?



##########
parquet/examples/read_with_rowgroup.rs:
##########
@@ -0,0 +1,220 @@
+// 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.
+
+use arrow::util::pretty::print_batches;
+use bytes::{Buf, Bytes};
+use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, RowGroups, RowSelection};
+use parquet::arrow::async_reader::AsyncFileReader;
+use parquet::arrow::{parquet_to_arrow_field_levels, ProjectionMask};
+use parquet::column::page::{PageIterator, PageReader};
+use parquet::errors::{ParquetError, Result};
+use parquet::file::metadata::RowGroupMetaData;
+use parquet::file::reader::{ChunkReader, Length};
+use parquet::file::serialized_reader::SerializedPageReader;
+use std::io::{Read, Seek};
+use std::sync::Arc;
+use tokio::fs::File;
+
+#[tokio::main(flavor = "current_thread")]
+async fn main() -> Result<()> {
+    let testdata = arrow::util::test_util::parquet_test_data();
+    let path = format!("{testdata}/alltypes_plain.parquet");
+    let mut file = File::open(&path).await.unwrap();
+
+    // The metadata could be cached in other places, this example only shows how to read
+    let metadata = file.get_metadata().await?;
+
+    for rg in metadata.row_groups() {
+        let mut rowgroup = InMemoryRowGroup::create(rg.clone(), ProjectionMask::all());
+        let mut reader = std::fs::File::open(&path).unwrap();
+        rowgroup.fetch_data(&mut reader, None)?;
+        let reader = rowgroup.build_reader(1024, None)?;
+
+        for batch in reader {
+            let batch = batch?;
+            print_batches(&[batch])?;
+        }
+    }
+
+    Ok(())
+}
+
+/// Implements [`PageIterator`] for a single column chunk, yielding a single [`PageReader`]
+struct ColumnChunkIterator {
+    reader: Option<Result<Box<dyn PageReader>>>,
+}
+
+impl Iterator for ColumnChunkIterator {
+    type Item = Result<Box<dyn PageReader>>;
+
+    fn next(&mut self) -> Option<Self::Item> {
+        self.reader.take()
+    }
+}
+
+impl PageIterator for ColumnChunkIterator {}
+
+/// An in-memory column chunk
+#[derive(Clone)]
+pub enum ColumnChunkData {
+    /// Column chunk data representing only a subset of data pages

Review Comment:
   Could we simplify this given the fetch code doesn't do page pushdown



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org