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/04/08 14:31:55 UTC

[GitHub] [druid] viongpanzi opened a new pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

viongpanzi opened a new pull request #11081:
URL: https://github.com/apache/druid/pull/11081


   
   <!-- If you are a committer, follow the PR action item checklist for committers:
   https://github.com/apache/druid/blob/master/dev/committer-instructions.md#pr-and-issue-action-item-checklist-for-committers. -->
   
   ### Description
   
   <!-- Describe the goal of this PR, what problem are you fixing. If there is a corresponding issue (referenced above), it's not necessary to repeat the description here, however, you may choose to keep one summary sentence. -->
   
   <!-- Describe your patch: what did you change in code? How did you fix the problem? -->
   
   <!-- If there are several relatively logically separate changes in this PR, create a mini-section for each of them. For example: -->
   
   We have a sql like this: 
   
   ```SQL
   select
       code
   from
       (
           select
               *
           from
               (
                   select
                       *,
                       CASE
                           when code = 1 then 0
                           else 5.3
                       end as pay
                   from
                       (
                           select
                               code,
                               __time as date_time
                               
                           from
                               test_datasource
                           where
                               __time >= '2020-02-27 08:00:00+08:00'
                               AND __time < '2021-02-28 07:59:59+08:00'
                               AND code in (0)
                               
                           group by
                               code,
                               __time
                       ) a
               ) b
           where
               pay > 0
       ) c
   group by
       code
   LIMIT
       10
   ```
   
   Expected result is 0, but got empty result!
   
   In order to reproduce this problem, we explain the above sql and add the following unit test into NestedQueryPushDownTest.java according to the explained plan (not the same but has the same pattern):
   
   ```Java
   @Test
     public void testVirtualColumnFilterOnInnerQuery()
     {
       QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(
           Collections.singletonList(Intervals.utc(1500000000000L, 1600000000000L))
       );
       GroupByQuery query = GroupByQuery
           .builder()
           .setDataSource("blah")
           .setDimensions(new DefaultDimensionSpec("dimA", "dimA"))
           .setAggregatorSpecs(
               new LongSumAggregatorFactory("metASum", "metA"),
               new LongSumAggregatorFactory("metBSum", "metB")
           )
           .setGranularity(Granularities.ALL)
           .setQuerySegmentSpec(intervalSpec)
           .build();
   
       GroupByQuery nestedQuery = GroupByQuery
           .builder()
           .setDataSource(query)
           .setDimensions(new DefaultDimensionSpec("dimA", "newDimA"))
           .setContext(
               ImmutableMap.of(
                   GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, false
               )
           )
           .setVirtualColumns(
               new ExpressionVirtualColumn(
                   "v0",
                   "case_searched((CAST(metBSum, 'LONG') == 10),0,5.3)",
                   ValueType.DOUBLE,
                   TestExprMacroTable.INSTANCE
               )
           )
           .setGranularity(Granularities.ALL)
           .setDimFilter(new BoundDimFilter("v0", "0", null, true, false, null, null, StringComparators.NUMERIC))
           .setQuerySegmentSpec(intervalSpec)
           .build();
   
       ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow(
           nestedQuery,
           "2017-07-14T02:40:00.000Z",
           "newDimA", "mango"
       );
   
       ResultRow expectedRow1 = GroupByQueryRunnerTestHelper.createExpectedRow(
           nestedQuery,
           "2017-07-14T02:40:00.000Z",
           "newDimA", "pomegranate"
       );
   
       QueryRunner<ResultRow> runner =
           groupByFactory.getToolchest().mergeResults(groupByFactory.mergeRunners(executorService, getQueryRunnerForSegment1()));
   
       Iterable<ResultRow> queryResult = GroupByQueryRunnerTestHelper.runQuery(groupByFactory, runner, nestedQuery);
       List<ResultRow> results = new ArrayList<>();
       queryResult.forEach(results::add);
   
       Assert.assertEquals(2, results.size());
       Assert.assertEquals(expectedRow0, results.get(0));
       Assert.assertEquals(expectedRow1, results.get(1));
     }
   ```
   Then we debug the above test.
   The dimFilter from the outer query will be used to filter out result rows of inner query (in method RowBasedGrouperHelper.getResultRowPredicate()). However, virtual column 'v0' does not exist in the result row signature of inner query lead to the results are filtered out.
   ![image](https://user-images.githubusercontent.com/8834263/114041840-7556ec00-98b7-11eb-81ca-864dae6ad897.png)
   
   The virtual column should be process or null value is returned to predicate which lead to data loss. 
   
   The patch is solved this issue and added the unit test.
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design (or naming) decision point and compare the alternatives with the designs that you've implemented (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), link to that discussion from this PR description and explain what have changed in your final design compared to your original proposal or the consensus version in the end of the discussion. If something hasn't changed since the original discussion, you can omit a detailed discussion of those aspects of the design here, perhaps apart from brief mentioning for the sake of readability of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `NestedQueryPushDownTest`
    * `RowBasedGrouperHelper`
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   


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


[GitHub] [druid] viongpanzi commented on pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on pull request #11081:
URL: https://github.com/apache/druid/pull/11081#issuecomment-816357088


   The failed test is irrelevant to this patch. Trigger the checks again.


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


[GitHub] [druid] clintropolis commented on a change in pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #11081:
URL: https://github.com/apache/druid/pull/11081#discussion_r610305869



##########
File path: processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
##########
@@ -398,8 +399,10 @@ public static ColumnSelectorFactory createResultRowBasedColumnSelectorFactory(
     );
 
     final SettableSupplier<ResultRow> rowSupplier = new SettableSupplier<>();
-    final ColumnSelectorFactory columnSelectorFactory =
-        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier);
+    final ColumnSelectorFactory columnSelectorFactory = new VirtualizedColumnSelectorFactory(
+        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier),
+        query.getVirtualColumns()
+    );

