You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/01/08 08:21:30 UTC

[GitHub] [druid] gianm commented on a change in pull request #10613: expression filter support for vectorized query engines

gianm commented on a change in pull request #10613:
URL: https://github.com/apache/druid/pull/10613#discussion_r553782719



##########
File path: core/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java
##########
@@ -74,7 +74,7 @@ public ExprType getOutputType(InputBindingInspector inspector)
   @Override
   public boolean canVectorize(InputBindingInspector inspector)
   {
-    return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);

Review comment:
       This opens not just STRING but also LONG_ARRAY and DOUBLE_ARRAY. Is that intentional (will numeric arrays work)?

##########
File path: core/src/main/java/org/apache/druid/math/expr/ExprTypeConversion.java
##########
@@ -20,24 +20,29 @@
 package org.apache.druid.math.expr;
 
 import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Pair;
 
 import javax.annotation.Nullable;
 import java.util.List;
 
 public class ExprTypeConversion
 {
+  public static Pair<ExprType, ExprType> coerceNull(Expr.InputBindingInspector inspector, Expr left, Expr right)

Review comment:
       This method deserves a javadoc.

##########
File path: processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
##########
@@ -305,17 +305,29 @@ private static ColumnCapabilities getEffectiveCapabilities(
         originalCapabilities == null || ValueType.COMPLEX.equals(originalCapabilities.getType());
 
     if (type == ValueType.STRING) {
-      if (!forceSingleValue && effectiveCapabilites.hasMultipleValues().isMaybeTrue()) {
-        return strategyFactory.makeMultiValueDimensionProcessor(
-            effectiveCapabilites,
-            selectorFactory.makeMultiValueDimensionSelector(dimensionSpec)
-        );
-      } else {
-        return strategyFactory.makeSingleValueDimensionProcessor(
-            effectiveCapabilites,
-            selectorFactory.makeSingleValueDimensionSelector(dimensionSpec)
-        );
+      if (!forceSingleValue) {
+        // if column is not dictionary encoded (and not non-existent or complex), use object selector
+        if (effectiveCapabilites.isDictionaryEncoded().isFalse() ||
+            effectiveCapabilites.areDictionaryValuesUnique().isFalse()

Review comment:
       Why does it matter if the dictionary values are unique? (Even if they're not, can't we still use a regular string selector?)

##########
File path: processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java
##########
@@ -64,7 +65,16 @@ public GroupByVectorColumnSelector makeMultiValueDimensionProcessor(
         ValueType.STRING == capabilities.getType(),
         "groupBy dimension processors must be STRING typed"
     );
-    throw new UnsupportedOperationException("Multi-value dimensions not yet implemented for vectorized groupBys");
+    throw new UnsupportedOperationException("Multi-value dimensions are not yet implemented for vectorized groupBys");
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeObjectProcessor(
+      ColumnCapabilities capabilities,
+      VectorObjectSelector selector
+  )
+  {
+    throw new UnsupportedOperationException("Object columns are not yet implemented for vectorized groupBys");

Review comment:
       "yet"? Will they ever be?

##########
File path: processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java
##########
@@ -55,6 +69,105 @@ public ExpressionFilter(final Supplier<Expr> expr, final FilterTuning filterTuni
     this.filterTuning = filterTuning;
   }
 
+  @Override
+  public boolean canVectorizeMatcher(ColumnInspector inspector)
+  {
+    return expr.get().canVectorize(inspector);
+  }
+
+  @Override
+  public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
+  {
+    final Expr theExpr = expr.get();
+
+    DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
+    {
+      @Override
+      public Predicate<String> makeStringPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidLongPredicate makeLongPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidFloatPredicate makeFloatPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidDoublePredicate makeDoublePredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      // The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
+      // DruidPredicateFactory doesn't really need equals or hashcode, in fact only the 'toString' method is called
+      // when testing equality of DimensionPredicateFilter, so it's the truly required method, but even that seems
+      // strange at best.
+      // Rather than tackle removing the annotation and reworking equality tests for now, will leave this to refactor
+      // as part of https://github.com/apache/druid/issues/8256 which suggests combining Filter and DimFilter
+      // implementations, which should clean up some of this mess.
+      @Override
+      public int hashCode()
+      {
+        return super.hashCode();
+      }
+
+      @Override
+      public boolean equals(Object obj)
+      {
+        return super.equals(obj);
+      }
+    };
+
+
+    final ExprType outputType = theExpr.getOutputType(factory);
+
+    if (outputType == null) {
+      // if an expression is vectorizable, but the output type is null, the result will be null (or the default
+      // value in default mode) because expression is either all null constants or missing columns
+
+      // in sql compatible mode, this means no matches ever, so just use the false matcher:
+      if (NullHandling.sqlCompatible()) {
+        return new FalseVectorMatcher(factory.getVectorSizeInspector());
+      }
+      // in default mode, just fallback to using a long matcher since nearly all boolean-ish expressions
+      // output a long value so it is probably a safe bet? idk, ending up here by using all null-ish things
+      // in default mode is dancing on the edge of insanity anyway...
+      return VectorValueMatcherColumnProcessorFactory.instance().makeLongProcessor(
+          ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG),
+          ExpressionVectorSelectors.makeVectorValueSelector(factory, theExpr)
+      ).makeMatcher(predicateFactory);
+    }
+
+    switch (outputType) {
+      case LONG:
+        return VectorValueMatcherColumnProcessorFactory.instance().makeLongProcessor(
+            ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG),
+            ExpressionVectorSelectors.makeVectorValueSelector(factory, theExpr)
+        ).makeMatcher(predicateFactory);
+      case DOUBLE:
+        return VectorValueMatcherColumnProcessorFactory.instance().makeDoubleProcessor(
+            ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE),
+            ExpressionVectorSelectors.makeVectorValueSelector(factory, theExpr)
+        ).makeMatcher(predicateFactory);
+      case STRING:
+        return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor(
+            // using 'numeric' capabilities creator so we are configured to NOT be dictionary encoded, etc
+            ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.STRING),

Review comment:
       `createSimpleNumericColumnCapabilities` is a weird method to call with type STRING. Is this a mistake? Or is the method just named poorly?

##########
File path: processing/src/main/java/org/apache/druid/segment/vector/VectorColumnSelectorFactory.java
##########
@@ -57,6 +57,15 @@ default int getMaxVectorSize()
    */
   MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(DimensionSpec dimensionSpec);
 
+  /**
+   * Returns an object selector for string columns, useful for non-dictionary encoded strings, or when

Review comment:
       What does it do when called on a non-string column? Cast the stuff to strings? Return null? Something else?

##########
File path: processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java
##########
@@ -27,6 +27,13 @@
 {
   Predicate<String> makeStringPredicate();
 
+  default Predicate<Object> makeObjectPredicate()
+  {
+    // default to try to use string predicate;

Review comment:
       "try" makes it sound like this could potentially not work out. Why might it not work out? What bad stuff will happen if it doesn't work out?

##########
File path: processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java
##########
@@ -70,6 +71,11 @@ default MultiValueDimensionVectorSelector decorate(MultiValueDimensionVectorSele
     throw new UOE("DimensionSpec[%s] cannot vectorize", getClass().getName());
   }
 
+  default VectorObjectSelector decorate(VectorObjectSelector selector)

Review comment:
       This method should have a javadoc. The others should too, and they don't, but don't let that prevent you from adding one here 🙂
   
   As I type this, it's not clear to me what this method does, but hopefully it becomes clear elsewhere in the patch.

##########
File path: processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java
##########
@@ -55,6 +69,105 @@ public ExpressionFilter(final Supplier<Expr> expr, final FilterTuning filterTuni
     this.filterTuning = filterTuning;
   }
 
+  @Override
+  public boolean canVectorizeMatcher(ColumnInspector inspector)
+  {
+    return expr.get().canVectorize(inspector);
+  }
+
+  @Override
+  public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
+  {
+    final Expr theExpr = expr.get();
+
+    DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
+    {
+      @Override
+      public Predicate<String> makeStringPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidLongPredicate makeLongPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidFloatPredicate makeFloatPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidDoublePredicate makeDoublePredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      // The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
+      // DruidPredicateFactory doesn't really need equals or hashcode, in fact only the 'toString' method is called
+      // when testing equality of DimensionPredicateFilter, so it's the truly required method, but even that seems
+      // strange at best.
+      // Rather than tackle removing the annotation and reworking equality tests for now, will leave this to refactor
+      // as part of https://github.com/apache/druid/issues/8256 which suggests combining Filter and DimFilter
+      // implementations, which should clean up some of this mess.
+      @Override
+      public int hashCode()
+      {
+        return super.hashCode();
+      }
+
+      @Override
+      public boolean equals(Object obj)
+      {
+        return super.equals(obj);
+      }
+    };
+
+
+    final ExprType outputType = theExpr.getOutputType(factory);
+
+    if (outputType == null) {
+      // if an expression is vectorizable, but the output type is null, the result will be null (or the default
+      // value in default mode) because expression is either all null constants or missing columns
+
+      // in sql compatible mode, this means no matches ever, so just use the false matcher:
+      if (NullHandling.sqlCompatible()) {
+        return new FalseVectorMatcher(factory.getVectorSizeInspector());
+      }
+      // in default mode, just fallback to using a long matcher since nearly all boolean-ish expressions
+      // output a long value so it is probably a safe bet? idk, ending up here by using all null-ish things
+      // in default mode is dancing on the edge of insanity anyway...

Review comment:
       Love the comment but the patch shouldn't go live like this 🙂
   
   Can you come up with a scenario where we'll actually hit this case and add some tests for it? Maybe in the process of doing that, a better way to behave or a better way to describe the behavior will come up.

##########
File path: processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java
##########
@@ -27,6 +27,13 @@
 {
   Predicate<String> makeStringPredicate();
 
+  default Predicate<Object> makeObjectPredicate()
+  {
+    // default to try to use string predicate;
+    final Predicate<String> stringPredicate = makeStringPredicate();
+    return o -> stringPredicate.apply((String) o);

Review comment:
       What guarantees that `o` is a String?

##########
File path: processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java
##########
@@ -55,6 +69,105 @@ public ExpressionFilter(final Supplier<Expr> expr, final FilterTuning filterTuni
     this.filterTuning = filterTuning;
   }
 
+  @Override
+  public boolean canVectorizeMatcher(ColumnInspector inspector)
+  {
+    return expr.get().canVectorize(inspector);
+  }
+
+  @Override
+  public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
+  {
+    final Expr theExpr = expr.get();
+
+    DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
+    {
+      @Override
+      public Predicate<String> makeStringPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidLongPredicate makeLongPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidFloatPredicate makeFloatPredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      @Override
+      public DruidDoublePredicate makeDoublePredicate()
+      {
+        return Evals::asBoolean;
+      }
+
+      // The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
+      // DruidPredicateFactory doesn't really need equals or hashcode, in fact only the 'toString' method is called

Review comment:
       Interesting. It sounds like a bug that DruidPredicateFactory is annotated with `@SubclassesMustOverrideEqualsAndHashCodeTest` but yet its toString method is used for equality checks. Is that right? If so, perhaps a good fix would be:
   
   - Keep the annotation on DimensionPredicateFilter
   - Switch the equality checks in DimensionPredicateFilter to use DruidPredicateFactory `equals` rather than `toString`
   
   If we do that, IMO it'd be good to do it separately from this patch and also separately from #8256.
   
   > DruidPredicateFactory doesn't really need equals or hashcode
   
   How else would DimensionPredicateFilter's `equals` be implemented?

##########
File path: core/src/main/java/org/apache/druid/math/expr/IdentifierExpr.java
##########
@@ -149,13 +150,14 @@ public boolean canVectorize(InputBindingInspector inspector)
     ExprType inputType = inspector.getType(binding);
 
     if (inputType == null) {
-      // nil column, we can be anything, why not be a double
-      return new IdentifierVectorProcessor<double[]>(ExprType.DOUBLE)
+      // nil column, we can be anything, so be a string because it's the most flexible
+      // (numbers will be populated with default values in default mode and non-null)
+      return new IdentifierVectorProcessor<String[]>(ExprType.STRING)
       {
         @Override
-        public ExprEvalVector<double[]> evalVector(VectorInputBinding bindings)
+        public ExprEvalVector<String[]> evalVector(VectorInputBinding bindings)
         {
-          return new ExprEvalDoubleVector(bindings.getDoubleVector(binding), bindings.getNullVector(binding));
+          return new ExprEvalStringVector(Arrays.stream(bindings.getObjectVector(binding)).map(x -> (String) x).toArray(String[]::new));

Review comment:
       What guarantees that the objects in the vector are Strings?




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



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