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

[GitHub] [arrow-rs] sundy-li opened a new pull request, #4332: feat: add parquet read by custom rowgroup examples

sundy-li opened a new pull request, #4332:
URL: https://github.com/apache/arrow-rs/pull/4332

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Continue #4307.
   
   # Rationale for this change
    
   <!--
   Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
   Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.
   -->
   
   @alamb  said: I also suggest someone write up a tested example of using these new APIs so they don't get inadvertently broken during some future refactor
   
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   add parquet read by custom rowgroup examples
   
   # Are there any user-facing changes?
   
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!---
   If there are any breaking changes to public APIs, please add the `breaking change` label.
   -->
   


-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212950401


##########
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?;

Review Comment:
   Oh I see, fair enough



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212962473


##########
parquet/examples/read_with_rowgroup.rs:
##########
@@ -40,8 +39,8 @@ async fn main() -> Result<()> {
 
     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 mut reader = File::open(&path).await.unwrap();
+        rowgroup.async_fetch_data(&mut reader, None).await?;

Review Comment:
   ```suggestion
           rowgroup.async_fetch_data(&mut file, None).await?;
   ```



-- 
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


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

Posted by "sundy-li (via GitHub)" <gi...@apache.org>.
sundy-li commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212947750


##########
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?;

Review Comment:
   it's the trait method of `AsyncFileReader`



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212938126


##########
parquet/Cargo.toml:
##########
@@ -111,6 +162,11 @@ name = "async_read_parquet"
 required-features = ["arrow", "async"]
 path = "./examples/async_read_parquet.rs"
 
+[[example]]
+name = "read_with_rowgroup"
+required-features = ["arrow", "async"]

Review Comment:
   It looks like this isn't getting tested in CI



-- 
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


[GitHub] [arrow-rs] tustvold merged pull request #4332: feat: add read parquet by custom rowgroup examples

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold merged PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332


-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212938126


##########
parquet/Cargo.toml:
##########
@@ -111,6 +162,11 @@ name = "async_read_parquet"
 required-features = ["arrow", "async"]
 path = "./examples/async_read_parquet.rs"
 
+[[example]]
+name = "read_with_rowgroup"
+required-features = ["arrow", "async"]

Review Comment:
   It looks like this isn't getting tested in CI
   
   Edit: nvm



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#issuecomment-1571875795

   Thank you


-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212953365


##########
parquet/examples/read_with_rowgroup.rs:
##########
@@ -0,0 +1,197 @@
+// 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)?;

Review Comment:
   Given we are using AsyncFileReader above we could use https://docs.rs/parquet/latest/parquet/arrow/async_reader/trait.AsyncFileReader.html#tymethod.get_bytes here perhaps



-- 
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


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

Posted by "sundy-li (via GitHub)" <gi...@apache.org>.
sundy-li commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212933690


##########
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:
   seems the `read_metadata` method needs async.



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212937146


##########
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?;

Review Comment:
   This appears to be calling tokio::fs::File::get_metadata not a parquet method?



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212936217


##########
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:
   https://docs.rs/parquet/latest/parquet/file/footer/fn.parse_metadata.html is the sync equivalent



-- 
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


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

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4332:
URL: https://github.com/apache/arrow-rs/pull/4332#discussion_r1212952603


##########
parquet/examples/read_with_rowgroup.rs:
##########
@@ -0,0 +1,197 @@
+// 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 struct ColumnChunkData {
+    offset: usize,
+    data: Bytes,
+}
+
+impl ColumnChunkData {
+    fn get(&self, start: u64) -> Result<Bytes> {
+        let start = start as usize - self.offset;
+        Ok(self.data.slice(start..))
+    }
+}
+
+impl Length for ColumnChunkData {
+    fn len(&self) -> u64 {
+        self.data.len() as u64
+    }
+}
+
+impl ChunkReader for ColumnChunkData {
+    type T = bytes::buf::Reader<Bytes>;
+
+    fn get_read(&self, start: u64) -> Result<Self::T> {
+        Ok(self.get(start)?.reader())
+    }
+
+    fn get_bytes(&self, start: u64, length: usize) -> Result<Bytes> {
+        Ok(self.get(start)?.slice(..length))
+    }
+}
+
+#[derive(Clone)]
+pub struct InMemoryRowGroup {
+    pub metadata: RowGroupMetaData,
+    mask: ProjectionMask,
+    column_chunks: Vec<Option<Arc<ColumnChunkData>>>,
+}
+
+impl RowGroups for InMemoryRowGroup {
+    fn num_rows(&self) -> usize {
+        self.metadata.num_rows() as usize
+    }
+
+    fn column_chunks(&self, i: usize) -> Result<Box<dyn PageIterator>> {
+        match &self.column_chunks[i] {
+            None => Err(ParquetError::General(format!(
+                "Invalid column index {i}, column was not fetched"
+            ))),
+            Some(data) => {
+                let page_reader: Box<dyn PageReader> =
+                    Box::new(SerializedPageReader::new(
+                        data.clone(),
+                        self.metadata.column(i),
+                        self.num_rows(),
+                        None,
+                    )?);
+
+                Ok(Box::new(ColumnChunkIterator {
+                    reader: Some(Ok(page_reader)),
+                }))
+            }
+        }
+    }
+}
+
+impl InMemoryRowGroup {
+    pub fn create(metadata: RowGroupMetaData, mask: ProjectionMask) -> Self {
+        let column_chunks = metadata.columns().iter().map(|_| None).collect::<Vec<_>>();
+
+        Self {
+            metadata,
+            mask,
+            column_chunks,
+        }
+    }
+
+    pub fn build_reader(
+        &self,
+        batch_size: usize,
+        selection: Option<RowSelection>,
+    ) -> Result<ParquetRecordBatchReader> {
+        let levels = parquet_to_arrow_field_levels(
+            &self.metadata.schema_descr_ptr(),
+            self.mask.clone(),
+            None,
+        )?;
+
+        ParquetRecordBatchReader::try_new_with_row_groups(
+            &levels, self, batch_size, selection,
+        )
+    }
+
+    /// fetch data from a reader in sync mode
+    pub fn fetch_data<R: Read + Seek>(
+        &mut self,
+        reader: &mut R,
+        _selection: Option<&RowSelection>,
+    ) -> Result<()> {
+        let mut vs = std::mem::take(&mut self.column_chunks);
+        for (leaf_idx, meta) in self.metadata.columns().iter().enumerate() {
+            if self.mask.leaf_included(leaf_idx) {
+                let (start, len) = meta.byte_range();
+                reader.seek(std::io::SeekFrom::Start(start))?;
+                let mut chunk = vec![0; len as usize];
+                reader.read_exact(&mut chunk)?;
+
+                vs[leaf_idx] = Some(Arc::new(ColumnChunkData {
+                    offset: start as usize,
+                    data: Bytes::from(chunk),
+                }));
+            }
+        }
+        self.column_chunks = std::mem::take(&mut vs);
+        Ok(())
+    }
+
+    /// fetch data from a reader in async mode
+    pub fn async_fetch_data<R: Read + Seek>(

Review Comment:
   Perhaps we could either consistently use sync or async IO instead of mixing the two together?



-- 
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