mirror of https://github.com/apache/druid.git
[MSQ] Handle dimensionless group by queries with partitioning, and multiple workers (#14678)
* fixup * add ut * review
This commit is contained in:
parent
25df122b41
commit
8232c03667
|
@ -130,7 +130,7 @@ public class StageDefinition
|
||||||
this.shuffleCheckHasMultipleValues = shuffleCheckHasMultipleValues;
|
this.shuffleCheckHasMultipleValues = shuffleCheckHasMultipleValues;
|
||||||
this.frameReader = Suppliers.memoize(() -> FrameReader.create(signature))::get;
|
this.frameReader = Suppliers.memoize(() -> FrameReader.create(signature))::get;
|
||||||
|
|
||||||
if (mustGatherResultKeyStatistics() && shuffleSpec.clusterBy().getColumns().isEmpty()) {
|
if (mustGatherResultKeyStatistics() && shuffleSpec.clusterBy().isEmpty()) {
|
||||||
throw new IAE("Cannot shuffle with spec [%s] and nil clusterBy", shuffleSpec);
|
throw new IAE("Cannot shuffle with spec [%s] and nil clusterBy", shuffleSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -120,12 +120,14 @@ public class GroupByQueryKit implements QueryKit<GroupByQuery>
|
||||||
// (i.e. no GROUP BY clause)
|
// (i.e. no GROUP BY clause)
|
||||||
// __time in such queries is generated using either an aggregator (e.g. sum(metric) as __time) or using a
|
// __time in such queries is generated using either an aggregator (e.g. sum(metric) as __time) or using a
|
||||||
// post-aggregator (e.g. TIMESTAMP '2000-01-01' as __time)
|
// post-aggregator (e.g. TIMESTAMP '2000-01-01' as __time)
|
||||||
if (intermediateClusterBy.getColumns().isEmpty() && resultClusterBy.isEmpty()) {
|
if (intermediateClusterBy.isEmpty() && resultClusterBy.isEmpty()) {
|
||||||
// Ignore shuffleSpecFactory, since we know only a single partition will come out, and we can save some effort.
|
// Ignore shuffleSpecFactory, since we know only a single partition will come out, and we can save some effort.
|
||||||
shuffleSpecFactoryPreAggregation = ShuffleSpecFactories.singlePartition();
|
shuffleSpecFactoryPreAggregation = ShuffleSpecFactories.singlePartition();
|
||||||
shuffleSpecFactoryPostAggregation = ShuffleSpecFactories.singlePartition();
|
shuffleSpecFactoryPostAggregation = ShuffleSpecFactories.singlePartition();
|
||||||
} else if (doOrderBy) {
|
} else if (doOrderBy) {
|
||||||
shuffleSpecFactoryPreAggregation = ShuffleSpecFactories.globalSortWithMaxPartitionCount(maxWorkerCount);
|
shuffleSpecFactoryPreAggregation = intermediateClusterBy.isEmpty()
|
||||||
|
? ShuffleSpecFactories.singlePartition()
|
||||||
|
: ShuffleSpecFactories.globalSortWithMaxPartitionCount(maxWorkerCount);
|
||||||
shuffleSpecFactoryPostAggregation = doLimitOrOffset
|
shuffleSpecFactoryPostAggregation = doLimitOrOffset
|
||||||
? ShuffleSpecFactories.singlePartition()
|
? ShuffleSpecFactories.singlePartition()
|
||||||
: resultShuffleSpecFactory;
|
: resultShuffleSpecFactory;
|
||||||
|
|
|
@ -437,6 +437,40 @@ public class MSQInsertTest extends MSQTestBase
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInsertOnFoo1WithTimeAggregatorAndMultipleWorkers()
|
||||||
|
{
|
||||||
|
Map<String, Object> localContext = new HashMap<>(context);
|
||||||
|
localContext.put(MultiStageQueryContext.CTX_TASK_ASSIGNMENT_STRATEGY, WorkerAssignmentStrategy.MAX.name());
|
||||||
|
localContext.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 4);
|
||||||
|
|
||||||
|
RowSignature rowSignature = RowSignature.builder()
|
||||||
|
.add("__time", ColumnType.LONG)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
testIngestQuery().setSql(
|
||||||
|
"INSERT INTO foo1 "
|
||||||
|
+ "SELECT MILLIS_TO_TIMESTAMP((SUM(CAST(\"m1\" AS BIGINT)))) AS __time "
|
||||||
|
+ "FROM foo "
|
||||||
|
+ "PARTITIONED BY DAY"
|
||||||
|
)
|
||||||
|
.setExpectedDataSource("foo1")
|
||||||
|
.setExpectedRowSignature(rowSignature)
|
||||||
|
.setQueryContext(localContext)
|
||||||
|
.setExpectedSegment(
|
||||||
|
ImmutableSet.of(
|
||||||
|
SegmentId.of("foo1", Intervals.of("1970-01-01/P1D"), "test", 0)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.setExpectedResultRows(
|
||||||
|
ImmutableList.of(
|
||||||
|
new Object[]{21L}
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.verifyResults();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testInsertOnFoo1WithTimePostAggregator()
|
public void testInsertOnFoo1WithTimePostAggregator()
|
||||||
{
|
{
|
||||||
|
|
Loading…
Reference in New Issue