Review comment:
       
   ```suggestion
       final ColumnSelectorFactory columnSelectorFactory = query.getVirtualColumns().wrap(
           RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier)
       );
   ```
   would be more consistent with how other methods such as `createGrouperAccumulatorPair` are doing this 
   
   Since `VirtualColumns.wrap` is currently the only 'production' method that is calling `VirtualizedColumnSelectorFactory` constructor right now, so I think we should probably change this to use `wrap`




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


[GitHub] [druid] viongpanzi commented on pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on pull request #11081:
URL: https://github.com/apache/druid/pull/11081#issuecomment-817245822


   @jihoonson Thanks for helping to delete the debug code. 


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


[GitHub] [druid] jihoonson commented on a change in pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #11081:
URL: https://github.com/apache/druid/pull/11081#discussion_r610961151



##########
File path: processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
##########
@@ -584,6 +590,70 @@ public void testDimensionFilterOnOuterQuery()
     Assert.assertEquals(expectedRow0, results.get(0));
   }
 
+  @Test
+  public void testVirtualColumnFilterOnInnerQuery()

Review comment:
       This class is supposed to test groupBy queries with `GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY` enabled. Could you please move this test to somewhere else such as `GroupByQueryRunnerTest`?




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


[GitHub] [druid] viongpanzi commented on a change in pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on a change in pull request #11081:
URL: https://github.com/apache/druid/pull/11081#discussion_r611001070



##########
File path: processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
##########
@@ -584,6 +590,70 @@ public void testDimensionFilterOnOuterQuery()
     Assert.assertEquals(expectedRow0, results.get(0));
   }
 
+  @Test
+  public void testVirtualColumnFilterOnInnerQuery()

Review comment:
       done




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


[GitHub] [druid] jihoonson commented on a change in pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #11081:
URL: https://github.com/apache/druid/pull/11081#discussion_r611091726



##########
File path: processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
##########
@@ -10767,6 +10767,80 @@ public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown()
     TestHelper.assertExpectedObjects(expectedResults, results, "limit-pushdown");
   }
 
