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 2020/12/31 21:54:33 UTC

[GitHub] [arrow] yordan-pavlov opened a new pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

yordan-pavlov opened a new pull request #9064:
URL: https://github.com/apache/arrow/pull/9064


   While profiling a DataFusion query I found that the code spends a lot of time in reading data from parquet files. Predicate / filter push-down is a commonly used performance optimization, where statistics data stored in parquet files (such as min / max values for columns in a parquet row group) is evaluated against query filters to determine which row groups could contain data requested by a query. In this way, by pushing down query filters all the way to the parquet data source, entire row groups or even parquet files can be skipped often resulting in significant performance improvements.
   
   I have been working on an implementation for a few weeks and initial results look promising - with predicate push-down, DataFusion is now faster than Apache Spark (`140ms for DataFusion vs 200ms for Spark`) for the same query against the same parquet files.
   
   My work is based on the following key ideas:
   * predicate-push down is implemented by filtering row group metadata entries to only those which could contain data which could satisfy query filters
   * it's best to reuse the existing code for evaluating physical expressions already implemented in DataFusion
   * filter expressions pushed down to a parquet table are rewritten to use parquet statistics (instead of the actual column data), for example `(column / 2) = 4`  becomes  `(column_min / 2) <= 4 && 4 <= (column_max / 2)` - this is done once for all files in a parquet table
   * for each parquet file, a RecordBatch containing all required statistics columns ( [`column_min`, `column_max`] in the example above) is produced, and the predicate expression from the previous step is evaluated, producing a binary array which is finally used to filter the row groups in each parquet file
   
   This is still work in progress - more tests left to write; I am publishing this now to gather feedback.
   
   @andygrove let me know what you think


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] andygrove commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
andygrove commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551040361



##########
File path: rust/datafusion/src/datasource/parquet.rs
##########
@@ -62,17 +64,37 @@ impl TableProvider for ParquetTable {
         self.schema.clone()
     }
 
