You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "gianm (via GitHub)" <gi...@apache.org> on 2023/05/10 18:53:13 UTC

[GitHub] [druid] gianm opened a new pull request, #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-lā€¦

gianm opened a new pull request, #14249:
URL: https://github.com/apache/druid/pull/14249

   Four main changes:
   
   1) Provide aggregatorBuilder, a more consistent way of defining the
      SqlAggFunction we need for all of our SQL aggregators. The mechanism
      is analogous to the one we already use for SQL functions
      (OperatorConversions.operatorBuilder).
   
   2) Allow CASTs of constants to be considered as "literalOperands". This
      fixes an issue where various of our operators are defined with
      OperandTypes.LITERAL as part of their checkers, which doesn't allow
      casts. However, in these cases we generally _do_ want to allow casts.
      The important piece is that the value must be reducible to a constant,
      not that the SQL text is literally a literal.
   
   3) Update DataSketches SQL aggregators to use the new aggregatorBuilder
      functionality. The main user-visible effect here is [2]: the aggregators
      would now accept, for example, "CAST(0.99 AS DOUBLE)" as a literal
      argument. Other aggregators could be updated in a future patch.
   
   4) Rename "requiredOperands" to "requiredOperandCount", because the
      old name was confusing. (It rhymes with "literalOperands" but the
      arguments mean different things.)
   


-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14249:
URL: https://github.com/apache/druid/pull/14249#discussion_r1190314822


##########
sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java:
##########
@@ -522,34 +546,53 @@
     /**
      * Creates a {@link SqlFunction} from this builder.
      */
