From 70c05bb23d25e911a834e03ac5a9ed284ee43771 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 5 Sep 2024 13:45:25 +0530 Subject: [PATCH 01/26] Add GlueingPartitioningOperator + Corresponding changes in window function layer to consume it for MSQ --- .../WindowOperatorQueryFrameProcessor.java | 3 +- .../msq/querykit/WindowOperatorQueryKit.java | 16 +- .../apache/druid/msq/exec/MSQWindowTest.java | 50 ++-- .../operator/GlueingPartitioningOperator.java | 261 ++++++++++++++++++ .../GlueingPartitioningOperatorFactory.java | 88 ++++++ .../operator/NaivePartitioningOperator.java | 2 +- .../apache/druid/query/operator/Operator.java | 4 +- .../druid/query/operator/OperatorFactory.java | 1 + ...lueingPartitioningOperatorFactoryTest.java | 34 +++ .../GlueingPartitioningOperatorTest.java | 199 +++++++++++++ .../NaivePartitioningOperatorTest.java | 43 +++ .../druid/sql/calcite/rel/Windowing.java | 26 +- 12 files changed, 686 insertions(+), 41 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index 3dc62f3a60de..a679f726cb0c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -213,7 +213,8 @@ Current approach with R&C and operators materialize a single R&C for processing. Most of the window operations like SUM(), RANK(), RANGE() etc. can be made with 2 passes of the data. We might think to reimplement them in the MSQ way so that we do not have to materialize so much data. */ - if (partitionColumnNames.isEmpty()) { + // todo: need to cleanup unused code and comments + if (true) { // Scenario 1: Query has atleast one window function with an OVER() clause without a PARTITION BY. if (inputChannel.canRead()) { final Frame frame = inputChannel.read(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index b3686359d2a4..a49c42fd1e34 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -37,7 +37,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Query; import org.apache.druid.query.operator.ColumnWithDirection; -import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; +import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; @@ -81,8 +81,8 @@ public QueryDefinition makeQueryDefinition( boolean isEmptyOverPresent = originalQuery.getOperators() .stream() - .filter(of -> of instanceof NaivePartitioningOperatorFactory) - .map(of -> (NaivePartitioningOperatorFactory) of) + .filter(of -> of instanceof GlueingPartitioningOperatorFactory) + .map(of -> (GlueingPartitioningOperatorFactory) of) .anyMatch(of -> of.getPartitionColumns().isEmpty()); List> operatorList = getOperatorListFromQuery(originalQuery); @@ -208,8 +208,8 @@ public QueryDefinition makeQueryDefinition( boolean partitionOperatorExists = false; List currentPartitionColumns = new ArrayList<>(); for (OperatorFactory of : operatorList.get(i)) { - if (of instanceof NaivePartitioningOperatorFactory) { - for (String s : ((NaivePartitioningOperatorFactory) of).getPartitionColumns()) { + if (of instanceof GlueingPartitioningOperatorFactory) { + for (String s : ((GlueingPartitioningOperatorFactory) of).getPartitionColumns()) { currentPartitionColumns.add(s); partitionOperatorExists = true; } @@ -287,11 +287,11 @@ private List> getOperatorListFromQuery(WindowOperatorQuery private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount) { - NaivePartitioningOperatorFactory partition = null; + GlueingPartitioningOperatorFactory partition = null; NaiveSortOperatorFactory sort = null; for (OperatorFactory of : operatorFactories) { - if (of instanceof NaivePartitioningOperatorFactory) { - partition = (NaivePartitioningOperatorFactory) of; + if (of instanceof GlueingPartitioningOperatorFactory) { + partition = (GlueingPartitioningOperatorFactory) of; } else if (of instanceof NaiveSortOperatorFactory) { sort = (NaiveSortOperatorFactory) of; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 3a1e3d95f800..218065730dab 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -45,7 +45,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.operator.ColumnWithDirection; -import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; +import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.operator.window.WindowFrame; @@ -125,7 +125,7 @@ public void testWindowOnFooWithPartitionByAndInnerGroupBy(String contextName, Ma RowSignature.builder().add("d0", ColumnType.FLOAT).add("w0", ColumnType.DOUBLE).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), null @@ -223,10 +223,10 @@ public void testWindowOnFooWithFirstWindowPartitionNextWindowEmpty(String contex .add("w1", ColumnType.DOUBLE) .build(), ImmutableList.of( - new NaivePartitioningOperatorFactory(ImmutableList.of()), + new GlueingPartitioningOperatorFactory(ImmutableList.of()), new WindowOperatorFactory(proc1), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), null @@ -334,10 +334,10 @@ public void testWindowOnFooWith2WindowsBothWindowsHavingPartitionByInnerGroupBy( .build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d1")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d1")), new WindowOperatorFactory(proc1) ), null @@ -447,10 +447,10 @@ public void testWindowOnFooWith2WindowsBothPartitionByWithOrderReversed( .build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d1")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d1")), new WindowOperatorFactory(proc), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc1) ), null @@ -541,7 +541,7 @@ public void testWindowOnFooWithEmptyOverWithGroupBy(String contextName, Map cont RowSignature.builder().add("dim2", ColumnType.STRING).add("w0", ColumnType.DOUBLE).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("dim2"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("dim2")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("dim2")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1340,7 +1340,7 @@ public void testWindowOnFooWithEmptyOverWithUnnest(String contextName, Map cont .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("cityName"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("cityName")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("cityName")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1894,7 +1894,7 @@ public void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("countryIsoCode"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("countryIsoCode")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("countryIsoCode")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1991,7 +1991,7 @@ public void testGroupByWithWikipedia(String contextName, Map con .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), + new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -2121,10 +2121,10 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers(String cont .add("w1", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"), ColumnWithDirection.ascending("d1"), ColumnWithDirection.ascending("d2"))), - new NaivePartitioningOperatorFactory(Collections.emptyList()), + new GlueingPartitioningOperatorFactory(Collections.emptyList()), new WindowOperatorFactory(new WindowRowNumberProcessor("w0")), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"), ColumnWithDirection.ascending("d0"), ColumnWithDirection.ascending("d2"))), - new NaivePartitioningOperatorFactory(Collections.singletonList("d1")), + new GlueingPartitioningOperatorFactory(Collections.singletonList("d1")), new WindowOperatorFactory(new WindowFramedAggregateProcessor(WindowFrame.forOrderBy("d0", "d1", "d2"), aggs)) ), ImmutableList.of() diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java new file mode 100644 index 000000000000..f8ff68f0bc21 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.query.rowsandcols.ConcatRowsAndColumns; +import org.apache.druid.query.rowsandcols.LimitedRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; +import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +/** + * todo: write detailed javadoc for the class, all the methods, etc. + */ +public class GlueingPartitioningOperator implements Operator +{ + private final List partitionColumns; + private final Operator child; + private RowsAndColumns previousRac; + + public GlueingPartitioningOperator( + List partitionColumns, + Operator child + ) + { + this.partitionColumns = partitionColumns; + this.child = child; + } + + @Override + public Closeable goOrContinue(Closeable continuation, Receiver receiver) + { + if (continuation != null) { + Continuation cont = (Continuation) continuation; + + if (cont.iter != null) { + while (cont.iter.hasNext()) { + RowsAndColumns next = cont.iter.next(); + + if (!cont.iter.hasNext()) { + // We are at the last RAC. Process it only if subContinuation is null, otherwise save it in previousRac. + if (cont.subContinuation == null) { + receiver.push(next); + receiver.completed(); + return null; + } else { + previousRac = next; + break; + } + } + + final Signal signal = receiver.push(next); + switch (signal) { + case GO: + break; + case PAUSE: + if (cont.iter.hasNext()) { + return cont; + } + + if (cont.subContinuation == null) { + // We were finished anyway + receiver.completed(); + return null; + } + + return new Continuation(null, cont.subContinuation); + case STOP: + receiver.completed(); + try { + cont.close(); + } + catch (IOException e) { + throw new RE(e, "Unable to close continuation"); + } + return null; + default: + throw new RE("Unknown signal[%s]", signal); + } + } + + if (cont.subContinuation == null) { + receiver.completed(); + return null; + } + } + + continuation = cont.subContinuation; + } + + AtomicReference> iterHolder = new AtomicReference<>(); + + final Closeable retVal = child.goOrContinue( + continuation, + new Receiver() + { + @Override + public Signal push(RowsAndColumns rac) + { + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } + + ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); + if (groupPartitioner == null) { + groupPartitioner = new DefaultClusteredGroupPartitioner(rac); + } + + int[] computedBoundaries = groupPartitioner.computeBoundaries(partitionColumns); + + final ArrayList gluedRACs = getGluedRACs(rac, computedBoundaries); + Iterator partitionsIter = gluedRACs.iterator(); + + Signal keepItGoing = Signal.GO; + while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { + RowsAndColumns rowsAndColumns = partitionsIter.next(); + if (partitionsIter.hasNext()) { + keepItGoing = receiver.push(rowsAndColumns); + } else { + // If it's the last element, save it in previousRac instead of pushing to receiver. + previousRac = rowsAndColumns; + } + } + + if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + + return keepItGoing; + } + + @Override + public void completed() + { + if (previousRac != null) { + receiver.push(previousRac); + previousRac = null; + } + if (iterHolder.get() == null) { + receiver.completed(); + } + } + } + ); + + if (iterHolder.get() != null || retVal != null) { + return new Continuation( + iterHolder.get(), + retVal + ); + } else { + return null; + } + } + + private static class Continuation implements Closeable + { + Iterator iter; + Closeable subContinuation; + + public Continuation(Iterator iter, Closeable subContinuation) + { + this.iter = iter; + this.subContinuation = subContinuation; + } + + @Override + public void close() throws IOException + { + if (subContinuation != null) { + subContinuation.close(); + } + } + } + + private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + { + if (previousRac == null) { + return false; + } + + final ConcatRowsAndColumns concatRac = getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac); + for (String column : partitionColumns) { + final Column theCol = concatRac.findColumn(column); + if (theCol == null) { + // The column doesn't exist. In this case, we assume it's always the same value: null. If it's always + // the same, then it doesn't impact grouping at all and can be entirely skipped. + continue; + } + final ColumnAccessor accessor = theCol.toAccessor(); + int comparison = accessor.compareRows(0, previousRac.numRows()); // Compare 1st row of previousRac and currentRac in [previousRac, currentRac] form. + if (comparison != 0) { + return false; + } + } + return true; + } + + private ArrayList getGluedRACs(RowsAndColumns rac, int[] boundaries) + { + final ArrayList gluedRACs = new ArrayList<>(); + for (int i = 1; i < boundaries.length; ++i) { + int start = boundaries[i - 1]; + int end = boundaries[i]; + final LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); + if (i == 1) { + handleFirstPartition(gluedRACs, limitedRAC); + } else { + gluedRACs.add(limitedRAC); + } + } + return gluedRACs; + } + + private void handleFirstPartition(ArrayList gluedRACs, LimitedRowsAndColumns firstPartitionOfCurrentRac) + { + if (isGlueingNeeded(previousRac, firstPartitionOfCurrentRac)) { + gluedRACs.add(getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac)); + } else { + if (previousRac != null) { + gluedRACs.add(previousRac); + } + gluedRACs.add(firstPartitionOfCurrentRac); + } + previousRac = null; + } + + private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + { + return new ConcatRowsAndColumns(new ArrayList<>(Arrays.asList(previousRac, firstPartitionOfCurrentRac))); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java new file mode 100644 index 000000000000..56b066828b37 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public class GlueingPartitioningOperatorFactory implements OperatorFactory +{ + private final List partitionColumns; + + @JsonCreator + public GlueingPartitioningOperatorFactory( + @JsonProperty("partitionColumns") List partitionColumns + ) + { + this.partitionColumns = partitionColumns == null ? new ArrayList<>() : partitionColumns; + } + + @JsonProperty("partitionColumns") + public List getPartitionColumns() + { + return partitionColumns; + } + + @Override + public Operator wrap(Operator op) + { + return new GlueingPartitioningOperator(partitionColumns, op); + } + + @Override + public boolean validateEquivalent(OperatorFactory other) + { + if (other instanceof GlueingPartitioningOperatorFactory) { + return partitionColumns.equals(((GlueingPartitioningOperatorFactory) other).getPartitionColumns()); + } + return false; + } + + @Override + public String toString() + { + return "GlueingPartitioningOperatorFactory{" + + "partitionColumns=" + partitionColumns + + '}'; + } + + @Override + public final int hashCode() + { + return Objects.hash(partitionColumns); + } + + @Override + public final boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || obj.getClass() != getClass()) { + return false; + } + GlueingPartitioningOperatorFactory other = (GlueingPartitioningOperatorFactory) obj; + return Objects.equals(partitionColumns, other.partitionColumns); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 0c68c3eea276..c2d2e0aa8100 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -84,7 +84,7 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) cont.close(); } catch (IOException e) { - throw new RE(e, "Unable to close continutation"); + throw new RE(e, "Unable to close continuation"); } return null; default: diff --git a/processing/src/main/java/org/apache/druid/query/operator/Operator.java b/processing/src/main/java/org/apache/druid/query/operator/Operator.java index 57bc1013fc44..6609f2455420 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/Operator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/Operator.java @@ -68,7 +68,7 @@ static void go(Operator op, Receiver receiver) * to indicate its degree of readiness for more data to be received. *

* If a Receiver returns a {@link Signal#PAUSE} signal, then if there is processing left to do, then it is expected - * that a non-null "continuation" object nwill be returned. This allows for flow control to be returned to the + * that a non-null "continuation" object will be returned. This allows for flow control to be returned to the * caller to, e.g., process another Operator or just exert backpressure. In this case, when the controller wants to * resume, it must call this method again and include the continuation object that it received. *

@@ -99,7 +99,7 @@ static void go(Operator op, Receiver receiver) * if there is any state that an Operator requires to be able to resume its processing, then it is expected that the * Operator will cast the object back to an instance of the type that it had originally returned. * - * @param receiver a receiver that will receiver data + * @param receiver a receiver that will receive data * @return null if processing is complete, non-null if the Receiver returned a {@link Signal#PAUSE} signal */ @Nullable diff --git a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java index a97c332505cf..a7d77aeb2d26 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java @@ -31,6 +31,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "naivePartition", value = NaivePartitioningOperatorFactory.class), + @JsonSubTypes.Type(name = "glueingPartition", value = GlueingPartitioningOperatorFactory.class), @JsonSubTypes.Type(name = "naiveSort", value = NaiveSortOperatorFactory.class), @JsonSubTypes.Type(name = "window", value = WindowOperatorFactory.class), @JsonSubTypes.Type(name = "scan", value = ScanOperatorFactory.class), diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactoryTest.java new file mode 100644 index 000000000000..cb5b0404f8be --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactoryTest.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class GlueingPartitioningOperatorFactoryTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(GlueingPartitioningOperatorFactory.class) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java new file mode 100644 index 000000000000..365a7f4ab728 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Test; + +import java.util.function.BiFunction; + +public class GlueingPartitioningOperatorTest +{ + @Test + public void testDefaultImplementation() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + ImmutableList.of("sorted"), + inlineScanOperator + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2}) + .expectColumn("unsorted", new int[]{20, 20}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1}) + .expectColumn("unsorted", new int[]{11}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testDefaultImplementationWithMultipleRACs() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{50, 51, 52, 53, 54, 55}) + ) + ); + RowsAndColumns rac3 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{70, 71, 72, 73, 74}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2, rac3); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + ImmutableList.of("sorted"), + inlineScanOperator + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2}) + .expectColumn("unsorted", new int[]{20, 20}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1, 1}) + .expectColumn("unsorted", new int[]{11, 50, 51, 52}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2}) + .expectColumn("unsorted", new int[]{53, 54}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1}) + .expectColumn("unsorted", new int[]{55, 70, 71}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2}) + .expectColumn("unsorted", new int[]{72, 73}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1}) + .expectColumn("unsorted", new int[]{74}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testStopMidStream() + { + RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + ImmutableList.of("sorted"), + inlineScanOperator + ); + + new OperatorTestHelper() + .expectAndStopAfter( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2}) + .expectColumn("unsorted", new int[]{20, 20}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testDoesNotValidateSort() + { + BiFunction singleHelperMaker = + (sorted, unsorted) -> + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{sorted}) + .expectColumn("unsorted", new int[]{unsorted}) + .allColumnsRegistered(); + + RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), + "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) + ) + ); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + ImmutableList.of("unsorted"), + InlineScanOperator.make(rac) + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + singleHelperMaker.apply(0, 3), + singleHelperMaker.apply(0, 54), + singleHelperMaker.apply(0, 21), + singleHelperMaker.apply(1, 1), + singleHelperMaker.apply(1, 5), + singleHelperMaker.apply(2, 54), + singleHelperMaker.apply(4, 2), + singleHelperMaker.apply(4, 3), + singleHelperMaker.apply(4, 92) + ) + .runToCompletion(op); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java index 07ab0c12aa0a..6dcd2848bc6a 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java @@ -68,6 +68,49 @@ public void testDefaultImplementation() .runToCompletion(op); } + @Test + public void testDefaultImplementationWithMultipleRACs() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1}), + "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 2, 2, 2}), + "unsorted", new IntArrayColumn(new int[]{10, 20, 30, 40}) + ) + ); + + NaivePartitioningOperator op = new NaivePartitioningOperator( + ImmutableList.of("sorted"), + InlineScanOperator.make(rac1, rac2) + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{0, 0, 0}) + .expectColumn("unsorted", new int[]{3, 54, 21}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1}) + .expectColumn("unsorted", new int[]{1, 5}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1}) + .expectColumn("unsorted", new int[]{10}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2, 2}) + .expectColumn("unsorted", new int[]{20, 30, 40}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + @Test public void testStopMidStream() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 8e1fc3ee2750..33d284d1f490 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -40,6 +40,7 @@ import org.apache.druid.query.QueryException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; @@ -63,6 +64,8 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rule.GroupByRules; +import org.apache.druid.sql.calcite.run.NativeSqlEngine; +import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nonnull; @@ -208,7 +211,12 @@ public static Windowing fromCalciteStuff( ))); } - List ops = computeWindowOperations(partialQuery, sourceRowSignature, windowGroupProcessors); + List ops = computeWindowOperations( + partialQuery, + sourceRowSignature, + windowGroupProcessors, + plannerContext.getEngine() + ); // Apply windowProject, if present. if (partialQuery.getWindowProject() != null) { @@ -248,7 +256,8 @@ public static Windowing fromCalciteStuff( private static List computeWindowOperations( final PartialDruidQuery partialQuery, final RowSignature sourceRowSignature, - List windowGroupProcessors + List windowGroupProcessors, + SqlEngine sqlEngine ) { final List ops = new ArrayList<>(); @@ -280,12 +289,21 @@ private static List computeWindowOperations( if (!sortMatches(priorSortColumns, sortColumns)) { // Sort order needs to change. Resort and repartition. ops.add(new NaiveSortOperatorFactory(new ArrayList<>(sortColumns))); - ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); + if (sqlEngine instanceof NativeSqlEngine) { + ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); + } else { + ops.add(new GlueingPartitioningOperatorFactory(group.getPartitionColumns())); + } priorSortColumns = sortColumns; priorPartitionColumns = group.getPartitionColumns(); } else if (!group.getPartitionColumns().equals(priorPartitionColumns)) { // Sort order doesn't need to change, but partitioning does. Only repartition. - ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); + if (sqlEngine instanceof NativeSqlEngine) { + ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); + } else { + ops.add(new GlueingPartitioningOperatorFactory(group.getPartitionColumns())); + } + priorPartitionColumns = group.getPartitionColumns(); } From dad20ee566ae63ad3259b041dd2cf02d4903f7e6 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Fri, 13 Sep 2024 12:01:05 +0530 Subject: [PATCH 02/26] Create GluedRACsIterator --- .../operator/GlueingPartitioningOperator.java | 124 ++++++++++-------- 1 file changed, 72 insertions(+), 52 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index f8ff68f0bc21..4a266b7cd47a 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -35,6 +35,7 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicReference; /** @@ -129,15 +130,7 @@ public Signal push(RowsAndColumns rac) throw DruidException.defensive("Should never get a null rac here."); } - ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); - if (groupPartitioner == null) { - groupPartitioner = new DefaultClusteredGroupPartitioner(rac); - } - - int[] computedBoundaries = groupPartitioner.computeBoundaries(partitionColumns); - - final ArrayList gluedRACs = getGluedRACs(rac, computedBoundaries); - Iterator partitionsIter = gluedRACs.iterator(); + Iterator partitionsIter = new GluedRACsIterator(rac); Signal keepItGoing = Signal.GO; while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { @@ -202,60 +195,87 @@ public void close() throws IOException } } - private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + private class GluedRACsIterator implements Iterator { - if (previousRac == null) { - return false; + private final RowsAndColumns rac; + private final int[] boundaries; + private int currentIndex = 0; + private boolean firstPartitionHandled = false; + + public GluedRACsIterator(RowsAndColumns rac) + { + this.rac = rac; + ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); + if (groupPartitioner == null) { + groupPartitioner = new DefaultClusteredGroupPartitioner(rac); + } + this.boundaries = groupPartitioner.computeBoundaries(partitionColumns); + } + + @Override + public boolean hasNext() + { + return currentIndex < boundaries.length - 1; } - final ConcatRowsAndColumns concatRac = getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac); - for (String column : partitionColumns) { - final Column theCol = concatRac.findColumn(column); - if (theCol == null) { - // The column doesn't exist. In this case, we assume it's always the same value: null. If it's always - // the same, then it doesn't impact grouping at all and can be entirely skipped. - continue; + @Override + public RowsAndColumns next() + { + if (!hasNext()) { + throw new NoSuchElementException(); } - final ColumnAccessor accessor = theCol.toAccessor(); - int comparison = accessor.compareRows(0, previousRac.numRows()); // Compare 1st row of previousRac and currentRac in [previousRac, currentRac] form. - if (comparison != 0) { - return false; + + if (!firstPartitionHandled) { + firstPartitionHandled = true; + int start = boundaries[currentIndex]; + int end = boundaries[currentIndex + 1]; + LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); + + if (isGlueingNeeded(previousRac, limitedRAC)) { + RowsAndColumns gluedRAC = getConcatRacForFirstPartition(previousRac, limitedRAC); + previousRac = null; + currentIndex++; + return gluedRAC; + } else { + if (previousRac != null) { + RowsAndColumns temp = previousRac; + previousRac = null; + return temp; + } + } } + + int start = boundaries[currentIndex]; + int end = boundaries[currentIndex + 1]; + currentIndex++; + return new LimitedRowsAndColumns(rac, start, end); } - return true; - } - private ArrayList getGluedRACs(RowsAndColumns rac, int[] boundaries) - { - final ArrayList gluedRACs = new ArrayList<>(); - for (int i = 1; i < boundaries.length; ++i) { - int start = boundaries[i - 1]; - int end = boundaries[i]; - final LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); - if (i == 1) { - handleFirstPartition(gluedRACs, limitedRAC); - } else { - gluedRACs.add(limitedRAC); + private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + { + if (previousRac == null) { + return false; } - } - return gluedRACs; - } - private void handleFirstPartition(ArrayList gluedRACs, LimitedRowsAndColumns firstPartitionOfCurrentRac) - { - if (isGlueingNeeded(previousRac, firstPartitionOfCurrentRac)) { - gluedRACs.add(getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac)); - } else { - if (previousRac != null) { - gluedRACs.add(previousRac); + final ConcatRowsAndColumns concatRac = getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac); + for (String column : partitionColumns) { + final Column theCol = concatRac.findColumn(column); + if (theCol == null) { + continue; + } + final ColumnAccessor accessor = theCol.toAccessor(); + // Compare 1st row of previousRac and firstPartitionOfCurrentRac in [previousRac, firstPartitionOfCurrentRac] form. + int comparison = accessor.compareRows(0, previousRac.numRows()); + if (comparison != 0) { + return false; + } } - gluedRACs.add(firstPartitionOfCurrentRac); + return true; } - previousRac = null; - } - private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) - { - return new ConcatRowsAndColumns(new ArrayList<>(Arrays.asList(previousRac, firstPartitionOfCurrentRac))); + private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + { + return new ConcatRowsAndColumns(new ArrayList<>(Arrays.asList(previousRac, firstPartitionOfCurrentRac))); + } } } From 52131ea227da9ab3f006420eb83df4f95b3ac2ed Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 18 Sep 2024 18:22:07 +0530 Subject: [PATCH 03/26] Modify MSQ window function execution to allow sending RACs into the operator pipeline without any extra logic --- .../WindowOperatorQueryFrameProcessor.java | 282 +++------------ ...dowOperatorQueryFrameProcessorFactory.java | 24 +- .../msq/querykit/WindowOperatorQueryKit.java | 236 ++++++------- .../apache/druid/msq/exec/MSQWindowTest.java | 102 ++++-- ...peratorQueryFrameProcessorFactoryTest.java | 2 +- ...WindowOperatorQueryFrameProcessorTest.java | 4 +- .../msq1.iq | 321 ++---------------- .../operator/BasePartitioningOperator.java | 121 +++++++ .../BasePartitioningOperatorFactory.java | 83 +++++ .../query/operator/BaseSortOperator.java | 44 +++ .../operator/BaseSortOperatorFactory.java | 82 +++++ .../operator/GlueingPartitioningOperator.java | 156 +++++---- .../GlueingPartitioningOperatorFactory.java | 38 +-- .../operator/NaivePartitioningOperator.java | 88 +---- .../NaivePartitioningOperatorFactory.java | 50 +-- .../query/operator/NaiveSortOperator.java | 8 +- .../operator/NaiveSortOperatorFactory.java | 47 +-- .../druid/query/operator/OperatorFactory.java | 1 + .../query/operator/PartitionSortOperator.java | 66 ++++ .../PartitionSortOperatorFactory.java | 42 +++ .../GlueingPartitioningOperatorTest.java | 234 ++++++++++++- .../PartitionSortOperatorFactoryTest.java | 34 ++ .../operator/PartitionSortOperatorTest.java | 58 ++++ .../druid/sql/calcite/rel/Windowing.java | 15 +- 24 files changed, 1129 insertions(+), 1009 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index a679f726cb0c..25c25ba1a415 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -38,23 +38,18 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsInAWindowFault; -import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.operator.Operator; import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.rowsandcols.ConcatRowsAndColumns; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; -import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.concrete.RowBasedFrameRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ColumnSelectorFactoryMaker; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.Cursor; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.NullableTypeStrategy; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; @@ -64,38 +59,27 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; public class WindowOperatorQueryFrameProcessor implements FrameProcessor { private static final Logger log = new Logger(WindowOperatorQueryFrameProcessor.class); - private final WindowOperatorQuery query; private final List operatorFactoryList; - private final List partitionColumnNames; - private final ObjectMapper jsonMapper; private final ArrayList frameRowsAndCols; private final ArrayList resultRowAndCols; + private int numRowsInFrameRowsAndCols; private final ReadableFrameChannel inputChannel; private final WritableFrameChannel outputChannel; private final FrameWriterFactory frameWriterFactory; private final FrameReader frameReader; private final int maxRowsMaterialized; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed - private Cursor frameCursor = null; - private Supplier rowSupplierFromFrameCursor; - private ResultRow outputRow = null; private FrameWriter frameWriter = null; private final VirtualColumns frameWriterVirtualColumns; private final SettableLongVirtualColumn partitionBoostVirtualColumn; - // List of type strategies to compare the partition columns across rows. - // Type strategies are pushed in the same order as column types in frameReader.signature() - private final NullableTypeStrategy[] typeStrategies; - - private final ArrayList rowsToProcess; - private int lastPartitionIndex = -1; + private Operator op = null; public WindowOperatorQueryFrameProcessor( WindowOperatorQuery query, @@ -105,28 +89,18 @@ public WindowOperatorQueryFrameProcessor( FrameReader frameReader, ObjectMapper jsonMapper, final List operatorFactoryList, - final RowSignature rowSignature, - final int maxRowsMaterializedInWindow, - final List partitionColumnNames + final int maxRowsMaterializedInWindow ) { this.inputChannel = inputChannel; this.outputChannel = outputChannel; this.frameWriterFactory = frameWriterFactory; this.operatorFactoryList = operatorFactoryList; - this.jsonMapper = jsonMapper; - this.query = query; this.frameRowsAndCols = new ArrayList<>(); this.resultRowAndCols = new ArrayList<>(); - this.rowsToProcess = new ArrayList<>(); this.maxRowsMaterialized = maxRowsMaterializedInWindow; - this.partitionColumnNames = partitionColumnNames; this.frameReader = frameReader; - this.typeStrategies = new NullableTypeStrategy[frameReader.signature().size()]; - for (int i = 0; i < frameReader.signature().size(); i++) { - typeStrategies[i] = frameReader.signature().getColumnType(i).get().getNullableStrategy(); - } // Get virtual columns to be added to the frame writer. this.partitionBoostVirtualColumn = new SettableLongVirtualColumn(QueryKitUtils.PARTITION_BOOST_COLUMN); @@ -137,6 +111,8 @@ public WindowOperatorQueryFrameProcessor( frameWriterVirtualColumns.add(segmentGranularityVirtualColumn); } this.frameWriterVirtualColumns = VirtualColumns.create(frameWriterVirtualColumns); + + initialiseOperator(); } @Override @@ -154,151 +130,60 @@ public List outputChannels() @Override public ReturnOrAwait runIncrementally(IntSet readableInputs) { - /* - There are 2 scenarios: - - *** Scenario 1: Query has atleast one window function with an OVER() clause without a PARTITION BY *** - - In this scenario, we add all the RACs to a single RowsAndColumns to be processed. We do it via ConcatRowsAndColumns, and run all the operators on the ConcatRowsAndColumns. - This is done because we anyway need to run the operators on the entire set of rows when we have an OVER() clause without a PARTITION BY. - This scenario corresponds to partitionColumnNames.isEmpty()=true code flow. - - *** Scenario 2: All window functions in the query have OVER() clause with a PARTITION BY *** - - In this scenario, we need to process rows for each PARTITION BY group together, but we can batch multiple PARTITION BY keys into the same RAC before passing it to the operators for processing. - Batching is fine since the operators list would have the required NaivePartitioningOperatorFactory to segregate each PARTITION BY group during the processing. - - The flow for this scenario can be summarised as following: - 1. Frame Reading and Cursor Initialization: We start by reading a frame from the inputChannel and initializing frameCursor to iterate over the rows in that frame. - 2. Row Comparison: For each row in the frame, we decide whether it belongs to the same PARTITION BY group as the previous row. - This is determined by comparePartitionKeys() method. - Please refer to the Javadoc of that method for further details and an example illustration. - 2.1. If the PARTITION BY columns of current row matches the PARTITION BY columns of the previous row, - they belong to the same PARTITION BY group, and gets added to rowsToProcess. - If the number of total rows materialized exceed maxRowsMaterialized, we process the pending batch via processRowsUpToLastPartition() method. - 2.2. If they don't match, then we have reached a partition boundary. - In this case, we update the value for lastPartitionIndex. - 3. End of Input: If the input channel is finished, any remaining rows in rowsToProcess are processed. - - *Illustration of Row Comparison step* + if (inputChannel.canRead()) { + final Frame frame = inputChannel.read(); + convertRowFrameToRowsAndColumns(frame); - Let's say we have window_function() OVER (PARTITION BY A ORDER BY B) in our query, and we get 3 frames in the input channel to process. - - Frame 1 - A, B - 1, 2 - 1, 3 - 2, 1 --> PARTITION BY key (column A) changed from 1 to 2. - 2, 2 - - Frame 2 - A, B - 3, 1 --> PARTITION BY key (column A) changed from 2 to 3. - 3, 2 - 3, 3 - 3, 4 - - Frame 3 - A, B - 3, 5 - 3, 6 - 4, 1 --> PARTITION BY key (column A) changed from 3 to 4. - 4, 2 - - *Why batching?* - We batch multiple PARTITION BY keys for processing together to avoid the overhead of creating different RACs for each PARTITION BY keys, as that would be unnecessary in scenarios where we have a large number of PARTITION BY keys, but each key having a single row. - - *Future thoughts: https://github.com/apache/druid/issues/16126* - Current approach with R&C and operators materialize a single R&C for processing. In case of data with low cardinality a single R&C might be too big to consume. Same for the case of empty OVER() clause. - Most of the window operations like SUM(), RANK(), RANGE() etc. can be made with 2 passes of the data. We might think to reimplement them in the MSQ way so that we do not have to materialize so much data. - */ - - // todo: need to cleanup unused code and comments - if (true) { - // Scenario 1: Query has atleast one window function with an OVER() clause without a PARTITION BY. - if (inputChannel.canRead()) { - final Frame frame = inputChannel.read(); - convertRowFrameToRowsAndColumns(frame); - return ReturnOrAwait.runAgain(); - } else if (inputChannel.isFinished()) { - runAllOpsOnMultipleRac(frameRowsAndCols); - return ReturnOrAwait.returnObject(Unit.instance()); - } else { - return ReturnOrAwait.awaitAll(inputChannels().size()); - } - } - - // Scenario 2: All window functions in the query have OVER() clause with a PARTITION BY - if (frameCursor == null || frameCursor.isDone()) { - if (readableInputs.isEmpty()) { - return ReturnOrAwait.awaitAll(1); - } else if (inputChannel.canRead()) { - final Frame frame = inputChannel.read(); - frameCursor = FrameProcessors.makeCursor(frame, frameReader); - makeRowSupplierFromFrameCursor(); - } else if (inputChannel.isFinished()) { - // Handle any remaining data. - lastPartitionIndex = rowsToProcess.size() - 1; - processRowsUpToLastPartition(); - return ReturnOrAwait.returnObject(Unit.instance()); - } else { - return ReturnOrAwait.runAgain(); - } - } - - while (!frameCursor.isDone()) { - final ResultRow currentRow = rowSupplierFromFrameCursor.get(); - if (outputRow == null) { - outputRow = currentRow; - rowsToProcess.add(currentRow); - } else if (comparePartitionKeys(outputRow, currentRow, partitionColumnNames)) { - // Add current row to the same batch of rows for processing. - rowsToProcess.add(currentRow); - if (rowsToProcess.size() > maxRowsMaterialized) { - // We don't want to materialize more than maxRowsMaterialized rows at any point in time, so process the pending batch. - processRowsUpToLastPartition(); + if (needToProcessBatch()) { + runAllOpsOnBatch(); + try { + flushAllRowsAndCols(resultRowAndCols); + } + catch (IOException e) { + throw new RuntimeException(e); } - ensureMaxRowsInAWindowConstraint(rowsToProcess.size()); - } else { - lastPartitionIndex = rowsToProcess.size() - 1; - outputRow = currentRow.copy(); - return ReturnOrAwait.runAgain(); } - frameCursor.advance(); + return ReturnOrAwait.runAgain(); + } else if (inputChannel.isFinished()) { + runAllOpsOnBatch(); + return ReturnOrAwait.returnObject(Unit.instance()); + } else { + return ReturnOrAwait.awaitAll(inputChannels().size()); } - return ReturnOrAwait.runAgain(); } - /** - * @param listOfRacs Concat this list of {@link RowsAndColumns} to a {@link ConcatRowsAndColumns} to use as a single input for the operators to be run - */ - private void runAllOpsOnMultipleRac(ArrayList listOfRacs) + private void initialiseOperator() { - Operator op = new Operator() + op = new Operator() { @Nullable @Override public Closeable goOrContinue(Closeable continuationObject, Receiver receiver) { - RowsAndColumns rac = new ConcatRowsAndColumns(listOfRacs); + RowsAndColumns rac = new ConcatRowsAndColumns(new ArrayList<>(frameRowsAndCols)); + frameRowsAndCols.clear(); + numRowsInFrameRowsAndCols = 0; ensureMaxRowsInAWindowConstraint(rac.numRows()); receiver.push(rac); - receiver.completed(); - return null; + + if (inputChannel.isFinished()) { + // Only call completed() when the input channel is finished. + receiver.completed(); + return null; // Signal that the operator has completed its work + } + + // Return a non-null continuation object to indicate that we want to continue processing. + return () -> {}; } }; - runOperatorsAfterThis(op); - } - - /** - * @param op Base operator for the operators to be run. Other operators are wrapped under this to run - */ - private void runOperatorsAfterThis(Operator op) - { for (OperatorFactory of : operatorFactoryList) { op = of.wrap(op); } - Operator.go(op, new Operator.Receiver() + } + + private void runAllOpsOnBatch() + { + op.goOrContinue(null, new Operator.Receiver() { @Override public Operator.Signal push(RowsAndColumns rac) @@ -315,16 +200,10 @@ public void completed() // due to the chain of calls across the ops // so we can clear after writing to output flushAllRowsAndCols(resultRowAndCols); - frameRowsAndCols.clear(); - } catch (IOException e) { throw new RuntimeException(e); } - finally { - frameRowsAndCols.clear(); - resultRowAndCols.clear(); - } } }); } @@ -339,6 +218,9 @@ private void flushAllRowsAndCols(ArrayList resultRowAndCols) thr AtomicInteger rowId = new AtomicInteger(0); createFrameWriterIfNeeded(rac, rowId); writeRacToFrame(rac, rowId); + frameRowsAndCols.clear(); + resultRowAndCols.clear(); + numRowsInFrameRowsAndCols = 0; } /** @@ -428,78 +310,7 @@ private void convertRowFrameToRowsAndColumns(Frame frame) // check if existing + newly added rows exceed guardrails ensureMaxRowsInAWindowConstraint(frameRowsAndCols.size() + ldrc.numRows()); frameRowsAndCols.add(ldrc); - } - - /** - * Compare two rows based on the columns in partitionColumnNames. - * If the partitionColumnNames is empty, the method will end up returning true. - *

- * For example, say: - *

    - *
  • partitionColumnNames = ["d1", "d2"]
  • - *
  • frameReader's row signature = {d1:STRING, d2:STRING, p0:STRING}
  • - *
  • frameReader.signature.indexOf("d1") = 0
  • - *
  • frameReader.signature.indexOf("d2") = 1
  • - *
  • row1 = [d1_row1, d2_row1, p0_row1]
  • - *
  • row2 = [d1_row2, d2_row2, p0_row2]
  • - *
- *

- * Then this method will return true if d1_row1==d1_row2 && d2_row1==d2_row2, false otherwise. - * Returning true would indicate that these 2 rows can be put into the same partition for window function processing. - */ - private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List partitionColumnNames) - { - int match = 0; - for (String columnName : partitionColumnNames) { - int i = frameReader.signature().indexOf(columnName); - if (typeStrategies[i].compare(row1.get(i), row2.get(i)) == 0) { - match++; - } - } - return match == partitionColumnNames.size(); - } - - private void makeRowSupplierFromFrameCursor() - { - final ColumnSelectorFactory frameColumnSelectorFactory = frameCursor.getColumnSelectorFactory(); - final Supplier[] fieldSuppliers = new Supplier[frameReader.signature().size()]; - for (int i = 0; i < fieldSuppliers.length; i++) { - final ColumnValueSelector selector = - frameColumnSelectorFactory.makeColumnValueSelector(frameReader.signature().getColumnName(i)); - fieldSuppliers[i] = selector::getObject; - } - rowSupplierFromFrameCursor = () -> { - final ResultRow row = ResultRow.create(fieldSuppliers.length); - for (int i = 0; i < fieldSuppliers.length; i++) { - row.set(i, fieldSuppliers[i].get()); - } - return row; - }; - } - - /** - * Process rows from rowsToProcess[0, lastPartitionIndex]. - */ - private void processRowsUpToLastPartition() - { - if (lastPartitionIndex == -1) { - return; - } - - RowsAndColumns singleRac = MapOfColumnsRowsAndColumns.fromResultRowTillIndex( - rowsToProcess, - frameReader.signature(), - lastPartitionIndex - ); - ArrayList rowsAndColumns = new ArrayList<>(); - rowsAndColumns.add(singleRac); - runAllOpsOnMultipleRac(rowsAndColumns); - - // Remove elements in the range [0, lastPartitionIndex] from the list. - // The call to list.subList(a, b).clear() deletes the elements in the range [a, b - 1], - // causing the remaining elements to shift and start from index 0. - rowsToProcess.subList(0, lastPartitionIndex + 1).clear(); - lastPartitionIndex = -1; + numRowsInFrameRowsAndCols += ldrc.numRows(); } private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) @@ -511,4 +322,9 @@ private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) )); } } + + private boolean needToProcessBatch() + { + return numRowsInFrameRowsAndCols >= maxRowsMaterialized / 2; // Can this be improved further? + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java index 9852f4f40988..2e301f4a18df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java @@ -27,7 +27,6 @@ import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; -import org.apache.druid.error.DruidException; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; @@ -61,26 +60,19 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor private final List operatorList; private final RowSignature stageRowSignature; private final int maxRowsMaterializedInWindow; - private final List partitionColumnNames; @JsonCreator public WindowOperatorQueryFrameProcessorFactory( @JsonProperty("query") WindowOperatorQuery query, @JsonProperty("operatorList") List operatorFactoryList, @JsonProperty("stageRowSignature") RowSignature stageRowSignature, - @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow, - @JsonProperty("partitionColumnNames") List partitionColumnNames + @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow ) { this.query = Preconditions.checkNotNull(query, "query"); this.operatorList = Preconditions.checkNotNull(operatorFactoryList, "bad operator"); this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature"); this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow; - - if (partitionColumnNames == null) { - throw DruidException.defensive("List of partition column names encountered as null."); - } - this.partitionColumnNames = partitionColumnNames; } @JsonProperty("query") @@ -95,12 +87,6 @@ public List getOperators() return operatorList; } - @JsonProperty("partitionColumnNames") - public List getPartitionColumnNames() - { - return partitionColumnNames; - } - @JsonProperty("stageRowSignature") public RowSignature getSignature() { @@ -153,6 +139,7 @@ public ProcessorsAndChannels makeProcessors( readableInput -> { final OutputChannel outputChannel = outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); + return new WindowOperatorQueryFrameProcessor( query, readableInput.getChannel(), @@ -161,9 +148,7 @@ public ProcessorsAndChannels makeProcessors( readableInput.getChannelFrameReader(), frameContext.jsonMapper(), operatorList, - stageRowSignature, - maxRowsMaterializedInWindow, - partitionColumnNames + maxRowsMaterializedInWindow ); } ); @@ -188,13 +173,12 @@ public boolean equals(Object o) return maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow && Objects.equals(query, that.query) && Objects.equals(operatorList, that.operatorList) - && Objects.equals(partitionColumnNames, that.partitionColumnNames) && Objects.equals(stageRowSignature, that.stageRowSignature); } @Override public int hashCode() { - return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, maxRowsMaterializedInWindow); + return Objects.hash(query, operatorList, stageRowSignature, maxRowsMaterializedInWindow); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index a49c42fd1e34..62f17024a683 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -36,10 +36,12 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Query; +import org.apache.druid.query.operator.BasePartitioningOperatorFactory; +import org.apache.druid.query.operator.BaseSortOperatorFactory; import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; -import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; +import org.apache.druid.query.operator.PartitionSortOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.operator.window.WindowOperatorFactory; import org.apache.druid.segment.column.ColumnType; @@ -71,20 +73,9 @@ public QueryDefinition makeQueryDefinition( int minStageNumber ) { - // Need to validate query first. - // Populate the group of operators to be processed at each stage. - // The size of the operators is the number of serialized stages. - // Later we should also check if these can be parallelized. - // Check if there is an empty OVER() clause or not. RowSignature rowSignature = originalQuery.getRowSignature(); log.info("Row signature received for query is [%s].", rowSignature); - boolean isEmptyOverPresent = originalQuery.getOperators() - .stream() - .filter(of -> of instanceof GlueingPartitioningOperatorFactory) - .map(of -> (GlueingPartitioningOperatorFactory) of) - .anyMatch(of -> of.getPartitionColumns().isEmpty()); - List> operatorList = getOperatorListFromQuery(originalQuery); log.info("Created operatorList with operator factories: [%s]", operatorList); @@ -120,127 +111,83 @@ public QueryDefinition makeQueryDefinition( maxRowsMaterialized = Limits.MAX_ROWS_MATERIALIZED_IN_WINDOW; } - if (isEmptyOverPresent) { - // Move everything to a single partition since we have to load all the data on a single worker anyway to compute empty over() clause. - log.info( - "Empty over clause is present in the query. Creating a single stage with all operator factories [%s].", - queryToRun.getOperators() - ); - queryDefBuilder.add( - StageDefinition.builder(firstStageNumber) - .inputs(new StageInputSpec(firstStageNumber - 1)) - .signature(finalWindowStageRowSignature) - .maxWorkerCount(maxWorkerCount) - .shuffleSpec(finalWindowStageShuffleSpec) - .processorFactory(new WindowOperatorQueryFrameProcessorFactory( - queryToRun, - queryToRun.getOperators(), - finalWindowStageRowSignature, - maxRowsMaterialized, - Collections.emptyList() - )) - ); - } else { - // There are multiple windows present in the query. - // Create stages for each window in the query. - // These stages will be serialized. - // The partition by clause of the next window will be the shuffle key for the previous window. - RowSignature.Builder bob = RowSignature.builder(); - RowSignature signatureFromInput = dataSourcePlan.getSubQueryDefBuilder().get().build().getFinalStageDefinition().getSignature(); - log.info("Row signature received from last stage is [%s].", signatureFromInput); - - for (int i = 0; i < signatureFromInput.getColumnNames().size(); i++) { - bob.add(signatureFromInput.getColumnName(i), signatureFromInput.getColumnType(i).get()); - } + // There are multiple windows present in the query. + // Create stages for each window in the query. + // These stages will be serialized. + // The partition by clause of the next window will be the shuffle key for the previous window. + RowSignature.Builder bob = RowSignature.builder(); + RowSignature signatureFromInput = dataSourcePlan.getSubQueryDefBuilder().get().build().getFinalStageDefinition().getSignature(); + log.info("Row signature received from last stage is [%s].", signatureFromInput); + + for (int i = 0; i < signatureFromInput.getColumnNames().size(); i++) { + bob.add(signatureFromInput.getColumnName(i), signatureFromInput.getColumnType(i).get()); + } - List partitionColumnNames = new ArrayList<>(); - - /* - operatorList is a List>, where each List corresponds to the operator factories - to be used for a different window stage. - - We iterate over operatorList, and add the definition for a window stage to QueryDefinitionBuilder. - */ - for (int i = 0; i < operatorList.size(); i++) { - for (OperatorFactory operatorFactory : operatorList.get(i)) { - if (operatorFactory instanceof WindowOperatorFactory) { - List outputColumnNames = ((WindowOperatorFactory) operatorFactory).getProcessor().getOutputColumnNames(); - - // Need to add column names which are present in outputColumnNames and rowSignature but not in bob, - // since they need to be present in the row signature for this window stage. - for (String columnName : outputColumnNames) { - int indexInRowSignature = rowSignature.indexOf(columnName); - if (indexInRowSignature != -1 && bob.build().indexOf(columnName) == -1) { - ColumnType columnType = rowSignature.getColumnType(indexInRowSignature).get(); - bob.add(columnName, columnType); - log.info("Added column [%s] of type [%s] to row signature for window stage.", columnName, columnType); - } else { - throw new ISE( - "Found unexpected column [%s] already present in row signature [%s].", - columnName, - rowSignature - ); - } + /* + operatorList is a List>, where each List corresponds to the operator factories + to be used for a different window stage. + + We iterate over operatorList, and add the definition for a window stage to QueryDefinitionBuilder. + */ + for (int i = 0; i < operatorList.size(); i++) { + for (OperatorFactory operatorFactory : operatorList.get(i)) { + if (operatorFactory instanceof WindowOperatorFactory) { + List outputColumnNames = ((WindowOperatorFactory) operatorFactory).getProcessor().getOutputColumnNames(); + + // Need to add column names which are present in outputColumnNames and rowSignature but not in bob, + // since they need to be present in the row signature for this window stage. + for (String columnName : outputColumnNames) { + int indexInRowSignature = rowSignature.indexOf(columnName); + if (indexInRowSignature != -1 && bob.build().indexOf(columnName) == -1) { + ColumnType columnType = rowSignature.getColumnType(indexInRowSignature).get(); + bob.add(columnName, columnType); + log.info("Added column [%s] of type [%s] to row signature for window stage.", columnName, columnType); + } else { + throw new ISE( + "Found unexpected column [%s] already present in row signature [%s].", + columnName, + rowSignature + ); } } } + } - final RowSignature intermediateSignature = bob.build(); - final RowSignature stageRowSignature; + final RowSignature intermediateSignature = bob.build(); + final RowSignature stageRowSignature; - if (i + 1 == operatorList.size()) { - stageRowSignature = finalWindowStageRowSignature; - nextShuffleSpec = finalWindowStageShuffleSpec; + if (i + 1 == operatorList.size()) { + stageRowSignature = finalWindowStageRowSignature; + nextShuffleSpec = finalWindowStageShuffleSpec; + } else { + nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(i + 1), maxWorkerCount); + if (nextShuffleSpec == null) { + stageRowSignature = intermediateSignature; } else { - nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(i + 1), maxWorkerCount); - if (nextShuffleSpec == null) { - stageRowSignature = intermediateSignature; - } else { - stageRowSignature = QueryKitUtils.sortableSignature( - intermediateSignature, - nextShuffleSpec.clusterBy().getColumns() - ); - } - } - - log.info("Using row signature [%s] for window stage.", stageRowSignature); - - boolean partitionOperatorExists = false; - List currentPartitionColumns = new ArrayList<>(); - for (OperatorFactory of : operatorList.get(i)) { - if (of instanceof GlueingPartitioningOperatorFactory) { - for (String s : ((GlueingPartitioningOperatorFactory) of).getPartitionColumns()) { - currentPartitionColumns.add(s); - partitionOperatorExists = true; - } - } - } - - if (partitionOperatorExists) { - partitionColumnNames = currentPartitionColumns; + stageRowSignature = QueryKitUtils.sortableSignature( + intermediateSignature, + nextShuffleSpec.clusterBy().getColumns() + ); } + } - log.info( - "Columns which would be used to define partitioning boundaries for this window stage are [%s]", - partitionColumnNames - ); + log.info("Using row signature [%s] for window stage.", stageRowSignature); - queryDefBuilder.add( - StageDefinition.builder(firstStageNumber + i) - .inputs(new StageInputSpec(firstStageNumber + i - 1)) - .signature(stageRowSignature) - .maxWorkerCount(maxWorkerCount) - .shuffleSpec(nextShuffleSpec) - .processorFactory(new WindowOperatorQueryFrameProcessorFactory( - queryToRun, - operatorList.get(i), - stageRowSignature, - maxRowsMaterialized, - partitionColumnNames - )) - ); - } + queryDefBuilder.add( + StageDefinition.builder(firstStageNumber + i) + .inputs(new StageInputSpec(firstStageNumber + i - 1)) + .signature(stageRowSignature) + .maxWorkerCount(maxWorkerCount) + .shuffleSpec(nextShuffleSpec) + .processorFactory(new WindowOperatorQueryFrameProcessorFactory( + queryToRun, + getOperatorFactoryListForStageDefinition(operatorList.get(i), maxRowsMaterialized), + stageRowSignature, + maxRowsMaterialized + )) + ); } + return queryDefBuilder.build(); } @@ -287,13 +234,13 @@ private List> getOperatorListFromQuery(WindowOperatorQuery private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount) { - GlueingPartitioningOperatorFactory partition = null; - NaiveSortOperatorFactory sort = null; + BasePartitioningOperatorFactory partition = null; + BaseSortOperatorFactory sort = null; for (OperatorFactory of : operatorFactories) { - if (of instanceof GlueingPartitioningOperatorFactory) { - partition = (GlueingPartitioningOperatorFactory) of; - } else if (of instanceof NaiveSortOperatorFactory) { - sort = (NaiveSortOperatorFactory) of; + if (of instanceof BasePartitioningOperatorFactory) { + partition = (BasePartitioningOperatorFactory) of; + } else if (of instanceof BaseSortOperatorFactory) { + sort = (BaseSortOperatorFactory) of; } } @@ -377,4 +324,37 @@ private static RowSignature computeSignatureForFinalWindowStage(RowSignature row finalWindowClusterBy.getColumns() ); } + + /** + * This method converts the operator chain received from native plan into MSQ plan. + * (NaiveSortOperator -> Naive/GlueingPartitioningOperator -> WindowOperator) is converted into (GlueingPartitioningOperator -> PartitionSortOperator -> WindowOperator). + * We rely on MSQ's shuffling to do the clustering on partitioning keys for us at every stage. + * This conversion allows us to blindly read N rows from input channel and push them into the operator chain, and repeat until the input channel isn't finished. + * @param operatorFactoryListFromQuery + * @param maxRowsMaterializedInWindow + * @return + */ + private List getOperatorFactoryListForStageDefinition(List operatorFactoryListFromQuery, int maxRowsMaterializedInWindow) + { + final List operatorFactoryList = new ArrayList<>(); + final List sortOperatorFactoryList = new ArrayList<>(); + for (OperatorFactory operatorFactory : operatorFactoryListFromQuery) { + if (operatorFactory instanceof BasePartitioningOperatorFactory) { + BasePartitioningOperatorFactory partition = (BasePartitioningOperatorFactory) operatorFactory; + operatorFactoryList.add(new GlueingPartitioningOperatorFactory(partition.getPartitionColumns(), maxRowsMaterializedInWindow)); + } else if (operatorFactory instanceof BaseSortOperatorFactory) { + BaseSortOperatorFactory sortOperatorFactory = (BaseSortOperatorFactory) operatorFactory; + sortOperatorFactoryList.add(new PartitionSortOperatorFactory(sortOperatorFactory.getSortColumns())); + } else { + // Add all the PartitionSortOperator(s) before every window operator. + operatorFactoryList.addAll(sortOperatorFactoryList); + sortOperatorFactoryList.clear(); + operatorFactoryList.add(operatorFactory); + } + } + + operatorFactoryList.addAll(sortOperatorFactoryList); + sortOperatorFactoryList.clear(); + return operatorFactoryList; + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 218065730dab..61aaf8858e45 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -45,7 +45,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.operator.ColumnWithDirection; -import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; +import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.operator.window.WindowFrame; @@ -125,7 +125,7 @@ public void testWindowOnFooWithPartitionByAndInnerGroupBy(String contextName, Ma RowSignature.builder().add("d0", ColumnType.FLOAT).add("w0", ColumnType.DOUBLE).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), null @@ -223,10 +223,10 @@ public void testWindowOnFooWithFirstWindowPartitionNextWindowEmpty(String contex .add("w1", ColumnType.DOUBLE) .build(), ImmutableList.of( - new GlueingPartitioningOperatorFactory(ImmutableList.of()), + new NaivePartitioningOperatorFactory(ImmutableList.of()), new WindowOperatorFactory(proc1), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), null @@ -334,10 +334,10 @@ public void testWindowOnFooWith2WindowsBothWindowsHavingPartitionByInnerGroupBy( .build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d1")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d1")), new WindowOperatorFactory(proc1) ), null @@ -447,10 +447,10 @@ public void testWindowOnFooWith2WindowsBothPartitionByWithOrderReversed( .build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d1")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d1")), new WindowOperatorFactory(proc), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc1) ), null @@ -541,7 +541,7 @@ public void testWindowOnFooWithEmptyOverWithGroupBy(String contextName, Map cont RowSignature.builder().add("dim2", ColumnType.STRING).add("w0", ColumnType.DOUBLE).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("dim2"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("dim2")), + new NaivePartitioningOperatorFactory(ImmutableList.of("dim2")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1340,7 +1340,7 @@ public void testWindowOnFooWithEmptyOverWithUnnest(String contextName, Map cont .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("cityName"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("cityName")), + new NaivePartitioningOperatorFactory(ImmutableList.of("cityName")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1894,7 +1894,7 @@ public void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("countryIsoCode"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("countryIsoCode")), + new NaivePartitioningOperatorFactory(ImmutableList.of("countryIsoCode")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -1991,7 +1991,7 @@ public void testGroupByWithWikipedia(String contextName, Map con .add("w0", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))), - new GlueingPartitioningOperatorFactory(ImmutableList.of("d0")), + new NaivePartitioningOperatorFactory(ImmutableList.of("d0")), new WindowOperatorFactory(proc) ), ImmutableList.of() @@ -2121,10 +2121,10 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers(String cont .add("w1", ColumnType.LONG).build(), ImmutableList.of( new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"), ColumnWithDirection.ascending("d1"), ColumnWithDirection.ascending("d2"))), - new GlueingPartitioningOperatorFactory(Collections.emptyList()), + new NaivePartitioningOperatorFactory(Collections.emptyList()), new WindowOperatorFactory(new WindowRowNumberProcessor("w0")), new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d1"), ColumnWithDirection.ascending("d0"), ColumnWithDirection.ascending("d2"))), - new GlueingPartitioningOperatorFactory(Collections.singletonList("d1")), + new NaivePartitioningOperatorFactory(Collections.singletonList("d1")), new WindowOperatorFactory(new WindowFramedAggregateProcessor(WindowFrame.forOrderBy("d0", "d1", "d2"), aggs)) ), ImmutableList.of() @@ -2266,22 +2266,64 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers(String cont 2, 0, "input0" ) .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(13).bytes(1379).frames(1), + CounterSnapshotMatcher.with().rows(13).bytes(989).frames(1), 2, 0, "output" ) - // Stage 3, Worker 0 + // Stage 3, Worker 1 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 6).bytes(0, 461).frames(0, 1), + 3, 1, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 6).bytes(0, 641).frames(0, 1), + 3, 1, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(6).bytes(617).frames(1), + 3, 1, "shuffle" + ) + + // Stage 3, Worker 2 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 1).bytes(0, 0, 114).frames(0, 0, 1), + 3, 2, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 1).bytes(0, 0, 144).frames(0, 0, 1), + 3, 2, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(1).bytes(140).frames(1), + 3, 2, "shuffle" + ) + + // Stage 3, Worker 3 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 0, 6).bytes(0, 0, 0, 482).frames(0, 0, 0, 1), + 3, 3, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 0, 6).bytes(0, 0, 0, 662).frames(0, 0, 0, 1), + 3, 3, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(6).bytes(638).frames(1), + 3, 3, "shuffle" + ) + + // Stage 4, Worker 0 .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(13).bytes(1327).frames(1), - 3, 0, "input0" + 4, 0, "input0" ) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(13).bytes(1379).frames(1), - 3, 0, "output" + 4, 0, "output" ) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(13).bytes(1327).frames(1), - 3, 0, "shuffle" + 4, 0, "shuffle" ) .verifyResults(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java index 802a4c52f172..e38bef1026f3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java @@ -28,7 +28,7 @@ public class WindowOperatorQueryFrameProcessorFactoryTest public void testEqualsAndHashcode() { EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class) - .withNonnullFields("query", "operatorList", "stageRowSignature", "maxRowsMaterializedInWindow", "partitionColumnNames") + .withNonnullFields("query", "operatorList", "stageRowSignature", "maxRowsMaterializedInWindow") .usingGetClass() .verify(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index 9d64fffe23e2..ef644f1f37f3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -170,9 +170,7 @@ public void validateBatching(int maxRowsMaterialized, int numFramesWritten) thro ImmutableList.of( new WindowOperatorFactory(new WindowRowNumberProcessor("w0")) ), - inputSignature, - maxRowsMaterialized, - ImmutableList.of("added") + maxRowsMaterialized ); exec.runFully(processor, null); diff --git a/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq b/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq index 502885fb3ae3..6288097aa1b2 100644 --- a/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq +++ b/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq @@ -1,308 +1,27 @@ !set plannerStrategy DECOUPLED +!set enableWindowing true +!set maxNumTasks 3 !use druidtest://?componentSupplier=DrillWindowQueryMSQComponentSupplier !set outputformat mysql -select cityName, count(case when delta > 0 then channel end) as cnt, count(1) as aall +select +countryName, added, +row_number() over (order by added) as c1 from wikipedia -where cityName in ('New York', 'Aarhus') -group by 1 -order by 1; -+----------+-----+------+ -| cityName | cnt | aall | -+----------+-----+------+ -| Aarhus | 0 | 1 | -| New York | 7 | 13 | -+----------+-----+------+ -(2 rows) +where countryName in ('Egypt', 'El Salvador') +group by countryName, cityName, added; ++-------------+-------+----+ +| countryName | added | c1 | ++-------------+-------+----+ +| Egypt | 1 | 2 | +| Egypt | 112 | 8 | +| Egypt | 14 | 4 | +| Egypt | 16 | 5 | +| Egypt | 27 | 6 | +| El Salvador | 0 | 1 | +| El Salvador | 106 | 7 | +| El Salvador | 9 | 3 | ++-------------+-------+----+ +(8 rows) !ok -[ { - "stageNumber" : 0, - "definition" : { - "id" : "_0", - "input" : [ { - "type" : "table", - "dataSource" : "wikipedia", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ], - "filter" : { - "type" : "inType", - "column" : "cityName", - "matchValueType" : "STRING", - "sortedValues" : [ "Aarhus", "New York" ] - }, - "filterFields" : [ "cityName" ] - } ], - "processor" : { - "type" : "groupByPreShuffle", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "inputNumber", - "inputNumber" : 0 - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "filter" : { - "type" : "inType", - "column" : "cityName", - "matchValueType" : "STRING", - "sortedValues" : [ "Aarhus", "New York" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "default", - "dimension" : "cityName", - "outputName" : "d0", - "outputType" : "STRING" - } ], - "aggregations" : [ { - "type" : "filtered", - "aggregator" : { - "type" : "count", - "name" : "a0" - }, - "filter" : { - "type" : "and", - "fields" : [ { - "type" : "not", - "field" : { - "type" : "null", - "column" : "channel" - } - }, { - "type" : "range", - "column" : "delta", - "matchValueType" : "LONG", - "lower" : 0, - "lowerOpen" : true - } ] - }, - "name" : "a0" - }, { - "type" : "count", - "name" : "a1" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - }, - "context" : { - "__user" : null, - "finalize" : true, - "maxParseExceptions" : 0, - "plannerStrategy" : "DECOUPLED", - "sqlQueryId" : __SQL_QUERY_ID__ - "sqlStringifyArrays" : false - } - } - }, - "signature" : [ { - "name" : "d0", - "type" : "STRING" - }, { - "name" : "a0", - "type" : "LONG" - }, { - "name" : "a1", - "type" : "LONG" - } ], - "shuffleSpec" : { - "type" : "maxCount", - "clusterBy" : { - "columns" : [ { - "columnName" : "d0", - "order" : "ASCENDING" - } ] - }, - "partitions" : 1, - "aggregate" : true - }, - "maxWorkerCount" : 1 - }, - "phase" : "FINISHED", - "workerCount" : 1, - "partitionCount" : 1, - "shuffle" : "globalSort", - "output" : "localStorage", - "startTime" : __TIMESTAMP__ - "duration" : __DURATION__ - "sort" : true -}, { - "stageNumber" : 1, - "definition" : { - "id" : "_1", - "input" : [ { - "type" : "stage", - "stage" : 0 - } ], - "processor" : { - "type" : "groupByPostShuffle", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "inputNumber", - "inputNumber" : 0 - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "filter" : { - "type" : "inType", - "column" : "cityName", - "matchValueType" : "STRING", - "sortedValues" : [ "Aarhus", "New York" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "default", - "dimension" : "cityName", - "outputName" : "d0", - "outputType" : "STRING" - } ], - "aggregations" : [ { - "type" : "filtered", - "aggregator" : { - "type" : "count", - "name" : "a0" - }, - "filter" : { - "type" : "and", - "fields" : [ { - "type" : "not", - "field" : { - "type" : "null", - "column" : "channel" - } - }, { - "type" : "range", - "column" : "delta", - "matchValueType" : "LONG", - "lower" : 0, - "lowerOpen" : true - } ] - }, - "name" : "a0" - }, { - "type" : "count", - "name" : "a1" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - }, - "context" : { - "__user" : null, - "finalize" : true, - "maxParseExceptions" : 0, - "plannerStrategy" : "DECOUPLED", - "sqlQueryId" : __SQL_QUERY_ID__ - "sqlStringifyArrays" : false - } - } - }, - "signature" : [ { - "name" : "d0", - "type" : "STRING" - }, { - "name" : "a0", - "type" : "LONG" - }, { - "name" : "a1", - "type" : "LONG" - } ], - "maxWorkerCount" : 1 - }, - "phase" : "FINISHED", - "workerCount" : 1, - "partitionCount" : 1, - "output" : "localStorage", - "startTime" : __TIMESTAMP__ - "duration" : __DURATION__ -} ] -!msqPlan -# 227 -LogicalSort(sort0=[$0], dir0=[ASC]) - LogicalAggregate(group=[{0}], cnt=[COUNT($1)], aall=[COUNT()]) - LogicalProject(cityName=[$2], $f1=[CASE(>($17, 0), $1, null:VARCHAR)]) - LogicalFilter(condition=[OR(=($2, 'New York'), =($2, 'Aarhus'))]) - LogicalTableScan(table=[[druid, wikipedia]]) - -!convertedPlan -LogicalSort(sort0=[$0], dir0=[ASC]) - LogicalAggregate(group=[{0}], cnt=[COUNT($1) FILTER $2], aall=[COUNT()]) - LogicalProject(cityName=[$2], channel=[$1], $f3=[IS TRUE(>($17, 0))]) - LogicalFilter(condition=[SEARCH($2, Sarg['Aarhus':VARCHAR, 'New York':VARCHAR]:VARCHAR)]) - LogicalTableScan(table=[[druid, wikipedia]]) - -!logicalPlan -DruidAggregate(group=[{0}], cnt=[COUNT($1) FILTER $2], aall=[COUNT()], druid=[logical]) - DruidProject(cityName=[$2], channel=[$1], $f3=[IS TRUE(>($17, 0))], druid=[logical]) - DruidFilter(condition=[SEARCH($2, Sarg['Aarhus':VARCHAR, 'New York':VARCHAR]:VARCHAR)]) - DruidTableScan(table=[[druid, wikipedia]], druid=[logical]) - -!druidPlan -{ - "queryType" : "groupBy", - "dataSource" : { - "type" : "table", - "name" : "wikipedia" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "filter" : { - "type" : "inType", - "column" : "cityName", - "matchValueType" : "STRING", - "sortedValues" : [ "Aarhus", "New York" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "default", - "dimension" : "cityName", - "outputName" : "d0", - "outputType" : "STRING" - } ], - "aggregations" : [ { - "type" : "filtered", - "aggregator" : { - "type" : "count", - "name" : "a0" - }, - "filter" : { - "type" : "and", - "fields" : [ { - "type" : "not", - "field" : { - "type" : "null", - "column" : "channel" - } - }, { - "type" : "range", - "column" : "delta", - "matchValueType" : "LONG", - "lower" : 0, - "lowerOpen" : true - } ] - }, - "name" : "a0" - }, { - "type" : "count", - "name" : "a1" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } -} -!nativePlan - - - diff --git a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java new file mode 100644 index 000000000000..ec0a5b717840 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.query.rowsandcols.RowsAndColumns; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +public abstract class BasePartitioningOperator implements Operator +{ + protected final List partitionColumns; + protected final Operator child; + + public BasePartitioningOperator( + List partitionColumns, + Operator child + ) + { + this.partitionColumns = partitionColumns; + this.child = child; + } + + protected static class Continuation implements Closeable + { + Iterator iter; + Closeable subContinuation; + + public Continuation(Iterator iter, Closeable subContinuation) + { + this.iter = iter; + this.subContinuation = subContinuation; + } + + @Override + public void close() throws IOException + { + if (subContinuation != null) { + subContinuation.close(); + } + } + } + + protected Signal handlePush(RowsAndColumns rac, Receiver receiver, AtomicReference> iterHolder) + { + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } + + Iterator partitionsIter = getIteratorForRAC(rac); + + AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); + while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { + handleKeepItGoing(keepItGoing, partitionsIter, receiver); + } + + if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + + return keepItGoing.get(); + } + + protected abstract Iterator getIteratorForRAC(RowsAndColumns rac); + + protected abstract void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver); + + protected Closeable handleNonGoCases(Signal signal, Iterator iter, Receiver receiver, Continuation cont) + { + switch (signal) { + case PAUSE: + if (iter.hasNext()) { + return cont; + } + + if (cont.subContinuation == null) { + // We were finished anyway + receiver.completed(); + return null; + } + + return new Continuation(null, cont.subContinuation); + + case STOP: + receiver.completed(); + try { + cont.close(); + } + catch (IOException e) { + throw new RE(e, "Unable to close continuation"); + } + return null; + + default: + throw new RE("Unknown signal[%s]", signal); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java new file mode 100644 index 000000000000..ffab6c95c657 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public abstract class BasePartitioningOperatorFactory implements OperatorFactory +{ + protected final List partitionColumns; + + @JsonCreator + public BasePartitioningOperatorFactory( + @JsonProperty("partitionColumns") List partitionColumns + ) + { + this.partitionColumns = partitionColumns == null ? new ArrayList<>() : partitionColumns; + } + + @JsonProperty("partitionColumns") + public List getPartitionColumns() + { + return partitionColumns; + } + + @Override + public abstract Operator wrap(Operator op); + + @Override + public boolean validateEquivalent(OperatorFactory other) + { + if (other instanceof BasePartitioningOperatorFactory) { + return partitionColumns.equals(((BasePartitioningOperatorFactory) other).getPartitionColumns()); + } + return false; + } + + @Override + public int hashCode() + { + return Objects.hash(partitionColumns); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + BasePartitioningOperatorFactory other = (BasePartitioningOperatorFactory) obj; + return Objects.equals(partitionColumns, other.partitionColumns); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{partitionColumns=" + partitionColumns + "}"; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java new file mode 100644 index 000000000000..8c643c933df3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import java.io.Closeable; +import java.util.List; + +/** + * Base class for sort operators. + */ +public abstract class BaseSortOperator implements Operator +{ + protected final Operator child; + protected final List sortColumns; + + public BaseSortOperator( + Operator child, + List sortColumns + ) + { + this.child = child; + this.sortColumns = sortColumns; + } + + @Override + public abstract Closeable goOrContinue(Closeable continuation, Receiver receiver); +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java new file mode 100644 index 000000000000..e69ee918f407 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public abstract class BaseSortOperatorFactory implements OperatorFactory +{ + protected final List sortColumns; + + @JsonCreator + public BaseSortOperatorFactory( + @JsonProperty("columns") List sortColumns + ) + { + this.sortColumns = sortColumns; + } + + @JsonProperty("columns") + public List getSortColumns() + { + return sortColumns; + } + + @Override + public abstract Operator wrap(Operator op); + + @Override + public boolean validateEquivalent(OperatorFactory other) + { + if (other instanceof BaseSortOperatorFactory) { + return sortColumns.equals(((BaseSortOperatorFactory) other).getSortColumns()); + } + return false; + } + + @Override + public int hashCode() + { + return Objects.hash(sortColumns); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + BaseSortOperatorFactory other = (BaseSortOperatorFactory) obj; + return Objects.equals(sortColumns, other.sortColumns); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{sortColumns=" + sortColumns + "}"; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index 4a266b7cd47a..e0c098ad60f4 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -19,8 +19,7 @@ package org.apache.druid.query.operator; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.RE; +import org.apache.druid.error.InvalidInput; import org.apache.druid.query.rowsandcols.ConcatRowsAndColumns; import org.apache.druid.query.rowsandcols.LimitedRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -30,7 +29,6 @@ import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.io.Closeable; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; @@ -39,21 +37,35 @@ import java.util.concurrent.atomic.AtomicReference; /** - * todo: write detailed javadoc for the class, all the methods, etc. + * This glueing partitioning operator is supposed to continuously receive data, and output batches of partitioned RACs. + * It maintains a last-partitioning-boundary of the last-pushed-RAC, and attempts to glue it with the next RAC it receives, + * ensuring that partitions are handled correctly, even across multiple RACs. + *

+ * Additionally, this assumes that data has been pre-sorted according to the partitioning columns. */ -public class GlueingPartitioningOperator implements Operator +public class GlueingPartitioningOperator extends BasePartitioningOperator { - private final List partitionColumns; - private final Operator child; + private final int maxRowsMaterialized; private RowsAndColumns previousRac; + private static final int MAX_ROWS_MATERIALIZED_NO_LIMIT = -1; + + public GlueingPartitioningOperator( + Operator child, + List partitionColumns + ) + { + this(child, partitionColumns, MAX_ROWS_MATERIALIZED_NO_LIMIT); + } + public GlueingPartitioningOperator( + Operator child, List partitionColumns, - Operator child + int maxRowsMaterialized ) { - this.partitionColumns = partitionColumns; - this.child = child; + super(partitionColumns, child); + this.maxRowsMaterialized = maxRowsMaterialized; } @Override @@ -79,32 +91,8 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) } final Signal signal = receiver.push(next); - switch (signal) { - case GO: - break; - case PAUSE: - if (cont.iter.hasNext()) { - return cont; - } - - if (cont.subContinuation == null) { - // We were finished anyway - receiver.completed(); - return null; - } - - return new Continuation(null, cont.subContinuation); - case STOP: - receiver.completed(); - try { - cont.close(); - } - catch (IOException e) { - throw new RE(e, "Unable to close continuation"); - } - return null; - default: - throw new RE("Unknown signal[%s]", signal); + if (signal != Signal.GO) { + return handleNonGoCases(signal, cont.iter, receiver, cont); } } @@ -126,29 +114,8 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) @Override public Signal push(RowsAndColumns rac) { - if (rac == null) { - throw DruidException.defensive("Should never get a null rac here."); - } - - Iterator partitionsIter = new GluedRACsIterator(rac); - - Signal keepItGoing = Signal.GO; - while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { - RowsAndColumns rowsAndColumns = partitionsIter.next(); - if (partitionsIter.hasNext()) { - keepItGoing = receiver.push(rowsAndColumns); - } else { - // If it's the last element, save it in previousRac instead of pushing to receiver. - previousRac = rowsAndColumns; - } - } - - if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { - iterHolder.set(partitionsIter); - return Signal.PAUSE; - } - - return keepItGoing; + ensureMaxRowsMaterializedConstraint(rac.numRows()); + return handlePush(rac, receiver, iterHolder); } @Override @@ -175,26 +142,11 @@ public void completed() } } - private static class Continuation implements Closeable - { - Iterator iter; - Closeable subContinuation; - - public Continuation(Iterator iter, Closeable subContinuation) - { - this.iter = iter; - this.subContinuation = subContinuation; - } - - @Override - public void close() throws IOException - { - if (subContinuation != null) { - subContinuation.close(); - } - } - } - + /** + * Iterator implementation for gluing partitioned RowsAndColumns. + * It handles the boundaries of partitions within a single RAC and potentially glues + * the first partition of the current RAC with the previous RAC if needed. + */ private class GluedRACsIterator implements Iterator { private final RowsAndColumns rac; @@ -218,6 +170,13 @@ public boolean hasNext() return currentIndex < boundaries.length - 1; } + /** + * Retrieves the next partition in the RowsAndColumns. If the first partition has not been handled yet, + * it may be glued with the previous RowsAndColumns if the partition columns match. + * + * @return The next RowsAndColumns partition, potentially glued with the previous one. + * @throws NoSuchElementException if there are no more partitions. + */ @Override public RowsAndColumns next() { @@ -233,6 +192,7 @@ public RowsAndColumns next() if (isGlueingNeeded(previousRac, limitedRAC)) { RowsAndColumns gluedRAC = getConcatRacForFirstPartition(previousRac, limitedRAC); + ensureMaxRowsMaterializedConstraint(gluedRAC.numRows()); previousRac = null; currentIndex++; return gluedRAC; @@ -251,6 +211,15 @@ public RowsAndColumns next() return new LimitedRowsAndColumns(rac, start, end); } + /** + * Determines whether gluing is needed between the previous RowsAndColumns and the first partition + * of the current RowsAndColumns based on the partition columns. If the columns match, the two RACs + * will be glued together. + * + * @param previousRac The previous RowsAndColumns. + * @param firstPartitionOfCurrentRac The first partition of the current RowsAndColumns. + * @return true if gluing is needed, false otherwise. + */ private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) { if (previousRac == null) { @@ -278,4 +247,33 @@ private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previo return new ConcatRowsAndColumns(new ArrayList<>(Arrays.asList(previousRac, firstPartitionOfCurrentRac))); } } + + private void ensureMaxRowsMaterializedConstraint(int numRows) + { + if (maxRowsMaterialized != MAX_ROWS_MATERIALIZED_NO_LIMIT && numRows > maxRowsMaterialized) { + throw InvalidInput.exception( + "Too many rows to process (requested = %d, max = %d).", + numRows, + maxRowsMaterialized + ); + } + } + + @Override + protected Iterator getIteratorForRAC(RowsAndColumns rac) + { + return new GluedRACsIterator(rac); + } + + @Override + protected void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) + { + RowsAndColumns rowsAndColumns = iterator.next(); + if (iterator.hasNext()) { + signalRef.set(receiver.push(rowsAndColumns)); + } else { + // If it's the last element, save it in previousRac instead of pushing to receiver. + previousRac = rowsAndColumns; + } + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java index 56b066828b37..c614fc5547a9 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java @@ -22,41 +22,40 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.ArrayList; import java.util.List; import java.util.Objects; -public class GlueingPartitioningOperatorFactory implements OperatorFactory +public class GlueingPartitioningOperatorFactory extends BasePartitioningOperatorFactory { - private final List partitionColumns; + private final int maxRowsMaterialized; @JsonCreator public GlueingPartitioningOperatorFactory( - @JsonProperty("partitionColumns") List partitionColumns + @JsonProperty("partitionColumns") List partitionColumns, + @JsonProperty("maxRowsMaterialized") int maxRowsMaterialized ) { - this.partitionColumns = partitionColumns == null ? new ArrayList<>() : partitionColumns; + super(partitionColumns); + this.maxRowsMaterialized = maxRowsMaterialized; } - @JsonProperty("partitionColumns") - public List getPartitionColumns() + @JsonProperty("maxRowsMaterialized") + public int getMaxRowsMaterialized() { - return partitionColumns; + return maxRowsMaterialized; } @Override public Operator wrap(Operator op) { - return new GlueingPartitioningOperator(partitionColumns, op); + return new GlueingPartitioningOperator(op, partitionColumns, maxRowsMaterialized); } @Override public boolean validateEquivalent(OperatorFactory other) { - if (other instanceof GlueingPartitioningOperatorFactory) { - return partitionColumns.equals(((GlueingPartitioningOperatorFactory) other).getPartitionColumns()); - } - return false; + return super.validateEquivalent(other) && + maxRowsMaterialized == ((GlueingPartitioningOperatorFactory) other).getMaxRowsMaterialized(); } @Override @@ -64,25 +63,20 @@ public String toString() { return "GlueingPartitioningOperatorFactory{" + "partitionColumns=" + partitionColumns + + "maxRowsMaterialized=" + maxRowsMaterialized + '}'; } @Override public final int hashCode() { - return Objects.hash(partitionColumns); + return Objects.hash(partitionColumns, maxRowsMaterialized); } @Override public final boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || obj.getClass() != getClass()) { - return false; - } - GlueingPartitioningOperatorFactory other = (GlueingPartitioningOperatorFactory) obj; - return Objects.equals(partitionColumns, other.partitionColumns); + return super.equals(obj) && + Objects.equals(maxRowsMaterialized, ((GlueingPartitioningOperatorFactory) obj).getMaxRowsMaterialized()); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index c2d2e0aa8100..c37658043878 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -19,14 +19,11 @@ package org.apache.druid.query.operator; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.RE; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.io.Closeable; -import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -40,18 +37,14 @@ * Additionally, this assumes that data has been pre-sorted according to the partitioning columns. If it is * given data that has not been pre-sorted, an exception is expected to be thrown. */ -public class NaivePartitioningOperator implements Operator +public class NaivePartitioningOperator extends BasePartitioningOperator { - private final List partitionColumns; - private final Operator child; - public NaivePartitioningOperator( List partitionColumns, Operator child ) { - this.partitionColumns = partitionColumns; - this.child = child; + super(partitionColumns, child); } @Override @@ -63,32 +56,8 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) if (cont.iter != null) { while (cont.iter.hasNext()) { final Signal signal = receiver.push(cont.iter.next()); - switch (signal) { - case GO: - break; - case PAUSE: - if (cont.iter.hasNext()) { - return cont; - } - - if (cont.subContinuation == null) { - // We were finished anyway - receiver.completed(); - return null; - } - - return new Continuation(null, cont.subContinuation); - case STOP: - receiver.completed(); - try { - cont.close(); - } - catch (IOException e) { - throw new RE(e, "Unable to close continuation"); - } - return null; - default: - throw new RE("Unknown signal[%s]", signal); + if (signal != Signal.GO) { + return handleNonGoCases(signal, cont.iter, receiver, cont); } } @@ -110,28 +79,7 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) @Override public Signal push(RowsAndColumns rac) { - if (rac == null) { - throw DruidException.defensive("Should never get a null rac here."); - } - ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); - if (groupPartitioner == null) { - groupPartitioner = new DefaultClusteredGroupPartitioner(rac); - } - - Iterator partitionsIter = - groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); - - Signal keepItGoing = Signal.GO; - while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { - keepItGoing = receiver.push(partitionsIter.next()); - } - - if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { - iterHolder.set(partitionsIter); - return Signal.PAUSE; - } - - return keepItGoing; + return handlePush(rac, receiver, iterHolder); } @Override @@ -154,23 +102,19 @@ public void completed() } } - private static class Continuation implements Closeable + @Override + protected Iterator getIteratorForRAC(RowsAndColumns rac) { - Iterator iter; - Closeable subContinuation; - - public Continuation(Iterator iter, Closeable subContinuation) - { - this.iter = iter; - this.subContinuation = subContinuation; + ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); + if (groupPartitioner == null) { + groupPartitioner = new DefaultClusteredGroupPartitioner(rac); } + return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); + } - @Override - public void close() throws IOException - { - if (subContinuation != null) { - subContinuation.close(); - } - } + @Override + protected void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) + { + signalRef.set(receiver.push(iterator.next())); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java index c836007e77ef..53d5b68c1766 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java @@ -22,26 +22,16 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.ArrayList; import java.util.List; -import java.util.Objects; -public class NaivePartitioningOperatorFactory implements OperatorFactory +public class NaivePartitioningOperatorFactory extends BasePartitioningOperatorFactory { - private final List partitionColumns; - @JsonCreator public NaivePartitioningOperatorFactory( @JsonProperty("partitionColumns") List partitionColumns ) { - this.partitionColumns = partitionColumns == null ? new ArrayList<>() : partitionColumns; - } - - @JsonProperty("partitionColumns") - public List getPartitionColumns() - { - return partitionColumns; + super(partitionColumns); } @Override @@ -49,40 +39,4 @@ public Operator wrap(Operator op) { return new NaivePartitioningOperator(partitionColumns, op); } - - @Override - public boolean validateEquivalent(OperatorFactory other) - { - if (other instanceof NaivePartitioningOperatorFactory) { - return partitionColumns.equals(((NaivePartitioningOperatorFactory) other).getPartitionColumns()); - } - return false; - } - - @Override - public String toString() - { - return "NaivePartitioningOperatorFactory{" + - "partitionColumns=" + partitionColumns + - '}'; - } - - @Override - public final int hashCode() - { - return Objects.hash(partitionColumns); - } - - @Override - public final boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || obj.getClass() != getClass()) { - return false; - } - NaivePartitioningOperatorFactory other = (NaivePartitioningOperatorFactory) obj; - return Objects.equals(partitionColumns, other.partitionColumns); - } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java index 486d20482824..68e420161a3f 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java @@ -31,18 +31,14 @@ * that it has to accumulate all of the data of its child operator first before it can sort. This limitation * means that hopefully this operator is only planned in a very small number of circumstances. */ -public class NaiveSortOperator implements Operator +public class NaiveSortOperator extends BaseSortOperator { - private final Operator child; - private final List sortColumns; - public NaiveSortOperator( Operator child, List sortColumns ) { - this.child = child; - this.sortColumns = sortColumns; + super(child, sortColumns); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java index 623d0ed0fe5b..6ba6828d164c 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java @@ -23,24 +23,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -import java.util.Objects; -public class NaiveSortOperatorFactory implements OperatorFactory +public class NaiveSortOperatorFactory extends BaseSortOperatorFactory { - private final List sortColumns; - @JsonCreator public NaiveSortOperatorFactory( @JsonProperty("columns") List sortColumns ) { - this.sortColumns = sortColumns; - } - - @JsonProperty("columns") - public List getSortColumns() - { - return sortColumns; + super(sortColumns); } @Override @@ -48,38 +39,4 @@ public Operator wrap(Operator op) { return new NaiveSortOperator(op, sortColumns); } - - @Override - public boolean validateEquivalent(OperatorFactory other) - { - if (other instanceof NaiveSortOperatorFactory) { - return sortColumns.equals(((NaiveSortOperatorFactory) other).getSortColumns()); - } - return false; - } - - @Override - public int hashCode() - { - return Objects.hash(sortColumns); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - NaiveSortOperatorFactory other = (NaiveSortOperatorFactory) obj; - return Objects.equals(sortColumns, other.sortColumns); - } - - @Override - public String toString() - { - return "NaiveSortOperatorFactory{sortColumns=" + sortColumns + "}"; - } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java index a7d77aeb2d26..95410b98dd5e 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java @@ -33,6 +33,7 @@ @JsonSubTypes.Type(name = "naivePartition", value = NaivePartitioningOperatorFactory.class), @JsonSubTypes.Type(name = "glueingPartition", value = GlueingPartitioningOperatorFactory.class), @JsonSubTypes.Type(name = "naiveSort", value = NaiveSortOperatorFactory.class), + @JsonSubTypes.Type(name = "partitionSort", value = PartitionSortOperatorFactory.class), @JsonSubTypes.Type(name = "window", value = WindowOperatorFactory.class), @JsonSubTypes.Type(name = "scan", value = ScanOperatorFactory.class), }) diff --git a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java new file mode 100644 index 000000000000..c10508727eb0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; + +/** + * This operator sorts rows inside partitioned RACs, on the sort columns. + * This operator expects to receive a "complete" partition of data. Each input RAC is expected to be a separate partition. + */ +public class PartitionSortOperator extends BaseSortOperator +{ + public PartitionSortOperator( + Operator child, + List sortColumns + ) + { + super(child, sortColumns); + } + + @Override + public Closeable goOrContinue(Closeable continuation, Receiver receiver) + { + return child.goOrContinue( + continuation, + new Receiver() + { + @Override + public Signal push(RowsAndColumns rac) + { + NaiveSortMaker.NaiveSorter sorter = NaiveSortMaker.fromRAC(rac).make(new ArrayList<>(sortColumns)); + receiver.push(sorter.complete()); + return Signal.GO; + } + + @Override + public void completed() + { + receiver.completed(); + } + } + ); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java new file mode 100644 index 000000000000..79386c313104 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class PartitionSortOperatorFactory extends BaseSortOperatorFactory +{ + @JsonCreator + public PartitionSortOperatorFactory( + @JsonProperty("columns") List sortColumns + ) + { + super(sortColumns); + } + + @Override + public Operator wrap(Operator op) + { + return new PartitionSortOperator(op, sortColumns); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java index 365a7f4ab728..d907c2d490ae 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java @@ -21,12 +21,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.DruidException; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; import org.junit.Test; +import java.util.Collections; import java.util.function.BiFunction; public class GlueingPartitioningOperatorTest @@ -44,8 +47,8 @@ public void testDefaultImplementation() InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); GlueingPartitioningOperator op = new GlueingPartitioningOperator( - ImmutableList.of("sorted"), - inlineScanOperator + inlineScanOperator, + ImmutableList.of("sorted") ); new OperatorTestHelper() @@ -91,8 +94,8 @@ public void testDefaultImplementationWithMultipleRACs() InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2, rac3); GlueingPartitioningOperator op = new GlueingPartitioningOperator( - ImmutableList.of("sorted"), - inlineScanOperator + inlineScanOperator, + ImmutableList.of("sorted") ); new OperatorTestHelper() @@ -129,6 +132,221 @@ public void testDefaultImplementationWithMultipleRACs() .runToCompletion(op); } + @Test + public void testDefaultImplementationWithMultipleConcatenationBetweenRACs() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1}), + "unsorted", new IntArrayColumn(new int[]{20, 20}) + ) + ); + RowsAndColumns rac3 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 2}), + "unsorted", new IntArrayColumn(new int[]{30, 40}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2, rac3); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + ImmutableList.of("sorted") + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1, 1, 1}) + .expectColumn("unsorted", new int[]{10, 10, 20, 20, 30}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2}) + .expectColumn("unsorted", new int[]{40}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testDefaultImplementationWithNoPartitionColumns() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + Collections.emptyList() + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1, 2, 2, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10, 20, 20, 11}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testDefaultImplementationWithMultipleRACsAndNoPartitionColumns() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{50, 51, 52, 53, 54, 55}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + Collections.emptyList() + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10, 20, 20, 11, 50, 51, 52, 53, 54, 55}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + + @Test + public void testMaxRowsConstraintViolation() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + ImmutableList.of("sorted"), + 2 + ); + + boolean exceptionThrown = false; + try { + new OperatorTestHelper() + .expectRowsAndColumns() + .runToCompletion(op); + } + catch (DruidException e) { + Assert.assertEquals( + e.getMessage(), + "Too many rows to process (requested = 3, max = 2)." + ); + exceptionThrown = true; + } + Assert.assertTrue(exceptionThrown); + } + + @Test + public void testMaxRowsConstraintViolationWhenGlueing() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 2, 3}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + ImmutableList.of("sorted"), + 3 + ); + + boolean exceptionThrown = false; + try { + new OperatorTestHelper() + .expectRowsAndColumns() + .runToCompletion(op); + } + catch (DruidException e) { + Assert.assertEquals( + e.getMessage(), + "Too many rows to process (requested = 4, max = 3)." + ); + exceptionThrown = true; + } + Assert.assertTrue(exceptionThrown); + } + + @Test + public void testMaxRowsConstraintWhenGlueing() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) + ) + ); + RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{2, 2, 2}), + "unsorted", new IntArrayColumn(new int[]{20, 20, 20}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + ImmutableList.of("sorted"), + 3 + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1}) + .expectColumn("unsorted", new int[]{10, 10, 10}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2, 2, 2}) + .expectColumn("unsorted", new int[]{20, 20, 20}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } + @Test public void testStopMidStream() { @@ -142,8 +360,8 @@ public void testStopMidStream() InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac); GlueingPartitioningOperator op = new GlueingPartitioningOperator( - ImmutableList.of("sorted"), - inlineScanOperator + inlineScanOperator, + ImmutableList.of("sorted") ); new OperatorTestHelper() @@ -178,8 +396,8 @@ public void testDoesNotValidateSort() ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( - ImmutableList.of("unsorted"), - InlineScanOperator.make(rac) + InlineScanOperator.make(rac), + ImmutableList.of("unsorted") ); new OperatorTestHelper() diff --git a/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorFactoryTest.java new file mode 100644 index 000000000000..43401c505ff5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorFactoryTest.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class PartitionSortOperatorFactoryTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NaiveSortOperatorFactory.class) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorTest.java new file mode 100644 index 000000000000..83f4afb405d3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/PartitionSortOperatorTest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.operator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Test; + +public class PartitionSortOperatorTest +{ + @Test + public void testDefaultImplementation() + { + RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), + "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) + ) + ); + + InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); + + PartitionSortOperator op = new PartitionSortOperator( + inlineScanOperator, + ImmutableList.of(new ColumnWithDirection("unsorted", ColumnWithDirection.Direction.ASC)) + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1, 1, 1, 2, 2}) + .expectColumn("unsorted", new int[]{10, 10, 10, 11, 20, 20}) + .allColumnsRegistered() + ) + .runToCompletion(op); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 33d284d1f490..b56e543fbd5b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -40,7 +40,6 @@ import org.apache.druid.query.QueryException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.operator.ColumnWithDirection; -import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; @@ -64,7 +63,6 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rule.GroupByRules; -import org.apache.druid.sql.calcite.run.NativeSqlEngine; import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -289,21 +287,12 @@ private static List computeWindowOperations( if (!sortMatches(priorSortColumns, sortColumns)) { // Sort order needs to change. Resort and repartition. ops.add(new NaiveSortOperatorFactory(new ArrayList<>(sortColumns))); - if (sqlEngine instanceof NativeSqlEngine) { - ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); - } else { - ops.add(new GlueingPartitioningOperatorFactory(group.getPartitionColumns())); - } + ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); priorSortColumns = sortColumns; priorPartitionColumns = group.getPartitionColumns(); } else if (!group.getPartitionColumns().equals(priorPartitionColumns)) { // Sort order doesn't need to change, but partitioning does. Only repartition. - if (sqlEngine instanceof NativeSqlEngine) { - ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); - } else { - ops.add(new GlueingPartitioningOperatorFactory(group.getPartitionColumns())); - } - + ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns())); priorPartitionColumns = group.getPartitionColumns(); } From ae2ab9e607a6a14a4b9c4d10b92b1bbad261b865 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 19 Sep 2024 19:12:40 +0530 Subject: [PATCH 04/26] Revert unnecessary changes --- .../msq1.iq | 321 ++++++++++++++++-- .../druid/sql/calcite/rel/Windowing.java | 11 +- 2 files changed, 303 insertions(+), 29 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq b/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq index 6288097aa1b2..502885fb3ae3 100644 --- a/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq +++ b/extensions-core/multi-stage-query/src/test/quidem/org.apache.druid.msq.quidem.MSQQuidemTest/msq1.iq @@ -1,27 +1,308 @@ !set plannerStrategy DECOUPLED -!set enableWindowing true -!set maxNumTasks 3 !use druidtest://?componentSupplier=DrillWindowQueryMSQComponentSupplier !set outputformat mysql -select -countryName, added, -row_number() over (order by added) as c1 +select cityName, count(case when delta > 0 then channel end) as cnt, count(1) as aall from wikipedia -where countryName in ('Egypt', 'El Salvador') -group by countryName, cityName, added; -+-------------+-------+----+ -| countryName | added | c1 | -+-------------+-------+----+ -| Egypt | 1 | 2 | -| Egypt | 112 | 8 | -| Egypt | 14 | 4 | -| Egypt | 16 | 5 | -| Egypt | 27 | 6 | -| El Salvador | 0 | 1 | -| El Salvador | 106 | 7 | -| El Salvador | 9 | 3 | -+-------------+-------+----+ -(8 rows) +where cityName in ('New York', 'Aarhus') +group by 1 +order by 1; ++----------+-----+------+ +| cityName | cnt | aall | ++----------+-----+------+ +| Aarhus | 0 | 1 | +| New York | 7 | 13 | ++----------+-----+------+ +(2 rows) !ok +[ { + "stageNumber" : 0, + "definition" : { + "id" : "_0", + "input" : [ { + "type" : "table", + "dataSource" : "wikipedia", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ], + "filter" : { + "type" : "inType", + "column" : "cityName", + "matchValueType" : "STRING", + "sortedValues" : [ "Aarhus", "New York" ] + }, + "filterFields" : [ "cityName" ] + } ], + "processor" : { + "type" : "groupByPreShuffle", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "inputNumber", + "inputNumber" : 0 + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "inType", + "column" : "cityName", + "matchValueType" : "STRING", + "sortedValues" : [ "Aarhus", "New York" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "cityName", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "and", + "fields" : [ { + "type" : "not", + "field" : { + "type" : "null", + "column" : "channel" + } + }, { + "type" : "range", + "column" : "delta", + "matchValueType" : "LONG", + "lower" : 0, + "lowerOpen" : true + } ] + }, + "name" : "a0" + }, { + "type" : "count", + "name" : "a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + }, + "context" : { + "__user" : null, + "finalize" : true, + "maxParseExceptions" : 0, + "plannerStrategy" : "DECOUPLED", + "sqlQueryId" : __SQL_QUERY_ID__ + "sqlStringifyArrays" : false + } + } + }, + "signature" : [ { + "name" : "d0", + "type" : "STRING" + }, { + "name" : "a0", + "type" : "LONG" + }, { + "name" : "a1", + "type" : "LONG" + } ], + "shuffleSpec" : { + "type" : "maxCount", + "clusterBy" : { + "columns" : [ { + "columnName" : "d0", + "order" : "ASCENDING" + } ] + }, + "partitions" : 1, + "aggregate" : true + }, + "maxWorkerCount" : 1 + }, + "phase" : "FINISHED", + "workerCount" : 1, + "partitionCount" : 1, + "shuffle" : "globalSort", + "output" : "localStorage", + "startTime" : __TIMESTAMP__ + "duration" : __DURATION__ + "sort" : true +}, { + "stageNumber" : 1, + "definition" : { + "id" : "_1", + "input" : [ { + "type" : "stage", + "stage" : 0 + } ], + "processor" : { + "type" : "groupByPostShuffle", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "inputNumber", + "inputNumber" : 0 + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "inType", + "column" : "cityName", + "matchValueType" : "STRING", + "sortedValues" : [ "Aarhus", "New York" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "cityName", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "and", + "fields" : [ { + "type" : "not", + "field" : { + "type" : "null", + "column" : "channel" + } + }, { + "type" : "range", + "column" : "delta", + "matchValueType" : "LONG", + "lower" : 0, + "lowerOpen" : true + } ] + }, + "name" : "a0" + }, { + "type" : "count", + "name" : "a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + }, + "context" : { + "__user" : null, + "finalize" : true, + "maxParseExceptions" : 0, + "plannerStrategy" : "DECOUPLED", + "sqlQueryId" : __SQL_QUERY_ID__ + "sqlStringifyArrays" : false + } + } + }, + "signature" : [ { + "name" : "d0", + "type" : "STRING" + }, { + "name" : "a0", + "type" : "LONG" + }, { + "name" : "a1", + "type" : "LONG" + } ], + "maxWorkerCount" : 1 + }, + "phase" : "FINISHED", + "workerCount" : 1, + "partitionCount" : 1, + "output" : "localStorage", + "startTime" : __TIMESTAMP__ + "duration" : __DURATION__ +} ] +!msqPlan +# 227 +LogicalSort(sort0=[$0], dir0=[ASC]) + LogicalAggregate(group=[{0}], cnt=[COUNT($1)], aall=[COUNT()]) + LogicalProject(cityName=[$2], $f1=[CASE(>($17, 0), $1, null:VARCHAR)]) + LogicalFilter(condition=[OR(=($2, 'New York'), =($2, 'Aarhus'))]) + LogicalTableScan(table=[[druid, wikipedia]]) + +!convertedPlan +LogicalSort(sort0=[$0], dir0=[ASC]) + LogicalAggregate(group=[{0}], cnt=[COUNT($1) FILTER $2], aall=[COUNT()]) + LogicalProject(cityName=[$2], channel=[$1], $f3=[IS TRUE(>($17, 0))]) + LogicalFilter(condition=[SEARCH($2, Sarg['Aarhus':VARCHAR, 'New York':VARCHAR]:VARCHAR)]) + LogicalTableScan(table=[[druid, wikipedia]]) + +!logicalPlan +DruidAggregate(group=[{0}], cnt=[COUNT($1) FILTER $2], aall=[COUNT()], druid=[logical]) + DruidProject(cityName=[$2], channel=[$1], $f3=[IS TRUE(>($17, 0))], druid=[logical]) + DruidFilter(condition=[SEARCH($2, Sarg['Aarhus':VARCHAR, 'New York':VARCHAR]:VARCHAR)]) + DruidTableScan(table=[[druid, wikipedia]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "table", + "name" : "wikipedia" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "inType", + "column" : "cityName", + "matchValueType" : "STRING", + "sortedValues" : [ "Aarhus", "New York" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "cityName", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0" + }, + "filter" : { + "type" : "and", + "fields" : [ { + "type" : "not", + "field" : { + "type" : "null", + "column" : "channel" + } + }, { + "type" : "range", + "column" : "delta", + "matchValueType" : "LONG", + "lower" : 0, + "lowerOpen" : true + } ] + }, + "name" : "a0" + }, { + "type" : "count", + "name" : "a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan + + + diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index b56e543fbd5b..8e1fc3ee2750 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -63,7 +63,6 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rule.GroupByRules; -import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nonnull; @@ -209,12 +208,7 @@ public static Windowing fromCalciteStuff( ))); } - List ops = computeWindowOperations( - partialQuery, - sourceRowSignature, - windowGroupProcessors, - plannerContext.getEngine() - ); + List ops = computeWindowOperations(partialQuery, sourceRowSignature, windowGroupProcessors); // Apply windowProject, if present. if (partialQuery.getWindowProject() != null) { @@ -254,8 +248,7 @@ public static Windowing fromCalciteStuff( private static List computeWindowOperations( final PartialDruidQuery partialQuery, final RowSignature sourceRowSignature, - List windowGroupProcessors, - SqlEngine sqlEngine + List windowGroupProcessors ) { final List ops = new ArrayList<>(); From 5d0594b00a70ac635ea9a50da11c0deba90df7a1 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 25 Sep 2024 11:58:31 +0530 Subject: [PATCH 05/26] Address review comments - part 1 --- .../WindowOperatorQueryFrameProcessor.java | 6 +-- ...dowOperatorQueryFrameProcessorFactory.java | 19 ++------ .../msq/querykit/WindowOperatorQueryKit.java | 11 +---- .../msq/util/MultiStageQueryContext.java | 8 ++++ ...peratorQueryFrameProcessorFactoryTest.java | 2 +- ...WindowOperatorQueryFrameProcessorTest.java | 43 ++++++------------- 6 files changed, 30 insertions(+), 59 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index 25c25ba1a415..ec6c84111226 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsInAWindowFault; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.operator.Operator; import org.apache.druid.query.operator.OperatorFactory; @@ -88,8 +89,7 @@ public WindowOperatorQueryFrameProcessor( FrameWriterFactory frameWriterFactory, FrameReader frameReader, ObjectMapper jsonMapper, - final List operatorFactoryList, - final int maxRowsMaterializedInWindow + final List operatorFactoryList ) { this.inputChannel = inputChannel; @@ -98,7 +98,7 @@ public WindowOperatorQueryFrameProcessor( this.operatorFactoryList = operatorFactoryList; this.frameRowsAndCols = new ArrayList<>(); this.resultRowAndCols = new ArrayList<>(); - this.maxRowsMaterialized = maxRowsMaterializedInWindow; + this.maxRowsMaterialized = MultiStageQueryContext.getMaxRowsMaterializedInWindow(query.context()); this.frameReader = frameReader; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java index 2e301f4a18df..8d0dea65c7f5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java @@ -59,20 +59,17 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor private final WindowOperatorQuery query; private final List operatorList; private final RowSignature stageRowSignature; - private final int maxRowsMaterializedInWindow; @JsonCreator public WindowOperatorQueryFrameProcessorFactory( @JsonProperty("query") WindowOperatorQuery query, @JsonProperty("operatorList") List operatorFactoryList, - @JsonProperty("stageRowSignature") RowSignature stageRowSignature, - @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow + @JsonProperty("stageRowSignature") RowSignature stageRowSignature ) { this.query = Preconditions.checkNotNull(query, "query"); this.operatorList = Preconditions.checkNotNull(operatorFactoryList, "bad operator"); this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature"); - this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow; } @JsonProperty("query") @@ -93,12 +90,6 @@ public RowSignature getSignature() return stageRowSignature; } - @JsonProperty("maxRowsMaterializedInWindow") - public int getMaxRowsMaterializedInWindow() - { - return maxRowsMaterializedInWindow; - } - @Override public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, @@ -147,8 +138,7 @@ public ProcessorsAndChannels makeProcessors( stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator(), removeNullBytes), readableInput.getChannelFrameReader(), frameContext.jsonMapper(), - operatorList, - maxRowsMaterializedInWindow + operatorList ); } ); @@ -170,8 +160,7 @@ public boolean equals(Object o) return false; } WindowOperatorQueryFrameProcessorFactory that = (WindowOperatorQueryFrameProcessorFactory) o; - return maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow - && Objects.equals(query, that.query) + return Objects.equals(query, that.query) && Objects.equals(operatorList, that.operatorList) && Objects.equals(stageRowSignature, that.stageRowSignature); } @@ -179,6 +168,6 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(query, operatorList, stageRowSignature, maxRowsMaterializedInWindow); + return Objects.hash(query, operatorList, stageRowSignature); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index 62f17024a683..d3a1acac2157 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.kernel.HashShuffleSpec; import org.apache.druid.msq.kernel.MixShuffleSpec; @@ -104,12 +103,7 @@ public QueryDefinition makeQueryDefinition( final ShuffleSpec finalWindowStageShuffleSpec = resultShuffleSpecFactory.build(finalWindowClusterBy, false); final RowSignature finalWindowStageRowSignature = computeSignatureForFinalWindowStage(rowSignature, finalWindowClusterBy, segmentGranularity); - final int maxRowsMaterialized; - if (originalQuery.context() != null && originalQuery.context().containsKey(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW)) { - maxRowsMaterialized = (int) originalQuery.context().get(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW); - } else { - maxRowsMaterialized = Limits.MAX_ROWS_MATERIALIZED_IN_WINDOW; - } + final int maxRowsMaterialized = MultiStageQueryContext.getMaxRowsMaterializedInWindow(originalQuery.context()); // There are multiple windows present in the query. // Create stages for each window in the query. @@ -182,8 +176,7 @@ public QueryDefinition makeQueryDefinition( .processorFactory(new WindowOperatorQueryFrameProcessorFactory( queryToRun, getOperatorFactoryListForStageDefinition(operatorList.get(i), maxRowsMaterialized), - stageRowSignature, - maxRowsMaterialized + stageRowSignature )) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index ed6a7c0e7b9b..cac42b4557e9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -200,6 +200,14 @@ public static String getMSQMode(final QueryContext queryContext) ); } + public static int getMaxRowsMaterializedInWindow(final QueryContext queryContext) + { + return queryContext.getInt( + MAX_ROWS_MATERIALIZED_IN_WINDOW, + Limits.MAX_ROWS_MATERIALIZED_IN_WINDOW + ); + } + public static int getMaxConcurrentStages(final QueryContext queryContext) { return queryContext.getInt( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java index e38bef1026f3..58affd228e17 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java @@ -28,7 +28,7 @@ public class WindowOperatorQueryFrameProcessorFactoryTest public void testEqualsAndHashcode() { EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class) - .withNonnullFields("query", "operatorList", "stageRowSignature", "maxRowsMaterializedInWindow") + .withNonnullFields("query", "operatorList", "stageRowSignature") .usingGetClass() .verify(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index ef644f1f37f3..3efcf006fb03 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -39,6 +39,7 @@ import org.apache.druid.msq.indexing.error.TooManyRowsInAWindowFault; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.test.LimitedFrameWriterFactory; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; @@ -67,47 +68,26 @@ public class WindowOperatorQueryFrameProcessorTest extends FrameProcessorTestBase { @Test - public void testBatchingOfPartitionByKeys_singleBatch() throws Exception + public void testProcessorRun() throws Exception { - // With maxRowsMaterialized=100, we will get 1 frame: - // [1, 1, 2, 2, 2, 3, 3] - validateBatching(100, 1); + runProcessor(100, 1); } @Test - public void testBatchingOfPartitionByKeys_multipleBatches_1() throws Exception - { - // With maxRowsMaterialized=5, we will get 2 frames: - // [1, 1, 2, 2, 2] - // [3, 3] - validateBatching(5, 2); - } - - @Test - public void testBatchingOfPartitionByKeys_multipleBatches_2() throws Exception - { - // With maxRowsMaterialized=4, we will get 3 frames: - // [1, 1] - // [2, 2, 2] - // [3, 3] - validateBatching(4, 3); - } - - @Test - public void testBatchingOfPartitionByKeys_TooManyRowsInAWindowFault() + public void testMaxRowsMaterializedConstraint() { final RuntimeException e = Assert.assertThrows( RuntimeException.class, - () -> validateBatching(2, 3) + () -> runProcessor(2, 3) ); MatcherAssert.assertThat( ((MSQException) e.getCause().getCause()).getFault(), CoreMatchers.instanceOf(TooManyRowsInAWindowFault.class) ); - Assert.assertTrue(e.getMessage().contains("TooManyRowsInAWindow: Too many rows in a window (requested = 3, max = 2)")); + Assert.assertTrue(e.getMessage().contains("TooManyRowsInAWindow: Too many rows in a window (requested = 7, max = 2)")); } - public void validateBatching(int maxRowsMaterialized, int numFramesWritten) throws Exception + public void runProcessor(int maxRowsMaterialized, int expectedNumFramesWritten) throws Exception { final ReadableInput factChannel = buildWindowTestInputChannel(); @@ -141,7 +121,9 @@ public void validateBatching(int maxRowsMaterialized, int numFramesWritten) thro .context(new HashMap<>()) .build()), new LegacySegmentSpec(Intervals.ETERNITY), - new HashMap<>(), + new HashMap<>( + ImmutableMap.of(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW, maxRowsMaterialized) + ), outputSignature, ImmutableList.of( new WindowOperatorFactory(new WindowRowNumberProcessor("w0")) @@ -169,8 +151,7 @@ public void validateBatching(int maxRowsMaterialized, int numFramesWritten) thro new ObjectMapper(), ImmutableList.of( new WindowOperatorFactory(new WindowRowNumberProcessor("w0")) - ), - maxRowsMaterialized + ) ); exec.runFully(processor, null); @@ -183,7 +164,7 @@ public void validateBatching(int maxRowsMaterialized, int numFramesWritten) thro final List> rows = rowsFromProcessor.toList(); long actualNumFrames = Arrays.stream(channelCounters.snapshot().getFrames()).findFirst().getAsLong(); - Assert.assertEquals(numFramesWritten, actualNumFrames); + Assert.assertEquals(expectedNumFramesWritten, actualNumFrames); Assert.assertEquals(7, rows.size()); } From 56c5942b6e74e223e4dff50da59e69bc160d5c69 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 25 Sep 2024 13:01:54 +0530 Subject: [PATCH 06/26] Address review comments - part 2 - rename base classes to abstract --- .../msq/querykit/WindowOperatorQueryKit.java | 24 +++++++++---------- ...java => AbstractPartitioningOperator.java} | 4 ++-- ... AbstractPartitioningOperatorFactory.java} | 10 ++++---- ...perator.java => AbstractSortOperator.java} | 4 ++-- ....java => AbstractSortOperatorFactory.java} | 10 ++++---- .../operator/GlueingPartitioningOperator.java | 2 +- .../GlueingPartitioningOperatorFactory.java | 2 +- .../operator/NaivePartitioningOperator.java | 2 +- .../NaivePartitioningOperatorFactory.java | 2 +- .../query/operator/NaiveSortOperator.java | 2 +- .../operator/NaiveSortOperatorFactory.java | 2 +- .../query/operator/PartitionSortOperator.java | 2 +- .../PartitionSortOperatorFactory.java | 2 +- 13 files changed, 34 insertions(+), 34 deletions(-) rename processing/src/main/java/org/apache/druid/query/operator/{BasePartitioningOperator.java => AbstractPartitioningOperator.java} (96%) rename processing/src/main/java/org/apache/druid/query/operator/{BasePartitioningOperatorFactory.java => AbstractPartitioningOperatorFactory.java} (83%) rename processing/src/main/java/org/apache/druid/query/operator/{BaseSortOperator.java => AbstractSortOperator.java} (93%) rename processing/src/main/java/org/apache/druid/query/operator/{BaseSortOperatorFactory.java => AbstractSortOperatorFactory.java} (85%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index d3a1acac2157..8fc8fc390257 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -35,8 +35,8 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Query; -import org.apache.druid.query.operator.BasePartitioningOperatorFactory; -import org.apache.druid.query.operator.BaseSortOperatorFactory; +import org.apache.druid.query.operator.AbstractPartitioningOperatorFactory; +import org.apache.druid.query.operator.AbstractSortOperatorFactory; import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.GlueingPartitioningOperatorFactory; import org.apache.druid.query.operator.OperatorFactory; @@ -227,13 +227,13 @@ private List> getOperatorListFromQuery(WindowOperatorQuery private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount) { - BasePartitioningOperatorFactory partition = null; - BaseSortOperatorFactory sort = null; + AbstractPartitioningOperatorFactory partition = null; + AbstractSortOperatorFactory sort = null; for (OperatorFactory of : operatorFactories) { - if (of instanceof BasePartitioningOperatorFactory) { - partition = (BasePartitioningOperatorFactory) of; - } else if (of instanceof BaseSortOperatorFactory) { - sort = (BaseSortOperatorFactory) of; + if (of instanceof AbstractPartitioningOperatorFactory) { + partition = (AbstractPartitioningOperatorFactory) of; + } else if (of instanceof AbstractSortOperatorFactory) { + sort = (AbstractSortOperatorFactory) of; } } @@ -332,11 +332,11 @@ private List getOperatorFactoryListForStageDefinition(List operatorFactoryList = new ArrayList<>(); final List sortOperatorFactoryList = new ArrayList<>(); for (OperatorFactory operatorFactory : operatorFactoryListFromQuery) { - if (operatorFactory instanceof BasePartitioningOperatorFactory) { - BasePartitioningOperatorFactory partition = (BasePartitioningOperatorFactory) operatorFactory; + if (operatorFactory instanceof AbstractPartitioningOperatorFactory) { + AbstractPartitioningOperatorFactory partition = (AbstractPartitioningOperatorFactory) operatorFactory; operatorFactoryList.add(new GlueingPartitioningOperatorFactory(partition.getPartitionColumns(), maxRowsMaterializedInWindow)); - } else if (operatorFactory instanceof BaseSortOperatorFactory) { - BaseSortOperatorFactory sortOperatorFactory = (BaseSortOperatorFactory) operatorFactory; + } else if (operatorFactory instanceof AbstractSortOperatorFactory) { + AbstractSortOperatorFactory sortOperatorFactory = (AbstractSortOperatorFactory) operatorFactory; sortOperatorFactoryList.add(new PartitionSortOperatorFactory(sortOperatorFactory.getSortColumns())); } else { // Add all the PartitionSortOperator(s) before every window operator. diff --git a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java similarity index 96% rename from processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java rename to processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java index ec0a5b717840..89711cd3a2b8 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java @@ -29,12 +29,12 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; -public abstract class BasePartitioningOperator implements Operator +public abstract class AbstractPartitioningOperator implements Operator { protected final List partitionColumns; protected final Operator child; - public BasePartitioningOperator( + public AbstractPartitioningOperator( List partitionColumns, Operator child ) diff --git a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperatorFactory.java similarity index 83% rename from processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java rename to processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperatorFactory.java index ffab6c95c657..fffc1ec50f4d 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/BasePartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperatorFactory.java @@ -26,12 +26,12 @@ import java.util.List; import java.util.Objects; -public abstract class BasePartitioningOperatorFactory implements OperatorFactory +public abstract class AbstractPartitioningOperatorFactory implements OperatorFactory { protected final List partitionColumns; @JsonCreator - public BasePartitioningOperatorFactory( + public AbstractPartitioningOperatorFactory( @JsonProperty("partitionColumns") List partitionColumns ) { @@ -50,8 +50,8 @@ public List getPartitionColumns() @Override public boolean validateEquivalent(OperatorFactory other) { - if (other instanceof BasePartitioningOperatorFactory) { - return partitionColumns.equals(((BasePartitioningOperatorFactory) other).getPartitionColumns()); + if (other instanceof AbstractPartitioningOperatorFactory) { + return partitionColumns.equals(((AbstractPartitioningOperatorFactory) other).getPartitionColumns()); } return false; } @@ -71,7 +71,7 @@ public boolean equals(Object obj) if (obj == null || getClass() != obj.getClass()) { return false; } - BasePartitioningOperatorFactory other = (BasePartitioningOperatorFactory) obj; + AbstractPartitioningOperatorFactory other = (AbstractPartitioningOperatorFactory) obj; return Objects.equals(partitionColumns, other.partitionColumns); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperator.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java rename to processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperator.java index 8c643c933df3..8adebbb210a7 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperator.java @@ -25,12 +25,12 @@ /** * Base class for sort operators. */ -public abstract class BaseSortOperator implements Operator +public abstract class AbstractSortOperator implements Operator { protected final Operator child; protected final List sortColumns; - public BaseSortOperator( + public AbstractSortOperator( Operator child, List sortColumns ) diff --git a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperatorFactory.java similarity index 85% rename from processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java rename to processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperatorFactory.java index e69ee918f407..7a9d9b4aab9c 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/BaseSortOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractSortOperatorFactory.java @@ -25,12 +25,12 @@ import java.util.List; import java.util.Objects; -public abstract class BaseSortOperatorFactory implements OperatorFactory +public abstract class AbstractSortOperatorFactory implements OperatorFactory { protected final List sortColumns; @JsonCreator - public BaseSortOperatorFactory( + public AbstractSortOperatorFactory( @JsonProperty("columns") List sortColumns ) { @@ -49,8 +49,8 @@ public List getSortColumns() @Override public boolean validateEquivalent(OperatorFactory other) { - if (other instanceof BaseSortOperatorFactory) { - return sortColumns.equals(((BaseSortOperatorFactory) other).getSortColumns()); + if (other instanceof AbstractSortOperatorFactory) { + return sortColumns.equals(((AbstractSortOperatorFactory) other).getSortColumns()); } return false; } @@ -70,7 +70,7 @@ public boolean equals(Object obj) if (obj == null || getClass() != obj.getClass()) { return false; } - BaseSortOperatorFactory other = (BaseSortOperatorFactory) obj; + AbstractSortOperatorFactory other = (AbstractSortOperatorFactory) obj; return Objects.equals(sortColumns, other.sortColumns); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index e0c098ad60f4..a7b5b269d1bb 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -43,7 +43,7 @@ *

* Additionally, this assumes that data has been pre-sorted according to the partitioning columns. */ -public class GlueingPartitioningOperator extends BasePartitioningOperator +public class GlueingPartitioningOperator extends AbstractPartitioningOperator { private final int maxRowsMaterialized; private RowsAndColumns previousRac; diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java index c614fc5547a9..2aeaa2397345 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.Objects; -public class GlueingPartitioningOperatorFactory extends BasePartitioningOperatorFactory +public class GlueingPartitioningOperatorFactory extends AbstractPartitioningOperatorFactory { private final int maxRowsMaterialized; diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index c37658043878..764c133b8bc1 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -37,7 +37,7 @@ * Additionally, this assumes that data has been pre-sorted according to the partitioning columns. If it is * given data that has not been pre-sorted, an exception is expected to be thrown. */ -public class NaivePartitioningOperator extends BasePartitioningOperator +public class NaivePartitioningOperator extends AbstractPartitioningOperator { public NaivePartitioningOperator( List partitionColumns, diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java index 53d5b68c1766..2b243bdadc9d 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java @@ -24,7 +24,7 @@ import java.util.List; -public class NaivePartitioningOperatorFactory extends BasePartitioningOperatorFactory +public class NaivePartitioningOperatorFactory extends AbstractPartitioningOperatorFactory { @JsonCreator public NaivePartitioningOperatorFactory( diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java index 68e420161a3f..0d3e5de78767 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java @@ -31,7 +31,7 @@ * that it has to accumulate all of the data of its child operator first before it can sort. This limitation * means that hopefully this operator is only planned in a very small number of circumstances. */ -public class NaiveSortOperator extends BaseSortOperator +public class NaiveSortOperator extends AbstractSortOperator { public NaiveSortOperator( Operator child, diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java index 6ba6828d164c..c281123a8f1d 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java @@ -24,7 +24,7 @@ import java.util.List; -public class NaiveSortOperatorFactory extends BaseSortOperatorFactory +public class NaiveSortOperatorFactory extends AbstractSortOperatorFactory { @JsonCreator public NaiveSortOperatorFactory( diff --git a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java index c10508727eb0..68d10fd77557 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperator.java @@ -30,7 +30,7 @@ * This operator sorts rows inside partitioned RACs, on the sort columns. * This operator expects to receive a "complete" partition of data. Each input RAC is expected to be a separate partition. */ -public class PartitionSortOperator extends BaseSortOperator +public class PartitionSortOperator extends AbstractSortOperator { public PartitionSortOperator( Operator child, diff --git a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java index 79386c313104..39f7a9afe5b8 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/PartitionSortOperatorFactory.java @@ -24,7 +24,7 @@ import java.util.List; -public class PartitionSortOperatorFactory extends BaseSortOperatorFactory +public class PartitionSortOperatorFactory extends AbstractSortOperatorFactory { @JsonCreator public PartitionSortOperatorFactory( From b80fed6c5e463694115ede9ed8a51117c91561eb Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 25 Sep 2024 13:14:53 +0530 Subject: [PATCH 07/26] Address review comments - part 3 - add RowsAndColumnsBuilder class --- .../WindowOperatorQueryFrameProcessor.java | 57 ++++++++++++++----- 1 file changed, 42 insertions(+), 15 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index ec6c84111226..7502427ed009 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -66,9 +66,8 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private static final Logger log = new Logger(WindowOperatorQueryFrameProcessor.class); private final List operatorFactoryList; - private final ArrayList frameRowsAndCols; private final ArrayList resultRowAndCols; - private int numRowsInFrameRowsAndCols; + private final RowsAndColumnsBuilder frameRowsAndColsBuilder; private final ReadableFrameChannel inputChannel; private final WritableFrameChannel outputChannel; private final FrameWriterFactory frameWriterFactory; @@ -96,7 +95,7 @@ public WindowOperatorQueryFrameProcessor( this.outputChannel = outputChannel; this.frameWriterFactory = frameWriterFactory; this.operatorFactoryList = operatorFactoryList; - this.frameRowsAndCols = new ArrayList<>(); + this.frameRowsAndColsBuilder = new RowsAndColumnsBuilder(); this.resultRowAndCols = new ArrayList<>(); this.maxRowsMaterialized = MultiStageQueryContext.getMaxRowsMaterializedInWindow(query.context()); @@ -160,9 +159,8 @@ private void initialiseOperator() @Override public Closeable goOrContinue(Closeable continuationObject, Receiver receiver) { - RowsAndColumns rac = new ConcatRowsAndColumns(new ArrayList<>(frameRowsAndCols)); - frameRowsAndCols.clear(); - numRowsInFrameRowsAndCols = 0; + RowsAndColumns rac = frameRowsAndColsBuilder.build(); + frameRowsAndColsBuilder.clear(); ensureMaxRowsInAWindowConstraint(rac.numRows()); receiver.push(rac); @@ -196,9 +194,6 @@ public Operator.Signal push(RowsAndColumns rac) public void completed() { try { - // resultRowsAndCols has reference to frameRowsAndCols - // due to the chain of calls across the ops - // so we can clear after writing to output flushAllRowsAndCols(resultRowAndCols); } catch (IOException e) { @@ -218,9 +213,8 @@ private void flushAllRowsAndCols(ArrayList resultRowAndCols) thr AtomicInteger rowId = new AtomicInteger(0); createFrameWriterIfNeeded(rac, rowId); writeRacToFrame(rac, rowId); - frameRowsAndCols.clear(); resultRowAndCols.clear(); - numRowsInFrameRowsAndCols = 0; + frameRowsAndColsBuilder.clear(); } /** @@ -308,9 +302,8 @@ private void convertRowFrameToRowsAndColumns(Frame frame) null ); // check if existing + newly added rows exceed guardrails - ensureMaxRowsInAWindowConstraint(frameRowsAndCols.size() + ldrc.numRows()); - frameRowsAndCols.add(ldrc); - numRowsInFrameRowsAndCols += ldrc.numRows(); + ensureMaxRowsInAWindowConstraint(frameRowsAndColsBuilder.getNumRows() + ldrc.numRows()); + frameRowsAndColsBuilder.add(ldrc); } private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) @@ -325,6 +318,40 @@ private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) private boolean needToProcessBatch() { - return numRowsInFrameRowsAndCols >= maxRowsMaterialized / 2; // Can this be improved further? + return frameRowsAndColsBuilder.getNumRows() >= maxRowsMaterialized / 2; // Can this be improved further? + } + + private static class RowsAndColumnsBuilder + { + private final List racList; + private int totalRows; + + public RowsAndColumnsBuilder() + { + this.racList = new ArrayList<>(); + this.totalRows = 0; + } + + public void add(RowsAndColumns rac) + { + racList.add(rac); + totalRows += rac.numRows(); + } + + public int getNumRows() + { + return totalRows; + } + + public RowsAndColumns build() + { + return new ConcatRowsAndColumns(new ArrayList<>(racList)); + } + + public void clear() + { + racList.clear(); + totalRows = 0; + } } } From 82e2bb104bf37df35e935d4c152d47d58aa09f85 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 3 Oct 2024 11:10:19 +0530 Subject: [PATCH 08/26] Fix MSQWindowTest --- .../org/apache/druid/msq/exec/MSQWindowTest.java | 4 ++-- .../query/rowsandcols/RearrangedRowsAndColumns.java | 13 +------------ 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 2d3325de532f..7ef0ad604a61 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -2331,7 +2331,7 @@ public void testFailurePartitionByMVD_1() .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Encountered a multi value column [v0]. Window processing does not support MVDs. Consider using UNNEST or MV_TO_ARRAY.")) + "Encountered a multi value column. Window processing does not support MVDs. Consider using UNNEST or MV_TO_ARRAY.")) )) .verifyExecutionError(); } @@ -2350,7 +2350,7 @@ public void testFailurePartitionByMVD_2() .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( CoreMatchers.instanceOf(ISE.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Encountered a multi value column [v0]. Window processing does not support MVDs. Consider using UNNEST or MV_TO_ARRAY.")) + "Encountered a multi value column. Window processing does not support MVDs. Consider using UNNEST or MV_TO_ARRAY.")) )) .verifyExecutionError(); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java index a5c2528dd1bd..e64f086edd7f 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java @@ -20,7 +20,6 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; @@ -29,7 +28,6 @@ import javax.annotation.Nullable; import java.util.Collection; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; /** @@ -130,16 +128,7 @@ public boolean isNull(int rowNum) @Override public Object getObject(int rowNum) { - Object value = accessor.getObject(pointers[start + rowNum]); - if (ColumnType.STRING.equals(getType()) && value instanceof List) { - // special handling to reject MVDs - throw new UOE( - "Encountered a multi value column [%s]. Window processing does not support MVDs. " - + "Consider using UNNEST or MV_TO_ARRAY.", - name - ); - } - return value; + return accessor.getObject(pointers[start + rowNum]); } @Override From 929f108df8c2cdf4ce2444689f1d1114715b859e Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 3 Oct 2024 14:20:50 +0530 Subject: [PATCH 09/26] Remove commented code --- .../druid/msq/querykit/WindowOperatorQueryFrameProcessor.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index b894d5bee98a..ff427da83a95 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -229,8 +229,6 @@ private void flushAllRowsAndCols() throws IOException RowsAndColumns rac = new ConcatRowsAndColumns(resultRowAndCols); createFrameWriterIfNeeded(rac); writeRacToFrame(rac); -// resultRowAndCols.clear(); -// frameRowsAndColsBuilder.clear(); } /** @@ -396,7 +394,6 @@ private boolean frameHasRowsPendingFlush() private void clearRACBuffers() { resultRowAndCols.clear(); -// frameRowsAndColsBuilder.clear(); rowId.set(0); } } From 348b7225c4263557849101469bcfbfc7ab5e1de7 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 7 Oct 2024 12:39:31 +0530 Subject: [PATCH 10/26] Make allocator capacity in LazilyDecoratedRowsAndColumns configurable to prevent OOM issues --- .../WindowOperatorQueryFrameProcessor.java | 3 ++- ...WindowOperatorQueryFrameProcessorTest.java | 7 +++---- .../LazilyDecoratedRowsAndColumns.java | 21 ++++++++++++++++--- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index ff427da83a95..b047f69615e3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -318,7 +318,8 @@ private void convertRowFrameToRowsAndColumns(Frame frame) null, OffsetLimit.limit(Integer.MAX_VALUE), null, - null + null, + (int) frameWriterFactory.allocatorCapacity() ); // check if existing + newly added rows exceed guardrails ensureMaxRowsInAWindowConstraint(frameRowsAndColsBuilder.getNumRows() + ldrc.numRows()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index 2ae039b47726..02cb02360d93 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -22,8 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.frame.allocation.HeapMemoryAllocator; -import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.read.FrameReader; @@ -116,7 +115,7 @@ public void testFrameWriterReachingCapacity() throws IOException final FrameWriterFactory frameWriterFactory = new LimitedFrameWriterFactory( FrameWriters.makeRowBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + new ArenaMemoryAllocatorFactory(1 << 20), outputSignature, Collections.emptyList(), false @@ -224,7 +223,7 @@ public void runProcessor(int maxRowsMaterialized, int expectedNumFramesWritten) // Limit output frames to 1 row to ensure we test edge cases final FrameWriterFactory frameWriterFactory = new LimitedFrameWriterFactory( FrameWriters.makeRowBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + new ArenaMemoryAllocatorFactory(1 << 20), outputSignature, Collections.emptyList(), false diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index bb35f6837976..d6a8ba0ceeaf 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -77,6 +77,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns private OffsetLimit limit; private LinkedHashSet viewableColumns; private List ordering; + private final Integer allocatorCapacity; public LazilyDecoratedRowsAndColumns( RowsAndColumns base, @@ -87,6 +88,20 @@ public LazilyDecoratedRowsAndColumns( List ordering, LinkedHashSet viewableColumns ) + { + this(base, interval, filter, virtualColumns, limit, ordering, viewableColumns, null); + } + + public LazilyDecoratedRowsAndColumns( + RowsAndColumns base, + Interval interval, + Filter filter, + VirtualColumns virtualColumns, + OffsetLimit limit, + List ordering, + LinkedHashSet viewableColumns, + Integer allocatorCapacity + ) { this.base = base; this.interval = interval; @@ -95,6 +110,7 @@ public LazilyDecoratedRowsAndColumns( this.limit = limit; this.ordering = ordering; this.viewableColumns = viewableColumns; + this.allocatorCapacity = allocatorCapacity != null ? allocatorCapacity : 200 << 20; } @Override @@ -268,7 +284,7 @@ private Pair materializeCursorFactory(CursorFactory cursor } final FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( - new ArenaMemoryAllocatorFactory(200 << 20), // 200 MB, because, why not? + new ArenaMemoryAllocatorFactory(allocatorCapacity), signature, sortColumns ); @@ -367,8 +383,7 @@ private Pair naiveMaterialize(RowsAndColumns rac) // is being left as an exercise for the future. final RowSignature.Builder sigBob = RowSignature.builder(); - final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(200 << 20); - + final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(allocatorCapacity); for (String column : columnsToGenerate) { final Column racColumn = rac.findColumn(column); From d5eab25bae046b9afc600804bdf4e7eeae23a91a Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 7 Oct 2024 13:19:37 +0530 Subject: [PATCH 11/26] Fix MSQWindowTest --- .../apache/druid/msq/exec/MSQWindowTest.java | 58 ++++++++++++++++--- 1 file changed, 50 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 7ef0ad604a61..effca1b06f76 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -2223,7 +2223,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() 2, 0, "output" ) - // Stage 3, Worker 1 + // Stage 3, Worker 1 (window stage) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(0, 6).bytes(0, 461).frames(0, 1), 3, 1, "input0" @@ -2233,11 +2233,11 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() 3, 1, "output" ) .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(6).bytes(617).frames(1), + CounterSnapshotMatcher.with().rows(1, 1, 2, 2).bytes(122, 132, 230, 235).frames(1, 1, 1, 1), 3, 1, "shuffle" ) - // Stage 3, Worker 2 + // Stage 3, Worker 2 (window stage) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(0, 0, 1).bytes(0, 0, 114).frames(0, 0, 1), 3, 2, "input0" @@ -2251,7 +2251,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() 3, 2, "shuffle" ) - // Stage 3, Worker 3 + // Stage 3, Worker 3 (window stage) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher.with().rows(0, 0, 0, 6).bytes(0, 0, 0, 482).frames(0, 0, 0, 1), 3, 3, "input0" @@ -2261,23 +2261,65 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() 3, 3, "output" ) .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(6).bytes(638).frames(1), + CounterSnapshotMatcher.with().rows(1, 1, 2, 2).bytes(143, 137, 222, 238).frames(1, 1, 1, 1), 3, 3, "shuffle" ) // Stage 4, Worker 0 .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(13).bytes(1327).frames(1), + CounterSnapshotMatcher.with().rows(3).bytes(337).frames(1), 4, 0, "input0" ) .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(13).bytes(1379).frames(1), + CounterSnapshotMatcher.with().rows(3).bytes(349).frames(1), 4, 0, "output" ) .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(13).bytes(1327).frames(1), + CounterSnapshotMatcher.with().rows(3).bytes(337).frames(1), 4, 0, "shuffle" ) + + // Stage 4, Worker 1 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 2).bytes(0, 235).frames(0, 1), + 4, 1, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(2).bytes(243).frames(1), + 4, 1, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(2).bytes(235).frames(1), + 4, 1, "shuffle" + ) + + // Stage 4, Worker 2 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 4).bytes(0, 0, 418).frames(0, 0, 1), + 4, 2, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(4).bytes(434).frames(1), + 4, 2, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(4).bytes(418).frames(1), + 4, 2, "shuffle" + ) + + // Stage 4, Worker 3 + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(0, 0, 0, 4).bytes(0, 0, 0, 439).frames(0, 0, 0, 1), + 4, 3, "input0" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(4).bytes(455).frames(1), + 4, 3, "output" + ) + .setExpectedCountersForStageWorkerChannel( + CounterSnapshotMatcher.with().rows(4).bytes(439).frames(1), + 4, 3, "shuffle" + ) .verifyResults(); } From 988f3f1a39ebd7221a9a460024bfc5b90852fb1f Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 9 Oct 2024 22:12:53 +0530 Subject: [PATCH 12/26] Address review comments --- .../WindowOperatorQueryFrameProcessor.java | 16 +- .../operator/GlueingPartitioningOperator.java | 41 ++- .../GlueingPartitioningOperatorFactory.java | 17 +- .../GlueingPartitioningOperatorTest.java | 335 ++++-------------- .../NaivePartitioningOperatorTest.java | 4 +- 5 files changed, 115 insertions(+), 298 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index b047f69615e3..e9cc5ef85e94 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -139,7 +139,9 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx if (inputChannel.canRead()) { final Frame frame = inputChannel.read(); - convertRowFrameToRowsAndColumns(frame); + LazilyDecoratedRowsAndColumns ldrc = convertRowFrameToRowsAndColumns(frame); + frameRowsAndColsBuilder.add(ldrc); + ensureMaxRowsInAWindowConstraint(frameRowsAndColsBuilder.getNumRows()); if (needToProcessBatch()) { runAllOpsOnBatch(); @@ -175,7 +177,6 @@ private void initialiseOperator() public Closeable goOrContinue(Closeable continuationObject, Receiver receiver) { RowsAndColumns rac = frameRowsAndColsBuilder.build(); - frameRowsAndColsBuilder.clear(); ensureMaxRowsInAWindowConstraint(rac.numRows()); receiver.push(rac); @@ -306,8 +307,9 @@ private long flushFrameWriter() throws IOException /** * @param frame Row based frame to be converted to a {@link RowsAndColumns} object * Throw an exception if the resultant rac used goes above the guardrail value + * @return A {@link LazilyDecoratedRowsAndColumns} encapsulating the frame. */ - private void convertRowFrameToRowsAndColumns(Frame frame) + private LazilyDecoratedRowsAndColumns convertRowFrameToRowsAndColumns(Frame frame) { final RowSignature signature = frameReader.signature(); RowBasedFrameRowsAndColumns frameRowsAndColumns = new RowBasedFrameRowsAndColumns(frame, signature); @@ -321,9 +323,7 @@ private void convertRowFrameToRowsAndColumns(Frame frame) null, (int) frameWriterFactory.allocatorCapacity() ); - // check if existing + newly added rows exceed guardrails - ensureMaxRowsInAWindowConstraint(frameRowsAndColsBuilder.getNumRows() + ldrc.numRows()); - frameRowsAndColsBuilder.add(ldrc); + return ldrc; } private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) @@ -365,7 +365,9 @@ public int getNumRows() public RowsAndColumns build() { - return new ConcatRowsAndColumns(new ArrayList<>(racList)); + ConcatRowsAndColumns concatRowsAndColumns = new ConcatRowsAndColumns(new ArrayList<>(racList)); + clear(); + return concatRowsAndColumns; } public void clear() diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index a7b5b269d1bb..a24f085aadbd 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -20,6 +20,7 @@ package org.apache.druid.query.operator; import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.ConcatRowsAndColumns; import org.apache.druid.query.rowsandcols.LimitedRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -31,6 +32,7 @@ import java.io.Closeable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -48,7 +50,7 @@ public class GlueingPartitioningOperator extends AbstractPartitioningOperator private final int maxRowsMaterialized; private RowsAndColumns previousRac; - private static final int MAX_ROWS_MATERIALIZED_NO_LIMIT = -1; + private static final Integer MAX_ROWS_MATERIALIZED_NO_LIMIT = Integer.MAX_VALUE; public GlueingPartitioningOperator( Operator child, @@ -61,7 +63,7 @@ public GlueingPartitioningOperator( public GlueingPartitioningOperator( Operator child, List partitionColumns, - int maxRowsMaterialized + Integer maxRowsMaterialized ) { super(partitionColumns, child); @@ -190,12 +192,12 @@ public RowsAndColumns next() int end = boundaries[currentIndex + 1]; LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); - if (isGlueingNeeded(previousRac, limitedRAC)) { - RowsAndColumns gluedRAC = getConcatRacForFirstPartition(previousRac, limitedRAC); - ensureMaxRowsMaterializedConstraint(gluedRAC.numRows()); + final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRac, limitedRAC); + if (previousRac != null && isGlueingNeeded(concatRacForFirstPartition, 0, previousRac.numRows())) { + ensureMaxRowsMaterializedConstraint(concatRacForFirstPartition.numRows()); previousRac = null; currentIndex++; - return gluedRAC; + return concatRacForFirstPartition; } else { if (previousRac != null) { RowsAndColumns temp = previousRac; @@ -212,29 +214,27 @@ public RowsAndColumns next() } /** - * Determines whether gluing is needed between the previous RowsAndColumns and the first partition - * of the current RowsAndColumns based on the partition columns. If the columns match, the two RACs - * will be glued together. - * - * @param previousRac The previous RowsAndColumns. - * @param firstPartitionOfCurrentRac The first partition of the current RowsAndColumns. + * Determines whether glueing is needed between 2 RACs represented as a ConcatRowsAndColumns, by comparing a row belonging to each RAC. + * The rows of different RACs are expected to be present at index1 and index2 respectively in the ConcatRAC. If the columns match, we + * can glue the 2 RACs and use the ConcatRAC. + * @param rac A {@link ConcatRowsAndColumns containing 2 RACs} + * @param index1 A row number belonging to the first RAC + * @param index2 A row number belonging to the second RAC * @return true if gluing is needed, false otherwise. */ - private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) + private boolean isGlueingNeeded(ConcatRowsAndColumns rac, int index1, int index2) { if (previousRac == null) { return false; } - final ConcatRowsAndColumns concatRac = getConcatRacForFirstPartition(previousRac, firstPartitionOfCurrentRac); for (String column : partitionColumns) { - final Column theCol = concatRac.findColumn(column); + final Column theCol = rac.findColumn(column); if (theCol == null) { - continue; + throw new ISE("Partition column [%s] not found in RAC."); } final ColumnAccessor accessor = theCol.toAccessor(); - // Compare 1st row of previousRac and firstPartitionOfCurrentRac in [previousRac, firstPartitionOfCurrentRac] form. - int comparison = accessor.compareRows(0, previousRac.numRows()); + int comparison = accessor.compareRows(index1, index2); if (comparison != 0) { return false; } @@ -244,13 +244,16 @@ private boolean isGlueingNeeded(RowsAndColumns previousRac, RowsAndColumns first private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previousRac, RowsAndColumns firstPartitionOfCurrentRac) { + if (previousRac == null) { + return new ConcatRowsAndColumns(new ArrayList<>(Collections.singletonList(firstPartitionOfCurrentRac))); + } return new ConcatRowsAndColumns(new ArrayList<>(Arrays.asList(previousRac, firstPartitionOfCurrentRac))); } } private void ensureMaxRowsMaterializedConstraint(int numRows) { - if (maxRowsMaterialized != MAX_ROWS_MATERIALIZED_NO_LIMIT && numRows > maxRowsMaterialized) { + if (numRows > maxRowsMaterialized) { throw InvalidInput.exception( "Too many rows to process (requested = %d, max = %d).", numRows, diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java index 2aeaa2397345..31a2686b3a93 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperatorFactory.java @@ -27,12 +27,12 @@ public class GlueingPartitioningOperatorFactory extends AbstractPartitioningOperatorFactory { - private final int maxRowsMaterialized; + private final Integer maxRowsMaterialized; @JsonCreator public GlueingPartitioningOperatorFactory( @JsonProperty("partitionColumns") List partitionColumns, - @JsonProperty("maxRowsMaterialized") int maxRowsMaterialized + @JsonProperty("maxRowsMaterialized") Integer maxRowsMaterialized ) { super(partitionColumns); @@ -40,7 +40,7 @@ public GlueingPartitioningOperatorFactory( } @JsonProperty("maxRowsMaterialized") - public int getMaxRowsMaterialized() + public Integer getMaxRowsMaterialized() { return maxRowsMaterialized; } @@ -54,8 +54,15 @@ public Operator wrap(Operator op) @Override public boolean validateEquivalent(OperatorFactory other) { - return super.validateEquivalent(other) && - maxRowsMaterialized == ((GlueingPartitioningOperatorFactory) other).getMaxRowsMaterialized(); + if (!super.validateEquivalent(other)) { + return false; + } + + if (!(other instanceof GlueingPartitioningOperatorFactory)) { + return false; + } + + return Objects.equals(maxRowsMaterialized, ((GlueingPartitioningOperatorFactory) other).getMaxRowsMaterialized()); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java index d907c2d490ae..01fe721f8556 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java @@ -30,163 +30,87 @@ import org.junit.Test; import java.util.Collections; -import java.util.function.BiFunction; public class GlueingPartitioningOperatorTest { @Test - public void testDefaultImplementation() + public void testPartitioning() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1, 2, 2, 1) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted") + ImmutableList.of("column") ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2}) - .expectColumn("unsorted", new int[]{20, 20}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1}) - .expectColumn("unsorted", new int[]{11}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1), + expectedSimpleRac(2, 2), + expectedSimpleRac(1) ) .runToCompletion(op); } @Test - public void testDefaultImplementationWithMultipleRACs() + public void testPartitioningWithMultipleRACs() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{50, 51, 52, 53, 54, 55}) - ) - ); - RowsAndColumns rac3 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{70, 71, 72, 73, 74}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1, 2, 2, 1), + makeSimpleRac(1, 1, 1, 2, 2, 1), + makeSimpleRac(1, 1, 2, 2, 1) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2, rac3); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted") + ImmutableList.of("column") ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2}) - .expectColumn("unsorted", new int[]{20, 20}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1, 1}) - .expectColumn("unsorted", new int[]{11, 50, 51, 52}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2}) - .expectColumn("unsorted", new int[]{53, 54}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1}) - .expectColumn("unsorted", new int[]{55, 70, 71}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2}) - .expectColumn("unsorted", new int[]{72, 73}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1}) - .expectColumn("unsorted", new int[]{74}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1), + expectedSimpleRac(2, 2), + expectedSimpleRac(1, 1, 1, 1), + expectedSimpleRac(2, 2), + expectedSimpleRac(1, 1, 1), + expectedSimpleRac(2, 2), + expectedSimpleRac(1) ) .runToCompletion(op); } @Test - public void testDefaultImplementationWithMultipleConcatenationBetweenRACs() + public void testPartitioningWithMultipleConcatenationBetweenRACs() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1}), - "unsorted", new IntArrayColumn(new int[]{20, 20}) - ) - ); - RowsAndColumns rac3 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 2}), - "unsorted", new IntArrayColumn(new int[]{30, 40}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1), + makeSimpleRac(1, 1), + makeSimpleRac(1, 2) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2, rac3); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted") + ImmutableList.of("column") ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1, 1, 1}) - .expectColumn("unsorted", new int[]{10, 10, 20, 20, 30}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2}) - .expectColumn("unsorted", new int[]{40}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1, 1, 1), + expectedSimpleRac(2) ) .runToCompletion(op); } @Test - public void testDefaultImplementationWithNoPartitionColumns() + public void testPartitioningWithNoPartitionColumns() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1, 2, 2, 1), + makeSimpleRac(1, 1, 1, 2, 2, 1) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, Collections.emptyList() @@ -194,43 +118,7 @@ public void testDefaultImplementationWithNoPartitionColumns() new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1, 2, 2, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10, 20, 20, 11}) - .allColumnsRegistered() - ) - .runToCompletion(op); - } - - @Test - public void testDefaultImplementationWithMultipleRACsAndNoPartitionColumns() - { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{50, 51, 52, 53, 54, 55}) - ) - ); - - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); - - GlueingPartitioningOperator op = new GlueingPartitioningOperator( - inlineScanOperator, - Collections.emptyList() - ); - - new OperatorTestHelper() - .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10, 20, 20, 11, 50, 51, 52, 53, 54, 55}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 1) ) .runToCompletion(op); } @@ -238,111 +126,62 @@ public void testDefaultImplementationWithMultipleRACsAndNoPartitionColumns() @Test public void testMaxRowsConstraintViolation() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted"), + ImmutableList.of("column"), 2 ); - boolean exceptionThrown = false; - try { - new OperatorTestHelper() - .expectRowsAndColumns() - .runToCompletion(op); - } - catch (DruidException e) { - Assert.assertEquals( - e.getMessage(), - "Too many rows to process (requested = 3, max = 2)." - ); - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); + Assert.assertThrows( + "Too many rows to process (requested = 3, max = 2).", + DruidException.class, + () -> new OperatorTestHelper().expectRowsAndColumns().runToCompletion(op) + ); } @Test public void testMaxRowsConstraintViolationWhenGlueing() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 2, 3}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1), + makeSimpleRac(1, 2, 3) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted"), + ImmutableList.of("column"), 3 ); - boolean exceptionThrown = false; - try { - new OperatorTestHelper() - .expectRowsAndColumns() - .runToCompletion(op); - } - catch (DruidException e) { - Assert.assertEquals( - e.getMessage(), - "Too many rows to process (requested = 4, max = 3)." - ); - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); + Assert.assertThrows( + "Too many rows to process (requested = 4, max = 3).", + DruidException.class, + () -> new OperatorTestHelper().expectRowsAndColumns().runToCompletion(op) + ); } @Test public void testMaxRowsConstraintWhenGlueing() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{2, 2, 2}), - "unsorted", new IntArrayColumn(new int[]{20, 20, 20}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1), + makeSimpleRac(2, 2, 2) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac1, rac2); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted"), + ImmutableList.of("column"), 3 ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2, 2}) - .expectColumn("unsorted", new int[]{20, 20, 20}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1), + expectedSimpleRac(2, 2, 2) ) .runToCompletion(op); } @@ -350,68 +189,34 @@ public void testMaxRowsConstraintWhenGlueing() @Test public void testStopMidStream() { - RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 1, 1, 2, 2, 1}), - "unsorted", new IntArrayColumn(new int[]{10, 10, 10, 20, 20, 11}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 1, 1, 2, 2, 1) ); - InlineScanOperator inlineScanOperator = InlineScanOperator.make(rac); - GlueingPartitioningOperator op = new GlueingPartitioningOperator( inlineScanOperator, - ImmutableList.of("sorted") + ImmutableList.of("column") ); new OperatorTestHelper() .expectAndStopAfter( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1, 1}) - .expectColumn("unsorted", new int[]{10, 10, 10}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2}) - .expectColumn("unsorted", new int[]{20, 20}) - .allColumnsRegistered() + expectedSimpleRac(1, 1, 1), + expectedSimpleRac(2, 2) ) .runToCompletion(op); } - @Test - public void testDoesNotValidateSort() + private RowsAndColumns makeSimpleRac(int... values) { - BiFunction singleHelperMaker = - (sorted, unsorted) -> - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{sorted}) - .expectColumn("unsorted", new int[]{unsorted}) - .allColumnsRegistered(); - - RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), - "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) - ) - ); - - GlueingPartitioningOperator op = new GlueingPartitioningOperator( - InlineScanOperator.make(rac), - ImmutableList.of("unsorted") + return MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of("column", new IntArrayColumn(values)) ); + } - new OperatorTestHelper() - .expectRowsAndColumns( - singleHelperMaker.apply(0, 3), - singleHelperMaker.apply(0, 54), - singleHelperMaker.apply(0, 21), - singleHelperMaker.apply(1, 1), - singleHelperMaker.apply(1, 5), - singleHelperMaker.apply(2, 54), - singleHelperMaker.apply(4, 2), - singleHelperMaker.apply(4, 3), - singleHelperMaker.apply(4, 92) - ) - .runToCompletion(op); + private RowsAndColumnsHelper expectedSimpleRac(int... values) + { + return new RowsAndColumnsHelper() + .expectColumn("column", values) + .allColumnsRegistered(); } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java index 6dcd2848bc6a..979c8ec85cee 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java @@ -32,7 +32,7 @@ public class NaivePartitioningOperatorTest { @Test - public void testDefaultImplementation() + public void testPartitioning() { RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( ImmutableMap.of( @@ -69,7 +69,7 @@ public void testDefaultImplementation() } @Test - public void testDefaultImplementationWithMultipleRACs() + public void testPartitioningWithMultipleRACs() { RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( ImmutableMap.of( From 56afb9cbba77d8d927c68f8f027ecf66d52e6c40 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 9 Oct 2024 22:20:58 +0530 Subject: [PATCH 13/26] Make the capacity field mandatory in LazilyDecoratedRowsAndColumns --- .../rowsandcols/LazilyDecoratedRowsAndColumns.java | 13 ------------- .../semantic/DefaultRowsAndColumnsDecorator.java | 3 ++- .../ColumnBasedFrameRowsAndColumnsTest.java | 1 + .../semantic/EvaluateRowsAndColumnsTest.java | 1 + 4 files changed, 4 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index d6a8ba0ceeaf..9b765dcb6080 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -79,19 +79,6 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns private List ordering; private final Integer allocatorCapacity; - public LazilyDecoratedRowsAndColumns( - RowsAndColumns base, - Interval interval, - Filter filter, - VirtualColumns virtualColumns, - OffsetLimit limit, - List ordering, - LinkedHashSet viewableColumns - ) - { - this(base, interval, filter, virtualColumns, limit, ordering, viewableColumns, null); - } - public LazilyDecoratedRowsAndColumns( RowsAndColumns base, Interval interval, diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java index 3cfcfeec6142..9710c5324512 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java @@ -133,7 +133,8 @@ public RowsAndColumns restrictColumns(List columns) virtualColumns, offsetLimit, ordering, - columns == null ? null : new LinkedHashSet<>(columns) + columns == null ? null : new LinkedHashSet<>(columns), + null ); } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java index f6a10e011464..be5be4ef672f 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java @@ -44,6 +44,7 @@ public static ColumnBasedFrameRowsAndColumns buildFrame(MapOfColumnsRowsAndColum null, OffsetLimit.limit(Integer.MAX_VALUE), null, + null, null ); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java index e2cee35a8e9a..a0f6319a6094 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java @@ -89,6 +89,7 @@ public void testMaterializeColumns() TestExprMacroTable.INSTANCE)), OffsetLimit.NONE, null, + null, null); // do the materialziation From e6167e5d1920f6de2e13964ade4f65b780664632 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 9 Oct 2024 23:06:39 +0530 Subject: [PATCH 14/26] Fix CodeQL --- .../druid/query/operator/GlueingPartitioningOperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index a24f085aadbd..e1f820ef6f19 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -231,7 +231,7 @@ private boolean isGlueingNeeded(ConcatRowsAndColumns rac, int index1, int index2 for (String column : partitionColumns) { final Column theCol = rac.findColumn(column); if (theCol == null) { - throw new ISE("Partition column [%s] not found in RAC."); + throw new ISE("Partition column [%s] not found in RAC.", column); } final ColumnAccessor accessor = theCol.toAccessor(); int comparison = accessor.compareRows(index1, index2); From d377788010fd650a234c15b37488f0d01a30933e Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 10:51:24 +0530 Subject: [PATCH 15/26] Make classes static --- .../AbstractPartitioningOperator.java | 122 +++++++---- .../operator/GlueingPartitioningOperator.java | 194 ++++++++++-------- .../operator/NaivePartitioningOperator.java | 108 +++++----- 3 files changed, 254 insertions(+), 170 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java index 89711cd3a2b8..6d290709deb9 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java @@ -19,7 +19,6 @@ package org.apache.druid.query.operator; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.RE; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -43,66 +42,63 @@ public AbstractPartitioningOperator( this.child = child; } - protected static class Continuation implements Closeable + @Override + public Closeable goOrContinue(Closeable continuation, Receiver receiver) { - Iterator iter; - Closeable subContinuation; + if (continuation != null) { + Continuation cont = (Continuation) continuation; - public Continuation(Iterator iter, Closeable subContinuation) - { - this.iter = iter; - this.subContinuation = subContinuation; - } + if (cont.iter != null) { + HandleContinuationResult handleContinuationResult = handleContinuation(receiver, cont); + if (!handleContinuationResult.needToContinueProcessing()) { + return handleContinuationResult.getContinuation(); + } - @Override - public void close() throws IOException - { - if (subContinuation != null) { - subContinuation.close(); + if (cont.subContinuation == null) { + receiver.completed(); + return null; + } } - } - } - protected Signal handlePush(RowsAndColumns rac, Receiver receiver, AtomicReference> iterHolder) - { - if (rac == null) { - throw DruidException.defensive("Should never get a null rac here."); + continuation = cont.subContinuation; } - Iterator partitionsIter = getIteratorForRAC(rac); + AtomicReference> iterHolder = new AtomicReference<>(); - AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); - while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { - handleKeepItGoing(keepItGoing, partitionsIter, receiver); - } + final Closeable retVal = child.goOrContinue( + continuation, + createReceiver(receiver, iterHolder) + ); - if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { - iterHolder.set(partitionsIter); - return Signal.PAUSE; + if (iterHolder.get() != null || retVal != null) { + return new Continuation( + iterHolder.get(), + retVal + ); + } else { + return null; } - - return keepItGoing.get(); } - protected abstract Iterator getIteratorForRAC(RowsAndColumns rac); + protected abstract HandleContinuationResult handleContinuation(Receiver receiver, Continuation cont); - protected abstract void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver); + protected abstract Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder); - protected Closeable handleNonGoCases(Signal signal, Iterator iter, Receiver receiver, Continuation cont) + protected HandleContinuationResult handleNonGoCases(Signal signal, Iterator iter, Receiver receiver, Continuation cont) { switch (signal) { case PAUSE: if (iter.hasNext()) { - return cont; + return HandleContinuationResult.of(cont); } if (cont.subContinuation == null) { // We were finished anyway receiver.completed(); - return null; + return HandleContinuationResult.of(null); } - return new Continuation(null, cont.subContinuation); + return HandleContinuationResult.of(new Continuation(null, cont.subContinuation)); case STOP: receiver.completed(); @@ -112,10 +108,62 @@ protected Closeable handleNonGoCases(Signal signal, Iterator ite catch (IOException e) { throw new RE(e, "Unable to close continuation"); } - return null; + return HandleContinuationResult.of(null); default: throw new RE("Unknown signal[%s]", signal); } } + + protected static class Continuation implements Closeable + { + Iterator iter; + Closeable subContinuation; + + public Continuation(Iterator iter, Closeable subContinuation) + { + this.iter = iter; + this.subContinuation = subContinuation; + } + + @Override + public void close() throws IOException + { + if (subContinuation != null) { + subContinuation.close(); + } + } + } + + /** + * This helper class helps us distinguish whether we need to continue processing or not. + */ + public static class HandleContinuationResult + { + private final Closeable continuation; + private final boolean continueProcessing; + + public static final HandleContinuationResult CONTINUE_PROCESSING = new HandleContinuationResult(null, true); + + private HandleContinuationResult(Closeable continuation, boolean continueProcessing) + { + this.continuation = continuation; + this.continueProcessing = continueProcessing; + } + + public static HandleContinuationResult of(Closeable closeable) + { + return new HandleContinuationResult(closeable, false); + } + + public boolean needToContinueProcessing() + { + return continueProcessing; + } + + public Closeable getContinuation() + { + return continuation; + } + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index e1f820ef6f19..a1693309c7af 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.ConcatRowsAndColumns; @@ -29,7 +30,6 @@ import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; -import java.io.Closeable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,7 +48,7 @@ public class GlueingPartitioningOperator extends AbstractPartitioningOperator { private final int maxRowsMaterialized; - private RowsAndColumns previousRac; + private final AtomicReference previousRacRef = new AtomicReference<>(null); private static final Integer MAX_ROWS_MATERIALIZED_NO_LIMIT = Integer.MAX_VALUE; @@ -71,76 +71,87 @@ public GlueingPartitioningOperator( } @Override - public Closeable goOrContinue(Closeable continuation, Receiver receiver) + protected HandleContinuationResult handleContinuation(Receiver receiver, Continuation cont) { - if (continuation != null) { - Continuation cont = (Continuation) continuation; - - if (cont.iter != null) { - while (cont.iter.hasNext()) { - RowsAndColumns next = cont.iter.next(); - - if (!cont.iter.hasNext()) { - // We are at the last RAC. Process it only if subContinuation is null, otherwise save it in previousRac. - if (cont.subContinuation == null) { - receiver.push(next); - receiver.completed(); - return null; - } else { - previousRac = next; - break; - } - } - - final Signal signal = receiver.push(next); - if (signal != Signal.GO) { - return handleNonGoCases(signal, cont.iter, receiver, cont); - } - } + while (cont.iter.hasNext()) { + RowsAndColumns next = cont.iter.next(); + if (!cont.iter.hasNext()) { + // We are at the last RAC. Process it only if subContinuation is null, otherwise save it in previousRac. if (cont.subContinuation == null) { + receiver.push(next); receiver.completed(); - return null; + return HandleContinuationResult.of(null); + } else { + previousRacRef.set(next); + break; } } - continuation = cont.subContinuation; + final Signal signal = receiver.push(next); + if (signal != Signal.GO) { + return handleNonGoCases(signal, cont.iter, receiver, cont); + } } + return HandleContinuationResult.CONTINUE_PROCESSING; + } - AtomicReference> iterHolder = new AtomicReference<>(); - - final Closeable retVal = child.goOrContinue( - continuation, - new Receiver() - { - @Override - public Signal push(RowsAndColumns rac) - { - ensureMaxRowsMaterializedConstraint(rac.numRows()); - return handlePush(rac, receiver, iterHolder); - } + private static class StaticReceiver implements Receiver + { + private final Receiver delegate; + private final AtomicReference> iterHolder; + private final AtomicReference previousRacRef; + private final int maxRowsMaterialized; + private final List partitionColumns; - @Override - public void completed() - { - if (previousRac != null) { - receiver.push(previousRac); - previousRac = null; - } - if (iterHolder.get() == null) { - receiver.completed(); - } - } - } - ); + public StaticReceiver( + Receiver delegate, + AtomicReference> iterHolder, + AtomicReference previousRacRef, + List partitionColumns, + int maxRowsMaterialized + ) + { + this.delegate = delegate; + this.iterHolder = iterHolder; + this.previousRacRef = previousRacRef; + this.partitionColumns = partitionColumns; + this.maxRowsMaterialized = maxRowsMaterialized; + } - if (iterHolder.get() != null || retVal != null) { - return new Continuation( - iterHolder.get(), - retVal - ); - } else { - return null; + @Override + public Signal push(RowsAndColumns rac) + { + ensureMaxRowsMaterializedConstraint(rac.numRows(), maxRowsMaterialized); + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } + + Iterator partitionsIter = getIteratorForRAC(rac, previousRacRef, partitionColumns, maxRowsMaterialized); + + AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); + while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { + handleKeepItGoing(keepItGoing, partitionsIter, delegate, previousRacRef); + } + + if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + + return keepItGoing.get(); + } + + @Override + public void completed() + { + if (previousRacRef.get() != null) { + delegate.push(previousRacRef.get()); + previousRacRef.set(null); + } + if (iterHolder.get() == null) { + delegate.completed(); + } } } @@ -149,14 +160,17 @@ public void completed() * It handles the boundaries of partitions within a single RAC and potentially glues * the first partition of the current RAC with the previous RAC if needed. */ - private class GluedRACsIterator implements Iterator + private static class GluedRACsIterator implements Iterator { private final RowsAndColumns rac; private final int[] boundaries; private int currentIndex = 0; private boolean firstPartitionHandled = false; + private final AtomicReference previousRacRef; + private final int maxRowsMaterialized; + private final List partitionColumns; - public GluedRACsIterator(RowsAndColumns rac) + public GluedRACsIterator(RowsAndColumns rac, AtomicReference previousRacRef, List partitionColumns, int maxRowsMaterialized) { this.rac = rac; ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); @@ -164,6 +178,9 @@ public GluedRACsIterator(RowsAndColumns rac) groupPartitioner = new DefaultClusteredGroupPartitioner(rac); } this.boundaries = groupPartitioner.computeBoundaries(partitionColumns); + this.previousRacRef = previousRacRef; + this.partitionColumns = partitionColumns; + this.maxRowsMaterialized = maxRowsMaterialized; } @Override @@ -192,16 +209,16 @@ public RowsAndColumns next() int end = boundaries[currentIndex + 1]; LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); - final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRac, limitedRAC); - if (previousRac != null && isGlueingNeeded(concatRacForFirstPartition, 0, previousRac.numRows())) { - ensureMaxRowsMaterializedConstraint(concatRacForFirstPartition.numRows()); - previousRac = null; + final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRacRef.get(), limitedRAC); + if (previousRacRef.get() != null && isGlueingNeeded(concatRacForFirstPartition, previousRacRef.get())) { + ensureMaxRowsMaterializedConstraint(concatRacForFirstPartition.numRows(), maxRowsMaterialized); + previousRacRef.set(null); currentIndex++; return concatRacForFirstPartition; } else { - if (previousRac != null) { - RowsAndColumns temp = previousRac; - previousRac = null; + if (previousRacRef.get() != null) { + RowsAndColumns temp = previousRacRef.get(); + previousRacRef.set(null); return temp; } } @@ -218,13 +235,12 @@ public RowsAndColumns next() * The rows of different RACs are expected to be present at index1 and index2 respectively in the ConcatRAC. If the columns match, we * can glue the 2 RACs and use the ConcatRAC. * @param rac A {@link ConcatRowsAndColumns containing 2 RACs} - * @param index1 A row number belonging to the first RAC - * @param index2 A row number belonging to the second RAC + * @param firstRac The 1st of two RACs present in the Concat RAC * @return true if gluing is needed, false otherwise. */ - private boolean isGlueingNeeded(ConcatRowsAndColumns rac, int index1, int index2) + private boolean isGlueingNeeded(ConcatRowsAndColumns rac, RowsAndColumns firstRac) { - if (previousRac == null) { + if (firstRac == null) { return false; } @@ -234,7 +250,7 @@ private boolean isGlueingNeeded(ConcatRowsAndColumns rac, int index1, int index2 throw new ISE("Partition column [%s] not found in RAC.", column); } final ColumnAccessor accessor = theCol.toAccessor(); - int comparison = accessor.compareRows(index1, index2); + int comparison = accessor.compareRows(0, firstRac.numRows()); if (comparison != 0) { return false; } @@ -251,7 +267,7 @@ private ConcatRowsAndColumns getConcatRacForFirstPartition(RowsAndColumns previo } } - private void ensureMaxRowsMaterializedConstraint(int numRows) + private static void ensureMaxRowsMaterializedConstraint(int numRows, int maxRowsMaterialized) { if (numRows > maxRowsMaterialized) { throw InvalidInput.exception( @@ -262,21 +278,35 @@ private void ensureMaxRowsMaterializedConstraint(int numRows) } } - @Override - protected Iterator getIteratorForRAC(RowsAndColumns rac) + protected static Iterator getIteratorForRAC( + RowsAndColumns rac, + AtomicReference previousRacRef, + List partitionColumns, + int maxRowsMaterialized + ) { - return new GluedRACsIterator(rac); + return new GluedRACsIterator(rac, previousRacRef, partitionColumns, maxRowsMaterialized); } - @Override - protected void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) + protected static void handleKeepItGoing( + AtomicReference signalRef, + Iterator iterator, + Receiver receiver, + AtomicReference previousRacRef + ) { RowsAndColumns rowsAndColumns = iterator.next(); if (iterator.hasNext()) { signalRef.set(receiver.push(rowsAndColumns)); } else { // If it's the last element, save it in previousRac instead of pushing to receiver. - previousRac = rowsAndColumns; + previousRacRef.set(rowsAndColumns); } } + + @Override + protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) + { + return new StaticReceiver(delegate, iterHolder, previousRacRef, partitionColumns, maxRowsMaterialized); + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 764c133b8bc1..0e65b3ef10dd 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -19,11 +19,11 @@ package org.apache.druid.query.operator; +import org.apache.druid.error.DruidException; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; -import java.io.Closeable; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -48,62 +48,62 @@ public NaivePartitioningOperator( } @Override - public Closeable goOrContinue(Closeable continuation, Receiver receiver) + protected HandleContinuationResult handleContinuation(Receiver receiver, Continuation cont) { - if (continuation != null) { - Continuation cont = (Continuation) continuation; - - if (cont.iter != null) { - while (cont.iter.hasNext()) { - final Signal signal = receiver.push(cont.iter.next()); - if (signal != Signal.GO) { - return handleNonGoCases(signal, cont.iter, receiver, cont); - } - } - - if (cont.subContinuation == null) { - receiver.completed(); - return null; - } + while (cont.iter.hasNext()) { + final Signal signal = receiver.push(cont.iter.next()); + if (signal != Signal.GO) { + return handleNonGoCases(signal, cont.iter, receiver, cont); } + } + return HandleContinuationResult.CONTINUE_PROCESSING; + } + + private static class StaticReceiver implements Receiver + { + Receiver delegate; + AtomicReference> iterHolder; + List partitionColumns; + + public StaticReceiver(Receiver delegate, AtomicReference> iterHolder, List partitionColumns) + { + this.delegate = delegate; + this.iterHolder = iterHolder; + this.partitionColumns = partitionColumns; + } + + @Override + public Signal push(RowsAndColumns rac) + { + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } + + Iterator partitionsIter = getIteratorForRAC(rac, partitionColumns); - continuation = cont.subContinuation; + AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); + while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { + handleKeepItGoing(keepItGoing, partitionsIter, delegate); + } + + if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + + return keepItGoing.get(); } - AtomicReference> iterHolder = new AtomicReference<>(); - - final Closeable retVal = child.goOrContinue( - continuation, - new Receiver() - { - @Override - public Signal push(RowsAndColumns rac) - { - return handlePush(rac, receiver, iterHolder); - } - - @Override - public void completed() - { - if (iterHolder.get() == null) { - receiver.completed(); - } - } - } - ); - - if (iterHolder.get() != null || retVal != null) { - return new Continuation( - iterHolder.get(), - retVal - ); - } else { - return null; + @Override + public void completed() + { + if (iterHolder.get() == null) { + delegate.completed(); + } } } - @Override - protected Iterator getIteratorForRAC(RowsAndColumns rac) + protected static Iterator getIteratorForRAC(RowsAndColumns rac, List partitionColumns) { ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); if (groupPartitioner == null) { @@ -112,9 +112,15 @@ protected Iterator getIteratorForRAC(RowsAndColumns rac) return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); } + protected static void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) + { + RowsAndColumns next = iterator.next(); + signalRef.set(receiver.push(next)); + } + @Override - protected void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) + protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) { - signalRef.set(receiver.push(iterator.next())); + return new StaticReceiver(delegate, iterHolder, partitionColumns); } } From 82a60cb6bb087f29db47a00a394abb912107418d Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 13:20:49 +0530 Subject: [PATCH 16/26] Remove separate method for keepItGoing logic --- .../operator/GlueingPartitioningOperator.java | 38 +++++++------------ .../operator/NaivePartitioningOperator.java | 22 ++++------- 2 files changed, 22 insertions(+), 38 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index a1693309c7af..783817521fcb 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -96,7 +96,7 @@ protected HandleContinuationResult handleContinuation(Receiver receiver, Continu return HandleContinuationResult.CONTINUE_PROCESSING; } - private static class StaticReceiver implements Receiver + private static class PartitioningReceiver implements Receiver { private final Receiver delegate; private final AtomicReference> iterHolder; @@ -104,7 +104,7 @@ private static class StaticReceiver implements Receiver private final int maxRowsMaterialized; private final List partitionColumns; - public StaticReceiver( + public PartitioningReceiver( Receiver delegate, AtomicReference> iterHolder, AtomicReference previousRacRef, @@ -129,17 +129,23 @@ public Signal push(RowsAndColumns rac) Iterator partitionsIter = getIteratorForRAC(rac, previousRacRef, partitionColumns, maxRowsMaterialized); - AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); - while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { - handleKeepItGoing(keepItGoing, partitionsIter, delegate, previousRacRef); + Signal keepItGoing = Signal.GO; + while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { + final RowsAndColumns rowsAndColumns = partitionsIter.next(); + if (partitionsIter.hasNext()) { + keepItGoing = delegate.push(rowsAndColumns); + } else { + // If it's the last element, save it in previousRac instead of pushing to receiver. + previousRacRef.set(rowsAndColumns); + } } - if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { + if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { iterHolder.set(partitionsIter); return Signal.PAUSE; } - return keepItGoing.get(); + return keepItGoing; } @Override @@ -288,25 +294,9 @@ protected static Iterator getIteratorForRAC( return new GluedRACsIterator(rac, previousRacRef, partitionColumns, maxRowsMaterialized); } - protected static void handleKeepItGoing( - AtomicReference signalRef, - Iterator iterator, - Receiver receiver, - AtomicReference previousRacRef - ) - { - RowsAndColumns rowsAndColumns = iterator.next(); - if (iterator.hasNext()) { - signalRef.set(receiver.push(rowsAndColumns)); - } else { - // If it's the last element, save it in previousRac instead of pushing to receiver. - previousRacRef.set(rowsAndColumns); - } - } - @Override protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) { - return new StaticReceiver(delegate, iterHolder, previousRacRef, partitionColumns, maxRowsMaterialized); + return new PartitioningReceiver(delegate, iterHolder, previousRacRef, partitionColumns, maxRowsMaterialized); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 0e65b3ef10dd..f85edabc2f70 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -59,13 +59,13 @@ protected HandleContinuationResult handleContinuation(Receiver receiver, Continu return HandleContinuationResult.CONTINUE_PROCESSING; } - private static class StaticReceiver implements Receiver + private static class PartitioningReceiver implements Receiver { Receiver delegate; AtomicReference> iterHolder; List partitionColumns; - public StaticReceiver(Receiver delegate, AtomicReference> iterHolder, List partitionColumns) + public PartitioningReceiver(Receiver delegate, AtomicReference> iterHolder, List partitionColumns) { this.delegate = delegate; this.iterHolder = iterHolder; @@ -81,17 +81,17 @@ public Signal push(RowsAndColumns rac) Iterator partitionsIter = getIteratorForRAC(rac, partitionColumns); - AtomicReference keepItGoing = new AtomicReference<>(Signal.GO); - while (keepItGoing.get() == Signal.GO && partitionsIter.hasNext()) { - handleKeepItGoing(keepItGoing, partitionsIter, delegate); + Signal keepItGoing = Signal.GO; + while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { + keepItGoing = delegate.push(partitionsIter.next()); } - if (keepItGoing.get() == Signal.PAUSE && partitionsIter.hasNext()) { + if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { iterHolder.set(partitionsIter); return Signal.PAUSE; } - return keepItGoing.get(); + return keepItGoing; } @Override @@ -112,15 +112,9 @@ protected static Iterator getIteratorForRAC(RowsAndColumns rac, return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); } - protected static void handleKeepItGoing(AtomicReference signalRef, Iterator iterator, Receiver receiver) - { - RowsAndColumns next = iterator.next(); - signalRef.set(receiver.push(next)); - } - @Override protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) { - return new StaticReceiver(delegate, iterHolder, partitionColumns); + return new PartitioningReceiver(delegate, iterHolder, partitionColumns); } } From db19325a272a75db734f4019bb3a6c7acc0cb381 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 13:31:37 +0530 Subject: [PATCH 17/26] Address review comments --- .../WindowOperatorQueryFrameProcessor.java | 12 ++- .../NaivePartitioningOperatorTest.java | 102 +++++++----------- 2 files changed, 44 insertions(+), 70 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index e9cc5ef85e94..5de95a18186b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -96,9 +96,9 @@ public WindowOperatorQueryFrameProcessor( this.outputChannel = outputChannel; this.frameWriterFactory = frameWriterFactory; this.operatorFactoryList = operatorFactoryList; - this.frameRowsAndColsBuilder = new RowsAndColumnsBuilder(); this.resultRowAndCols = new ArrayList<>(); this.maxRowsMaterialized = MultiStageQueryContext.getMaxRowsMaterializedInWindow(query.context()); + this.frameRowsAndColsBuilder = new RowsAndColumnsBuilder(this.maxRowsMaterialized); this.frameReader = frameReader; @@ -141,7 +141,6 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx final Frame frame = inputChannel.read(); LazilyDecoratedRowsAndColumns ldrc = convertRowFrameToRowsAndColumns(frame); frameRowsAndColsBuilder.add(ldrc); - ensureMaxRowsInAWindowConstraint(frameRowsAndColsBuilder.getNumRows()); if (needToProcessBatch()) { runAllOpsOnBatch(); @@ -177,7 +176,7 @@ private void initialiseOperator() public Closeable goOrContinue(Closeable continuationObject, Receiver receiver) { RowsAndColumns rac = frameRowsAndColsBuilder.build(); - ensureMaxRowsInAWindowConstraint(rac.numRows()); + ensureMaxRowsInAWindowConstraint(rac.numRows(), maxRowsMaterialized); receiver.push(rac); if (inputChannel.isFinished()) { @@ -326,7 +325,7 @@ private LazilyDecoratedRowsAndColumns convertRowFrameToRowsAndColumns(Frame fram return ldrc; } - private void ensureMaxRowsInAWindowConstraint(int numRowsInWindow) + private static void ensureMaxRowsInAWindowConstraint(int numRowsInWindow, int maxRowsMaterialized) { if (numRowsInWindow > maxRowsMaterialized) { throw new MSQException(new TooManyRowsInAWindowFault( @@ -345,17 +344,20 @@ private static class RowsAndColumnsBuilder { private final List racList; private int totalRows; + private final int maxRowsMaterialized; - public RowsAndColumnsBuilder() + public RowsAndColumnsBuilder(int maxRowsMaterialized) { this.racList = new ArrayList<>(); this.totalRows = 0; + this.maxRowsMaterialized = maxRowsMaterialized; } public void add(RowsAndColumns rac) { racList.add(rac); totalRows += rac.numRows(); + ensureMaxRowsInAWindowConstraint(getNumRows(), maxRowsMaterialized); } public int getNumRows() diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java index 979c8ec85cee..0a7edba868e3 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java @@ -34,36 +34,21 @@ public class NaivePartitioningOperatorTest @Test public void testPartitioning() { - RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), - "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(0, 0, 0, 1, 1, 2, 4, 4, 4) ); NaivePartitioningOperator op = new NaivePartitioningOperator( - ImmutableList.of("sorted"), - InlineScanOperator.make(rac) + ImmutableList.of("column"), + inlineScanOperator ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{0, 0, 0}) - .expectColumn("unsorted", new int[]{3, 54, 21}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1}) - .expectColumn("unsorted", new int[]{1, 5}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2}) - .expectColumn("unsorted", new int[]{54}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{4, 4, 4}) - .expectColumn("unsorted", new int[]{2, 3, 92}) - .allColumnsRegistered() + expectedSimpleRac(0, 0, 0), + expectedSimpleRac(1, 1), + expectedSimpleRac(2), + expectedSimpleRac(4, 4, 4) ) .runToCompletion(op); } @@ -71,42 +56,22 @@ public void testPartitioning() @Test public void testPartitioningWithMultipleRACs() { - RowsAndColumns rac1 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1}), - "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5}) - ) - ); - RowsAndColumns rac2 = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{1, 2, 2, 2}), - "unsorted", new IntArrayColumn(new int[]{10, 20, 30, 40}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(0, 0, 0, 1, 1), + makeSimpleRac(1, 2, 2, 2) ); NaivePartitioningOperator op = new NaivePartitioningOperator( - ImmutableList.of("sorted"), - InlineScanOperator.make(rac1, rac2) + ImmutableList.of("column"), + inlineScanOperator ); new OperatorTestHelper() .expectRowsAndColumns( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{0, 0, 0}) - .expectColumn("unsorted", new int[]{3, 54, 21}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1}) - .expectColumn("unsorted", new int[]{1, 5}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1}) - .expectColumn("unsorted", new int[]{10}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2, 2, 2}) - .expectColumn("unsorted", new int[]{20, 30, 40}) - .allColumnsRegistered() + expectedSimpleRac(0, 0, 0), + expectedSimpleRac(1, 1), + expectedSimpleRac(1), + expectedSimpleRac(2, 2, 2) ) .runToCompletion(op); } @@ -114,26 +79,19 @@ public void testPartitioningWithMultipleRACs() @Test public void testStopMidStream() { - RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), - "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) - ) + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(0, 0, 0, 1, 1, 2, 4, 4, 4) ); NaivePartitioningOperator op = new NaivePartitioningOperator( - ImmutableList.of("sorted"), - InlineScanOperator.make(rac) + ImmutableList.of("column"), + inlineScanOperator ); new OperatorTestHelper() .expectAndStopAfter( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{0, 0, 0}) - .expectColumn("unsorted", new int[]{3, 54, 21}), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1}) - .expectColumn("unsorted", new int[]{1, 5}) + expectedSimpleRac(0, 0, 0), + expectedSimpleRac(1, 1) ) .runToCompletion(op); } @@ -174,4 +132,18 @@ public void testDoesNotValidateSort() ) .runToCompletion(op); } + + private RowsAndColumns makeSimpleRac(int... values) + { + return MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of("column", new IntArrayColumn(values)) + ); + } + + private RowsAndColumnsHelper expectedSimpleRac(int... values) + { + return new RowsAndColumnsHelper() + .expectColumn("column", values) + .allColumnsRegistered(); + } } From a865fdfe99d447a7b1f2ea9b88629383185b06e2 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 13:40:13 +0530 Subject: [PATCH 18/26] Remove firstPartitionHandled boolean, use currentIndex instead --- .../operator/GlueingPartitioningOperator.java | 4 +-- .../GlueingPartitioningOperatorTest.java | 29 +++++++++++++++++++ 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index 783817521fcb..6397773f04e0 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -171,7 +171,6 @@ private static class GluedRACsIterator implements Iterator private final RowsAndColumns rac; private final int[] boundaries; private int currentIndex = 0; - private boolean firstPartitionHandled = false; private final AtomicReference previousRacRef; private final int maxRowsMaterialized; private final List partitionColumns; @@ -209,8 +208,7 @@ public RowsAndColumns next() throw new NoSuchElementException(); } - if (!firstPartitionHandled) { - firstPartitionHandled = true; + if (currentIndex == 0) { int start = boundaries[currentIndex]; int end = boundaries[currentIndex + 1]; LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java index 01fe721f8556..fef29ffd5259 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java @@ -103,6 +103,35 @@ public void testPartitioningWithMultipleConcatenationBetweenRACs() .runToCompletion(op); } + @Test + public void testPartitioningWithNoGlueing() + { + InlineScanOperator inlineScanOperator = InlineScanOperator.make( + makeSimpleRac(1, 2, 3), + makeSimpleRac(4, 5, 6), + makeSimpleRac(7, 8, 9) + ); + + GlueingPartitioningOperator op = new GlueingPartitioningOperator( + inlineScanOperator, + ImmutableList.of("column") + ); + + new OperatorTestHelper() + .expectRowsAndColumns( + expectedSimpleRac(1), + expectedSimpleRac(2), + expectedSimpleRac(3), + expectedSimpleRac(4), + expectedSimpleRac(5), + expectedSimpleRac(6), + expectedSimpleRac(7), + expectedSimpleRac(8), + expectedSimpleRac(9) + ) + .runToCompletion(op); + } + @Test public void testPartitioningWithNoPartitionColumns() { From 3075b96b6471d656641200c823f7540de1d6130e Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 13:48:01 +0530 Subject: [PATCH 19/26] Change isGlueingNeeded() to compare first and last rows of ConcatRAC --- .../operator/GlueingPartitioningOperator.java | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index 6397773f04e0..65be65848a97 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -214,7 +214,7 @@ public RowsAndColumns next() LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRacRef.get(), limitedRAC); - if (previousRacRef.get() != null && isGlueingNeeded(concatRacForFirstPartition, previousRacRef.get())) { + if (isGlueingNeeded(concatRacForFirstPartition)) { ensureMaxRowsMaterializedConstraint(concatRacForFirstPartition.numRows(), maxRowsMaterialized); previousRacRef.set(null); currentIndex++; @@ -236,25 +236,20 @@ public RowsAndColumns next() /** * Determines whether glueing is needed between 2 RACs represented as a ConcatRowsAndColumns, by comparing a row belonging to each RAC. - * The rows of different RACs are expected to be present at index1 and index2 respectively in the ConcatRAC. If the columns match, we - * can glue the 2 RACs and use the ConcatRAC. + * We do this by comparing the first and last rows of the Concat RAC, as they would belong to the two respective RACs. + * If the columns match, we can glue the 2 RACs and use the ConcatRAC. * @param rac A {@link ConcatRowsAndColumns containing 2 RACs} - * @param firstRac The 1st of two RACs present in the Concat RAC * @return true if gluing is needed, false otherwise. */ - private boolean isGlueingNeeded(ConcatRowsAndColumns rac, RowsAndColumns firstRac) + private boolean isGlueingNeeded(ConcatRowsAndColumns rac) { - if (firstRac == null) { - return false; - } - for (String column : partitionColumns) { final Column theCol = rac.findColumn(column); if (theCol == null) { throw new ISE("Partition column [%s] not found in RAC.", column); } final ColumnAccessor accessor = theCol.toAccessor(); - int comparison = accessor.compareRows(0, firstRac.numRows()); + int comparison = accessor.compareRows(0, rac.numRows() - 1); if (comparison != 0) { return false; } From 071cfed4962af4b9e8c88d0850e880999c47f8a4 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 13:55:49 +0530 Subject: [PATCH 20/26] Pass allocatorCapacity as Long, convert to integer in the ldrc layer --- .../druid/msq/querykit/WindowOperatorQueryFrameProcessor.java | 2 +- .../query/rowsandcols/LazilyDecoratedRowsAndColumns.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index 5de95a18186b..bf074446a47e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -320,7 +320,7 @@ private LazilyDecoratedRowsAndColumns convertRowFrameToRowsAndColumns(Frame fram OffsetLimit.limit(Integer.MAX_VALUE), null, null, - (int) frameWriterFactory.allocatorCapacity() + frameWriterFactory.allocatorCapacity() ); return ldrc; } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 9b765dcb6080..a45c999b3c8c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -87,7 +87,7 @@ public LazilyDecoratedRowsAndColumns( OffsetLimit limit, List ordering, LinkedHashSet viewableColumns, - Integer allocatorCapacity + Long allocatorCapacity ) { this.base = base; @@ -97,7 +97,7 @@ public LazilyDecoratedRowsAndColumns( this.limit = limit; this.ordering = ordering; this.viewableColumns = viewableColumns; - this.allocatorCapacity = allocatorCapacity != null ? allocatorCapacity : 200 << 20; + this.allocatorCapacity = allocatorCapacity != null ? allocatorCapacity.intValue() : 200 << 20; } @Override From f947fc763efab5ca914aef41fce512a2e8e808f2 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 14:42:43 +0530 Subject: [PATCH 21/26] Fix CodeQL --- .../druid/query/operator/GlueingPartitioningOperator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index 65be65848a97..b8f6eed52cf5 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -122,11 +122,10 @@ public PartitioningReceiver( @Override public Signal push(RowsAndColumns rac) { - ensureMaxRowsMaterializedConstraint(rac.numRows(), maxRowsMaterialized); if (rac == null) { throw DruidException.defensive("Should never get a null rac here."); } - + ensureMaxRowsMaterializedConstraint(rac.numRows(), maxRowsMaterialized); Iterator partitionsIter = getIteratorForRAC(rac, previousRacRef, partitionColumns, maxRowsMaterialized); Signal keepItGoing = Signal.GO; From b72533225bede3a462c2b79e46c059300357f610 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 17:04:50 +0530 Subject: [PATCH 22/26] Address review comment: Change order of conditionals to simplify the logic --- .../operator/GlueingPartitioningOperator.java | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index b8f6eed52cf5..891036d827b7 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -207,28 +207,30 @@ public RowsAndColumns next() throw new NoSuchElementException(); } - if (currentIndex == 0) { - int start = boundaries[currentIndex]; - int end = boundaries[currentIndex + 1]; - LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); + int start = boundaries[currentIndex]; + int end = boundaries[currentIndex + 1]; + + if (previousRacRef.get() != null) { + if (currentIndex != 0) { + throw new ISE("previousRac should be non-null only while handling the first partition boundary."); + } - final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRacRef.get(), limitedRAC); + final RowsAndColumns previousRac = previousRacRef.get(); + previousRacRef.set(null); + + final LimitedRowsAndColumns limitedRAC = new LimitedRowsAndColumns(rac, start, end); + + final ConcatRowsAndColumns concatRacForFirstPartition = getConcatRacForFirstPartition(previousRac, limitedRAC); if (isGlueingNeeded(concatRacForFirstPartition)) { ensureMaxRowsMaterializedConstraint(concatRacForFirstPartition.numRows(), maxRowsMaterialized); - previousRacRef.set(null); currentIndex++; return concatRacForFirstPartition; } else { - if (previousRacRef.get() != null) { - RowsAndColumns temp = previousRacRef.get(); - previousRacRef.set(null); - return temp; - } + return previousRac; } } - int start = boundaries[currentIndex]; - int end = boundaries[currentIndex + 1]; + // If previousRac is null, just return the next partitioned RAC. currentIndex++; return new LimitedRowsAndColumns(rac, start, end); } From 00ce202b03bde6ea75aa62a6ec46e709d8609c63 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 20:12:21 +0530 Subject: [PATCH 23/26] Create AbstractReceiver --- .../AbstractPartitioningOperator.java | 31 ++++++++++++++++ .../operator/GlueingPartitioningOperator.java | 35 ++++++------------ .../operator/NaivePartitioningOperator.java | 37 +++++++------------ 3 files changed, 57 insertions(+), 46 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java index 6d290709deb9..17aed31ad7ae 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java @@ -80,6 +80,37 @@ public Closeable goOrContinue(Closeable continuation, Receiver receiver) } } + protected abstract static class AbstractReceiver implements Receiver + { + protected final Receiver delegate; + protected final AtomicReference> iterHolder; + protected final List partitionColumns; + + public AbstractReceiver( + Receiver delegate, + AtomicReference> iterHolder, + List partitionColumns + ) + { + this.delegate = delegate; + this.iterHolder = iterHolder; + this.partitionColumns = partitionColumns; + } + + @Override + public abstract Signal push(RowsAndColumns rac); + + @Override + public void completed() + { + if (iterHolder.get() == null) { + delegate.completed(); + } + } + + protected abstract Iterator getIteratorForRAC(RowsAndColumns rac); + } + protected abstract HandleContinuationResult handleContinuation(Receiver receiver, Continuation cont); protected abstract Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder); diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index 891036d827b7..d479e27cac8d 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -96,15 +96,12 @@ protected HandleContinuationResult handleContinuation(Receiver receiver, Continu return HandleContinuationResult.CONTINUE_PROCESSING; } - private static class PartitioningReceiver implements Receiver + private static class GlueingReceiver extends AbstractReceiver { - private final Receiver delegate; - private final AtomicReference> iterHolder; private final AtomicReference previousRacRef; private final int maxRowsMaterialized; - private final List partitionColumns; - public PartitioningReceiver( + public GlueingReceiver( Receiver delegate, AtomicReference> iterHolder, AtomicReference previousRacRef, @@ -112,10 +109,8 @@ public PartitioningReceiver( int maxRowsMaterialized ) { - this.delegate = delegate; - this.iterHolder = iterHolder; + super(delegate, iterHolder, partitionColumns); this.previousRacRef = previousRacRef; - this.partitionColumns = partitionColumns; this.maxRowsMaterialized = maxRowsMaterialized; } @@ -126,7 +121,7 @@ public Signal push(RowsAndColumns rac) throw DruidException.defensive("Should never get a null rac here."); } ensureMaxRowsMaterializedConstraint(rac.numRows(), maxRowsMaterialized); - Iterator partitionsIter = getIteratorForRAC(rac, previousRacRef, partitionColumns, maxRowsMaterialized); + Iterator partitionsIter = getIteratorForRAC(rac); Signal keepItGoing = Signal.GO; while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { @@ -154,9 +149,13 @@ public void completed() delegate.push(previousRacRef.get()); previousRacRef.set(null); } - if (iterHolder.get() == null) { - delegate.completed(); - } + super.completed(); + } + + @Override + protected Iterator getIteratorForRAC(RowsAndColumns rac) + { + return new GluedRACsIterator(rac, previousRacRef, partitionColumns, maxRowsMaterialized); } } @@ -278,19 +277,9 @@ private static void ensureMaxRowsMaterializedConstraint(int numRows, int maxRows } } - protected static Iterator getIteratorForRAC( - RowsAndColumns rac, - AtomicReference previousRacRef, - List partitionColumns, - int maxRowsMaterialized - ) - { - return new GluedRACsIterator(rac, previousRacRef, partitionColumns, maxRowsMaterialized); - } - @Override protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) { - return new PartitioningReceiver(delegate, iterHolder, previousRacRef, partitionColumns, maxRowsMaterialized); + return new GlueingReceiver(delegate, iterHolder, previousRacRef, partitionColumns, maxRowsMaterialized); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index f85edabc2f70..f9bbf5460e41 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -59,17 +59,15 @@ protected HandleContinuationResult handleContinuation(Receiver receiver, Continu return HandleContinuationResult.CONTINUE_PROCESSING; } - private static class PartitioningReceiver implements Receiver + private static class NaiveReceiver extends AbstractReceiver { - Receiver delegate; - AtomicReference> iterHolder; - List partitionColumns; - - public PartitioningReceiver(Receiver delegate, AtomicReference> iterHolder, List partitionColumns) + public NaiveReceiver( + Receiver delegate, + AtomicReference> iterHolder, + List partitionColumns + ) { - this.delegate = delegate; - this.iterHolder = iterHolder; - this.partitionColumns = partitionColumns; + super(delegate, iterHolder, partitionColumns); } @Override @@ -79,7 +77,7 @@ public Signal push(RowsAndColumns rac) throw DruidException.defensive("Should never get a null rac here."); } - Iterator partitionsIter = getIteratorForRAC(rac, partitionColumns); + Iterator partitionsIter = getIteratorForRAC(rac); Signal keepItGoing = Signal.GO; while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { @@ -95,26 +93,19 @@ public Signal push(RowsAndColumns rac) } @Override - public void completed() + protected Iterator getIteratorForRAC(RowsAndColumns rac) { - if (iterHolder.get() == null) { - delegate.completed(); + ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); + if (groupPartitioner == null) { + groupPartitioner = new DefaultClusteredGroupPartitioner(rac); } + return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); } } - protected static Iterator getIteratorForRAC(RowsAndColumns rac, List partitionColumns) - { - ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); - if (groupPartitioner == null) { - groupPartitioner = new DefaultClusteredGroupPartitioner(rac); - } - return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); - } - @Override protected Receiver createReceiver(Receiver delegate, AtomicReference> iterHolder) { - return new PartitioningReceiver(delegate, iterHolder, partitionColumns); + return new NaiveReceiver(delegate, iterHolder, partitionColumns); } } From bb8f9fedd7c93c4d57601fff6ef4d38d004bd56b Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 20:34:58 +0530 Subject: [PATCH 24/26] Move push() method logic to AbstractReceiver --- .../AbstractPartitioningOperator.java | 28 +++++++++++++++- .../operator/GlueingPartitioningOperator.java | 32 ++++++++----------- .../operator/NaivePartitioningOperator.java | 23 ------------- 3 files changed, 40 insertions(+), 43 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java index 17aed31ad7ae..ae563f2135c9 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.RE; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -98,7 +99,27 @@ public AbstractReceiver( } @Override - public abstract Signal push(RowsAndColumns rac); + public Signal push(RowsAndColumns rac) + { + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } + + Iterator partitionsIter = getIteratorForRAC(rac); + + Signal keepItGoing = Signal.GO; + while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { + final RowsAndColumns rowsAndColumns = partitionsIter.next(); + keepItGoing = pushPartition(rowsAndColumns, !partitionsIter.hasNext(), keepItGoing); + } + + if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + + return keepItGoing; + } @Override public void completed() @@ -108,6 +129,11 @@ public void completed() } } + protected Signal pushPartition(RowsAndColumns partition, boolean isLastPartition, Signal previousSignal) + { + return delegate.push(partition); + } + protected abstract Iterator getIteratorForRAC(RowsAndColumns rac); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index d479e27cac8d..f82dd54d9071 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -121,25 +121,7 @@ public Signal push(RowsAndColumns rac) throw DruidException.defensive("Should never get a null rac here."); } ensureMaxRowsMaterializedConstraint(rac.numRows(), maxRowsMaterialized); - Iterator partitionsIter = getIteratorForRAC(rac); - - Signal keepItGoing = Signal.GO; - while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { - final RowsAndColumns rowsAndColumns = partitionsIter.next(); - if (partitionsIter.hasNext()) { - keepItGoing = delegate.push(rowsAndColumns); - } else { - // If it's the last element, save it in previousRac instead of pushing to receiver. - previousRacRef.set(rowsAndColumns); - } - } - - if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { - iterHolder.set(partitionsIter); - return Signal.PAUSE; - } - - return keepItGoing; + return super.push(rac); } @Override @@ -152,6 +134,18 @@ public void completed() super.completed(); } + @Override + protected Signal pushPartition(RowsAndColumns partition, boolean isLastPartition, Signal previousSignal) + { + if (isLastPartition) { + // If it's the last partition, save it in previousRac instead of pushing to receiver. + previousRacRef.set(partition); + return previousSignal; + } else { + return super.pushPartition(partition, isLastPartition, previousSignal); + } + } + @Override protected Iterator getIteratorForRAC(RowsAndColumns rac) { diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index f9bbf5460e41..452d14ade790 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -19,7 +19,6 @@ package org.apache.druid.query.operator; -import org.apache.druid.error.DruidException; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; @@ -70,28 +69,6 @@ public NaiveReceiver( super(delegate, iterHolder, partitionColumns); } - @Override - public Signal push(RowsAndColumns rac) - { - if (rac == null) { - throw DruidException.defensive("Should never get a null rac here."); - } - - Iterator partitionsIter = getIteratorForRAC(rac); - - Signal keepItGoing = Signal.GO; - while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { - keepItGoing = delegate.push(partitionsIter.next()); - } - - if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { - iterHolder.set(partitionsIter); - return Signal.PAUSE; - } - - return keepItGoing; - } - @Override protected Iterator getIteratorForRAC(RowsAndColumns rac) { From aa2e46f440c69c762fcd25f22204a6f728f7677a Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 10 Oct 2024 20:41:27 +0530 Subject: [PATCH 25/26] Improve some access modifiers --- .../query/operator/AbstractPartitioningOperator.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java index ae563f2135c9..ed45ff5b8084 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/AbstractPartitioningOperator.java @@ -195,12 +195,12 @@ public void close() throws IOException /** * This helper class helps us distinguish whether we need to continue processing or not. */ - public static class HandleContinuationResult + protected static class HandleContinuationResult { private final Closeable continuation; private final boolean continueProcessing; - public static final HandleContinuationResult CONTINUE_PROCESSING = new HandleContinuationResult(null, true); + protected static final HandleContinuationResult CONTINUE_PROCESSING = new HandleContinuationResult(null, true); private HandleContinuationResult(Closeable continuation, boolean continueProcessing) { @@ -208,17 +208,17 @@ private HandleContinuationResult(Closeable continuation, boolean continueProcess this.continueProcessing = continueProcessing; } - public static HandleContinuationResult of(Closeable closeable) + protected static HandleContinuationResult of(Closeable closeable) { return new HandleContinuationResult(closeable, false); } - public boolean needToContinueProcessing() + private boolean needToContinueProcessing() { return continueProcessing; } - public Closeable getContinuation() + private Closeable getContinuation() { return continuation; } From b73f47266be86c6cbd2923ed3b0f7ad0edf3e89c Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 16 Oct 2024 18:54:12 +0530 Subject: [PATCH 26/26] Address review comments --- .../WindowOperatorQueryFrameProcessor.java | 15 +-- .../operator/GlueingPartitioningOperator.java | 8 +- .../operator/NaivePartitioningOperator.java | 6 +- .../GlueingPartitioningOperatorTest.java | 106 ++++++++---------- .../NaivePartitioningOperatorTest.java | 42 +++---- .../operator/window/RowsAndColumnsHelper.java | 15 +++ 6 files changed, 82 insertions(+), 110 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index bf074446a47e..6e91b19df4d4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -78,7 +78,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private final VirtualColumns frameWriterVirtualColumns; private final SettableLongVirtualColumn partitionBoostVirtualColumn; - private Operator op = null; + private Operator operator = null; final AtomicInteger rowId = new AtomicInteger(0); @@ -144,12 +144,7 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx if (needToProcessBatch()) { runAllOpsOnBatch(); - try { - flushAllRowsAndCols(); - } - catch (IOException e) { - throw new RuntimeException(e); - } + flushAllRowsAndCols(); } return ReturnOrAwait.runAgain(); } else if (inputChannel.isFinished()) { @@ -169,7 +164,7 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx private void initialiseOperator() { - op = new Operator() + operator = new Operator() { @Nullable @Override @@ -191,13 +186,13 @@ public Closeable goOrContinue(Closeable continuationObject, Receiver receiver) } }; for (OperatorFactory of : operatorFactoryList) { - op = of.wrap(op); + operator = of.wrap(operator); } } private void runAllOpsOnBatch() { - op.goOrContinue(null, new Operator.Receiver() + operator.goOrContinue(null, new Operator.Receiver() { @Override public Operator.Signal push(RowsAndColumns rac) diff --git a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java index f82dd54d9071..7601ec703b3a 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/GlueingPartitioningOperator.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator; +import com.google.common.base.Preconditions; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.ISE; @@ -28,7 +29,6 @@ import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; -import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.util.ArrayList; import java.util.Arrays; @@ -67,6 +67,7 @@ public GlueingPartitioningOperator( ) { super(partitionColumns, child); + Preconditions.checkNotNull(maxRowsMaterialized, "maxRowsMaterialized cannot be null"); this.maxRowsMaterialized = maxRowsMaterialized; } @@ -170,10 +171,7 @@ private static class GluedRACsIterator implements Iterator public GluedRACsIterator(RowsAndColumns rac, AtomicReference previousRacRef, List partitionColumns, int maxRowsMaterialized) { this.rac = rac; - ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); - if (groupPartitioner == null) { - groupPartitioner = new DefaultClusteredGroupPartitioner(rac); - } + final ClusteredGroupPartitioner groupPartitioner = ClusteredGroupPartitioner.fromRAC(rac); this.boundaries = groupPartitioner.computeBoundaries(partitionColumns); this.previousRacRef = previousRacRef; this.partitionColumns = partitionColumns; diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 452d14ade790..cde6c8674b73 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -21,7 +21,6 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; -import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.util.Iterator; import java.util.List; @@ -72,10 +71,7 @@ public NaiveReceiver( @Override protected Iterator getIteratorForRAC(RowsAndColumns rac) { - ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); - if (groupPartitioner == null) { - groupPartitioner = new DefaultClusteredGroupPartitioner(rac); - } + final ClusteredGroupPartitioner groupPartitioner = ClusteredGroupPartitioner.fromRAC(rac); return groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java index fef29ffd5259..6f0d70015884 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/GlueingPartitioningOperatorTest.java @@ -20,12 +20,8 @@ package org.apache.druid.query.operator; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.error.DruidException; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; -import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; -import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.junit.Assert; import org.junit.Test; @@ -37,7 +33,7 @@ public class GlueingPartitioningOperatorTest public void testPartitioning() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1, 2, 2, 1) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -47,9 +43,9 @@ public void testPartitioning() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1, 1, 1), - expectedSimpleRac(2, 2), - expectedSimpleRac(1) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2), + RowsAndColumnsHelper.expectedSingleColumnRac(1) ) .runToCompletion(op); } @@ -58,9 +54,9 @@ public void testPartitioning() public void testPartitioningWithMultipleRACs() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1, 2, 2, 1), - makeSimpleRac(1, 1, 1, 2, 2, 1), - makeSimpleRac(1, 1, 2, 2, 1) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 2, 2, 1) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -70,13 +66,13 @@ public void testPartitioningWithMultipleRACs() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1, 1, 1), - expectedSimpleRac(2, 2), - expectedSimpleRac(1, 1, 1, 1), - expectedSimpleRac(2, 2), - expectedSimpleRac(1, 1, 1), - expectedSimpleRac(2, 2), - expectedSimpleRac(1) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2), + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2), + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2), + RowsAndColumnsHelper.expectedSingleColumnRac(1) ) .runToCompletion(op); } @@ -85,9 +81,9 @@ public void testPartitioningWithMultipleRACs() public void testPartitioningWithMultipleConcatenationBetweenRACs() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1), - makeSimpleRac(1, 1), - makeSimpleRac(1, 2) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 2) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -97,8 +93,8 @@ public void testPartitioningWithMultipleConcatenationBetweenRACs() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1, 1, 1, 1, 1), - expectedSimpleRac(2) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2) ) .runToCompletion(op); } @@ -107,9 +103,9 @@ public void testPartitioningWithMultipleConcatenationBetweenRACs() public void testPartitioningWithNoGlueing() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 2, 3), - makeSimpleRac(4, 5, 6), - makeSimpleRac(7, 8, 9) + RowsAndColumnsHelper.makeSingleColumnRac(1, 2, 3), + RowsAndColumnsHelper.makeSingleColumnRac(4, 5, 6), + RowsAndColumnsHelper.makeSingleColumnRac(7, 8, 9) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -119,15 +115,15 @@ public void testPartitioningWithNoGlueing() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1), - expectedSimpleRac(2), - expectedSimpleRac(3), - expectedSimpleRac(4), - expectedSimpleRac(5), - expectedSimpleRac(6), - expectedSimpleRac(7), - expectedSimpleRac(8), - expectedSimpleRac(9) + RowsAndColumnsHelper.expectedSingleColumnRac(1), + RowsAndColumnsHelper.expectedSingleColumnRac(2), + RowsAndColumnsHelper.expectedSingleColumnRac(3), + RowsAndColumnsHelper.expectedSingleColumnRac(4), + RowsAndColumnsHelper.expectedSingleColumnRac(5), + RowsAndColumnsHelper.expectedSingleColumnRac(6), + RowsAndColumnsHelper.expectedSingleColumnRac(7), + RowsAndColumnsHelper.expectedSingleColumnRac(8), + RowsAndColumnsHelper.expectedSingleColumnRac(9) ) .runToCompletion(op); } @@ -136,8 +132,8 @@ public void testPartitioningWithNoGlueing() public void testPartitioningWithNoPartitionColumns() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1, 2, 2, 1), - makeSimpleRac(1, 1, 1, 2, 2, 1) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -147,7 +143,7 @@ public void testPartitioningWithNoPartitionColumns() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 1) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 1) ) .runToCompletion(op); } @@ -156,7 +152,7 @@ public void testPartitioningWithNoPartitionColumns() public void testMaxRowsConstraintViolation() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -176,8 +172,8 @@ public void testMaxRowsConstraintViolation() public void testMaxRowsConstraintViolationWhenGlueing() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1), - makeSimpleRac(1, 2, 3) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 2, 3) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -197,8 +193,8 @@ public void testMaxRowsConstraintViolationWhenGlueing() public void testMaxRowsConstraintWhenGlueing() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1), - makeSimpleRac(2, 2, 2) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.makeSingleColumnRac(2, 2, 2) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -209,8 +205,8 @@ public void testMaxRowsConstraintWhenGlueing() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(1, 1, 1), - expectedSimpleRac(2, 2, 2) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2, 2) ) .runToCompletion(op); } @@ -219,7 +215,7 @@ public void testMaxRowsConstraintWhenGlueing() public void testStopMidStream() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(1, 1, 1, 2, 2, 1) + RowsAndColumnsHelper.makeSingleColumnRac(1, 1, 1, 2, 2, 1) ); GlueingPartitioningOperator op = new GlueingPartitioningOperator( @@ -229,23 +225,9 @@ public void testStopMidStream() new OperatorTestHelper() .expectAndStopAfter( - expectedSimpleRac(1, 1, 1), - expectedSimpleRac(2, 2) + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2) ) .runToCompletion(op); } - - private RowsAndColumns makeSimpleRac(int... values) - { - return MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of("column", new IntArrayColumn(values)) - ); - } - - private RowsAndColumnsHelper expectedSimpleRac(int... values) - { - return new RowsAndColumnsHelper() - .expectColumn("column", values) - .allColumnsRegistered(); - } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java index 0a7edba868e3..e6e340a25306 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java @@ -35,7 +35,7 @@ public class NaivePartitioningOperatorTest public void testPartitioning() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(0, 0, 0, 1, 1, 2, 4, 4, 4) + RowsAndColumnsHelper.makeSingleColumnRac(0, 0, 0, 1, 1, 2, 4, 4, 4) ); NaivePartitioningOperator op = new NaivePartitioningOperator( @@ -45,10 +45,10 @@ public void testPartitioning() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(0, 0, 0), - expectedSimpleRac(1, 1), - expectedSimpleRac(2), - expectedSimpleRac(4, 4, 4) + RowsAndColumnsHelper.expectedSingleColumnRac(0, 0, 0), + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(2), + RowsAndColumnsHelper.expectedSingleColumnRac(4, 4, 4) ) .runToCompletion(op); } @@ -57,8 +57,8 @@ public void testPartitioning() public void testPartitioningWithMultipleRACs() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(0, 0, 0, 1, 1), - makeSimpleRac(1, 2, 2, 2) + RowsAndColumnsHelper.makeSingleColumnRac(0, 0, 0, 1, 1), + RowsAndColumnsHelper.makeSingleColumnRac(1, 2, 2, 2) ); NaivePartitioningOperator op = new NaivePartitioningOperator( @@ -68,10 +68,10 @@ public void testPartitioningWithMultipleRACs() new OperatorTestHelper() .expectRowsAndColumns( - expectedSimpleRac(0, 0, 0), - expectedSimpleRac(1, 1), - expectedSimpleRac(1), - expectedSimpleRac(2, 2, 2) + RowsAndColumnsHelper.expectedSingleColumnRac(0, 0, 0), + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1), + RowsAndColumnsHelper.expectedSingleColumnRac(1), + RowsAndColumnsHelper.expectedSingleColumnRac(2, 2, 2) ) .runToCompletion(op); } @@ -80,7 +80,7 @@ public void testPartitioningWithMultipleRACs() public void testStopMidStream() { InlineScanOperator inlineScanOperator = InlineScanOperator.make( - makeSimpleRac(0, 0, 0, 1, 1, 2, 4, 4, 4) + RowsAndColumnsHelper.makeSingleColumnRac(0, 0, 0, 1, 1, 2, 4, 4, 4) ); NaivePartitioningOperator op = new NaivePartitioningOperator( @@ -90,8 +90,8 @@ public void testStopMidStream() new OperatorTestHelper() .expectAndStopAfter( - expectedSimpleRac(0, 0, 0), - expectedSimpleRac(1, 1) + RowsAndColumnsHelper.expectedSingleColumnRac(0, 0, 0), + RowsAndColumnsHelper.expectedSingleColumnRac(1, 1) ) .runToCompletion(op); } @@ -132,18 +132,4 @@ public void testDoesNotValidateSort() ) .runToCompletion(op); } - - private RowsAndColumns makeSimpleRac(int... values) - { - return MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of("column", new IntArrayColumn(values)) - ); - } - - private RowsAndColumnsHelper expectedSimpleRac(int... values) - { - return new RowsAndColumnsHelper() - .expectColumn("column", values) - .allColumnsRegistered(); - } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java index cdc84620ab0f..e6ed59d5d1de 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java @@ -19,13 +19,16 @@ package org.apache.druid.query.operator.window; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import it.unimi.dsi.fastutil.ints.IntArrayList; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; @@ -129,6 +132,18 @@ public RowsAndColumnsHelper expectColumn(String col, Object[] expectedVals, Colu return this; } + public static RowsAndColumns makeSingleColumnRac(int... values) + { + return MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of("column", new IntArrayColumn(values)) + ); + } + + public static RowsAndColumnsHelper expectedSingleColumnRac(int... values) + { + return new RowsAndColumnsHelper().expectColumn("column", values).allColumnsRegistered(); + } + public ColumnHelper columnHelper(String column, int expectedSize, ColumnType expectedType) { if (this.expectedSize.get() == null) {