+    fn supports_filter_pushdown(
+        &self,
+        _filter: &Expr,
+    ) -> Result<TableProviderFilterPushDown> {
+        Ok(TableProviderFilterPushDown::Inexact)
+    }
+
     /// Scan the file(s), using the provided projection, and return one BatchIterator per
     /// partition.
     fn scan(
         &self,
         projection: &Option<Vec<usize>>,
         batch_size: usize,
-        _filters: &[Expr],
+        filters: &[Expr],
     ) -> Result<Arc<dyn ExecutionPlan>> {
+        let predicate = if filters.is_empty() {
+            None
+        } else {
+            Some(
+                filters

Review comment:
       Immediately after posting that comment I see how it works now, but I think a comment would still be helpful




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] sunchao commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551680435



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       What I was thinking is that, we can have another constructor for `SerializedFileReader` which takes a custom metadata:
   ```rust
       pub fn new_with_metadata(chunk_reader: R, metadata: ParquetMetaData) -> Result<Self> {
           Ok(Self {
               chunk_reader: Arc::new(chunk_reader),
               metadata: metadata,
           })
       }
   ```
   and we move the metadata filtering part to data fusion, or a util function in `footer.rs`.
   
   In the long term though, I think we should do something similar to [parquet-mr is doing](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L656), that is, having a `ParquetReadOptions`-like struct which allows user to specify various configs, properties, filters, etc when reading a parquet file. The struct is extendable as well to accommodate new features in future such as filtering with column indexes or bloom filters. The constructor can become like this:
   ```rust
       pub fn new(chunk_reader: R, options: ParquetReadOptions) -> Result<Self> {
           Ok(Self {
               chunk_reader: Arc::new(chunk_reader),
               options: options,
           })
       }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-761773174


   I apologize for the delay in merging Rust PRs -- the 3.0 release is being finalized now and are planning to minimize entropy by postponing merging  changes not critical for the release until the process was complete. I hope the process is complete in the next few days. There is more [discussion](https://lists.apache.org/list.html?dev@arrow.apache.org) in the mailing list 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r554469567



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(

Review comment:
       @alamb I have just pushed a change which removes this repetition and makes that part of the code cleaner; not long to go now, some more tests to add for the execution of the row group predicate in the next couple of days and this work should be ready to merge




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753214950


   https://issues.apache.org/jira/browse/ARROW-11074


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551621749



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       there is another possibility - I have just noticed `FilePageIterator::with_row_groups` which could be used to filter row groups based on a list of row group indexes; this could replace the `filter_row_groups` method but would require the row group indexes to be passed down all the way to `build_for_primitive_type_inner` where `FilePageIterator` is created; this could be done through a new field in `ArrayReaderBuilderContext`.
   It's a deeper change but would mean that `filter_row_groups` method is no longer necessary. @sunchao  do you think this would be a better way to go about filtering of row groups?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551063977



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {

Review comment:
       done




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r556752615



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       @jorgecarleitao actually what is wrote in my previous comment is incorrect - boolean is valid statistics type; although in most cases I suspect that it may not provide very helpful statistics (because it only has two values - true and false); anyway I will look into a better implementation for the `build_statistics_array` method and support for more types, but probably in a separate PR as this one is already fairly large




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r558505683



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -950,9 +944,88 @@ mod tests {
             .downcast_ref::<Int32Array>()
             .unwrap();
         let int32_vec = int32_array.into_iter().collect::<Vec<_>>();
+        // here the first max value is None and not the Some(10) value which was actually set
+        // because the min value is None
         assert_eq!(int32_vec, vec![None, Some(20), Some(30)]);
     }
 
+    #[test]
+    fn build_statistics_array_utf8() {
+        // build row group metadata array
+        let s1 = ParquetStatistics::byte_array(None, Some("10".into()), None, 0, false);
+        let s2 = ParquetStatistics::byte_array(
+            Some("2".into()),
+            Some("20".into()),
+            None,
+            0,
+            false,
+        );
+        let s3 = ParquetStatistics::byte_array(
+            Some("3".into()),
+            Some("30".into()),
+            None,
+            0,
+            false,
+        );
+        let statistics = vec![Some(&s1), Some(&s2), Some(&s3)];
+
+        let statistics_array =
+            build_statistics_array(&statistics, StatisticsType::Min, &DataType::Utf8);
+        let string_array = statistics_array
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .unwrap();
+        let string_vec = string_array.into_iter().collect::<Vec<_>>();
+        assert_eq!(string_vec, vec![None, Some("2"), Some("3")]);
+
+        let statistics_array =
+            build_statistics_array(&statistics, StatisticsType::Max, &DataType::Utf8);
+        let string_array = statistics_array
+            .as_any()
+            .downcast_ref::<StringArray>()
+            .unwrap();
+        let string_vec = string_array.into_iter().collect::<Vec<_>>();
+        // here the first max value is None and not the Some("10") value which was actually set

Review comment:
       👍 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551156780



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),

Review comment:
       I would error or `panic!` here or before that, or validate that the predicate is a boolean array.

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(

Review comment:
       I would have split this in N functions, one per array type (via generics), and write `build_statistics_array` as simply `match data_type { each implementation }`.
   
   This would follow the convention in other places and reduces the risk of mistakes, particularly in matching datatypes.

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       This is only valid for primitive types. In general, I would recommend using `PrimitiveArray<T>::from_iter`, `BooleanArray::from_iter` and `StringArray::from_iter`.  Using `MutableBuffer` in this high level is prone to errors. E.g. if we add a filter for boolean types (e.g. eq and neq), this does not panic but the array is not valid (as the size is measured in bits, not bytes).

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {

Review comment:
       I would use a match, as this is a bit brittle against matching specific datatypes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551587504



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),

Review comment:
       My thinking in designing this has been that pushing the predicate down to parquet is optional, because even if it fails the query will still compute, just slower; because of that the code tries to avoid panicking and instead returns a predicate which returns true - it doesn't filter any row groups and lets them be processed by downstream operators. 
   It is even possible to have a partial predicate expression, where multiple conditions are joined using a logical `AND`, and some of them can't be translated for some reason to physical expressions, they will be replaced by `true`, but the rest will still be evaluated and could still filter some row groups.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io edited a comment on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (a03ccf9) into [master](https://codecov.io/gh/apache/arrow/commit/dfef236f7587e4168ac1e07bd09e42d9373beb70?el=desc) (dfef236) will **decrease** coverage by `0.34%`.
   > The diff coverage is `26.19%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.60%   82.25%   -0.35%     
   ==========================================
     Files         204      204              
     Lines       50189    50478     +289     
   ==========================================
   + Hits        41459    41522      +63     
   - Misses       8730     8956     +226     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `45.73% <22.30%> (-39.15%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `91.54% <36.36%> (-4.67%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `75.71% <100.00%> (+0.36%)` | :arrow_up: |
   | [rust/arrow/src/record\_batch.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvcmVjb3JkX2JhdGNoLnJz) | `73.83% <0.00%> (-2.24%)` | :arrow_down: |
   | [rust/arrow/src/array/array\_struct.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfc3RydWN0LnJz) | `88.43% <0.00%> (-0.18%)` | :arrow_down: |
   | [rust/arrow/src/array/equal/mod.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvZXF1YWwvbW9kLnJz) | `92.69% <0.00%> (+0.37%)` | :arrow_up: |
   | [rust/arrow/src/array/array\_primitive.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfcHJpbWl0aXZlLnJz) | `92.82% <0.00%> (+0.49%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [dfef236...a03ccf9](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r552200495



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       > however I think this falls outside of the scope of this PR;
   
   I agree -- this is already a large enough PR (and important enough). If we need to add some non ideal api to parquet and then upgrade it later I think that is the better approach. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r552147401



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       the second option, with the `ParquetReadOptions` parameter, sounds better (compared to the `new_with_metadata` method) - more extensible as you have described; however I think this falls outside of the scope of this PR




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io edited a comment on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (5d7dda7) into [master](https://codecov.io/gh/apache/arrow/commit/fdf5e88a67f33c0a76673a32938274f063c9cb93?el=desc) (fdf5e88) will **decrease** coverage by `0.82%`.
   > The diff coverage is `71.20%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.57%   81.75%   -0.83%     
   ==========================================
     Files         204      214      +10     
     Lines       50327    51751    +1424     
   ==========================================
   + Hits        41560    42307     +747     
   - Misses       8767     9444     +677     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/operators.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vb3BlcmF0b3JzLnJz) | `75.00% <ø> (ø)` | |
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `74.60% <70.00%> (-10.02%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `95.51% <92.00%> (-0.70%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `72.09% <100.00%> (+0.40%)` | :arrow_up: |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `78.51% <0.00%> (-1.49%)` | :arrow_down: |
   | [rust/arrow/src/util/integration\_util.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvdXRpbC9pbnRlZ3JhdGlvbl91dGlsLnJz) | `66.73% <0.00%> (-0.75%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/expressions.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2V4cHJlc3Npb25zLnJz) | `83.77% <0.00%> (-0.71%)` | :arrow_down: |
   | [rust/datafusion/src/sql/utils.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9zcWwvdXRpbHMucnM=) | `53.92% <0.00%> (-0.68%)` | :arrow_down: |
   | [rust/datafusion/src/optimizer/utils.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9vcHRpbWl6ZXIvdXRpbHMucnM=) | `58.18% <0.00%> (-0.54%)` | :arrow_down: |
   | ... and [33 more](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [fdf5e88...5d7dda7](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551588681



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {

Review comment:
       this may not be very idiomatic Rust, but allows the code to handle this single special case separately




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551597243



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       Thank you for your feedback. I was looking for a (mostly) generic approach to building statistics arrays and this is the simplest implementation I could come up with. Using `MutableBuffer` may be prone to errors, but I have added a test to confirm it's working. Your questions make me think if this could be done with generics though.
   From what I have seen, parquet statistics are only stored as Int32, Int64, Float, Double or ByteArray (used for strings and other complex types); may be someone with more experience with parquet can advise on how statistics work for boolean columns.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-761136953


   Sounds good. I'll plan to merge it once master is opened for 4.0 commits (eta tomorrow). Thanks again


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551621749



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       there is another possibility - I have just noticed `FilePageIterator::with_row_groups` which could be used to filter row groups based on a list of row group indexes; this could replace the `filter_row_groups` method but would require the row group indexes to be passed down all the way to `build_for_primitive_type_inner` where `FilePageIterator` is created; this could be done through a new field in `ArrayReaderBuilderContext`.
   It's a deeper change but would mean that `filter_row_groups` method is no longer necessary. @sunchao  do you think this would be a better way to go about filtering of row groups? I am not sure the complexity is worth it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] andygrove commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
andygrove commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551040214



##########
File path: rust/datafusion/src/datasource/parquet.rs
##########
@@ -62,17 +64,37 @@ impl TableProvider for ParquetTable {
         self.schema.clone()
     }
 
+    fn supports_filter_pushdown(
+        &self,
+        _filter: &Expr,
+    ) -> Result<TableProviderFilterPushDown> {
+        Ok(TableProviderFilterPushDown::Inexact)
+    }
+
     /// Scan the file(s), using the provided projection, and return one BatchIterator per
     /// partition.
     fn scan(
         &self,
         projection: &Option<Vec<usize>>,
         batch_size: usize,
-        _filters: &[Expr],
+        filters: &[Expr],
     ) -> Result<Arc<dyn ExecutionPlan>> {
+        let predicate = if filters.is_empty() {
+            None
+        } else {
+            Some(
+                filters

Review comment:
       Could you add a comment explaining the logic here? It isn't immediately obvious to me.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r554469567



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(

Review comment:
       @alamb I have just pushed a change which removes this repetition and makes that part of the code cleaner; not long to go now, some more tests to add for the execution of the row group predicate in the next couple of days and this work should be ready to merge




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551525125



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {

Review comment:
       makes sense




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-761122377


   @alamb yes I think this is ready to merge and as you said, already large 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.

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



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r554556262



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(

Review comment:
       Awesome -- thanks @yordan-pavlov  -- I am excited for this one. When it is ready I'll re-review the code and get it merged asap.
   
   Thanks again for introducing this feature. 🎉 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551525987



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {

Review comment:
       I don't feel strongly either way either -- no need to change it




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r556752615



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       @jorgecarleitao actually what I wrote in my previous comment is incorrect - boolean is valid statistics type; although in most cases I suspect that it may not provide very helpful statistics (because it only has two values - true and false); anyway I will look into a better implementation for the `build_statistics_array` method and support for more types, but probably in a separate PR as this one is already fairly large




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-761120500


   @yordan-pavlov  do you think this PR is ready to merge?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] Dandandan commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
Dandandan commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-762951266


   I think this is one of the big features of 4.0 already! Thanks @yordan-pavlov great work


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551006563



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {

Review comment:
       nit: probably this doesn't need to be a `pub` struct given that it seems to be tied to the parquet scan implementation

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max

Review comment:
       these comments are quite helpful. Thank you

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {

Review comment:
       I wonder if you could use `NullArray` here instead: https://github.com/apache/arrow/blob//rust/arrow/src/array/null.rs

##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       this is a fancy way to filter out the row groups -- it is probably worth adding documentation here.
   
   I don't know if there are assumptions in the parquet reader code that the row group metadata matches what was read from the file or not
   
   I suggest you consider filtering the row groups at the DataFusion (aka skip them in the datafusion physical operator) level rather than in the parquet reader level and avoid that potential problem completely. 

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(

Review comment:
       I suggest copying the (nicely written) summary of your algorithm from this PR' description somewhere into this file
   
   It is probably good to mention the assumptions of this predicate expression -- which I think is that it will return `true` if a rowgroup *may* contain rows that match the predicate, and will return `false` if and only if all rows in the row group can *not* match the predicate.
   
   The idea of creating arrays of `(col1_min, col1_max, col2_min, col2_max ...)` is clever (and could likely be applied to sources other than parquet files). 

##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(

Review comment:
       stylistically, you might be able to hoist out the repeated calls to 
   
   ```
   rewrite_column_expr(
                       expr_builder.column_expr(),
                       expr_builder.column_name(),
                       max_column_name.as_str()
   ```
   
   and 
   
   ```
   rewrite_column_expr(
                       expr_builder.column_expr(),
                       expr_builder.column_name(),
                       min_column_name.as_str()
   ```
   
    by evaluating them once before the `match` expression:
   
   ```
   let min_col_expr = rewrite_column_expr(
                       expr_builder.column_expr(),
                       expr_builder.column_name(),
                       min_column_name.as_str());
   
   let max_col_expr = rewrite_column_expr(
                       expr_builder.column_expr(),
                       expr_builder.column_name(),
                       max_column_name.as_str())
   ```
   
   But they way you have it works well too




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] sunchao commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r552258182



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       Yeah I didn't mean we should tackle it here - which is why I said "in the long term" :-)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r556138748



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {

Review comment:
       @alamb  I have now changed the code to use `NullArray` but have had to add a new `new_with_type` constructor function (for the reason explained in my previous comment)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb closed pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb closed pull request #9064:
URL: https://github.com/apache/arrow/pull/9064


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r556752615



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       @jorgecarleitao actually what I wrote in my previous comment is incorrect - boolean is a valid statistics type; although in most cases I suspect that it may not provide very helpful statistics (because it only has two values - true and false); anyway I will look into a better implementation for the `build_statistics_array` method and support for more types, but probably in a separate PR as this one is already fairly large




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r552147401



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       the second option, with the `ParquetReadOptions` parameter, sounds better - more extensible as you have described; however I think this falls outside of the scope of this PR




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551900306



##########
File path: rust/datafusion/src/datasource/parquet.rs
##########
@@ -62,17 +64,37 @@ impl TableProvider for ParquetTable {
         self.schema.clone()
     }
 
+    fn supports_filter_pushdown(
+        &self,
+        _filter: &Expr,
+    ) -> Result<TableProviderFilterPushDown> {
+        Ok(TableProviderFilterPushDown::Inexact)
+    }
+
     /// Scan the file(s), using the provided projection, and return one BatchIterator per
     /// partition.
     fn scan(
         &self,
         projection: &Option<Vec<usize>>,
         batch_size: usize,
-        _filters: &[Expr],
+        filters: &[Expr],

Review comment:
       I agree that starting with this PR and then extending the approach to be something more generic is a good approach.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551574592



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {

Review comment:
       thinking some more about this, it could be done by moving the creation of PredicateExpressionBuilder from `ParquetExec::try_from_files`  into `(ExecutionPlan for ParquetExec)::execute`, but then this work would be repeated for each partition, where as currently it's only done once; at this point I don't think it's worth it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] sunchao commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551558307



##########
File path: rust/datafusion/src/datasource/parquet.rs
##########
@@ -62,17 +64,37 @@ impl TableProvider for ParquetTable {
         self.schema.clone()
     }
 
+    fn supports_filter_pushdown(
+        &self,
+        _filter: &Expr,
+    ) -> Result<TableProviderFilterPushDown> {
+        Ok(TableProviderFilterPushDown::Inexact)
+    }
+
     /// Scan the file(s), using the provided projection, and return one BatchIterator per
     /// partition.
     fn scan(
         &self,
         projection: &Option<Vec<usize>>,
         batch_size: usize,
-        _filters: &[Expr],
+        filters: &[Expr],

Review comment:
       Ideally, I feel we should have a proper filter API defined in data fusion which can be shared among various data sources. On the other hand, the actual filtering logic should be implemented by different data sources / formats, probably via converting the data fusion's filter API to the corresponding ones from the latter. 
   
   But this is a very good start and we can probably do them as follow ups (if we don't care much for API changes).

##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       Yeah I think we can either move this to the application layer (i.e., data fusion), or expose it as a util function from `footer.rs`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (5b2b6f2) into [master](https://codecov.io/gh/apache/arrow/commit/dfef236f7587e4168ac1e07bd09e42d9373beb70?el=desc) (dfef236) will **decrease** coverage by `0.35%`.
   > The diff coverage is `22.74%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.60%   82.25%   -0.36%     
   ==========================================
     Files         204      204              
     Lines       50189    50478     +289     
   ==========================================
   + Hits        41459    41520      +61     
   - Misses       8730     8958     +228     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `45.73% <22.30%> (-39.15%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `91.54% <36.36%> (-4.67%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `75.37% <100.00%> (+0.02%)` | :arrow_up: |
   | [rust/arrow/src/array/array\_primitive.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfcHJpbWl0aXZlLnJz) | `92.82% <0.00%> (+0.49%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [dfef236...5b2b6f2](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] sunchao commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551680435



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       What I was thinking is that, we can have another constructor for `SerializedFileReader` which takes a custom metadata:
   ```rust
       pub fn new_with_metadata(chunk_reader: R, metadata: ParquetMetaData) -> Result<Self> {
           Ok(Self {
               chunk_reader: Arc::new(chunk_reader),
               metadata: metadata,
           })
       }
   ```
   and we move the metadata filtering part to data fusion, or a util function in `footer.rs`.
   
   In the long term though, I think we should do something similar to [parquet-mr is doing](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L656), that is, having a `ParquetReadOptions`-like struct which allows user to specify various configs, properties, filters, etc when reading a parquet file. The struct is extendable as well to accommodate new features in future such as filtering with column indexes or bloom filters, so we don't need to have multiple constructors. The constructor can become like this:
   ```rust
       pub fn new(chunk_reader: R, options: ParquetReadOptions) -> Result<Self> {
           Ok(Self {
               chunk_reader: Arc::new(chunk_reader),
               options: options,
           })
       }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] alamb commented on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-762933045


   Thanks again @yordan-pavlov -- I am totally stoked for this feature


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551597243



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,477 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                *statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: Operator) -> Operator {
+        if !self.reverse_operator {
+            return op;
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op,
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == &statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, *op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == Operator::And || op == Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => (min, max) = literal => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Copy, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {
+    let mut null_bitmap_builder = BooleanBufferBuilder::new(length);
+    null_bitmap_builder.append_n(length, false);
+    let array_data = ArrayData::builder(data_type.clone())
+        .len(length)
+        .null_bit_buffer(null_bitmap_builder.finish())
+        .build();
+    make_array(array_data)
+}
+
+fn build_statistics_array(
+    statistics: &[Option<&ParquetStatistics>],
+    statistics_type: StatisticsType,
+    data_type: &DataType,
+) -> ArrayRef {
+    let statistics_count = statistics.len();
+    // this should be handled by the condition below
+    // if statistics_count < 1 {
+    //     return build_null_array(data_type, 0);
+    // }
+
+    let first_group_stats = statistics.iter().find(|s| s.is_some());
+    let first_group_stats = if let Some(Some(statistics)) = first_group_stats {
+        // found first row group with statistics defined
+        statistics
+    } else {
+        // no row group has statistics defined
+        return build_null_array(data_type, statistics_count);
+    };
+
+    let (data_size, arrow_type) = match first_group_stats {
+        ParquetStatistics::Int32(_) => (std::mem::size_of::<i32>(), DataType::Int32),
+        ParquetStatistics::Int64(_) => (std::mem::size_of::<i64>(), DataType::Int64),
+        ParquetStatistics::Float(_) => (std::mem::size_of::<f32>(), DataType::Float32),
+        ParquetStatistics::Double(_) => (std::mem::size_of::<f64>(), DataType::Float64),
+        ParquetStatistics::ByteArray(_) if data_type == &DataType::Utf8 => {
+            (0, DataType::Utf8)
+        }
+        _ => {
+            // type of statistics not supported
+            return build_null_array(data_type, statistics_count);
+        }
+    };
+
+    let statistics = statistics.iter().map(|s| {
+        s.filter(|s| s.has_min_max_set())
+            .map(|s| match statistics_type {
+                StatisticsType::Min => s.min_bytes(),
+                StatisticsType::Max => s.max_bytes(),
+            })
+    });
+
+    if arrow_type == DataType::Utf8 {
+        let data_size = statistics
+            .clone()
+            .map(|x| x.map(|b| b.len()).unwrap_or(0))
+            .sum();
+        let mut builder =
+            arrow::array::StringBuilder::with_capacity(statistics_count, data_size);
+        let string_statistics =
+            statistics.map(|x| x.and_then(|bytes| std::str::from_utf8(bytes).ok()));
+        for maybe_string in string_statistics {
+            match maybe_string {
+                Some(string_value) => builder.append_value(string_value).unwrap(),
+                None => builder.append_null().unwrap(),
+            };
+        }
+        return Arc::new(builder.finish());
+    }
+
+    let mut data_buffer = MutableBuffer::new(statistics_count * data_size);
+    let mut bitmap_builder = BooleanBufferBuilder::new(statistics_count);
+    let mut null_count = 0;
+    for s in statistics {
+        if let Some(stat_data) = s {
+            bitmap_builder.append(true);
+            data_buffer.extend_from_slice(stat_data);
+        } else {
+            bitmap_builder.append(false);
+            data_buffer.resize(data_buffer.len() + data_size);
+            null_count += 1;
+        }
+    }
+
+    let mut builder = ArrayData::builder(arrow_type)
+        .len(statistics_count)
+        .add_buffer(data_buffer.into());
+    if null_count > 0 {
+        builder = builder.null_bit_buffer(bitmap_builder.finish());
+    }
+    let array_data = builder.build();
+    let statistics_array = make_array(array_data);
+    if statistics_array.data_type() == data_type {
+        return statistics_array;
+    }

Review comment:
       Thank you for your feedback. I was looking for a (mostly) generic approach to building statistics arrays and this is the simplest implementation I could come up with. Your questions make me think if this could be done with generics though.
   From what I have seen, parquet statistics are only stored as Int32, Int64, Float, Double or ByteArray (used for strings and other complex types); may be someone with more experience with parquet can advise on how statistics work for boolean columns.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io edited a comment on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (61f5656) into [master](https://codecov.io/gh/apache/arrow/commit/fdf5e88a67f33c0a76673a32938274f063c9cb93?el=desc) (fdf5e88) will **decrease** coverage by `0.91%`.
   > The diff coverage is `88.73%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.57%   81.66%   -0.92%     
   ==========================================
     Files         204      215      +11     
     Lines       50327    52093    +1766     
   ==========================================
   + Hits        41560    42540     +980     
   - Misses       8767     9553     +786     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/operators.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vb3BlcmF0b3JzLnJz) | `75.00% <ø> (ø)` | |
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `87.98% <89.70%> (+3.36%)` | :arrow_up: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `94.30% <91.30%> (-1.91%)` | :arrow_down: |
   | [rust/arrow/src/array/null.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvbnVsbC5ycw==) | `89.58% <100.00%> (+2.91%)` | :arrow_up: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `72.09% <100.00%> (+0.40%)` | :arrow_up: |
   | [rust/datafusion/src/datasource/csv.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL2Nzdi5ycw==) | `65.00% <0.00%> (-16.25%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/csv.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Nzdi5ycw==) | `74.26% <0.00%> (-8.53%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `72.29% <0.00%> (-7.71%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/limit.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2xpbWl0LnJz) | `57.47% <0.00%> (-5.82%)` | :arrow_down: |
   | ... and [52 more](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [fdf5e88...61f5656](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io edited a comment on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (5db2c6f) into [master](https://codecov.io/gh/apache/arrow/commit/dfef236f7587e4168ac1e07bd09e42d9373beb70?el=desc) (dfef236) will **decrease** coverage by `0.35%`.
   > The diff coverage is `26.11%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.60%   82.25%   -0.36%     
   ==========================================
     Files         204      204              
     Lines       50189    50479     +290     
   ==========================================
   + Hits        41459    41521      +62     
   - Misses       8730     8958     +228     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/operators.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vb3BlcmF0b3JzLnJz) | `75.00% <ø> (ø)` | |
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `45.63% <22.22%> (-39.25%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `91.54% <36.36%> (-4.67%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `75.71% <100.00%> (+0.36%)` | :arrow_up: |
   | [rust/arrow/src/record\_batch.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvcmVjb3JkX2JhdGNoLnJz) | `73.83% <0.00%> (-2.24%)` | :arrow_down: |
   | [rust/parquet/src/encodings/encoding.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9lbmNvZGluZ3MvZW5jb2RpbmcucnM=) | `95.24% <0.00%> (-0.20%)` | :arrow_down: |
   | [rust/arrow/src/array/array\_struct.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfc3RydWN0LnJz) | `88.43% <0.00%> (-0.18%)` | :arrow_down: |
   | [rust/arrow/src/array/equal/mod.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvZXF1YWwvbW9kLnJz) | `92.69% <0.00%> (+0.37%)` | :arrow_up: |
   | [rust/arrow/src/array/array\_primitive.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfcHJpbWl0aXZlLnJz) | `92.82% <0.00%> (+0.49%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [dfef236...5db2c6f](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551583777



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       Good point about documentation - will add some shortly. 
   
   As long as row group metadata is filtered immediately after creating a SerializedFileReader, this approach will work.
   
   That's the simplest way I could think of to allow filtering of row groups using statistics metadata; not sure how this could be done within DataFusion itself, because it reads data in batches (of configurable size) which could potentially span multiple row groups; it could be done, but would probably move a lot of complexity into DataFusion which today is nicely abstracted into the parquet library. This would also expose a lot more about the internals of a parquet file format to the outside as the user would have to be aware of row groups rather than just requesting batches of data.
   May be I misunderstand what you are suggesting?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551064584



##########
File path: rust/datafusion/src/datasource/parquet.rs
##########
@@ -62,17 +64,37 @@ impl TableProvider for ParquetTable {
         self.schema.clone()
     }
 
+    fn supports_filter_pushdown(
+        &self,
+        _filter: &Expr,
+    ) -> Result<TableProviderFilterPushDown> {
+        Ok(TableProviderFilterPushDown::Inexact)
+    }
+
     /// Scan the file(s), using the provided projection, and return one BatchIterator per
     /// partition.
     fn scan(
         &self,
         projection: &Option<Vec<usize>>,
         batch_size: usize,
-        _filters: &[Expr],
+        filters: &[Expr],
     ) -> Result<Arc<dyn ExecutionPlan>> {
+        let predicate = if filters.is_empty() {
+            None
+        } else {
+            Some(
+                filters

Review comment:
       👍 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] Dandandan commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r550998581



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]

Review comment:
       This can use `Copy` too so functions can use `statistics_type: StatisticsType` 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551064489



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]
+enum StatisticsType {
+    Min,
+    Max,
+}
+
+fn build_null_array(data_type: &DataType, length: usize) -> ArrayRef {

Review comment:
       that's what I thought at first, and then realized that `NullArray` returns data type of `DataType::Null`, which doesn't work when the statistics record batch is created as it checks that types from the schema fields and from arrays are the same; that's why I wrote the `build_null_array` function




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r552147401



##########
File path: rust/parquet/src/file/serialized_reader.rs
##########
@@ -137,6 +137,22 @@ impl<R: 'static + ChunkReader> SerializedFileReader<R> {
             metadata,
         })
     }
+
+    pub fn filter_row_groups(

Review comment:
       the second option, with the `ParquetReadOptions` parameter, sounds better (compared to the `new_with_metadata` method) - more extensible as you have described; however I think this falls outside of the scope of this PR;
   
   one issue I can think of, though, is that the code needs to read the statistics metadata from the parquet file, in order create the statistics record batch, execute the predicate expression on it, and then use the results to filter the parquet row groups; this could still work, if the parquet metadata can be read before `SerializedFileReader` is crated using the proposed constructor




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] Dandandan commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r550998137



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {

Review comment:
       I think we should add a `Copy` to the `Operator` enum so we can dereference `op`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551062848



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max
+            // (column / 2) = 4 => (column_min / 2) <= 4 && 4 <= (column_max / 2)
+            expr_builder
+                .scalar_expr()
+                .gt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    min_column_name.as_str(),
+                )?)
+                .and(expr_builder.scalar_expr().lt_eq(rewrite_column_expr(
+                    expr_builder.column_expr(),
+                    expr_builder.column_name(),
+                    max_column_name.as_str(),
+                )?))
+        }
+        Operator::Gt => {
+            let max_column_name = expr_builder.add_max_column();
+            // column > literal => (min, max) > literal => max > literal
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt(expr_builder.scalar_expr().clone())
+        }
+        Operator::GtEq => {
+            // column >= literal => (min, max) >= literal => max >= literal
+            let max_column_name = expr_builder.add_max_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                max_column_name.as_str(),
+            )?
+            .gt_eq(expr_builder.scalar_expr().clone())
+        }
+        Operator::Lt => {
+            // (input < input) => (predicate)
+            // column < literal => (min, max) < literal => min < literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt(expr_builder.scalar_expr().clone())
+        }
+        Operator::LtEq => {
+            // (input <= input) => (predicate)
+            // column <= literal => (min, max) <= literal => min <= literal
+            let min_column_name = expr_builder.add_min_column();
+            rewrite_column_expr(
+                expr_builder.column_expr(),
+                expr_builder.column_name(),
+                min_column_name.as_str(),
+            )?
+            .lt_eq(expr_builder.scalar_expr().clone())
+        }
+        // other expressions are not supported
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    expr_builder.build(&statistics_expr)
+}
+
+/// replaces a column with an old name with a new name in an expression
+fn rewrite_column_expr(
+    expr: &Expr,
+    column_old_name: &str,
+    column_new_name: &str,
+) -> Result<Expr> {
+    let expressions = utils::expr_sub_expressions(&expr)?;
+    let expressions = expressions
+        .iter()
+        .map(|e| rewrite_column_expr(e, column_old_name, column_new_name))
+        .collect::<Result<Vec<_>>>()?;
+
+    if let Expr::Column(name) = expr {
+        if name == column_old_name {
+            return Ok(Expr::Column(column_new_name.to_string()));
+        }
+    }
+    utils::rewrite_expression(&expr, &expressions)
+}
+
+#[derive(Debug, Clone, PartialEq)]

Review comment:
       you are right - I have implemented the `Copy` trait for `StatisticsType` - this makes the code a bit cleaner as it's not necessary to use references all the time




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551062412



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {
+    parquet_schema: Schema,
+    predicate_expr: Arc<dyn PhysicalExpr>,
+    stat_column_req: Vec<(String, StatisticsType, Field)>,
+}
+
+impl PredicateExpressionBuilder {
+    /// Try to create a new instance of PredicateExpressionBuilder
+    pub fn try_new(expr: &Expr, parquet_schema: Schema) -> Result<Self> {
+        // build predicate expression once
+        let mut stat_column_req = Vec::<(String, StatisticsType, Field)>::new();
+        let predicate_expr =
+            build_predicate_expression(expr, &parquet_schema, &mut stat_column_req)?;
+
+        Ok(Self {
+            parquet_schema,
+            predicate_expr,
+            stat_column_req,
+        })
+    }
+
+    /// Generate a predicate function used to filter row group metadata
+    pub fn build_row_group_predicate(
+        &self,
+        row_group_metadata: &[RowGroupMetaData],
+    ) -> Box<dyn Fn(&RowGroupMetaData, usize) -> bool> {
+        // build statistics record batch
+        let predicate_result = build_row_group_record_batch(
+            row_group_metadata,
+            &self.parquet_schema,
+            &self.stat_column_req,
+        )
+        .and_then(|statistics_batch| {
+            // execute predicate expression
+            self.predicate_expr.evaluate(&statistics_batch)
+        })
+        .and_then(|v| match v {
+            ColumnarValue::Array(array) => Ok(array),
+            ColumnarValue::Scalar(_) => Err(DataFusionError::Plan(
+                "predicate expression didn't return an array".to_string(),
+            )),
+        });
+
+        let predicate_array = match predicate_result {
+            Ok(array) => array,
+            _ => return Box::new(|_r, _i| true),
+        };
+
+        let predicate_array = predicate_array.as_any().downcast_ref::<BooleanArray>();
+        match predicate_array {
+            // return row group predicate function
+            Some(array) => {
+                let predicate_values =
+                    array.iter().map(|x| x.unwrap_or(false)).collect::<Vec<_>>();
+                Box::new(move |_, i| predicate_values[i])
+            }
+            // predicate result is not a BooleanArray
+            _ => Box::new(|_r, _i| true),
+        }
+    }
+}
+
+fn build_row_group_record_batch(
+    row_groups: &[RowGroupMetaData],
+    parquet_schema: &Schema,
+    stat_column_req: &Vec<(String, StatisticsType, Field)>,
+) -> Result<RecordBatch> {
+    let mut fields = Vec::<Field>::new();
+    let mut arrays = Vec::<ArrayRef>::new();
+    for (column_name, statistics_type, stat_field) in stat_column_req {
+        if let Some((column_index, _)) = parquet_schema.column_with_name(column_name) {
+            let statistics = row_groups
+                .iter()
+                .map(|g| g.column(column_index).statistics())
+                .collect::<Vec<_>>();
+            let array = build_statistics_array(
+                &statistics,
+                statistics_type,
+                stat_field.data_type(),
+            );
+            fields.push(stat_field.clone());
+            arrays.push(array);
+        }
+    }
+    let schema = Arc::new(Schema::new(fields));
+    RecordBatch::try_new(schema, arrays)
+        .map_err(|err| DataFusionError::Plan(err.to_string()))
+}
+
+struct PhysicalExpressionBuilder<'a> {
+    column_name: String,
+    column_expr: &'a Expr,
+    scalar_expr: &'a Expr,
+    parquet_field: &'a Field,
+    statistics_fields: Vec<Field>,
+    stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    reverse_operator: bool,
+}
+
+impl<'a> PhysicalExpressionBuilder<'a> {
+    fn try_new(
+        left: &'a Expr,
+        right: &'a Expr,
+        parquet_schema: &'a Schema,
+        stat_column_req: &'a mut Vec<(String, StatisticsType, Field)>,
+    ) -> Result<Self> {
+        // find column name; input could be a more complicated expression
+        let mut left_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(left, &mut left_columns)?;
+        let mut right_columns = HashSet::<String>::new();
+        utils::expr_to_column_names(right, &mut right_columns)?;
+        let (column_expr, scalar_expr, column_names, reverse_operator) =
+            match (left_columns.len(), right_columns.len()) {
+                (1, 0) => (left, right, left_columns, false),
+                (0, 1) => (right, left, right_columns, true),
+                _ => {
+                    // if more than one column used in expression - not supported
+                    return Err(DataFusionError::Plan(
+                        "Multi-column expressions are not currently supported"
+                            .to_string(),
+                    ));
+                }
+            };
+        let column_name = column_names.iter().next().unwrap().clone();
+        let field = match parquet_schema.column_with_name(&column_name) {
+            Some((_, f)) => f,
+            _ => {
+                // field not found in parquet schema
+                return Err(DataFusionError::Plan(
+                    "Field not found in parquet schema".to_string(),
+                ));
+            }
+        };
+
+        Ok(Self {
+            column_name,
+            column_expr,
+            scalar_expr,
+            parquet_field: field,
+            statistics_fields: Vec::new(),
+            stat_column_req,
+            reverse_operator,
+        })
+    }
+
+    fn correct_operator(&self, op: &Operator) -> Operator {
+        if !self.reverse_operator {
+            return op.clone();
+        }
+
+        match op {
+            Operator::Lt => Operator::Gt,
+            Operator::Gt => Operator::Lt,
+            Operator::LtEq => Operator::GtEq,
+            Operator::GtEq => Operator::LtEq,
+            _ => op.clone(),
+        }
+    }
+
+    fn column_expr(&self) -> &Expr {
+        self.column_expr
+    }
+
+    fn scalar_expr(&self) -> &Expr {
+        self.scalar_expr
+    }
+
+    fn column_name(&self) -> &String {
+        &self.column_name
+    }
+
+    fn is_stat_column_missing(&self, statistics_type: &StatisticsType) -> bool {
+        self.stat_column_req
+            .iter()
+            .filter(|(c, t, _f)| c == &self.column_name && t == statistics_type)
+            .count()
+            == 0
+    }
+
+    fn add_min_column(&mut self) -> String {
+        let min_field = Field::new(
+            format!("{}_min", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let min_column_name = min_field.name().clone();
+        self.statistics_fields.push(min_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Min) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Min,
+                min_field,
+            ));
+        }
+        min_column_name
+    }
+
+    fn add_max_column(&mut self) -> String {
+        let max_field = Field::new(
+            format!("{}_max", self.column_name).as_str(),
+            self.parquet_field.data_type().clone(),
+            self.parquet_field.is_nullable(),
+        );
+        let max_column_name = max_field.name().clone();
+        self.statistics_fields.push(max_field.clone());
+        if self.is_stat_column_missing(&StatisticsType::Max) {
+            // only add statistics column if not previously added
+            self.stat_column_req.push((
+                self.column_name.to_string(),
+                StatisticsType::Max,
+                max_field,
+            ));
+        }
+        max_column_name
+    }
+
+    fn build(&self, statistics_expr: &Expr) -> Result<Arc<dyn PhysicalExpr>> {
+        let execution_context_state = ExecutionContextState {
+            datasources: HashMap::new(),
+            scalar_functions: HashMap::new(),
+            var_provider: HashMap::new(),
+            aggregate_functions: HashMap::new(),
+            config: ExecutionConfig::new(),
+        };
+        let schema = Schema::new(self.statistics_fields.clone());
+        DefaultPhysicalPlanner::default().create_physical_expr(
+            statistics_expr,
+            &schema,
+            &execution_context_state,
+        )
+    }
+}
+
+/// Translate logical filter expression into parquet statistics physical filter expression
+fn build_predicate_expression(
+    expr: &Expr,
+    parquet_schema: &Schema,
+    stat_column_req: &mut Vec<(String, StatisticsType, Field)>,
+) -> Result<Arc<dyn PhysicalExpr>> {
+    // predicate expression can only be a binary expression
+    let (left, op, right) = match expr {
+        Expr::BinaryExpr { left, op, right } => (left, op, right),
+        _ => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+
+    if op == &Operator::And || op == &Operator::Or {
+        let left_expr =
+            build_predicate_expression(left, parquet_schema, stat_column_req)?;
+        let right_expr =
+            build_predicate_expression(right, parquet_schema, stat_column_req)?;
+        let stat_fields = stat_column_req
+            .iter()
+            .map(|(_, _, f)| f.clone())
+            .collect::<Vec<_>>();
+        let stat_schema = Schema::new(stat_fields);
+        let result = expressions::binary(left_expr, op.clone(), right_expr, &stat_schema);
+        return result;
+    }
+
+    let mut expr_builder = match PhysicalExpressionBuilder::try_new(
+        left,
+        right,
+        parquet_schema,
+        stat_column_req,
+    ) {
+        Ok(builder) => builder,
+        // allow partial failure in predicate expression generation
+        // this can still produce a useful predicate when multiple conditions are joined using AND
+        Err(_) => {
+            return Ok(expressions::lit(ScalarValue::Boolean(Some(true))));
+        }
+    };
+    let corrected_op = expr_builder.correct_operator(op);
+    let statistics_expr = match corrected_op {
+        Operator::Eq => {
+            let min_column_name = expr_builder.add_min_column();
+            let max_column_name = expr_builder.add_max_column();
+            // column = literal => column = (min, max) => min <= literal && literal <= max

Review comment:
       this particular comment should be `// column = literal => (min, max) = literal => min <= literal && literal <= max` :), but yes, it does require some thinking so I thought it would be good to add these comments to help with the process




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] codecov-io edited a comment on pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#issuecomment-753605727


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=h1) Report
   > Merging [#9064](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=desc) (5d7dda7) into [master](https://codecov.io/gh/apache/arrow/commit/fdf5e88a67f33c0a76673a32938274f063c9cb93?el=desc) (fdf5e88) will **decrease** coverage by `0.82%`.
   > The diff coverage is `71.20%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9064/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9064      +/-   ##
   ==========================================
   - Coverage   82.57%   81.75%   -0.83%     
   ==========================================
     Files         204      214      +10     
     Lines       50327    51751    +1424     
   ==========================================
   + Hits        41560    42307     +747     
   - Misses       8767     9444     +677     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/logical\_plan/operators.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9sb2dpY2FsX3BsYW4vb3BlcmF0b3JzLnJz) | `75.00% <ø> (ø)` | |
   | [rust/parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9maWxlL3NlcmlhbGl6ZWRfcmVhZGVyLnJz) | `93.42% <0.00%> (-2.19%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL3BhcnF1ZXQucnM=) | `74.60% <70.00%> (-10.02%)` | :arrow_down: |
   | [rust/datafusion/src/datasource/parquet.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9kYXRhc291cmNlL3BhcnF1ZXQucnM=) | `95.51% <92.00%> (-0.70%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `72.09% <100.00%> (+0.40%)` | :arrow_up: |
   | [rust/datafusion/src/physical\_plan/functions.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2Z1bmN0aW9ucy5ycw==) | `78.51% <0.00%> (-1.49%)` | :arrow_down: |
   | [rust/arrow/src/util/integration\_util.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvdXRpbC9pbnRlZ3JhdGlvbl91dGlsLnJz) | `66.73% <0.00%> (-0.75%)` | :arrow_down: |
   | [rust/datafusion/src/physical\_plan/expressions.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2V4cHJlc3Npb25zLnJz) | `83.77% <0.00%> (-0.71%)` | :arrow_down: |
   | [rust/datafusion/src/sql/utils.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9zcWwvdXRpbHMucnM=) | `53.92% <0.00%> (-0.68%)` | :arrow_down: |
   | [rust/datafusion/src/optimizer/utils.rs](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9vcHRpbWl6ZXIvdXRpbHMucnM=) | `58.18% <0.00%> (-0.54%)` | :arrow_down: |
   | ... and [33 more](https://codecov.io/gh/apache/arrow/pull/9064/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=footer). Last update [fdf5e88...5d7dda7](https://codecov.io/gh/apache/arrow/pull/9064?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [arrow] yordan-pavlov commented on a change in pull request #9064: ARROW-11074: [Rust][DataFusion] Implement predicate push-down for parquet tables

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #9064:
URL: https://github.com/apache/arrow/pull/9064#discussion_r551063325



##########
File path: rust/datafusion/src/physical_plan/parquet.rs
##########
@@ -209,6 +251,479 @@ impl ParquetExec {
     }
 }
 
+#[derive(Debug, Clone)]
+/// Predicate builder used for generating of predicate functions, used to filter row group metadata
+pub struct PredicateExpressionBuilder {

Review comment:
       I don't feel strongly about this either way, but at the moment it has to be public because it is used as a parameter in `pub ParquetExec::new`




----------------------------------------------------------------
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:
users@infra.apache.org