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 2020/03/09 16:24:05 UTC

[GitHub] [druid] gianm opened a new pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

gianm opened a new pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484
 
 
   - Migrate ValueMatcherColumnSelectorStrategy to newer ColumnProcessorFactory
     system, and set defaultType COMPLEX so unknown types can be dynamically matched.
   - Remove ValueGetters in favor of ColumnComparisonFilter doing its own thing.
   - Switch various methods to use convertObjectToX when casting to numbers, rather
     than ad-hoc and inconsistent logic.
   - Fix bug in RowBasedExpressionColumnValueSelector: isBindingArray should return
     true even for 0- or 1- element arrays.
   - Adjust various javadocs.
   

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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-597108855
 
 
   > Tagged release notes because this PR changes the behavior of complex metric aggregation at ingestion time when SQL compatible null handling is _disabled_ (the default mode) to now aggregate the default 0 values for rows instead of skipping them. This change is for the better imo since it makes things symmetrical to as if you ingested the raw data and built the sketch at query time, but it is different so worth calling out, and you can see the effects in some of the test changes in this PR.
   
   Thanks for pointing that out. Yes, I agree, it is for the better since it makes the ingest-time behavior and query-time behavior the same. This is part of the promise of Druid rollup in the first place (you can move aggregations to ingest time if you want).
   
   Btw, this patch also ends up making ingest-time transforms and filters behave more consistently with query-time ones.

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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-596717367
 
 
   Pushed some updates to address test failures in InputRowSerdeTest. I had to add a `throwParseExceptions` option to the RowBasedColumnSelectorFactory, since it turns out some users want that behavior and some don't.

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


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-597053167
 
 
   Tagged release notes because this PR changes the behavior of complex metric aggregation at ingestion time when SQL compatible null handling is _disabled_ (the default mode) to now aggregate the default 0 values for rows instead of skipping them. This change is for the better imo since it makes things symmetrical to as if you ingested the raw data and built the sketch at query time, but it is different so worth calling out.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#discussion_r389845742
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java
 ##########
 @@ -0,0 +1,365 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.filter;
