You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by fj...@apache.org on 2019/05/21 18:32:15 UTC

[incubator-druid] branch master updated: SQL: Fix exception with OR of impossible filters. (#7707)

This is an automated email from the ASF dual-hosted git repository.

fjy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new bcea05e  SQL: Fix exception with OR of impossible filters. (#7707)
bcea05e is described below

commit bcea05e4e8d4cabd698aedbc8f60fdfa8666e2ca
Author: Gian Merlino <gi...@imply.io>
AuthorDate: Tue May 21 11:32:09 2019 -0700

    SQL: Fix exception with OR of impossible filters. (#7707)
    
    Fixes #7671.
---
 .../filtration/CombineAndSimplifyBounds.java       | 89 +++++++++++-----------
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 22 ++++++
 2 files changed, 66 insertions(+), 45 deletions(-)

diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java
index a8eb0a5..ce29494 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java
@@ -118,48 +118,23 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
    * Simplify BoundDimFilters that are children of an OR or an AND.
    *
    * @param children    the filters
-   * @param disjunction true for disjunction, false for conjunction
+   * @param disjunction true for OR, false for AND
    *
    * @return simplified filters
    */
   private static DimFilter doSimplify(final List<DimFilter> children, boolean disjunction)
   {
-    // Copy children list
+    // Copy the list of child filters. We'll modify the copy and eventually return it.
     final List<DimFilter> newChildren = Lists.newArrayList(children);
 
     // Group Bound filters by dimension, extractionFn, and comparator and compute a RangeSet for each one.
     final Map<BoundRefKey, List<BoundDimFilter>> bounds = new HashMap<>();
 
-    final Iterator<DimFilter> iterator = newChildren.iterator();
-    while (iterator.hasNext()) {
-      final DimFilter child = iterator.next();
-
-      if (child.equals(Filtration.matchNothing())) {
-        // Child matches nothing, equivalent to FALSE
-        // OR with FALSE => ignore
-        // AND with FALSE => always false, short circuit
-        if (disjunction) {
-          iterator.remove();
-        } else {
-          return Filtration.matchNothing();
-        }
-      } else if (child.equals(Filtration.matchEverything())) {
-        // Child matches everything, equivalent to TRUE
-        // OR with TRUE => always true, short circuit
-        // AND with TRUE => ignore
-        if (disjunction) {
-          return Filtration.matchEverything();
-        } else {
-          iterator.remove();
-        }
-      } else if (child instanceof BoundDimFilter) {
+    for (final DimFilter child : newChildren) {
+      if (child instanceof BoundDimFilter) {
         final BoundDimFilter bound = (BoundDimFilter) child;
         final BoundRefKey boundRefKey = BoundRefKey.from(bound);
-        List<BoundDimFilter> filterList = bounds.get(boundRefKey);
-        if (filterList == null) {
-          filterList = new ArrayList<>();
-          bounds.put(boundRefKey, filterList);
-        }
+        final List<BoundDimFilter> filterList = bounds.computeIfAbsent(boundRefKey, k -> new ArrayList<>());
         filterList.add(bound);
       }
     }
@@ -184,25 +159,13 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
 
         if (rangeSet.asRanges().isEmpty()) {
           // range set matches nothing, equivalent to FALSE
-          // OR with FALSE => ignore
-          // AND with FALSE => always false, short circuit
-          if (disjunction) {
-            newChildren.add(Filtration.matchNothing());
-          } else {
-            return Filtration.matchNothing();
-          }
+          newChildren.add(Filtration.matchNothing());
         }
 
         for (final Range<BoundValue> range : rangeSet.asRanges()) {
           if (!range.hasLowerBound() && !range.hasUpperBound()) {
             // range matches all, equivalent to TRUE
-            // AND with TRUE => ignore
-            // OR with TRUE => always true; short circuit
-            if (disjunction) {
-              return Filtration.matchEverything();
-            } else {
-              newChildren.add(Filtration.matchEverything());
-            }
+            newChildren.add(Filtration.matchEverything());
           } else {
             newChildren.add(Bounds.toFilter(boundRefKey, range));
           }
@@ -210,8 +173,44 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
       }
     }
 
+    // Finally: Go through newChildren, removing or potentially exiting early based on TRUE / FALSE marker filters.
     Preconditions.checkState(newChildren.size() > 0, "newChildren.size > 0");
-    if (newChildren.size() == 1) {
+
+    final Iterator<DimFilter> iterator = newChildren.iterator();
+    while (iterator.hasNext()) {
+      final DimFilter newChild = iterator.next();
+
+      if (Filtration.matchNothing().equals(newChild)) {
+        // Child matches nothing, equivalent to FALSE
+        // OR with FALSE => ignore
+        // AND with FALSE => always false, short circuit
+        if (disjunction) {
+          iterator.remove();
+        } else {
+          return Filtration.matchNothing();
+        }
+      } else if (Filtration.matchEverything().equals(newChild)) {
+        // Child matches everything, equivalent to TRUE
+        // OR with TRUE => always true, short circuit
+        // AND with TRUE => ignore
+        if (disjunction) {
+          return Filtration.matchEverything();
+        } else {
+          iterator.remove();
+        }
+      }
+    }
+
+    if (newChildren.isEmpty()) {
+      // If "newChildren" is empty at this point, it must have consisted entirely of TRUE / FALSE marker filters.
+      if (disjunction) {
+        // Must have been all FALSE filters (the only kind we would have removed above).
+        return Filtration.matchNothing();
+      } else {
+        // Must have been all TRUE filters (the only kind we would have removed above).
+        return Filtration.matchEverything();
+      }
+    } else if (newChildren.size() == 1) {
       return newChildren.get(0);
     } else {
       return disjunction ? new OrDimFilter(newChildren) : new AndDimFilter(newChildren);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index d2e1f75..ecef1b5 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -1892,6 +1892,28 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
   }
 
   @Test
+  public void testGroupByNothingWithImpossibleTimeFilter() throws Exception
+  {
+    // Regression test for https://github.com/apache/incubator-druid/issues/7671
+
+    testQuery(
+        "SELECT COUNT(*) FROM druid.foo\n"
+        + "WHERE FLOOR(__time TO DAY) = TIMESTAMP '2000-01-02 01:00:00'\n"
+        + "OR FLOOR(__time TO DAY) = TIMESTAMP '2000-01-02 02:00:00'",
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(CalciteTests.DATASOURCE1)
+                  .intervals(querySegmentSpec())
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(TIMESERIES_CONTEXT_DEFAULT)
+                  .build()
+        ),
+        ImmutableList.of()
+    );
+  }
+
+  @Test
   public void testGroupByOneColumnWithLiterallyFalseFilter() throws Exception
   {
     testQuery(


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