You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/07/03 11:20:26 UTC

[GitHub] [arrow-datafusion] alamb commented on a change in pull request #657: Add End-to-end test for parquet pruning + metrics for ParquetExec

alamb commented on a change in pull request #657:
URL: https://github.com/apache/arrow-datafusion/pull/657#discussion_r663353376



##########
File path: datafusion/src/physical_plan/parquet.rs
##########
@@ -59,6 +61,8 @@ use crate::datasource::datasource::{ColumnStatistics, Statistics};
 use async_trait::async_trait;
 use futures::stream::{Stream, StreamExt};
 
+use super::SQLMetric;

Review comment:
       The changes to this file are to add metrics on the pruning (that are then used in the test)

##########
File path: datafusion/src/test/mod.rs
##########
@@ -251,11 +251,11 @@ pub fn make_timestamps() -> RecordBatch {
     let arr_names = StringArray::from(names);
 
     let schema = Schema::new(vec![
-        Field::new("nanos", arr_nanos.data_type().clone(), false),
-        Field::new("micros", arr_micros.data_type().clone(), false),
-        Field::new("millis", arr_millis.data_type().clone(), false),
-        Field::new("secs", arr_secs.data_type().clone(), false),
-        Field::new("name", arr_names.data_type().clone(), false),
+        Field::new("nanos", arr_nanos.data_type().clone(), true),

Review comment:
       This was a bug I found while using similar code for the end to end test - the actual data has `None` so the schema needs to be marked "nullable"

##########
File path: datafusion/tests/parquet_pruning.rs
##########
@@ -0,0 +1,343 @@
+// 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.
+
+// This file contains an end to end test of parquet pruning. It writes
+// data into a parquet file and then
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, StringArray, TimestampMicrosecondArray, TimestampMillisecondArray,
+        TimestampNanosecondArray, TimestampSecondArray,
+    },
+    datatypes::{Field, Schema},
+    record_batch::RecordBatch,
+    util::pretty::pretty_format_batches,
+};
+use chrono::Duration;
+use datafusion::{
+    physical_plan::{plan_metrics, SQLMetric},
+    prelude::ExecutionContext,
+};
+use hashbrown::HashMap;
+use parquet::{arrow::ArrowWriter, file::properties::WriterProperties};
+use tempfile::NamedTempFile;
+
+#[tokio::test]

Review comment:
       Here are the new end to end tests -- they make an actual parquet file and then run a query against it, validating the pruning metrics -- they currently "pass" but they show there is no actual pruning occuring




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