You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/05/29 11:40:46 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6464: Clippy lints

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


##########
datafusion/core/src/avro_to_arrow/reader.rs:
##########
@@ -153,7 +153,15 @@ impl<'a, R: Read> Reader<'a, R> {
     }
 
     /// Returns the next batch of results (defined by `self.batch_size`), or `None` if there
-    /// are no more results
+    /// are no more results.
+    //
+    // TODO(clippy): The clippy `allow` could be removed by renaming this method to `next_batch`.
+    // This would also make the intent of the method clearer.
+    //
+    // Another option could be to rework `AvroArrowArrayReader::next_batch` so it returns an

Review Comment:
   I think this second option would be my preference



##########
datafusion/core/src/catalog/information_schema.rs:
##########
@@ -486,37 +486,35 @@ struct InformationSchemaColumnsBuilder {
 }
 
 impl InformationSchemaColumnsBuilder {
-    #[allow(clippy::too_many_arguments)]
     fn add_column(
         &mut self,
-        catalog_name: impl AsRef<str>,
-        schema_name: impl AsRef<str>,
-        table_name: impl AsRef<str>,
-        column_name: impl AsRef<str>,
-        column_position: usize,
-        is_nullable: bool,
-        data_type: &DataType,
+        catalog_name: &str,
+        schema_name: &str,
+        table_name: &str,
+        field_position: usize,
+        field: &Field,
     ) {
         use DataType::*;
 
         // Note: append_value is actually infallable.
-        self.catalog_names.append_value(catalog_name.as_ref());
-        self.schema_names.append_value(schema_name.as_ref());
-        self.table_names.append_value(table_name.as_ref());
+        self.catalog_names.append_value(catalog_name);
+        self.schema_names.append_value(schema_name);
+        self.table_names.append_value(table_name);
 
-        self.column_names.append_value(column_name.as_ref());
+        self.column_names.append_value(field.name());
 
-        self.ordinal_positions.append_value(column_position as u64);
+        self.ordinal_positions.append_value(field_position as u64);
 
         // DataFusion does not support column default values, so null
         self.column_defaults.append_null();
 
         // "YES if the column is possibly nullable, NO if it is known not nullable. "
-        let nullable_str = if is_nullable { "YES" } else { "NO" };
+        let nullable_str = if field.is_nullable() { "YES" } else { "NO" };

Review Comment:
   this is a nice cleanup



##########
datafusion/core/src/physical_plan/file_format/parquet/row_groups.rs:
##########
@@ -263,6 +263,15 @@ mod tests {
     };
     use std::sync::Arc;
 
+    struct PrimitiveTypeField<'a> {
+        name: &'a str,

Review Comment:
   I don't think it is important but given that `name` seems to always be a constant string, you could do something like
   
   ```suggestion
           name: &'static str,
   ```
   
   And avoid having to have the named lifetime `'a` on this struct



##########
datafusion/core/src/physical_plan/file_format/parquet/row_groups.rs:
##########
@@ -272,14 +281,14 @@ mod tests {
         let expr = logical2physical(&expr, &schema);
         let pruning_predicate =
             PruningPredicate::try_new(expr, Arc::new(schema)).unwrap();
-        let schema_descr = get_test_schema_descr(vec![(
-            "c1",
-            PhysicalType::INT32,
-            None,
-            None,
-            None,
-            None,
-        )]);
+        let schema_descr = get_test_schema_descr(vec![PrimitiveTypeField {
+            name: "c1",

Review Comment:
   I am a big fan of explicit field names as a way to better document what is going on here. 



##########
datafusion/core/src/physical_plan/aggregates/mod.rs:
##########
@@ -551,52 +549,21 @@ impl AggregateExec {
         partition: usize,
         context: Arc<TaskContext>,
     ) -> Result<StreamType> {
-        let batch_size = context.session_config().batch_size();
-        let scalar_update_factor = context.session_config().agg_scalar_update_factor();
-        let input = self.input.execute(partition, Arc::clone(&context))?;
-        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
-
         if self.group_by.expr.is_empty() {
             Ok(StreamType::AggregateStream(AggregateStream::new(
-                self.mode,
-                self.schema.clone(),
-                self.aggr_expr.clone(),
-                self.filter_expr.clone(),
-                input,
-                baseline_metrics,
-                context,
-                partition,
+                self, context, partition,

Review Comment:
   this certainly looks much nicer 😍 



##########
datafusion/core/src/catalog/mod.rs:
##########
@@ -18,6 +18,10 @@
 //! This module contains interfaces and default implementations
 //! of table namespacing concepts, including catalogs and schemas.
 
+// TODO(clippy): Having a `datasource::datasource` module path is unclear and ambiguous.

Review Comment:
   👍 



##########
datafusion/core/src/physical_plan/file_format/parquet/row_groups.rs:
##########
@@ -263,6 +263,15 @@ mod tests {
     };
     use std::sync::Arc;
 
+    struct PrimitiveTypeField<'a> {

Review Comment:
   👍 



##########
datafusion/core/src/physical_plan/file_format/parquet/row_groups.rs:
##########
@@ -347,8 +356,22 @@ mod tests {
         let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
 
         let schema_descr = get_test_schema_descr(vec![
-            ("c1", PhysicalType::INT32, None, None, None, None),
-            ("c2", PhysicalType::INT32, None, None, None, None),
+            PrimitiveTypeField {
+                name: "c1",
+                physical_ty: PhysicalType::INT32,
+                logical_ty: None,
+                precision: None,
+                scale: None,
+                byte_len: None,
+            },

Review Comment:
   When I was reading this, I was thinking these tests might be even more readable if they had a builder style, something like
   
   ```rust
    PrimtiveTypeField::new("c1", PhysicalType::INT32)
   ```
   
   And then for tests that need other optional fields set:
   
   ```
    PrimtiveTypeField::new("c1", PhysicalType::INT32)
     .with_logical_type(LogicalType::Decimal {
                   scale: 0,
                   precision: 9,
               });
   ```
   



##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -677,8 +677,8 @@ mod hash_join_tests {
             ..
         }) = plan.as_any().downcast_ref::<HashJoinExec>()
         {
-            let left_changed = Arc::ptr_eq(left, &right_exec);
-            let right_changed = Arc::ptr_eq(right, &left_exec);
+            let left_changed = Arc::data_ptr_eq(left, &right_exec);

Review Comment:
   I like the fact that you have moved the (potentially dubious) pointer comparisons into a trait that has much better documentation about rationale. 🙏  thank you



##########
datafusion/expr/src/logical_plan/extension.rs:
##########
@@ -84,6 +84,12 @@ pub trait UserDefinedLogicalNode: fmt::Debug + Send + Sync {
     /// of self.inputs and self.exprs.
     ///
     /// So, `self.from_template(exprs, ..).expressions() == exprs
+    //
+    // TODO(clippy): This should probably be renamed to use a `with_*` prefix. Something
+    // like `with_template`, or `with_exprs_and_inputs`.
+    //
+    // Also, I think `ExtensionPlanNode` has been renamed to `UserDefinedLogicalNode`

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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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