You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "imply-cheddar (via GitHub)" <gi...@apache.org> on 2023/05/02 01:39:25 UTC

[GitHub] [druid] imply-cheddar commented on a diff in pull request #14139: fix issues with filtering nulls on values coerced to numeric types

imply-cheddar commented on code in PR #14139:
URL: https://github.com/apache/druid/pull/14139#discussion_r1181990948


##########
processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java:
##########
@@ -106,22 +106,27 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
       }
       final NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
       if (rangeIndex != null) {
-        final Number lower = boundDimFilter.hasLowerBound() ? Double.parseDouble(boundDimFilter.getLower()) : null;
-        final Number upper = boundDimFilter.hasUpperBound() ? Double.parseDouble(boundDimFilter.getUpper()) : null;
-        final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
-            lower,
-            boundDimFilter.isLowerStrict(),
-            upper,
-            boundDimFilter.isUpperStrict()
-        );
-        if (rangeBitmaps != null) {
-          // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
-          if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
-            return rangeBitmaps;
-          } else {
-            return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
+        try {
+          final Number lower = boundDimFilter.hasLowerBound() ? Double.parseDouble(boundDimFilter.getLower()) : null;
+          final Number upper = boundDimFilter.hasUpperBound() ? Double.parseDouble(boundDimFilter.getUpper()) : null;
+          final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
+              lower,
+              boundDimFilter.isLowerStrict(),
+              upper,
+              boundDimFilter.isUpperStrict()
+          );
+          if (rangeBitmaps != null) {
+            // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
+            if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
+              return rangeBitmaps;
+            } else {
+              return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
+            }
           }
         }
+        catch (NumberFormatException ignored) {
+          // bounds are not numeric?
+        }

Review Comment:
   This seems like a really late place to be catching this, shouldn't we be noticing that we are a numerical bounds filter with non-numerical arguments much sooner?



##########
processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java:
##########
@@ -217,20 +224,27 @@ public BitmapColumnIndex forValue(@Nullable String value)
     {
       final boolean inputNull = value == null;
       final Double doubleValue = Strings.isNullOrEmpty(value) ? null : Doubles.tryParse(value);
+      final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
+      int defaultValueIndex = dictionary.indexOf(NullHandling.defaultDoubleValue());
+
       return new SimpleBitmapColumnIndex()
       {
-        final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
-
         @Override
         public double estimateSelectivity(int totalRows)
         {
           if (doubleValue == null) {
-            if (inputNull) {
+            if (inputNull && NullHandling.sqlCompatible()) {
               return (double) getBitmap(0).size() / totalRows;
             } else {
               return 0.0;
             }
           }
+          if (NullHandling.replaceWithDefault() && doubleValue.equals(NullHandling.defaultDoubleValue())) {
+            if (defaultValueIndex >= 0) {
+              return ((double) getBitmap(0).size() + (double) getBitmap(defaultValueIndex).size()) / totalRows;
+            }
+            return (double) getBitmap(0).size() / totalRows;
+          }

Review Comment:
   Instead of doing the `NullHandling` checks in here, can you have this object take a constructor parameter that tells it which mode it is in and pass that value in from the call to `.as()`?



##########
processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumn.java:
##########
@@ -91,6 +92,9 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector)
       @Override
       public boolean isNull()
       {
+        if (NullHandling.replaceWithDefault()) {
+          return false;
+        }

Review Comment:
   Instead of doing the `NullHandling` check here, can you make it a field and have the Supplier pass it in?



##########
processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java:
##########
@@ -459,11 +461,12 @@ public Object getObject()
         if (0 <= dimIndex && dimIndex < dims.length) {
           final StructuredData data = (StructuredData) dims[dimIndex];
           if (data != null) {
-            return ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
+            final Object o = ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
+            return o == null ? defaultValue : o;

Review Comment:
   Anything that is calling `.getObject()` had better be doing null checks on the response and doing good things with it.  Seeing this code makes me wonder what call-site is doing bad things such that we require doing this to protect them.  I kinda wish we could go and fix all of the call-sites doing bad things instead of materializing things here.



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java:
##########
@@ -4561,4 +4562,245 @@ public void testJsonValueUnsupportedOptions()
         }
     );
   }
+
+  @Test
+  public void testGroupByPathSelectorFilterVariantNull()
+  {
+    testQuery(
+        "SELECT "
+        + "JSON_VALUE(nest, '$.x'), "
+        + "JSON_VALUE(nester, '$.n.x' RETURNING BIGINT), "
+        + "SUM(cnt) "
+        + "FROM druid.nested WHERE JSON_VALUE(nester, '$.n.x' RETURNING BIGINT) IS NULL GROUP BY 1, 2",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(DATA_SOURCE)
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(
+                            new NestedFieldVirtualColumn("nester", "$.n.x", "v0", ColumnType.LONG),
+                            new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING)
+                        )
+                        .setDimensions(
+                            dimensions(
+                                new DefaultDimensionSpec("v1", "d0"),
+                                new DefaultDimensionSpec("v0", "d1", ColumnType.LONG)
+                            )
+                        )
+                        .setDimFilter(selector("v0", null, null))
+                        .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.defaultStringValue(), NullHandling.defaultLongValue(), 4L},
+            new Object[]{"100", NullHandling.defaultLongValue(), 1L},
+            new Object[]{"200", NullHandling.defaultLongValue(), 1L}
+        ),

Review Comment:
   This appears to be validating the default value instead of `null`, but it's all using `JSON_VALUE` so I would've expected it to get `null` back instead.



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