+  @Test
+  public void testVirtualColumnFilterOnInnerQuery()
+  {
+    GroupByQuery subquery = makeQueryBuilder()
+        .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
+        .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
+        .setDimensions(new DefaultDimensionSpec("quality", "alias"))
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                Collections.singletonList(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING)),
+                12
+            )
+        )
+        .setAggregatorSpecs(
+            QueryRunnerTestHelper.ROWS_COUNT,
+            new LongSumAggregatorFactory("idx", "index")
+        )
+        .setGranularity(QueryRunnerTestHelper.DAY_GRAN)
+        .build();
+
+    GroupByQuery query = makeQueryBuilder()
+        .setDataSource(subquery)
+        .setQuerySegmentSpec(
+            new MultipleIntervalSegmentSpec(
+                ImmutableList.of(
+                    Intervals.of("2011-04-01T00:00:00.000Z/2011-04-01T23:58:00.000Z"),
+                    Intervals.of("2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z")
+                )
+            )
+        ).setDimensions(new DefaultDimensionSpec("alias", "alias"))
+        .setVirtualColumns(
+            new ExpressionVirtualColumn(
+                "v",
+                "case_searched(idx > 1000, 1, 0)",
+                ValueType.LONG,
+                TestExprMacroTable.INSTANCE
+            )
+        )
+        .setDimFilter(
+            new BoundDimFilter(
+                "v",
+                "0",
+                null,
+                true,
+                false,
+                null,
+                null,
+                StringComparators.NUMERIC
+            )
+        )
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                Collections.singletonList(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.DESCENDING)),
+                15
+            )
+        )
+        .setAggregatorSpecs(
+            new LongSumAggregatorFactory("rows", "rows"),
+            new LongSumAggregatorFactory("idx", "idx")
+        )
+        .setGranularity(QueryRunnerTestHelper.DAY_GRAN)
+        .build();
+
+    List<ResultRow> expectedResults = Arrays.asList(
+        makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L),
+        makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L),
+        makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L)
+    );
+
+    Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    System.out.println(results);

Review comment:
       ```suggestion
   ```




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


[GitHub] [druid] jihoonson commented on pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #11081:
URL: https://github.com/apache/druid/pull/11081#issuecomment-817086492


   @viongpanzi thank you! The latest patch LGTM. +1 after Travis passes. I will retrigger the CI if any flaky job fails.


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


[GitHub] [druid] viongpanzi commented on a change in pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on a change in pull request #11081:
URL: https://github.com/apache/druid/pull/11081#discussion_r610317324



##########
File path: processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
##########
@@ -398,8 +399,10 @@ public static ColumnSelectorFactory createResultRowBasedColumnSelectorFactory(
     );
 
     final SettableSupplier<ResultRow> rowSupplier = new SettableSupplier<>();
-    final ColumnSelectorFactory columnSelectorFactory =
-        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier);
+    final ColumnSelectorFactory columnSelectorFactory = new VirtualizedColumnSelectorFactory(
+        RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(subquery, rowSupplier),
+        query.getVirtualColumns()
+    );

Review comment:
       Done




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


[GitHub] [druid] viongpanzi commented on pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on pull request #11081:
URL: https://github.com/apache/druid/pull/11081#issuecomment-816539033


   Travis CI failed due to too many docker requests.


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


[GitHub] [druid] viongpanzi commented on pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
viongpanzi commented on pull request #11081:
URL: https://github.com/apache/druid/pull/11081#issuecomment-816393531


   (https://repo.maven.apache.org/maven2): Read timed out, trigger the checks again! 


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


[GitHub] [druid] jihoonson merged pull request #11081: Nested GroupBy query got wrong/empty result when using virtual column and filter

Posted by GitBox <gi...@apache.org>.
jihoonson merged pull request #11081:
URL: https://github.com/apache/druid/pull/11081


   


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