+
+import com.google.common.base.Predicate;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.filter.DruidDoublePredicate;
+import org.apache.druid.query.filter.DruidFloatPredicate;
+import org.apache.druid.query.filter.DruidLongPredicate;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.query.filter.ValueMatcher;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+import org.apache.druid.segment.BaseFloatColumnValueSelector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+import org.apache.druid.segment.DimensionDictionarySelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.DimensionSelector;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+/**
+ * Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link ConstantValueMatcherFactory}
+ * and {@link PredicateValueMatcherFactory}.
+ */
+public class ValueMatchers
+{
+  private ValueMatchers()
+  {
+    // No instantiation.
+  }
+
+  /**
+   * Creates a constant-based {@link ValueMatcher} for a string-typed selector.
+   *
+   * @param selector          column selector
+   * @param value             value to match
+   * @param hasMultipleValues whether the column selector *might* have multiple values
+   */
+  public static ValueMatcher makeStringValueMatcher(
+      final DimensionSelector selector,
+      final String value,
+      final boolean hasMultipleValues
+  )
+  {
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
+        selector,
+        hasMultipleValues,
+        s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
+    );
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    } else {
+      return selector.makeValueMatcher(value);
+    }
+  }
+
+  /**
+   * Creates a predicate-based {@link ValueMatcher} for a string-typed selector.
+   *
+   * @param selector          column selector
+   * @param predicateFactory  predicate to match
+   * @param hasMultipleValues whether the column selector *might* have multiple values
+   */
+  public static ValueMatcher makeStringValueMatcher(
+      final DimensionSelector selector,
+      final DruidPredicateFactory predicateFactory,
+      final boolean hasMultipleValues
+  )
+  {
+    final Predicate<String> predicate = predicateFactory.makeStringPredicate();
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    } else {
+      return selector.makeValueMatcher(predicate);
+    }
+  }
+
+  /**
+   * Creates a constant-based {@link ValueMatcher} for a float-typed selector.
+   *
+   * @param selector column selector
+   * @param value    value to match
+   */
+  public static ValueMatcher makeFloatValueMatcher(
+      final BaseFloatColumnValueSelector selector,
+      final String value
+  )
+  {
+    final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
+    if (matchVal == null) {
+      return makeNumericNullValueMatcher(selector);
+    }
+
+    // Use "floatToIntBits" to canonicalize NaN values.
+    final int matchValIntBits = Float.floatToIntBits(matchVal);
+    return new ValueMatcher()
+    {
+      @Override
+      public boolean matches()
+      {
+        if (selector.isNull()) {
+          return false;
+        }
+        return Float.floatToIntBits(selector.getFloat()) == matchValIntBits;
+      }
+
+      @Override
+      public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+      {
+        inspector.visit("selector", selector);
+      }
+    };
+  }
+
+  public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value)
+  {
+    final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
+    if (matchVal == null) {
+      return makeNumericNullValueMatcher(selector);
+    }
+    final long matchValLong = matchVal;
+    return new ValueMatcher()
+    {
+      @Override
+      public boolean matches()
+      {
+        if (selector.isNull()) {
+          return false;
+        }
+        return selector.getLong() == matchValLong;
+      }
+
+      @Override
+      public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+      {
+        inspector.visit("selector", selector);
+      }
+    };
+  }
+
+  public static ValueMatcher makeLongValueMatcher(
 
 Review comment:
   same nit re javadoc

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
gianm merged pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484
 
 
   

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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-596633079
 
 
   Note to reviewers — some of the bugs fixed here aren't tested by existing tests, but I plan to add tests for them in a future patch that also adds a RowBasedStorageAdapter. That's because the simplest & best way to test them is to add a row-based cursor to BaseFilterTest, which won't exist until the future patch.

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


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis edited a comment on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
clintropolis edited a comment on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-597053167
 
 
   Tagged release notes because this PR changes the behavior of complex metric aggregation at ingestion time when SQL compatible null handling is _disabled_ (the default mode) to now aggregate the default 0 values for rows instead of skipping them. This change is for the better imo since it makes things symmetrical to as if you ingested the raw data and built the sketch at query time, but it is different so worth calling out, and you can see the effects in some of the test changes in 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


With regards,
Apache Git Services

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


[GitHub] [druid] gianm edited a comment on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
gianm edited a comment on issue #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#issuecomment-597108855
 
 
   > Tagged release notes because this PR changes the behavior of complex metric aggregation at ingestion time when SQL compatible null handling is _disabled_ (the default mode) to now aggregate the default 0 values for rows instead of skipping them. This change is for the better imo since it makes things symmetrical to as if you ingested the raw data and built the sketch at query time, but it is different so worth calling out, and you can see the effects in some of the test changes in this PR.
   
   Thanks for pointing that out. Yes, I agree, it is for the better since it makes the ingest-time behavior and query-time behavior the same. This is part of the promise of Druid rollup in the first place (you can move aggregations to ingest time if you want).
   
   Btw, this patch also ends up making ingest-time transforms and filters behave more consistently with query-time ones.
   
   The reason is that all this ingest-time stuff runs in unknown-type mode, which til now had various inconsistencies with known-type mode (which is used at query 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#discussion_r389843860
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/virtual/RowBasedExpressionColumnValueSelector.java
 ##########
 @@ -95,7 +95,7 @@ private boolean isBindingArray(String x)
   {
     Object binding = bindings.get(x);
     if (binding != null) {
-      if (binding instanceof String[] && ((String[]) binding).length > 1) {
+      if (binding instanceof String[]) {
 
 Review comment:
   :+1:

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9484: Harmonization and bug-fixing for selector and filter behavior on unknown types.
URL: https://github.com/apache/druid/pull/9484#discussion_r389845620
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java
 ##########
 @@ -0,0 +1,365 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.filter;
+
+import com.google.common.base.Predicate;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.filter.DruidDoublePredicate;
+import org.apache.druid.query.filter.DruidFloatPredicate;
+import org.apache.druid.query.filter.DruidLongPredicate;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.query.filter.ValueMatcher;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+import org.apache.druid.segment.BaseFloatColumnValueSelector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+import org.apache.druid.segment.DimensionDictionarySelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.DimensionSelector;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+/**
+ * Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link ConstantValueMatcherFactory}
+ * and {@link PredicateValueMatcherFactory}.
+ */
+public class ValueMatchers
+{
+  private ValueMatchers()
+  {
+    // No instantiation.
+  }
+
+  /**
+   * Creates a constant-based {@link ValueMatcher} for a string-typed selector.
+   *
+   * @param selector          column selector
+   * @param value             value to match
+   * @param hasMultipleValues whether the column selector *might* have multiple values
+   */
+  public static ValueMatcher makeStringValueMatcher(
+      final DimensionSelector selector,
+      final String value,
+      final boolean hasMultipleValues
+  )
+  {
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
+        selector,
+        hasMultipleValues,
+        s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
+    );
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    } else {
+      return selector.makeValueMatcher(value);
+    }
+  }
+
+  /**
+   * Creates a predicate-based {@link ValueMatcher} for a string-typed selector.
+   *
+   * @param selector          column selector
+   * @param predicateFactory  predicate to match
+   * @param hasMultipleValues whether the column selector *might* have multiple values
+   */
+  public static ValueMatcher makeStringValueMatcher(
+      final DimensionSelector selector,
+      final DruidPredicateFactory predicateFactory,
+      final boolean hasMultipleValues
+  )
+  {
+    final Predicate<String> predicate = predicateFactory.makeStringPredicate();
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    } else {
+      return selector.makeValueMatcher(predicate);
+    }
+  }
+
+  /**
+   * Creates a constant-based {@link ValueMatcher} for a float-typed selector.
+   *
+   * @param selector column selector
+   * @param value    value to match
+   */
+  public static ValueMatcher makeFloatValueMatcher(
+      final BaseFloatColumnValueSelector selector,
+      final String value
+  )
+  {
+    final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
+    if (matchVal == null) {
+      return makeNumericNullValueMatcher(selector);
+    }
+
+    // Use "floatToIntBits" to canonicalize NaN values.
+    final int matchValIntBits = Float.floatToIntBits(matchVal);
+    return new ValueMatcher()
+    {
+      @Override
+      public boolean matches()
+      {
+        if (selector.isNull()) {
+          return false;
+        }
+        return Float.floatToIntBits(selector.getFloat()) == matchValIntBits;
+      }
+
+      @Override
+      public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+      {
+        inspector.visit("selector", selector);
+      }
+    };
+  }
+
+  public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value)
 
 Review comment:
   super nit: missing javadoc (since almost all the others have 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


With regards,
Apache Git Services

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