-    public SqlFunction build()
+    @SuppressWarnings("unchecked")
+    public T build()
+    {
+      final IntSet nullableOperands = buildNullableOperands();
+      return (T) new SqlFunction(
+          name,
+          kind,
+          Preconditions.checkNotNull(returnTypeInference, "returnTypeInference"),
+          buildOperandTypeInference(nullableOperands),
+          buildOperandTypeChecker(nullableOperands),
+          functionCategory
+      );
+    }
+
+    protected IntSet buildNullableOperands()
     {
       // Create "nullableOperands" set including all optional arguments.
       final IntSet nullableOperands = new IntArraySet();
-      if (requiredOperands != null) {
-        IntStream.range(requiredOperands, operandTypes.size()).forEach(nullableOperands::add);
+      if (requiredOperandCount != null) {
+        IntStream.range(requiredOperandCount, operandTypes.size()).forEach(nullableOperands::add);
       }
+      return nullableOperands;
+    }
 
-      final SqlOperandTypeChecker theOperandTypeChecker;
-
+    protected SqlOperandTypeChecker buildOperandTypeChecker(final IntSet nullableOperands)
+    {
       if (operandTypeChecker == null) {
-        theOperandTypeChecker = new DefaultOperandTypeChecker(
+        return new DefaultOperandTypeChecker(
             operandTypes,
-            requiredOperands == null ? operandTypes.size() : requiredOperands,
+            requiredOperandCount == null ? operandTypes.size() : requiredOperandCount,

Review Comment:
   ## Dereferenced variable may be null
   
   Variable [operandTypes](1) may be null at this access as suggested by [this](2) null guard.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4944)



-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm merged pull request #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.

Posted by "gianm (via GitHub)" <gi...@apache.org>.
gianm merged PR #14249:
URL: https://github.com/apache/druid/pull/14249


-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a diff in pull request #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #14249:
URL: https://github.com/apache/druid/pull/14249#discussion_r1239581927


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java:
##########
@@ -21,24 +21,29 @@
 
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.InferTypes;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
 import org.apache.druid.sql.calcite.aggregation.Aggregation;
 import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.OperatorConversions;
 
 import java.util.Collections;
 
 public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator
 {
   public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
-
-  private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchApproxCountDistinctSqlAggFunction();
+  private static final SqlAggFunction FUNCTION_INSTANCE =
+      OperatorConversions.aggregatorBuilder(NAME)
+                         .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)

Review Comment:
   not a big deal, but does this lose the nice signature that provides the parameter names in the validation error? similar comment on other changes that drop the `OperandTypes.sequence`.
   
   I personally find the more verbose `OperandTypes.or` form of using `operandTypeChecker` to be a bit easier to understand the shape of the function at a glance than trying to piece it together from looking at all of `operandTypes`, `literalOperands` and `requiredOperandCount`, but i guess it isn't a big deal and is also consistent with what is allowed when using the builder to spit out `SqlFunction`.



-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a diff in pull request #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.

Posted by "gianm (via GitHub)" <gi...@apache.org>.
gianm commented on code in PR #14249:
URL: https://github.com/apache/druid/pull/14249#discussion_r1240294783


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java:
##########
@@ -21,24 +21,29 @@
 
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.InferTypes;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
 import org.apache.druid.sql.calcite.aggregation.Aggregation;
 import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.OperatorConversions;
 
 import java.util.Collections;
 
 public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator
 {
   public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
-
-  private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchApproxCountDistinctSqlAggFunction();
+  private static final SqlAggFunction FUNCTION_INSTANCE =
+      OperatorConversions.aggregatorBuilder(NAME)
+                         .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)

Review Comment:
   In the latest patch, I added `operandNames` which are used to generate the signature for validation errors.
   
   About the syntax, yeah, I was wanting to match it to `SqlFunction`. Feel free to introduce a new syntax in a future patch šŸ™‚. IMO it would make sense to have a builder for `DefaultOperandTypeChecker` that builds up the operand list one at a time. That would enable an alternate syntax like:
   
   ```
   OperatorConversions
     .aggregatorBuilder(NAME)
     .operandTypeChecker(
         OperandConversions
           .checkerBuilder()
           .operand("column", SqlTypeFamily.ANY)
           .operand("lgK", SqlTypeFamily.NUMERIC).optional().literal()
           .operand("tgtHllType", SqlTypeFamily.STRING).optional().literal()        
           .build()
     )
     .returnTypeNonNull(SqlTypeName.BIGINT)
     .functionCategory(SqlFunctionCategory.NUMERIC)
     .build();
   ```



-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a diff in pull request #14249: SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.

Posted by "gianm (via GitHub)" <gi...@apache.org>.
gianm commented on code in PR #14249:
URL: https://github.com/apache/druid/pull/14249#discussion_r1240294783


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java:
##########
@@ -21,24 +21,29 @@
 
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.InferTypes;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
 import org.apache.druid.sql.calcite.aggregation.Aggregation;
 import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.OperatorConversions;
 
 import java.util.Collections;
 
 public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator
 {
   public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
-
-  private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchApproxCountDistinctSqlAggFunction();
+  private static final SqlAggFunction FUNCTION_INSTANCE =
+      OperatorConversions.aggregatorBuilder(NAME)
+                         .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)

Review Comment:
   In the latest patch, I added `operandNames` which are used to generate the signature for validation errors.
   
   About the syntax, yeah, I was wanting to match it to `SqlFunction`. Feel free to introduce a new syntax in a future patch šŸ™‚. IMO it would make sense to have a builder for `DefaultOperandTypeChecker` that builds up the operand list one at a time. That would enable an alternate syntax like:
   
   ```
   OperatorConversions
     .aggregatorBuilder(NAME)
     .operandTypeChecker(
         OperandConversions
           .checkerBuilder()
           .operand("column", SqlTypeFamily.ANY)
           .operand("lgK", SqlTypeFamily.NUMERIC).optional().literal()
           .operand("tgtHllType", SqlTypeFamily.STRING).optional().literal()        
     )
     .returnTypeNonNull(SqlTypeName.BIGINT)
     .functionCategory(SqlFunctionCategory.NUMERIC)
     .build();
   ```



-- 
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: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org