From 77fa36d06ff9ed10d5b4a4b2cdf166abd38417fb Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Mon, 24 Oct 2022 19:19:31 -0700 Subject: [PATCH] [hotfix] add integration test for distinct (#9637) * add integration test for distinct * fix distinct with limit can't be zero test Co-authored-by: Rong Rong --- .../tests/ClusterIntegrationTestUtils.java | 2 +- .../integration/tests/QueryGenerator.java | 47 +++++++++++++++---- 2 files changed, 38 insertions(+), 11 deletions(-) diff --git a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java index 6d7554f33d0a..52a27b47728e 100644 --- a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java +++ b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java @@ -612,7 +612,7 @@ private static void testQueryInternal(String pinotQuery, String brokerUrl, comparePinotResultsWithExpectedValues(expectedValues, expectedOrderByValues, resultTableResultSet, orderByColumns, pinotQuery, h2Query, h2NumRows, pinotNumRecordsSelected); } else { - if (queryContext.getGroupByExpressions() == null) { + if (queryContext.getGroupByExpressions() == null && !QueryContextUtils.isDistinctQuery(queryContext)) { // aggregation only // compare the single row diff --git a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java index d3967f51fcb7..c1e38453b934 100644 --- a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java +++ b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java @@ -768,29 +768,56 @@ private class AggregationQueryGenerationStrategy implements QueryGenerationStrat @Override public Query generateQuery() { - // Generate at most MAX_NUM_AGGREGATION_COLUMNS columns on which to aggregate, map 0 to 'COUNT(*)'. + // Generate at most MAX_NUM_GROUP_BY_COLUMNS columns on which to group. + int groupColumnCount = Math.min(RANDOM.nextInt(MAX_NUM_GROUP_BY_COLUMNS + 1), _singleValueColumnNames.size()); + Set groupColumns = new HashSet<>(); + while (groupColumns.size() < groupColumnCount) { + groupColumns.add(pickRandom(_singleValueColumnNames)); + } + + // Generate at most MAX_NUM_AGGREGATION_COLUMNS columns on which to aggregate int aggregationColumnCount = RANDOM.nextInt(MAX_NUM_AGGREGATION_COLUMNS + 1); Set aggregationColumnsAndFunctions = new HashSet<>(); + boolean isDistinctQuery = false; if (aggregationColumnCount == 0) { - aggregationColumnsAndFunctions.add("COUNT(*)"); + // if no aggregation function being randomly generated, pick the group by columns into the select list + // this should generate a distinct query using query rewriter. + // TODO: noted that we don't support distinct/agg rewrite with only part of the group by columns. change this + // test once we support such queries. + if (groupColumnCount != 0) { + aggregationColumnsAndFunctions.addAll(groupColumns); + isDistinctQuery = true; + } else { + aggregationColumnsAndFunctions.add("COUNT(*)"); + } } else { while (aggregationColumnsAndFunctions.size() < aggregationColumnCount) { aggregationColumnsAndFunctions.add(createRandomAggregationFunction()); } } + // Generate a predicate. PredicateQueryFragment predicate = generatePredicate(); - // Generate at most MAX_NUM_GROUP_BY_COLUMNS columns on which to group. - int groupColumnCount = Math.min(RANDOM.nextInt(MAX_NUM_GROUP_BY_COLUMNS + 1), _singleValueColumnNames.size()); - Set groupColumns = new HashSet<>(); - while (groupColumns.size() < groupColumnCount) { - groupColumns.add(pickRandom(_singleValueColumnNames)); - } + //Generate a HAVING predicate ArrayList arrayOfAggregationColumnsAndFunctions = new ArrayList<>(aggregationColumnsAndFunctions); - HavingQueryFragment havingPredicate = generateHavingPredicate(arrayOfAggregationColumnsAndFunctions); + HavingQueryFragment havingPredicate; + if (isDistinctQuery) { + havingPredicate = new HavingQueryFragment(Collections.emptyList(), Collections.emptyList(), + Collections.emptyList()); + } else { + havingPredicate = generateHavingPredicate(arrayOfAggregationColumnsAndFunctions); + } + // Generate a result limit of at most MAX_RESULT_LIMIT. - LimitQueryFragment limit = new LimitQueryFragment(RANDOM.nextInt(MAX_RESULT_LIMIT + 1)); + LimitQueryFragment limit; + if (isDistinctQuery) { + // Distinct query must have positive LIMIT + limit = new LimitQueryFragment(RANDOM.nextInt(MAX_RESULT_LIMIT) + 1); + } else { + limit = new LimitQueryFragment(RANDOM.nextInt(MAX_RESULT_LIMIT + 1)); + } + return new AggregationQuery(arrayOfAggregationColumnsAndFunctions, predicate, groupColumns, havingPredicate, limit); }