alamb commented on code in PR #8254:
URL: https://github.com/apache/arrow-datafusion/pull/8254#discussion_r1397564462


##########
datafusion/core/src/datasource/file_format/parquet.rs:
##########
@@ -256,167 +253,52 @@ impl FileFormat for ParquetFormat {
     }
 }
 
-fn summarize_min_max(
-    max_values: &mut [Option<MaxAccumulator>],
-    min_values: &mut [Option<MinAccumulator>],
-    fields: &Fields,
-    i: usize,
+/// Convert the statistics for a RowGroup ([`ParquetStatistics`]) to a
+/// [`ColumnStatistics`].
+fn column_chunk_statisics_to_column_statistics(
     stat: &ParquetStatistics,
-) {
-    match stat {
-        ParquetStatistics::Boolean(s) => {
-            if let DataType::Boolean = fields[i].data_type() {
-                if s.has_min_max_set() {
-                    if let Some(max_value) = &mut max_values[i] {
-                        match 
max_value.update_batch(&[Arc::new(BooleanArray::from(
-                            vec![Some(*s.max())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                max_values[i] = None;
-                            }
-                        }
-                    }
-                    if let Some(min_value) = &mut min_values[i] {
-                        match 
min_value.update_batch(&[Arc::new(BooleanArray::from(
-                            vec![Some(*s.min())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                min_values[i] = None;
-                            }
-                        }
-                    }
-                    return;
-                }
-            }
-            max_values[i] = None;
-            min_values[i] = None;
-        }
-        ParquetStatistics::Int32(s) => {
-            if let DataType::Int32 = fields[i].data_type() {
-                if s.has_min_max_set() {
-                    if let Some(max_value) = &mut max_values[i] {
-                        match 
max_value.update_batch(&[Arc::new(Int32Array::from_value(
-                            *s.max(),
-                            1,
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                max_values[i] = None;
-                            }
-                        }
-                    }
-                    if let Some(min_value) = &mut min_values[i] {
-                        match 
min_value.update_batch(&[Arc::new(Int32Array::from_value(
-                            *s.min(),
-                            1,
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                min_values[i] = None;
-                            }
-                        }
-                    }
-                    return;
-                }
-            }
-            max_values[i] = None;
-            min_values[i] = None;
-        }
-        ParquetStatistics::Int64(s) => {
-            if let DataType::Int64 = fields[i].data_type() {
-                if s.has_min_max_set() {
-                    if let Some(max_value) = &mut max_values[i] {
-                        match 
max_value.update_batch(&[Arc::new(Int64Array::from_value(
-                            *s.max(),
-                            1,
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                max_values[i] = None;
-                            }
-                        }
-                    }
-                    if let Some(min_value) = &mut min_values[i] {
-                        match 
min_value.update_batch(&[Arc::new(Int64Array::from_value(
-                            *s.min(),
-                            1,
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                min_values[i] = None;
-                            }
-                        }
-                    }
-                    return;
-                }
-            }
-            max_values[i] = None;
-            min_values[i] = None;
-        }
-        ParquetStatistics::Float(s) => {
-            if let DataType::Float32 = fields[i].data_type() {
-                if s.has_min_max_set() {
-                    if let Some(max_value) = &mut max_values[i] {
-                        match 
max_value.update_batch(&[Arc::new(Float32Array::from(
-                            vec![Some(*s.max())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                max_values[i] = None;
-                            }
-                        }
-                    }
-                    if let Some(min_value) = &mut min_values[i] {
-                        match 
min_value.update_batch(&[Arc::new(Float32Array::from(
-                            vec![Some(*s.min())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                min_values[i] = None;
-                            }
-                        }
-                    }
-                    return;
-                }
-            }
-            max_values[i] = None;
-            min_values[i] = None;
-        }
-        ParquetStatistics::Double(s) => {
-            if let DataType::Float64 = fields[i].data_type() {
-                if s.has_min_max_set() {
-                    if let Some(max_value) = &mut max_values[i] {
-                        match 
max_value.update_batch(&[Arc::new(Float64Array::from(
-                            vec![Some(*s.max())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                max_values[i] = None;
-                            }
-                        }
-                    }
-                    if let Some(min_value) = &mut min_values[i] {
-                        match 
min_value.update_batch(&[Arc::new(Float64Array::from(
-                            vec![Some(*s.min())],
-                        ))]) {
-                            Ok(_) => {}
-                            Err(_) => {
-                                min_values[i] = None;
-                            }
-                        }
-                    }
-                    return;
-                }
-            }
-            max_values[i] = None;
-            min_values[i] = None;
-        }
-        _ => {
-            max_values[i] = None;
-            min_values[i] = None;
+) -> ColumnStatistics {
+    let (min_value, max_value) = if stat.has_min_max_set() {
+        match stat {
+            ParquetStatistics::Boolean(s) => (
+                Some(ScalarValue::Boolean(Some(*s.min()))),

Review Comment:
   This may look like a regression in performance, but I think it will actually 
perform better (as the old code is creating a single row array just to call an 
accumulator method 😱 )
   
   ```rust
    match max_value.update_batch(&[Arc::new(BooleanArray::from(
                               vec![Some(*s.max())],
                           ))]
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to