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 2022/05/10 08:31:54 UTC

[GitHub] [arrow-datafusion] Ted-Jiang opened a new pull request, #2499: Add metrics for ParquetExec

Ted-Jiang opened a new pull request, #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499

   # Which issue does this PR close?
   add metrics=`[output_rows, elapsed_compute]` in ParquetExec
   
   Got this in this pr
   ```
   ParquetExec: 
   limit=None, 
   partitions=[/Users/yangjiang/test-data/tchp-1g/orders/part-00001-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet, /Users/yangjiang/test-data/tchp-1g/orders/part-00000-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet, /Users/yangjiang/test-data/tchp-1g/orders/part-00002-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet, /Users/yangjiang/test-data/tchp-1g/orders/part-00004-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet, /Users/yangjiang/test-data/tchp-1g/orders/part-00003-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet], 
   projection=[o_totalprice, o_orderdate, o_orderpriority],
   metrics=[output_rows=0, elapsed_compute=924.453343ms, spill_count=0, spilled_bytes=0, mem_used=0, bytes_scanned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00000-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=2411369, predicate_evaluation_errors{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00001-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0, row_groups_pruned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00001-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0, bytes_scanned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00002-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=2411574, num_predicate_creation_errors=0, predicate_evaluation_errors{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00002-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0, bytes_scanned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00001-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=2
 411368, predicate_evaluation_errors{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00000-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0, row_groups_pruned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00002-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0, row_groups_pruned{filename=/Users/yangjiang/test-data/tchp-1g/orders/part-00000-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet}=0]
   ```
   
   Closes #2497.
   
    # 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.  
   -->
   
   # 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.
   -->
   
   # 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 `api 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-datafusion] Ted-Jiang commented on a diff in pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499#discussion_r870003789


##########
datafusion/core/src/physical_plan/file_format/parquet.rs:
##########
@@ -425,7 +437,8 @@ impl Stream for ParquetExecStream {
         mut self: Pin<&mut Self>,
         _cx: &mut Context<'_>,
     ) -> Poll<Option<Self::Item>> {
-        Poll::Ready(Iterator::next(&mut *self))
+        let poll = Poll::Ready(Iterator::next(&mut *self));
+        self.baseline_metrics.record_poll(poll)

Review Comment:
   I think we remove `async` so will never call poll. But i think we should keep this change.



-- 
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-datafusion] alamb commented on a diff in pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499#discussion_r869306017


##########
datafusion/core/src/physical_plan/file_format/parquet.rs:
##########
@@ -227,6 +228,7 @@ impl ExecutionPlan for ParquetExec {
             files: self.base_config.file_groups[partition_index].clone().into(),
             projector: partition_col_proj,
             adapter: SchemaAdapter::new(self.base_config.file_schema.clone()),
+            baseline_metrics: BaselineMetrics::new(&self.metrics, partition_index),
         };
 
         // Use spawn_blocking only if running from a tokio context (#2201)

Review Comment:
   I think the issue is that the `ParquetExecStream` is not yet used by default (#2201) (#2202) by @tustvold 
   
   I think you also have to add code that updates the row count in the code below that creates a `RecordBatchReceiverStream`



-- 
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-datafusion] alamb merged pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
alamb merged PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499


-- 
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-datafusion] Ted-Jiang commented on a diff in pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499#discussion_r870000794


##########
datafusion/core/src/physical_plan/file_format/parquet.rs:
##########
@@ -227,6 +228,7 @@ impl ExecutionPlan for ParquetExec {
             files: self.base_config.file_groups[partition_index].clone().into(),
             projector: partition_col_proj,
             adapter: SchemaAdapter::new(self.base_config.file_schema.clone()),
+            baseline_metrics: BaselineMetrics::new(&self.metrics, partition_index),
         };
 
         // Use spawn_blocking only if running from a tokio context (#2201)

Review Comment:
   @alamb Thanks fix row count in [ada393f](https://github.com/apache/arrow-datafusion/pull/2499/commits/ada393fde7fe9c535d63698d7aeba970a3d8d1d9)
   which get 
   ```
   ParquetExec: limit=None, partitions=[//Users/yangjiang/test-data/tchp-1g/orders/part-00003-e87df013-b3f8-493f-93c2-3da94f34e357-c000.snappy.parquet], projection=[o_totalprice, o_orderdate, o_orderpriority], metrics=[output_rows=899999,elapsed_compute=878.750494ms, spill_count=0,
   ```
   



-- 
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-datafusion] alamb commented on a diff in pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499#discussion_r870249201


##########
datafusion/core/src/physical_plan/file_format/parquet.rs:
##########
@@ -425,7 +437,8 @@ impl Stream for ParquetExecStream {
         mut self: Pin<&mut Self>,
         _cx: &mut Context<'_>,
     ) -> Poll<Option<Self::Item>> {
-        Poll::Ready(Iterator::next(&mut *self))
+        let poll = Poll::Ready(Iterator::next(&mut *self));
+        self.baseline_metrics.record_poll(poll)

Review Comment:
   I think the reason poll is not called is that the new scheduler is not (yet) connected up. I agree we should keep this change. 
   
   cc @tustvold  
   
   



-- 
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-datafusion] Ted-Jiang commented on pull request #2499: Add metrics for ParquetExec

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on PR #2499:
URL: https://github.com/apache/arrow-datafusion/pull/2499#issuecomment-1122096325

   @alamb  plz take a look. Sorry to bother you, one weird thing  i check many sqls, but the output_rows still be zero. Is there something i missed?


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