jorgecarleitao commented on a change in pull request #9086:
URL: https://github.com/apache/arrow/pull/9086#discussion_r551054748



##########
File path: rust/datafusion/src/physical_plan/filter.rs
##########
@@ -103,25 +103,23 @@ impl ExecutionPlan for FilterExec {
     }
 
     async fn execute(&self, partition: usize) -> 
Result<SendableRecordBatchStream> {
-        Ok(Box::pin(FilterExecStream {
-            schema: self.input.schema().clone(),
-            predicate: self.predicate.clone(),
-            input: self.input.execute(partition).await?,
-        }))
+        let predicate = self.predicate.clone();
+
+        let stream = self.input.execute(partition).await?;
+        let stream = stream.then(move |batch| {
+            let predicate = predicate.clone();
+            async move {
+                // Filtering batches is CPU-bounded and therefore justifies a 
dedicated thread pool
+                task::spawn_blocking(move || batch_filter(&batch?, &predicate))
+                    .await
+                    .unwrap()
+            }
+        });
+
+        Ok(Box::pin(SizedRecordBatchStream::new(stream, self.schema())))

Review comment:
       This is the core idea: wrap the input stream with a blocking thread that 
runs `batch_filter` (the offending blocking code).

##########
File path: rust/datafusion/src/physical_plan/filter.rs
##########
@@ -103,25 +103,23 @@ impl ExecutionPlan for FilterExec {
     }
 
     async fn execute(&self, partition: usize) -> 
Result<SendableRecordBatchStream> {
-        Ok(Box::pin(FilterExecStream {
-            schema: self.input.schema().clone(),
-            predicate: self.predicate.clone(),
-            input: self.input.execute(partition).await?,
-        }))
+        let predicate = self.predicate.clone();
+
+        let stream = self.input.execute(partition).await?;
+        let stream = stream.then(move |batch| {
+            let predicate = predicate.clone();
+            async move {
+                // Filtering batches is CPU-bounded and therefore justifies a 
dedicated thread pool
+                task::spawn_blocking(move || batch_filter(&batch?, &predicate))
+                    .await
+                    .unwrap()
+            }
+        });
+
+        Ok(Box::pin(SizedRecordBatchStream::new(stream, self.schema())))

Review comment:
       This is the core idea: wrap the input stream with a blocking task that 
runs `batch_filter` (the offending blocking code).




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

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


Reply via email to