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 2019/07/22 22:25:22 UTC

[GitHub] [incubator-druid] himanshug commented on a change in pull request #8109: fix merging of groupBy subtotal spec results

himanshug commented on a change in pull request #8109: fix merging of groupBy subtotal spec results
URL: https://github.com/apache/incubator-druid/pull/8109#discussion_r306058870
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
 ##########
 @@ -419,36 +436,81 @@ public Row apply(final Row row)
               false
           )
       );
+
+
       List<Sequence<Row>> subtotalsResults = new ArrayList<>(subtotals.size());
 
       Map<String, DimensionSpec> queryDimensionSpecs = new HashMap(queryWithoutSubtotalsSpec.getDimensions().size());
+      List<String> queryDimNames = new ArrayList(queryWithoutSubtotalsSpec.getDimensions().size());
+
       for (DimensionSpec dimSpec : queryWithoutSubtotalsSpec.getDimensions()) {
         queryDimensionSpecs.put(dimSpec.getOutputName(), dimSpec);
+        queryDimNames.add(dimSpec.getOutputName());
       }
 
       for (List<String> subtotalSpec : subtotals) {
         GroupByQuery subtotalQuery = queryWithoutSubtotalsSpec.withDimensionSpecs(
             subtotalSpec.stream()
-                        .map(s -> new DefaultDimensionSpec(s, s, queryDimensionSpecs.get(s).getOutputType()))
+                        .map(queryDimensionSpecs::get)
                         .collect(Collectors.toList())
         );
 
-        subtotalsResults.add(applyPostProcessing(
-            mergeResults(new QueryRunner<Row>()
-            {
-              @Override
-              public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+        if (Utils.isPrefix(subtotalSpec, queryDimNames)) {
+          subtotalsResults.add(applyPostProcessing(
+              mergeResults(new QueryRunner<Row>()
               {
-                return GroupByRowProcessor.getRowsFromGrouper(
-                    queryWithoutSubtotalsSpec,
-                    subtotalSpec,
-                    grouperSupplier
-                );
-              }
-            }, subtotalQuery, null),
-            subtotalQuery
-                             )
-        );
+                @Override
+                public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+                {
+                  return GroupByRowProcessor.getRowsFromGrouper(
+                      queryWithoutSubtotalsSpec,
+                      subtotalSpec,
+                      grouperSupplier
+                  );
+                }
+              }, subtotalQuery, null),
+              subtotalQuery
+                               )
+          );
+        } else {
+          subtotalsResults.add(applyPostProcessing(
+              mergeResults(new QueryRunner<Row>()
+              {
+                @Override
+                public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
+                {
+                  List<Closeable> closeables = new ArrayList<>();
+
+                  Sequence<Row> result = GroupByRowProcessor.getRowsFromGrouper(
+                      subtotalQuery,
+                      subtotalSpec,
+                      Suppliers.memoize(() -> GroupByRowProcessor.createGrouper(
+                          subtotalQuery,
+                          GroupByRowProcessor.getRowsFromGrouper(
+                              queryWithoutSubtotalsSpec,
+                              subtotalSpec,
+                              grouperSupplier
+                          ),
+                          GroupByQueryHelper.rowSignatureFor(subtotalQuery),
+                          configSupplier.get(),
+                          resource,
+                          spillMapper,
+                          processingConfig.getTmpDir(),
+                          processingConfig.intermediateComputeSizeBytes(),
+                          closeables,
+                          false,
+                          false
+                                        )
+                      )
+                  );
+
+                  return Sequences.withBaggage(result, () -> Lists.reverse(closeables).forEach(closeable -> CloseQuietly.close(closeable)));
+                }
+              }, subtotalQuery, null),
 
 Review comment:
   it was passing the intellij formatter , introduced few newlines and it looks better now.

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