indexList = new ArrayList<>();
- for (OperatorFactory of : operatorFactoryList) {
- if (of instanceof NaivePartitioningOperatorFactory) {
- for (String s : ((NaivePartitioningOperatorFactory) of).getPartitionColumns()) {
- indexList.add(rowSignature.indexOf(s));
- }
- }
- }
- return indexList;
- }
-
/**
- *
- * Compare two rows based only the columns in the partitionIndices
- * In case the parition indices is empty or null compare entire row
- *
+ * Compare two rows based on the columns in partitionColumnNames.
+ * If the partitionColumnNames is empty or null, compare entire row.
+ *
+ * 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 partitionIndices)
+ private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List partitionColumnNames)
{
- if (partitionIndices == null || partitionIndices.isEmpty()) {
+ if (partitionColumnNames == null || partitionColumnNames.isEmpty()) {
return row1.equals(row2);
} else {
int match = 0;
- for (int i : partitionIndices) {
+ for (String columnName : partitionColumnNames) {
+ int i = frameReader.signature().indexOf(columnName);
if (Objects.equals(row1.get(i), row2.get(i))) {
match++;
}
}
- return match == partitionIndices.size();
+ return match == partitionColumnNames.size();
}
}
}
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 fbbc0a0fc3e..d9c14390736 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
@@ -61,6 +61,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
private final RowSignature stageRowSignature;
private final boolean isEmptyOver;
private final int maxRowsMaterializedInWindow;
+ private final List partitionColumnNames;
@JsonCreator
public WindowOperatorQueryFrameProcessorFactory(
@@ -68,7 +69,8 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
@JsonProperty("operatorList") List operatorFactoryList,
@JsonProperty("stageRowSignature") RowSignature stageRowSignature,
@JsonProperty("emptyOver") boolean emptyOver,
- @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow
+ @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow,
+ @JsonProperty("partitionColumnNames") List partitionColumnNames
)
{
this.query = Preconditions.checkNotNull(query, "query");
@@ -76,6 +78,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature");
this.isEmptyOver = emptyOver;
this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow;
+ this.partitionColumnNames = partitionColumnNames;
}
@JsonProperty("query")
@@ -90,6 +93,12 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
return operatorList;
}
+ @JsonProperty("partitionColumnNames")
+ public List getPartitionColumnNames()
+ {
+ return partitionColumnNames;
+ }
+
@JsonProperty("stageRowSignature")
public RowSignature getSignature()
{
@@ -148,7 +157,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
readableInput -> {
final OutputChannel outputChannel =
outputChannels.get(readableInput.getStagePartition().getPartitionNumber());
-
return new WindowOperatorQueryFrameProcessor(
query,
readableInput.getChannel(),
@@ -159,7 +167,8 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
operatorList,
stageRowSignature,
isEmptyOver,
- maxRowsMaterializedInWindow
+ maxRowsMaterializedInWindow,
+ partitionColumnNames
);
}
);
@@ -185,12 +194,13 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
&& 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, stageRowSignature, isEmptyOver, maxRowsMaterializedInWindow);
+ return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, isEmptyOver, 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 d08d78ef791..3754f081a27 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
@@ -24,9 +24,12 @@ import com.google.common.collect.ImmutableMap;
import org.apache.druid.frame.key.ClusterBy;
import org.apache.druid.frame.key.KeyColumn;
import org.apache.druid.frame.key.KeyOrder;
+import org.apache.druid.java.util.common.ISE;
+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;
import org.apache.druid.msq.kernel.QueryDefinition;
import org.apache.druid.msq.kernel.QueryDefinitionBuilder;
import org.apache.druid.msq.kernel.ShuffleSpec;
@@ -39,6 +42,7 @@ import org.apache.druid.query.operator.NaiveSortOperatorFactory;
import org.apache.druid.query.operator.OperatorFactory;
import org.apache.druid.query.operator.WindowOperatorQuery;
import org.apache.druid.query.operator.window.WindowOperatorFactory;
+import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import java.util.ArrayList;
@@ -48,6 +52,7 @@ import java.util.Map;
public class WindowOperatorQueryKit implements QueryKit
{
+ private static final Logger log = new Logger(WindowOperatorQueryKit.class);
private final ObjectMapper jsonMapper;
public WindowOperatorQueryKit(ObjectMapper jsonMapper)
@@ -65,13 +70,22 @@ public class WindowOperatorQueryKit implements QueryKit
int minStageNumber
)
{
- // need to validate query first
- // populate the group of operators to be processed as each stage
- // the size of the operators is the number of serialized stages
- // later we should also check if these can be parallelized
- // check there is an empty over clause or not
- List> operatorList = new ArrayList<>();
- boolean isEmptyOverFound = ifEmptyOverPresentInWindowOperstors(originalQuery, operatorList);
+ // 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 NaivePartitioningOperatorFactory)
+ .map(of -> (NaivePartitioningOperatorFactory) of)
+ .anyMatch(of -> of.getPartitionColumns().isEmpty());
+
+ List> operatorList = getOperatorListFromQuery(originalQuery);
+ log.info("Created operatorList with operator factories: [%s]", operatorList);
ShuffleSpec nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(0), maxWorkerCount);
// add this shuffle spec to the last stage of the inner query
@@ -102,16 +116,14 @@ public class WindowOperatorQueryKit implements QueryKit
final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber());
final WindowOperatorQuery queryToRun = (WindowOperatorQuery) originalQuery.withDataSource(dataSourcePlan.getNewDataSource());
final int maxRowsMaterialized;
- RowSignature rowSignature = queryToRun.getRowSignature();
+
if (originalQuery.context() != null && originalQuery.context().containsKey(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW)) {
- maxRowsMaterialized = (int) originalQuery.context()
- .get(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;
}
-
- if (isEmptyOverFound) {
+ if (isEmptyOverPresent) {
// empty over clause found
// moving everything to a single partition
queryDefBuilder.add(
@@ -125,28 +137,59 @@ public class WindowOperatorQueryKit implements QueryKit
queryToRun.getOperators(),
rowSignature,
true,
- maxRowsMaterialized
+ maxRowsMaterialized,
+ new ArrayList<>()
))
);
} 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
+ // 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();
- final int numberOfWindows = operatorList.size();
- final int baseSize = rowSignature.size() - numberOfWindows;
- for (int i = 0; i < baseSize; i++) {
- bob.add(rowSignature.getColumnName(i), rowSignature.getColumnType(i).get());
+ 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());
}
- for (int i = 0; i < numberOfWindows; i++) {
- bob.add(rowSignature.getColumnName(baseSize + i), rowSignature.getColumnType(baseSize + i).get()).build();
+ 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
+ );
+ }
+ }
+ }
+ }
+
// find the shuffle spec of the next stage
// if it is the last stage set the next shuffle spec to single partition
- if (i + 1 == numberOfWindows) {
- nextShuffleSpec = ShuffleSpecFactories.singlePartition()
- .build(ClusterBy.none(), false);
+ if (i + 1 == operatorList.size()) {
+ nextShuffleSpec = MixShuffleSpec.instance();
} else {
nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(i + 1), maxWorkerCount);
}
@@ -162,6 +205,28 @@ public class WindowOperatorQueryKit implements QueryKit
);
}
+ 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 NaivePartitioningOperatorFactory) {
+ for (String s : ((NaivePartitioningOperatorFactory) of).getPartitionColumns()) {
+ currentPartitionColumns.add(s);
+ partitionOperatorExists = true;
+ }
+ }
+ }
+
+ if (partitionOperatorExists) {
+ partitionColumnNames = currentPartitionColumns;
+ }
+
+ log.info(
+ "Columns which would be used to define partitioning boundaries for this window stage are [%s]",
+ partitionColumnNames
+ );
+
queryDefBuilder.add(
StageDefinition.builder(firstStageNumber + i)
.inputs(new StageInputSpec(firstStageNumber + i - 1))
@@ -173,7 +238,8 @@ public class WindowOperatorQueryKit implements QueryKit
operatorList.get(i),
stageRowSignature,
false,
- maxRowsMaterialized
+ maxRowsMaterialized,
+ partitionColumnNames
))
);
}
@@ -184,14 +250,12 @@ public class WindowOperatorQueryKit implements QueryKit
/**
*
* @param originalQuery
- * @param operatorList
- * @return true if the operator List has a partitioning operator with an empty OVER clause, false otherwise
+ * @return A list of list of operator factories, where each list represents the operator factories for a particular
+ * window stage.
*/
- private boolean ifEmptyOverPresentInWindowOperstors(
- WindowOperatorQuery originalQuery,
- List> operatorList
- )
+ private List> getOperatorListFromQuery(WindowOperatorQuery originalQuery)
{
+ List> operatorList = new ArrayList<>();
final List operators = originalQuery.getOperators();
List operatorFactoryList = new ArrayList<>();
for (OperatorFactory of : operators) {
@@ -203,18 +267,17 @@ public class WindowOperatorQueryKit implements QueryKit
if (((NaivePartitioningOperatorFactory) of).getPartitionColumns().isEmpty()) {
operatorList.clear();
operatorList.add(originalQuery.getOperators());
- return true;
+ return operatorList;
}
}
}
- return false;
+ return operatorList;
}
private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount)
{
NaivePartitioningOperatorFactory partition = null;
NaiveSortOperatorFactory sort = null;
- List keyColsOfWindow = new ArrayList<>();
for (OperatorFactory of : operatorFactories) {
if (of instanceof NaivePartitioningOperatorFactory) {
partition = (NaivePartitioningOperatorFactory) of;
@@ -222,29 +285,31 @@ public class WindowOperatorQueryKit implements QueryKit
sort = (NaiveSortOperatorFactory) of;
}
}
- Map colMap = new HashMap<>();
+
+ Map sortColumnsMap = new HashMap<>();
if (sort != null) {
for (ColumnWithDirection sortColumn : sort.getSortColumns()) {
- colMap.put(sortColumn.getColumn(), sortColumn.getDirection());
+ sortColumnsMap.put(sortColumn.getColumn(), sortColumn.getDirection());
}
}
- assert partition != null;
- if (partition.getPartitionColumns().isEmpty()) {
+
+ if (partition == null || partition.getPartitionColumns().isEmpty()) {
+ // If operatorFactories doesn't have any partitioning factory, then we should keep the shuffle spec from previous stage.
+ // This indicates that we already have the data partitioned correctly, and hence we don't need to do any shuffling.
return null;
}
+
+ List keyColsOfWindow = new ArrayList<>();
for (String partitionColumn : partition.getPartitionColumns()) {
KeyColumn kc;
- if (colMap.containsKey(partitionColumn)) {
- if (colMap.get(partitionColumn) == ColumnWithDirection.Direction.ASC) {
- kc = new KeyColumn(partitionColumn, KeyOrder.ASCENDING);
- } else {
- kc = new KeyColumn(partitionColumn, KeyOrder.DESCENDING);
- }
+ if (sortColumnsMap.get(partitionColumn) == ColumnWithDirection.Direction.DESC) {
+ kc = new KeyColumn(partitionColumn, KeyOrder.DESCENDING);
} else {
kc = new KeyColumn(partitionColumn, KeyOrder.ASCENDING);
}
keyColsOfWindow.add(kc);
}
+
return new HashShuffleSpec(new ClusterBy(keyColsOfWindow, 0), maxWorkerCount);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java
similarity index 57%
rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java
rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java
index c78aec4f39b..2049c0194ed 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java
@@ -17,30 +17,19 @@
* under the License.
*/
-package org.apache.druid.indexing.common.actions;
+package org.apache.druid.msq.querykit;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.junit.Test;
-/**
- * The configuration for task audit logging.
- * This class will be removed in future releases. See https://github.com/apache/druid/issues/5859.
- */
-@Deprecated
-public class TaskAuditLogConfig
+public class WindowOperatorQueryFrameProcessorFactoryTest
{
- @JsonProperty
- private final boolean enabled;
-
- @JsonCreator
- public TaskAuditLogConfig(@JsonProperty("enabled") boolean enabled)
+ @Test
+ public void testEqualsAndHashcode()
{
- this.enabled = enabled;
- }
-
- @JsonProperty("enabled")
- public boolean isEnabled()
- {
- return enabled;
+ EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class)
+ .withNonnullFields("query", "operatorList", "stageRowSignature", "isEmptyOver", "maxRowsMaterializedInWindow", "partitionColumnNames")
+ .usingGetClass()
+ .verify();
}
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
index b2582c0b9a7..51322eafe98 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
@@ -39,6 +39,7 @@ import org.apache.druid.guice.JoinableFactoryModule;
import org.apache.druid.guice.annotations.Self;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.math.expr.ExprMacroTable;
@@ -62,12 +63,14 @@ import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.TestGroupByBuffers;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.IndexIO;
+import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.column.ColumnConfig;
+import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
@@ -91,6 +94,7 @@ import javax.annotation.Nullable;
import java.io.File;
import java.util.List;
import java.util.Set;
+import java.util.UUID;
import java.util.function.Function;
import java.util.function.Supplier;
@@ -99,6 +103,7 @@ import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE1;
import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE2;
import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE3;
import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE5;
+import static org.apache.druid.sql.calcite.util.CalciteTests.WIKIPEDIA;
import static org.apache.druid.sql.calcite.util.TestDataBuilder.INDEX_SCHEMA_LOTS_O_COLUMNS;
import static org.apache.druid.sql.calcite.util.TestDataBuilder.INDEX_SCHEMA_NUMERIC_DIMS;
import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1;
@@ -205,6 +210,17 @@ public class CalciteMSQTestsHelper
{
final QueryableIndex index;
switch (segmentId.getDataSource()) {
+ case WIKIPEDIA:
+ try {
+ final File directory = new File(tempFolderProducer.apply("tmpDir"), StringUtils.format("wikipedia-index-%s", UUID.randomUUID()));
+ final IncrementalIndex incrementalIndex = TestIndex.makeWikipediaIncrementalIndex();
+ TestIndex.INDEX_MERGER.persist(incrementalIndex, directory, IndexSpec.DEFAULT, null);
+ index = TestIndex.INDEX_IO.loadIndex(directory);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ break;
case DATASOURCE1:
IncrementalIndexSchema foo1Schema = new IncrementalIndexSchema.Builder()
.withMetrics(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java
index 276a9fe2270..e319dc24266 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java
@@ -121,12 +121,6 @@ public class CheckPointDataSourceMetadataAction implements TaskAction
);
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
index bd0be9f7411..1d0059335ed 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
@@ -22,8 +22,6 @@ package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.TaskStorage;
-import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
@@ -37,45 +35,21 @@ public class LocalTaskActionClient implements TaskActionClient
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
private final Task task;
- private final TaskStorage storage;
private final TaskActionToolbox toolbox;
- private final TaskAuditLogConfig auditLogConfig;
public LocalTaskActionClient(
Task task,
- TaskStorage storage,
- TaskActionToolbox toolbox,
- TaskAuditLogConfig auditLogConfig
+ TaskActionToolbox toolbox
)
{
this.task = task;
- this.storage = storage;
this.toolbox = toolbox;
- this.auditLogConfig = auditLogConfig;
}
@Override
public RetType submit(TaskAction taskAction)
{
log.debug("Performing action for task[%s]: %s", task.getId(), taskAction);
-
- if (auditLogConfig.isEnabled() && taskAction.isAudited()) {
- // Add audit log
- try {
- final long auditLogStartTime = System.currentTimeMillis();
- storage.addAuditLog(task, taskAction);
- emitTimerMetric("task/action/log/time", taskAction, System.currentTimeMillis() - auditLogStartTime);
- }
- catch (Exception e) {
- final String actionClass = taskAction.getClass().getName();
- log.makeAlert(e, "Failed to record action in audit log")
- .addData("task", task.getId())
- .addData("actionClass", actionClass)
- .emit();
- throw new ISE(e, "Failed to record action [%s] in audit log", actionClass);
- }
- }
-
final long performStartTime = System.currentTimeMillis();
final RetType result = performAction(taskAction);
emitTimerMetric("task/action/run/time", taskAction, System.currentTimeMillis() - performStartTime);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java
index 60fc01a2390..c485b166038 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java
@@ -21,27 +21,22 @@ package org.apache.druid.indexing.common.actions;
import com.google.inject.Inject;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.TaskStorage;
/**
*/
public class LocalTaskActionClientFactory implements TaskActionClientFactory
{
- private final TaskStorage storage;
private final TaskActionToolbox toolbox;
- private final TaskAuditLogConfig auditLogConfig;
@Inject
- public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox, TaskAuditLogConfig auditLogConfig)
+ public LocalTaskActionClientFactory(TaskActionToolbox toolbox)
{
- this.storage = storage;
this.toolbox = toolbox;
- this.auditLogConfig = auditLogConfig;
}
@Override
public TaskActionClient create(Task task)
{
- return new LocalTaskActionClient(task, storage, toolbox, auditLogConfig);
+ return new LocalTaskActionClient(task, toolbox);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java
index 22af74f26b5..c60c5740798 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java
@@ -39,12 +39,6 @@ public class LockListAction implements TaskAction>
return toolbox.getTaskLockbox().findLocksForTask(task);
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java
index fa8e1d6bea2..32974117cc7 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java
@@ -56,12 +56,6 @@ public class LockReleaseAction implements TaskAction
return null;
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java
index 93cb75280fa..35a8ed3e35c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java
@@ -67,9 +67,4 @@ public class MarkSegmentsAsUnusedAction implements TaskAction
.markSegmentsAsUnusedWithinInterval(dataSource, interval);
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java
index c853a00c58d..42259b38183 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java
@@ -64,12 +64,6 @@ public class ResetDataSourceMetadataAction implements TaskAction
return toolbox.getSupervisorManager().resetSupervisor(dataSource, resetMetadata);
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java
index 88d3703f4b0..a4ca90ac62e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java
@@ -74,12 +74,6 @@ public class RetrieveSegmentsByIdAction implements TaskAction>
.retrieveSegmentsById(dataSource, segmentIds);
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public boolean equals(Object o)
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
index 3f8d4725835..fb58328b3d7 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
@@ -101,12 +101,6 @@ public class RetrieveUnusedSegmentsAction implements TaskAction
}
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
deleted file mode 100644
index 478e0b89d3d..00000000000
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.indexing.common.actions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.segment.SegmentSchemaMapping;
-import org.apache.druid.segment.SegmentUtils;
-import org.apache.druid.timeline.DataSegment;
-
-import javax.annotation.Nullable;
-import java.util.Set;
-
-/**
- * Word of warning: Very large "segments" sets can cause oversized audit log entries, which is bad because it means
- * that the task cannot actually complete. Callers should avoid this by avoiding inserting too many segments in the
- * same action.
- */
-public class SegmentInsertAction implements TaskAction>
-{
- private final Set segments;
-
- @Nullable
- private final SegmentSchemaMapping segmentSchemaMapping;
-
- @JsonCreator
- public SegmentInsertAction(
- @JsonProperty("segments") Set segments,
- @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping
- )
- {
- this.segments = ImmutableSet.copyOf(segments);
- this.segmentSchemaMapping = segmentSchemaMapping;
- }
-
- @JsonProperty
- public Set getSegments()
- {
- return segments;
- }
-
- @JsonProperty
- @Nullable
- public SegmentSchemaMapping getSegmentSchemaMapping()
- {
- return segmentSchemaMapping;
- }
-
- @Override
- public TypeReference> getReturnTypeReference()
- {
- return new TypeReference>()
- {
- };
- }
-
- /**
- * Behaves similarly to
- * {@link org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#commitSegments},
- * with startMetadata and endMetadata both null.
- */
- @Override
- public Set perform(Task task, TaskActionToolbox toolbox)
- {
- return SegmentTransactionalInsertAction.appendAction(segments, null, null, segmentSchemaMapping).perform(task, toolbox).getSegments();
- }
-
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
- @Override
- public String toString()
- {
- return "SegmentInsertAction{" +
- "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) +
- '}';
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
index 70c81225b2f..e17af3847ca 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
@@ -119,12 +119,6 @@ public class SegmentLockAcquireAction implements TaskAction
}
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
index 7728574c075..c5084eb61cd 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
@@ -102,12 +102,6 @@ public class SegmentLockTryAcquireAction implements TaskAction>
.collect(Collectors.toList());
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
index cbad8bfcc5c..b2a5ee5843b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
@@ -99,12 +99,6 @@ public class SegmentMetadataUpdateAction implements TaskAction
return null;
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
index 2856f161e0e..326907c8f0a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
@@ -101,12 +101,6 @@ public class SegmentNukeAction implements TaskAction
return null;
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java
index 4871e65e162..2326176885b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java
@@ -44,7 +44,6 @@ import java.util.Set;
import java.util.stream.Collectors;
/**
- *
* Append segments to metadata storage. The segment versions must all be less than or equal to a lock held by
* your task for the segment intervals.
*
@@ -209,12 +208,6 @@ public class SegmentTransactionalAppendAction implements TaskAction
RetType perform(Task task, TaskActionToolbox toolbox);
- boolean isAudited();
-
default boolean canPerformAsync(Task task, TaskActionToolbox toolbox)
{
return false;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
index 305a0d7b284..42f4e40d0e6 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
@@ -97,12 +97,6 @@ public class TimeChunkLockAcquireAction implements TaskAction
}
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
index 07bcfa5c101..b2dc6e3d5e6 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
@@ -81,12 +81,6 @@ public class TimeChunkLockTryAcquireAction implements TaskAction
return result.getTaskLock();
}
- @Override
- public boolean isAudited()
- {
- return false;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java
index dce9f00c35b..2c7c265a7f4 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java
@@ -63,12 +63,6 @@ public class UpdateLocationAction implements TaskAction
return null;
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java
index 2ff8375b7f6..d02020acb9d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java
@@ -85,12 +85,6 @@ public class UpdateStatusAction implements TaskAction
return null;
}
- @Override
- public boolean isAudited()
- {
- return true;
- }
-
@Override
public String toString()
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
index e84976db86c..5109abe2377 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
@@ -327,24 +327,6 @@ public class HeapMemoryTaskStorage implements TaskStorage
}
}
- @Deprecated
- @Override
- public void addAuditLog(Task task, TaskAction taskAction)
- {
- synchronized (taskActions) {
- taskActions.put(task.getId(), taskAction);
- }
- }
-
- @Deprecated
- @Override
- public List getAuditLogs(String taskid)
- {
- synchronized (taskActions) {
- return ImmutableList.copyOf(taskActions.get(taskid));
- }
- }
-
private static class TaskStuff
{
final Task task;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
index 6d23fcec327..c3e498aa18f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
@@ -25,6 +25,7 @@ import org.apache.druid.indexer.TaskInfo;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.metadata.TaskLookup;
import org.joda.time.Interval;
import java.util.Comparator;
@@ -32,16 +33,16 @@ import java.util.Optional;
public class IndexerMetadataStorageAdapter
{
- private final TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private final TaskStorage taskStorage;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
@Inject
public IndexerMetadataStorageAdapter(
- TaskStorageQueryAdapter taskStorageQueryAdapter,
+ TaskStorage taskStorage,
IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator
)
{
- this.taskStorageQueryAdapter = taskStorageQueryAdapter;
+ this.taskStorage = taskStorage;
this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
}
@@ -49,8 +50,8 @@ public class IndexerMetadataStorageAdapter
{
// Find the earliest active task created for the specified datasource; if one exists,
// check if its interval overlaps with the delete interval.
- final Optional> earliestActiveTaskOptional = taskStorageQueryAdapter
- .getActiveTaskInfo(dataSource)
+ final Optional> earliestActiveTaskOptional = taskStorage
+ .getTaskInfos(TaskLookup.activeTasksOnly(), dataSource)
.stream()
.min(Comparator.comparing(TaskInfo::getCreatedTime));
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
index 15730d48bb1..3608f023899 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
@@ -75,14 +75,6 @@ public class MetadataTaskStorage implements TaskStorage
};
}
- @Override
- public TypeReference getLogType()
- {
- return new TypeReference()
- {
- };
- }
-
@Override
public TypeReference getLockType()
{
@@ -319,24 +311,6 @@ public class MetadataTaskStorage implements TaskStorage
);
}
- @Deprecated
- @Override
- public void addAuditLog(final Task task, final TaskAction taskAction)
- {
- Preconditions.checkNotNull(taskAction, "taskAction");
-
- log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
-
- handler.addLog(task.getId(), taskAction);
- }
-
- @Deprecated
- @Override
- public List getAuditLogs(final String taskId)
- {
- return handler.getLogs(taskId);
- }
-
private Map getLocksWithIds(final String taskid)
{
return handler.getLocks(taskid);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java
similarity index 62%
rename from indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java
rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java
index ba2ca3c7066..29ca16f5aa9 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java
@@ -24,43 +24,34 @@ import com.google.inject.Inject;
import org.apache.druid.indexer.TaskInfo;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.TaskStatusPlus;
-import org.apache.druid.indexing.common.actions.SegmentInsertAction;
-import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
-import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.metadata.LockFilterPolicy;
import org.apache.druid.metadata.TaskLookup;
-import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup;
import org.apache.druid.metadata.TaskLookup.TaskLookupType;
-import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import javax.annotation.Nullable;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Set;
/**
- * Wraps a {@link TaskStorage}, providing a useful collection of read-only methods.
+ * Provides read-only methods to fetch information related to tasks.
+ * This class may serve information that is cached in memory in {@link TaskQueue}
+ * or {@link TaskLockbox}. If not present in memory, then the underlying
+ * {@link TaskStorage} is queried.
*/
-public class TaskStorageQueryAdapter
+public class TaskQueryTool
{
private final TaskStorage storage;
private final TaskLockbox taskLockbox;
- private final Optional taskQueue;
+ private final TaskMaster taskMaster;
@Inject
- public TaskStorageQueryAdapter(TaskStorage storage, TaskLockbox taskLockbox, TaskMaster taskMaster)
+ public TaskQueryTool(TaskStorage storage, TaskLockbox taskLockbox, TaskMaster taskMaster)
{
this.storage = storage;
this.taskLockbox = taskLockbox;
- this.taskQueue = taskMaster.getTaskQueue();
- }
-
- public List getActiveTasks()
- {
- return storage.getActiveTasks();
+ this.taskMaster = taskMaster;
}
/**
@@ -91,7 +82,7 @@ public class TaskStorageQueryAdapter
public List> getActiveTaskInfo(@Nullable String dataSource)
{
return storage.getTaskInfos(
- ActiveTaskLookup.getInstance(),
+ TaskLookup.activeTasksOnly(),
dataSource
);
}
@@ -104,20 +95,21 @@ public class TaskStorageQueryAdapter
return storage.getTaskStatusPlusList(taskLookups, dataSource);
}
- public Optional getTask(final String taskid)
+ public Optional getTask(final String taskId)
{
+ final Optional taskQueue = taskMaster.getTaskQueue();
if (taskQueue.isPresent()) {
- Optional activeTask = taskQueue.get().getActiveTask(taskid);
+ Optional activeTask = taskQueue.get().getActiveTask(taskId);
if (activeTask.isPresent()) {
return activeTask;
}
}
- return storage.getTask(taskid);
+ return storage.getTask(taskId);
}
- public Optional getStatus(final String taskid)
+ public Optional getStatus(final String taskId)
{
- return storage.getStatus(taskid);
+ return storage.getStatus(taskId);
}
@Nullable
@@ -126,27 +118,4 @@ public class TaskStorageQueryAdapter
return storage.getTaskInfo(taskId);
}
- /**
- * Returns all segments created by this task.
- *
- * This method is useful when you want to figure out all of the things a single task spawned. It does pose issues
- * with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your
- * own risk and know that at some point, we might adjust this to actually enforce some sort of limits.
- *
- * @param taskid task ID
- * @return set of segments created by the specified task
- */
- @Deprecated
- public Set getInsertedSegments(final String taskid)
- {
- final Set segments = new HashSet<>();
- for (final TaskAction action : storage.getAuditLogs(taskid)) {
- if (action instanceof SegmentInsertAction) {
- segments.addAll(((SegmentInsertAction) action).getSegments());
- } else if (action instanceof SegmentTransactionalInsertAction) {
- segments.addAll(((SegmentTransactionalInsertAction) action).getSegments());
- }
- }
- return segments;
- }
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
index 235c763e1f7..b231b3f37c2 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
@@ -24,7 +24,6 @@ import org.apache.druid.indexer.TaskInfo;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.indexing.common.TaskLock;
-import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.metadata.TaskLookup;
import org.apache.druid.metadata.TaskLookup.TaskLookupType;
@@ -110,26 +109,6 @@ public interface TaskStorage
@Nullable
TaskInfo getTaskInfo(String taskId);
- /**
- * Add an action taken by a task to the audit log.
- *
- * @param task task to record action for
- * @param taskAction task action to record
- * @param task action return type
- */
- @Deprecated
- void addAuditLog(Task task, TaskAction taskAction);
-
- /**
- * Returns all actions taken by a task.
- *
- * @param taskid task ID
- *
- * @return list of task actions
- */
- @Deprecated
- List getAuditLogs(String taskid);
-
/**
* Returns a list of currently running or pending tasks as stored in the storage facility. No particular order
* is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
index 56123a561f6..f945cc95c1e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
@@ -47,10 +47,10 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter;
import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.TaskQueue;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
import org.apache.druid.indexing.overlord.WorkerTaskRunner;
import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter;
import org.apache.druid.indexing.overlord.autoscaling.ProvisioningStrategy;
@@ -84,7 +84,6 @@ import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.tasklogs.TaskLogStreamer;
-import org.apache.druid.timeline.DataSegment;
import org.joda.time.Duration;
import org.joda.time.Interval;
@@ -126,7 +125,7 @@ public class OverlordResource
private static final Logger log = new Logger(OverlordResource.class);
private final TaskMaster taskMaster;
- private final TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private final TaskQueryTool taskQueryTool;
private final IndexerMetadataStorageAdapter indexerMetadataStorageAdapter;
private final TaskLogStreamer taskLogStreamer;
private final JacksonConfigManager configManager;
@@ -163,7 +162,7 @@ public class OverlordResource
@Inject
public OverlordResource(
TaskMaster taskMaster,
- TaskStorageQueryAdapter taskStorageQueryAdapter,
+ TaskQueryTool taskQueryTool,
IndexerMetadataStorageAdapter indexerMetadataStorageAdapter,
TaskLogStreamer taskLogStreamer,
JacksonConfigManager configManager,
@@ -175,7 +174,7 @@ public class OverlordResource
)
{
this.taskMaster = taskMaster;
- this.taskStorageQueryAdapter = taskStorageQueryAdapter;
+ this.taskQueryTool = taskQueryTool;
this.indexerMetadataStorageAdapter = indexerMetadataStorageAdapter;
this.taskLogStreamer = taskLogStreamer;
this.configManager = configManager;
@@ -285,7 +284,7 @@ public class OverlordResource
}
// Build the response
- return Response.ok(taskStorageQueryAdapter.getLockedIntervals(minTaskPriority)).build();
+ return Response.ok(taskQueryTool.getLockedIntervals(minTaskPriority)).build();
}
@POST
@@ -299,7 +298,7 @@ public class OverlordResource
}
// Build the response
- return Response.ok(taskStorageQueryAdapter.getLockedIntervals(lockFilterPolicies)).build();
+ return Response.ok(taskQueryTool.getLockedIntervals(lockFilterPolicies)).build();
}
@GET
@@ -310,7 +309,7 @@ public class OverlordResource
{
final TaskPayloadResponse response = new TaskPayloadResponse(
taskid,
- taskStorageQueryAdapter.getTask(taskid).orNull()
+ taskQueryTool.getTask(taskid).orNull()
);
final Response.Status status = response.getPayload() == null
@@ -326,7 +325,7 @@ public class OverlordResource
@ResourceFilters(TaskResourceFilter.class)
public Response getTaskStatus(@PathParam("taskid") String taskid)
{
- final TaskInfo taskInfo = taskStorageQueryAdapter.getTaskInfo(taskid);
+ final TaskInfo taskInfo = taskQueryTool.getTaskInfo(taskid);
TaskStatusResponse response = null;
if (taskInfo != null) {
@@ -400,8 +399,12 @@ public class OverlordResource
@ResourceFilters(TaskResourceFilter.class)
public Response getTaskSegments(@PathParam("taskid") String taskid)
{
- final Set segments = taskStorageQueryAdapter.getInsertedSegments(taskid);
- return Response.ok().entity(segments).build();
+ final String errorMsg =
+ "Segment IDs committed by a task action are not persisted anymore."
+ + " Use the metric 'segment/added/bytes' to identify the segments created by a task.";
+ return Response.status(Status.NOT_FOUND)
+ .entity(Collections.singletonMap("error", errorMsg))
+ .build();
}
@POST
@@ -437,7 +440,7 @@ public class OverlordResource
@Override
public Response apply(TaskQueue taskQueue)
{
- final List> tasks = taskStorageQueryAdapter.getActiveTaskInfo(dataSource);
+ final List> tasks = taskQueryTool.getActiveTaskInfo(dataSource);
if (tasks.isEmpty()) {
return Response.status(Status.NOT_FOUND).build();
} else {
@@ -468,7 +471,7 @@ public class OverlordResource
if (taskQueue.isPresent()) {
optional = taskQueue.get().getTaskStatus(taskId);
} else {
- optional = taskStorageQueryAdapter.getStatus(taskId);
+ optional = taskQueryTool.getStatus(taskId);
}
if (optional.isPresent()) {
result.put(taskId, optional.get());
@@ -863,7 +866,7 @@ public class OverlordResource
throw new IAE("Unknown state: [%s]", state);
}
- final Stream taskStatusPlusStream = taskStorageQueryAdapter.getTaskStatusPlusList(
+ final Stream taskStatusPlusStream = taskQueryTool.getTaskStatusPlusList(
taskLookups,
dataSource
).stream();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java
index ad1f526eda6..a9f66ce30e7 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java
@@ -26,7 +26,7 @@ import com.google.inject.Inject;
import com.sun.jersey.spi.container.ContainerRequest;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.server.http.security.AbstractResourceFilter;
import org.apache.druid.server.security.Access;
@@ -49,16 +49,16 @@ import javax.ws.rs.core.Response;
*/
public class TaskResourceFilter extends AbstractResourceFilter
{
- private final TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private final TaskQueryTool taskQueryTool;
@Inject
public TaskResourceFilter(
- TaskStorageQueryAdapter taskStorageQueryAdapter,
+ TaskQueryTool taskQueryTool,
AuthorizerMapper authorizerMapper
)
{
super(authorizerMapper);
- this.taskStorageQueryAdapter = taskStorageQueryAdapter;
+ this.taskQueryTool = taskQueryTool;
}
@Override
@@ -76,7 +76,7 @@ public class TaskResourceFilter extends AbstractResourceFilter
IdUtils.validateId("taskId", taskId);
- Optional taskOptional = taskStorageQueryAdapter.getTask(taskId);
+ Optional taskOptional = taskQueryTool.getTask(taskId);
if (!taskOptional.isPresent()) {
throw new WebApplicationException(
Response.status(Response.Status.NOT_FOUND)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java
deleted file mode 100644
index c8999c2f5d4..00000000000
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * 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.indexing.common.actions;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.indexing.common.TaskLockType;
-import org.apache.druid.indexing.common.task.NoopTask;
-import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.CriticalAction;
-import org.apache.druid.indexing.overlord.LockResult;
-import org.apache.druid.indexing.overlord.Segments;
-import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.LinearShardSpec;
-import org.assertj.core.api.Assertions;
-import org.hamcrest.CoreMatchers;
-import org.joda.time.Interval;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.util.Collections;
-import java.util.Set;
-
-public class SegmentInsertActionTest
-{
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- @Rule
- public TaskActionTestKit actionTestKit = new TaskActionTestKit();
-
- private static final String DATA_SOURCE = "none";
- private static final Interval INTERVAL = Intervals.of("2020/2020T01");
- private static final String PARTY_YEAR = "1999";
- private static final String THE_DISTANT_FUTURE = "3000";
-
- private static final DataSegment SEGMENT1 = new DataSegment(
- DATA_SOURCE,
- INTERVAL,
- PARTY_YEAR,
- ImmutableMap.of(),
- ImmutableList.of(),
- ImmutableList.of(),
- new LinearShardSpec(0),
- 9,
- 1024
- );
-
- private static final DataSegment SEGMENT2 = new DataSegment(
- DATA_SOURCE,
- INTERVAL,
- PARTY_YEAR,
- ImmutableMap.of(),
- ImmutableList.of(),
- ImmutableList.of(),
- new LinearShardSpec(1),
- 9,
- 1024
- );
-
- private static final DataSegment SEGMENT3 = new DataSegment(
- DATA_SOURCE,
- INTERVAL,
- THE_DISTANT_FUTURE,
- ImmutableMap.of(),
- ImmutableList.of(),
- ImmutableList.of(),
- new LinearShardSpec(1),
- 9,
- 1024
- );
-
- private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs)
- throws InterruptedException
- {
- return actionTestKit.getTaskLockbox().lock(task, new TimeChunkLockRequest(lockType, task, interval, null), timeoutMs);
- }
-
- @Test
- public void testSimple() throws Exception
- {
- final Task task = NoopTask.create();
- final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2), null);
- actionTestKit.getTaskLockbox().add(task);
- acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
- actionTestKit.getTaskLockbox().doInCriticalSection(
- task,
- Collections.singleton(INTERVAL),
- CriticalAction.builder()
- .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox()))
- .onInvalidLocks(
- () -> {
- Assert.fail();
- return null;
- }
- )
- .build()
- );
-
- Assertions.assertThat(
- actionTestKit.getMetadataStorageCoordinator()
- .retrieveUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
- ).containsExactlyInAnyOrder(SEGMENT1, SEGMENT2);
- }
-
- @Test
- public void testFailBadVersion() throws Exception
- {
- final Task task = NoopTask.create();
- final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3), null);
- actionTestKit.getTaskLockbox().add(task);
- acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
- final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection(
- task,
- Collections.singleton(INTERVAL),
- CriticalAction.>builder()
- .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox()))
- .onInvalidLocks(
- () -> {
- Assert.fail();
- return null;
- }
- )
- .build()
- );
-
- Assert.assertEquals(ImmutableSet.of(SEGMENT3), segments);
- }
-}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java
index 63380e84fd9..cad2feb0deb 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java
@@ -24,7 +24,6 @@ import org.apache.druid.indexing.common.actions.SurrogateAction;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
-import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.overlord.TaskStorage;
import java.util.concurrent.ConcurrentHashMap;
@@ -42,7 +41,7 @@ public class CountingLocalTaskActionClientForTest implements TaskActionClient
TaskActionToolbox toolbox
)
{
- delegate = new LocalTaskActionClient(task, storage, toolbox, new TaskAuditLogConfig(false));
+ delegate = new LocalTaskActionClient(task, toolbox);
}
@Override
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
index d6687efbf34..ca4d7ee7fca 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
@@ -43,7 +43,6 @@ import org.apache.druid.indexer.report.TaskReport;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.TestUtils;
-import org.apache.druid.indexing.common.actions.SegmentInsertAction;
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
@@ -353,7 +352,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
public class TestLocalTaskActionClient extends CountingLocalTaskActionClientForTest
{
private final Set publishedSegments = new HashSet<>();
- private SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION);
+ private final SegmentSchemaMapping segmentSchemaMapping
+ = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION);
private TestLocalTaskActionClient(Task task)
{
@@ -365,11 +365,9 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
{
final RetType result = super.submit(taskAction);
if (taskAction instanceof SegmentTransactionalInsertAction) {
- publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments());
- segmentSchemaMapping.merge(((SegmentTransactionalInsertAction) taskAction).getSegmentSchemaMapping());
- } else if (taskAction instanceof SegmentInsertAction) {
- publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments());
- segmentSchemaMapping.merge(((SegmentInsertAction) taskAction).getSegmentSchemaMapping());
+ SegmentTransactionalInsertAction insertAction = (SegmentTransactionalInsertAction) taskAction;
+ publishedSegments.addAll(insertAction.getSegments());
+ segmentSchemaMapping.merge(insertAction.getSegmentSchemaMapping());
}
return result;
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java
index 633d861410d..5226c9d735b 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java
@@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.metadata.TaskLookup;
import org.easymock.EasyMock;
import org.hamcrest.MatcherAssert;
import org.joda.time.Interval;
@@ -39,7 +40,7 @@ import java.util.List;
public class IndexerMetadataStorageAdapterTest
{
- private TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private TaskStorage taskStorage;
private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter;
@@ -47,9 +48,9 @@ public class IndexerMetadataStorageAdapterTest
public void setup()
{
indexerMetadataStorageCoordinator = EasyMock.strictMock(IndexerMetadataStorageCoordinator.class);
- taskStorageQueryAdapter = EasyMock.strictMock(TaskStorageQueryAdapter.class);
+ taskStorage = EasyMock.strictMock(TaskStorage.class);
indexerMetadataStorageAdapter = new IndexerMetadataStorageAdapter(
- taskStorageQueryAdapter,
+ taskStorage,
indexerMetadataStorageCoordinator
);
}
@@ -73,7 +74,7 @@ public class IndexerMetadataStorageAdapterTest
NoopTask.create()
)
);
- EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos);
+ EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource")).andReturn(taskInfos);
final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01");
EasyMock
@@ -84,7 +85,7 @@ public class IndexerMetadataStorageAdapterTest
)
)
.andReturn(10);
- EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator);
+ EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator);
Assert.assertEquals(10, indexerMetadataStorageAdapter.deletePendingSegments("dataSource", deleteInterval));
}
@@ -109,7 +110,8 @@ public class IndexerMetadataStorageAdapterTest
)
);
- EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos);
+ EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource"))
+ .andReturn(taskInfos);
final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01");
EasyMock
@@ -120,7 +122,7 @@ public class IndexerMetadataStorageAdapterTest
)
)
.andReturn(10);
- EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator);
+ EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator);
MatcherAssert.assertThat(
Assert.assertThrows(
@@ -155,7 +157,8 @@ public class IndexerMetadataStorageAdapterTest
)
);
- EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos);
+ EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource"))
+ .andReturn(taskInfos);
final Interval deleteInterval = Intervals.of("2017-01-01/2018-12-01");
EasyMock
@@ -166,7 +169,7 @@ public class IndexerMetadataStorageAdapterTest
)
)
.andReturn(10);
- EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator);
+ EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator);
MatcherAssert.assertThat(
Assert.assertThrows(
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java
index 68f0ff77bb9..02d09b6fc6c 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java
@@ -20,14 +20,13 @@
package org.apache.druid.indexing.overlord;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.LockReleaseAction;
-import org.apache.druid.indexing.common.actions.SegmentInsertAction;
+import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
import org.apache.druid.indexing.common.config.TaskConfig;
@@ -38,6 +37,7 @@ import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert;
+import java.util.Collections;
import java.util.List;
/**
@@ -97,18 +97,7 @@ public class RealtimeishTask extends AbstractTask
Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2);
// Push first segment
- SegmentInsertAction firstSegmentInsertAction = new SegmentInsertAction(
- ImmutableSet.of(
- DataSegment.builder()
- .dataSource("foo")
- .interval(interval1)
- .version(lock1.getVersion())
- .size(0)
- .build()
- ),
- null
- );
- toolbox.getTaskActionClient().submit(firstSegmentInsertAction);
+ toolbox.getTaskActionClient().submit(createSegmentInsertAction(interval1, lock1.getVersion()));
// Release first lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1));
@@ -118,18 +107,7 @@ public class RealtimeishTask extends AbstractTask
Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3);
// Push second segment
- SegmentInsertAction secondSegmentInsertAction = new SegmentInsertAction(
- ImmutableSet.of(
- DataSegment.builder()
- .dataSource("foo")
- .interval(interval2)
- .version(lock2.getVersion())
- .size(0)
- .build()
- ),
- null
- );
- toolbox.getTaskActionClient().submit(secondSegmentInsertAction);
+ toolbox.getTaskActionClient().submit(createSegmentInsertAction(interval2, lock2.getVersion()));
// Release second lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2));
@@ -141,4 +119,17 @@ public class RealtimeishTask extends AbstractTask
// Exit
return TaskStatus.success(getId());
}
+
+ private SegmentTransactionalInsertAction createSegmentInsertAction(Interval interval, String version)
+ {
+ final DataSegment segmentToInsert
+ = DataSegment.builder()
+ .dataSource("foo")
+ .interval(interval)
+ .version(version)
+ .size(0)
+ .build();
+ return SegmentTransactionalInsertAction
+ .appendAction(Collections.singleton(segmentToInsert), null, null, null);
+ }
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
index 7e421519963..7b1209e7929 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
@@ -63,11 +63,10 @@ import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.LockListAction;
-import org.apache.druid.indexing.common.actions.SegmentInsertAction;
+import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
-import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
@@ -235,7 +234,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
private final String taskStorageType;
private ObjectMapper mapper;
- private TaskStorageQueryAdapter tsqa = null;
+ private TaskQueryTool tsqa = null;
private TaskStorage taskStorage = null;
private TaskLockbox taskLockbox = null;
private TaskQueue taskQueue = null;
@@ -478,7 +477,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()).anyTimes();
EasyMock.replay(taskMaster);
- tsqa = new TaskStorageQueryAdapter(taskStorage, taskLockbox, taskMaster);
+ tsqa = new TaskQueryTool(taskStorage, taskLockbox, taskMaster);
return taskStorage;
}
@@ -592,7 +591,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
taskLockbox = new TaskLockbox(taskStorage, mdc);
tac = new LocalTaskActionClientFactory(
- taskStorage,
new TaskActionToolbox(
taskLockbox,
taskStorage,
@@ -600,8 +598,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
emitter,
EasyMock.createMock(SupervisorManager.class),
mapper
- ),
- new TaskAuditLogConfig(true)
+ )
);
taskConfig = new TaskConfigBuilder()
.setBaseDir(temporaryFolder.newFolder().toString())
@@ -747,12 +744,10 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
final TaskStatus mergedStatus = runTask(indexTask);
final TaskStatus status = taskStorage.getStatus(indexTask.getId()).get();
final List publishedSegments = BY_INTERVAL_ORDERING.sortedCopy(mdc.getPublished());
- final List loggedSegments = BY_INTERVAL_ORDERING.sortedCopy(tsqa.getInsertedSegments(indexTask.getId()));
Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode());
Assert.assertEquals(taskLocation, status.getLocation());
Assert.assertEquals("merged statusCode", TaskState.SUCCESS, mergedStatus.getStatusCode());
- Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments);
Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
@@ -1103,7 +1098,9 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
.size(0)
.build();
- toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null));
+ toolbox.getTaskActionClient().submit(
+ SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null)
+ );
return TaskStatus.success(getId());
}
};
@@ -1144,7 +1141,9 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
.size(0)
.build();
- toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null));
+ toolbox.getTaskActionClient().submit(
+ SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null)
+ );
return TaskStatus.success(getId());
}
};
@@ -1186,7 +1185,9 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
.size(0)
.build();
- toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null));
+ toolbox.getTaskActionClient().submit(
+ SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null)
+ );
return TaskStatus.success(getId());
}
};
@@ -1244,11 +1245,9 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
final TaskStatus status = taskStorage.getStatus(indexTask.getId()).get();
final List publishedSegments = BY_INTERVAL_ORDERING.sortedCopy(mdc.getPublished());
- final List loggedSegments = BY_INTERVAL_ORDERING.sortedCopy(tsqa.getInsertedSegments(indexTask.getId()));
Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode());
Assert.assertEquals(taskLocation, status.getLocation());
- Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments);
Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
index 51b0cfe742e..4f2c3a38794 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
@@ -42,10 +42,10 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter;
import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.TaskQueue;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
import org.apache.druid.indexing.overlord.WorkerTaskRunner;
import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter;
import org.apache.druid.indexing.overlord.autoscaling.AutoScaler;
@@ -106,7 +106,7 @@ public class OverlordResourceTest
private JacksonConfigManager configManager;
private ProvisioningStrategy provisioningStrategy;
private AuthConfig authConfig;
- private TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private TaskQueryTool taskQueryTool;
private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter;
private HttpServletRequest req;
private TaskRunner taskRunner;
@@ -126,7 +126,7 @@ public class OverlordResourceTest
provisioningStrategy = EasyMock.createMock(ProvisioningStrategy.class);
authConfig = EasyMock.createMock(AuthConfig.class);
taskMaster = EasyMock.createStrictMock(TaskMaster.class);
- taskStorageQueryAdapter = EasyMock.createStrictMock(TaskStorageQueryAdapter.class);
+ taskQueryTool = EasyMock.createStrictMock(TaskQueryTool.class);
indexerMetadataStorageAdapter = EasyMock.createStrictMock(IndexerMetadataStorageAdapter.class);
req = EasyMock.createStrictMock(HttpServletRequest.class);
workerTaskRunnerQueryAdapter = EasyMock.createStrictMock(WorkerTaskRunnerQueryAdapter.class);
@@ -171,7 +171,7 @@ public class OverlordResourceTest
overlordResource = new OverlordResource(
taskMaster,
- taskStorageQueryAdapter,
+ taskQueryTool,
indexerMetadataStorageAdapter,
null,
configManager,
@@ -189,7 +189,7 @@ public class OverlordResourceTest
EasyMock.verify(
taskRunner,
taskMaster,
- taskStorageQueryAdapter,
+ taskQueryTool,
indexerMetadataStorageAdapter,
req,
workerTaskRunnerQueryAdapter,
@@ -197,19 +197,27 @@ public class OverlordResourceTest
);
}
+ private void replayAll()
+ {
+ EasyMock.replay(
+ taskRunner,
+ taskMaster,
+ taskQueryTool,
+ indexerMetadataStorageAdapter,
+ req,
+ workerTaskRunnerQueryAdapter,
+ authConfig,
+ configManager,
+ auditManager,
+ provisioningStrategy
+ );
+ }
+
@Test
public void testLeader()
{
EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("boz").once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.getLeader();
Assert.assertEquals("boz", response.getEntity());
@@ -221,15 +229,7 @@ public class OverlordResourceTest
{
EasyMock.expect(taskMaster.isLeader()).andReturn(true).once();
EasyMock.expect(taskMaster.isLeader()).andReturn(false).once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
// true
final Response response1 = overlordResource.isLeader();
@@ -247,7 +247,7 @@ public class OverlordResourceTest
{
expectAuthorizationTokenCheck();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()),
null
)
@@ -267,15 +267,7 @@ public class OverlordResourceTest
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
List responseObjects = (List) overlordResource.getWaitingTasks(req)
.getEntity();
@@ -290,7 +282,7 @@ public class OverlordResourceTest
List tasksIds = ImmutableList.of("id_1", "id_2", "id_3");
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null)
).andStubReturn(
ImmutableList.of(
@@ -299,15 +291,8 @@ public class OverlordResourceTest
createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow")
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
List responseObjects = (List) overlordResource
.getCompleteTasks(null, req).getEntity();
@@ -328,7 +313,7 @@ public class OverlordResourceTest
)
);
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()),
null
)
@@ -341,15 +326,7 @@ public class OverlordResourceTest
EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.RUNNING);
EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andStubReturn(RunnerTaskState.RUNNING);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
List responseObjects = (List) overlordResource.getRunningTasks(null, req)
.getEntity();
@@ -363,7 +340,7 @@ public class OverlordResourceTest
{
expectAuthorizationTokenCheck();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.ACTIVE,
ActiveTaskLookup.getInstance(),
@@ -390,15 +367,8 @@ public class OverlordResourceTest
)
).atLeastOnce();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
List responseObjects = (List) overlordResource
.getTasks(null, null, null, null, null, req)
.getEntity();
@@ -411,7 +381,7 @@ public class OverlordResourceTest
expectAuthorizationTokenCheck();
//completed tasks
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.COMPLETE,
CompleteTaskLookup.of(null, null),
@@ -437,15 +407,7 @@ public class OverlordResourceTest
new MockTaskRunnerWorkItem("id_4")
)
).atLeastOnce();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
List responseObjects = (List) overlordResource
.getTasks(null, "allow", null, null, null, req)
@@ -461,7 +423,7 @@ public class OverlordResourceTest
expectAuthorizationTokenCheck();
//active tasks
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.ACTIVE,
ActiveTaskLookup.getInstance()
@@ -484,15 +446,8 @@ public class OverlordResourceTest
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
List responseObjects = (List) overlordResource
.getTasks(
"waiting",
@@ -511,7 +466,7 @@ public class OverlordResourceTest
{
expectAuthorizationTokenCheck();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.ACTIVE,
ActiveTaskLookup.getInstance()
@@ -537,15 +492,7 @@ public class OverlordResourceTest
EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING);
EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andReturn(RunnerTaskState.RUNNING);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
List responseObjects = (List) overlordResource
.getTasks("running", "allow", null, null, null, req)
@@ -569,7 +516,7 @@ public class OverlordResourceTest
)
);
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()),
null
)
@@ -587,15 +534,7 @@ public class OverlordResourceTest
EasyMock.expect(taskRunner.getRunnerTaskState("id_3")).andStubReturn(RunnerTaskState.RUNNING);
EasyMock.expect(taskRunner.getRunnerTaskState("id_4")).andStubReturn(RunnerTaskState.RUNNING);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
List responseObjects = (List) overlordResource
.getTasks("pending", null, null, null, null, req)
@@ -611,7 +550,7 @@ public class OverlordResourceTest
{
expectAuthorizationTokenCheck();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)),
null
)
@@ -622,15 +561,8 @@ public class OverlordResourceTest
createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow")
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
List responseObjects = (List) overlordResource
.getTasks("complete", null, null, null, null, req)
.getEntity();
@@ -645,7 +577,7 @@ public class OverlordResourceTest
expectAuthorizationTokenCheck();
Duration duration = new Period("PT86400S").toStandardDuration();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
EasyMock.anyObject(),
EasyMock.anyObject()
)
@@ -657,15 +589,8 @@ public class OverlordResourceTest
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
String interval = "2010-01-01_P1D";
List responseObjects = (List) overlordResource
.getTasks("complete", null, interval, null, null, req)
@@ -684,7 +609,7 @@ public class OverlordResourceTest
// Setup mocks to return completed, active, known, pending and running tasks
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.COMPLETE,
CompleteTaskLookup.of(null, null),
@@ -712,16 +637,7 @@ public class OverlordResourceTest
EasyMock.expect(taskRunner.getRunnerTaskState("id_4")).andReturn(RunnerTaskState.PENDING);
EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING);
- // Replay all mocks
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
// Verify that only the tasks of read access datasource are returned
List responseObjects = (List) overlordResource
@@ -742,7 +658,7 @@ public class OverlordResourceTest
// Setup mocks to return completed, active, known, pending and running tasks
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.COMPLETE,
CompleteTaskLookup.of(null, null),
@@ -769,16 +685,7 @@ public class OverlordResourceTest
EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING);
- // Replay all mocks
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
// Verify that only the tasks of read access datasource are returned
List responseObjects = (List) overlordResource
@@ -797,16 +704,7 @@ public class OverlordResourceTest
// and no access to "buzzfeed"
expectAuthorizationTokenCheck(Users.WIKI_READER);
- // Replay all mocks
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
// Verify that only the tasks of read access datasource are returned
expectedException.expect(WebApplicationException.class);
@@ -818,7 +716,7 @@ public class OverlordResourceTest
{
expectAuthorizationTokenCheck();
EasyMock.expect(
- taskStorageQueryAdapter.getTaskStatusPlusList(
+ taskQueryTool.getTaskStatusPlusList(
ImmutableMap.of(
TaskLookupType.COMPLETE,
CompleteTaskLookup.of(null, null)
@@ -832,15 +730,8 @@ public class OverlordResourceTest
createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow")
)
);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
List responseObjects = (List) overlordResource
.getTasks("complete", null, null, null, null, req)
.getEntity();
@@ -852,15 +743,8 @@ public class OverlordResourceTest
@Test
public void testGetTasksNegativeState()
{
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
Object responseObject = overlordResource
.getTasks("blah", "ds_test", null, null, null, req)
.getEntity();
@@ -877,15 +761,8 @@ public class OverlordResourceTest
expectAuthorizationTokenCheck();
EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(false);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+
Task task = NoopTask.create();
overlordResource.taskPost(task, req);
}
@@ -914,17 +791,7 @@ public class OverlordResourceTest
auditManager.doAudit(EasyMock.capture(auditEntryCapture));
EasyMock.expectLastCall().once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskQueue,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig,
- auditManager
- );
+ replayAll();
Task task = new KillUnusedSegmentsTask("kill_all", "allow", Intervals.ETERNITY, null, null, 10, null, null);
overlordResource.taskPost(task, req);
@@ -943,15 +810,7 @@ public class OverlordResourceTest
expectAuthorizationTokenCheck(Users.WIKI_READER);
EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(false);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
// Verify that taskPost fails for user who has only datasource read access
Task task = NoopTask.forDatasource(Datasources.WIKIPEDIA);
@@ -975,15 +834,7 @@ public class OverlordResourceTest
)
.andReturn(2);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource
.killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req);
@@ -1008,15 +859,7 @@ public class OverlordResourceTest
.andThrow(InvalidInput.exception(exceptionMsg))
.once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource
.killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req);
@@ -1042,15 +885,7 @@ public class OverlordResourceTest
.andThrow(DruidException.defensive(exceptionMsg))
.once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource
.killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req);
@@ -1076,15 +911,7 @@ public class OverlordResourceTest
.andThrow(new IllegalStateException(exceptionMsg))
.once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource
.killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req);
@@ -1100,15 +927,7 @@ public class OverlordResourceTest
EasyMock.expect(taskMaster.isLeader()).andReturn(false);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource
.killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req);
@@ -1125,21 +944,13 @@ public class OverlordResourceTest
// This should be fixed in https://github.com/apache/druid/issues/6685.
// expectAuthorizationTokenCheck();
final NoopTask task = NoopTask.create();
- EasyMock.expect(taskStorageQueryAdapter.getTask("mytask"))
+ EasyMock.expect(taskQueryTool.getTask("mytask"))
.andReturn(Optional.of(task));
- EasyMock.expect(taskStorageQueryAdapter.getTask("othertask"))
+ EasyMock.expect(taskQueryTool.getTask("othertask"))
.andReturn(Optional.absent());
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response1 = overlordResource.getTaskPayload("mytask");
final TaskPayloadResponse taskPayloadResponse1 = TestHelper.makeJsonMapper().readValue(
@@ -1167,7 +978,7 @@ public class OverlordResourceTest
final String taskId = task.getId();
final TaskStatus status = TaskStatus.running(taskId);
- EasyMock.expect(taskStorageQueryAdapter.getTaskInfo(taskId))
+ EasyMock.expect(taskQueryTool.getTaskInfo(taskId))
.andReturn(new TaskInfo(
task.getId(),
DateTimes.of("2018-01-01"),
@@ -1176,21 +987,13 @@ public class OverlordResourceTest
task
));
- EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("othertask"))
+ EasyMock.expect(taskQueryTool.getTaskInfo("othertask"))
.andReturn(null);
EasyMock.>expect(taskRunner.getKnownTasks())
.andReturn(ImmutableList.of());
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response1 = overlordResource.getTaskStatus(taskId);
final TaskStatusResponse taskStatusResponse1 = TestHelper.makeJsonMapper().readValue(
@@ -1239,17 +1042,9 @@ public class OverlordResourceTest
)
);
- EasyMock.expect(taskStorageQueryAdapter.getLockedIntervals(minTaskPriority))
+ EasyMock.expect(taskQueryTool.getLockedIntervals(minTaskPriority))
.andReturn(expectedLockedIntervals);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.getDatasourceLockedIntervals(minTaskPriority);
Assert.assertEquals(200, response.getStatus());
@@ -1268,15 +1063,7 @@ public class OverlordResourceTest
@Test
public void testGetLockedIntervalsWithEmptyBody()
{
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
Response response = overlordResource.getDatasourceLockedIntervals(null);
Assert.assertEquals(400, response.getStatus());
@@ -1302,16 +1089,8 @@ public class OverlordResourceTest
mockQueue.shutdown("id_1", "Shutdown request from user");
EasyMock.expectLastCall();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- mockQueue,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+ EasyMock.replay(mockQueue);
final Map response = (Map) overlordResource
.doShutdown("id_1")
@@ -1333,7 +1112,7 @@ public class OverlordResourceTest
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(
Optional.of(mockQueue)
).anyTimes();
- EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("datasource")).andStubReturn(ImmutableList.of(
+ EasyMock.expect(taskQueryTool.getActiveTaskInfo("datasource")).andStubReturn(ImmutableList.of(
new TaskInfo<>(
"id_1",
DateTime.now(ISOChronology.getInstanceUTC()),
@@ -1354,16 +1133,8 @@ public class OverlordResourceTest
mockQueue.shutdown("id_2", "Shutdown request from user");
EasyMock.expectLastCall();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- mockQueue,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
+ EasyMock.replay(mockQueue);
final Map response = (Map) overlordResource
.shutdownTasksForDataSource("datasource")
@@ -1377,16 +1148,8 @@ public class OverlordResourceTest
final TaskQueue taskQueue = EasyMock.createMock(TaskQueue.class);
EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes();
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
- EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo(EasyMock.anyString())).andReturn(Collections.emptyList());
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ EasyMock.expect(taskQueryTool.getActiveTaskInfo(EasyMock.anyString())).andReturn(Collections.emptyList());
+ replayAll();
final Response response = overlordResource.shutdownTasksForDataSource("notExisting");
Assert.assertEquals(Status.NOT_FOUND.getStatusCode(), response.getStatus());
@@ -1400,15 +1163,7 @@ public class OverlordResourceTest
workerTaskRunnerQueryAdapter.enableWorker(host);
EasyMock.expectLastCall().once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.enableWorker(host);
@@ -1424,15 +1179,7 @@ public class OverlordResourceTest
workerTaskRunnerQueryAdapter.disableWorker(host);
EasyMock.expectLastCall().once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.disableWorker(host);
@@ -1448,15 +1195,7 @@ public class OverlordResourceTest
workerTaskRunnerQueryAdapter.enableWorker(host);
EasyMock.expectLastCall().andThrow(new RE("Worker API returns error!")).once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.enableWorker(host);
@@ -1472,15 +1211,7 @@ public class OverlordResourceTest
workerTaskRunnerQueryAdapter.disableWorker(host);
EasyMock.expectLastCall().andThrow(new RE("Worker API returns error!")).once();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.disableWorker(host);
@@ -1495,16 +1226,7 @@ public class OverlordResourceTest
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(
Optional.absent()
).anyTimes();
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- authConfig
- );
+ replayAll();
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE.getCode(), response.getStatus());
}
@@ -1517,16 +1239,8 @@ public class OverlordResourceTest
EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference);
EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1);
EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- authConfig
- );
+ replayAll();
+
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity());
@@ -1541,16 +1255,8 @@ public class OverlordResourceTest
EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference);
EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1);
EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- authConfig
- );
+ replayAll();
+
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity());
@@ -1566,16 +1272,8 @@ public class OverlordResourceTest
EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference);
EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1);
EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1);
- EasyMock.replay(
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- authConfig
- );
+ replayAll();
+
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity());
@@ -1617,17 +1315,9 @@ public class OverlordResourceTest
EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference);
EasyMock.replay(
workerTaskRunner,
- autoScaler,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- provisioningStrategy,
- authConfig
+ autoScaler
);
+ replayAll();
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
Assert.assertEquals(expectedWorkerCapacity, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity());
@@ -1670,17 +1360,9 @@ public class OverlordResourceTest
EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference);
EasyMock.replay(
workerTaskRunner,
- autoScaler,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter,
- configManager,
- provisioningStrategy,
- authConfig
+ autoScaler
);
+ replayAll();
final Response response = overlordResource.getTotalWorkerCapacity();
Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
Assert.assertEquals(workerInfos.stream().findFirst().get().getWorker().getCapacity(), ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity());
@@ -1706,16 +1388,8 @@ public class OverlordResourceTest
Action.READ
)));
- EasyMock.replay(
- task,
- authConfig,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter
- );
+ EasyMock.replay(task);
+ replayAll();
Set expectedResourceActions = ImmutableSet.of(
new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE),
@@ -1728,7 +1402,6 @@ public class OverlordResourceTest
@Test
public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled()
{
-
final String dataSource = "dataSourceTest";
final UOE expectedException = new UOE("unsupported");
Task task = EasyMock.createMock(Task.class);
@@ -1739,17 +1412,8 @@ public class OverlordResourceTest
EasyMock.expect(task.getDestinationResource()).andReturn(java.util.Optional.of(new Resource(dataSource, ResourceType.DATASOURCE)));
EasyMock.expect(task.getInputSourceResources()).andThrow(expectedException);
- EasyMock.replay(
- task,
- authConfig,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter
- );
-
+ EasyMock.replay(task);
+ replayAll();
final UOE e = Assert.assertThrows(
UOE.class,
@@ -1762,7 +1426,6 @@ public class OverlordResourceTest
@Test
public void testResourceActionsForTaskWithInputTypeAndInputSecurityDisabled()
{
-
final String dataSource = "dataSourceTest";
final String inputSourceType = "local";
Task task = EasyMock.createMock(Task.class);
@@ -1776,16 +1439,8 @@ public class OverlordResourceTest
Action.READ
)));
- EasyMock.replay(
- task,
- authConfig,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter
- );
+ EasyMock.replay(task);
+ replayAll();
Set expectedResourceActions = ImmutableSet.of(
new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE)
@@ -1797,24 +1452,14 @@ public class OverlordResourceTest
@Test
public void testGetMultipleTaskStatuses_presentTaskQueue()
{
- // Needed for teardown
- EasyMock.replay(
- authConfig,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter
- );
+ replayAll();
TaskQueue taskQueue = EasyMock.createMock(TaskQueue.class);
EasyMock.expect(taskQueue.getTaskStatus("task"))
.andReturn(Optional.of(TaskStatus.running("task")));
- EasyMock.replay(taskQueue);
TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue));
- EasyMock.replay(taskMaster);
+ EasyMock.replay(taskMaster, taskQueue);
OverlordResource overlordResource = new OverlordResource(
taskMaster,
null,
@@ -1835,27 +1480,18 @@ public class OverlordResourceTest
@Test
public void testGetMultipleTaskStatuses_absentTaskQueue()
{
- // Needed for teardown
- EasyMock.replay(
- authConfig,
- taskRunner,
- taskMaster,
- taskStorageQueryAdapter,
- indexerMetadataStorageAdapter,
- req,
- workerTaskRunnerQueryAdapter
- );
+ replayAll();
- TaskStorageQueryAdapter taskStorageQueryAdapter = EasyMock.createMock(TaskStorageQueryAdapter.class);
- EasyMock.expect(taskStorageQueryAdapter.getStatus("task"))
+ TaskQueryTool taskQueryTool = EasyMock.createMock(TaskQueryTool.class);
+ EasyMock.expect(taskQueryTool.getStatus("task"))
.andReturn(Optional.of(TaskStatus.running("task")));
- EasyMock.replay(taskStorageQueryAdapter);
+
TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent());
- EasyMock.replay(taskMaster);
+ EasyMock.replay(taskMaster, taskQueryTool);
OverlordResource overlordResource = new OverlordResource(
taskMaster,
- taskStorageQueryAdapter,
+ taskQueryTool,
null,
null,
null,
@@ -1870,6 +1506,24 @@ public class OverlordResourceTest
Assert.assertEquals(ImmutableMap.of("task", TaskStatus.running("task")), response);
}
+ @Test
+ public void testGetTaskSegmentsReturns404()
+ {
+ replayAll();
+ OverlordResource overlordResource =
+ new OverlordResource(null, null, null, null, null, null, null, null, null, null);
+ final Response response = overlordResource.getTaskSegments("taskId");
+ Assert.assertEquals(404, response.getStatus());
+ Assert.assertEquals(
+ Collections.singletonMap(
+ "error",
+ "Segment IDs committed by a task action are not persisted anymore."
+ + " Use the metric 'segment/added/bytes' to identify the segments created by a task."
+ ),
+ response.getEntity()
+ );
+ }
+
private void expectAuthorizationTokenCheck()
{
expectAuthorizationTokenCheck(Users.DRUID);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
index aab3639b2e6..9bbf2e9fc8a 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
@@ -52,12 +52,12 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.apache.druid.indexing.overlord.TaskRunnerFactory;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.indexing.overlord.TaskStorage;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter;
import org.apache.druid.indexing.overlord.autoscaling.ScalingStats;
import org.apache.druid.indexing.overlord.config.DefaultTaskConfig;
@@ -269,14 +269,14 @@ public class OverlordTest
Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort());
Assert.assertEquals(Optional.absent(), taskMaster.getRedirectLocation());
- final TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(taskStorage, taskLockbox, taskMaster);
+ final TaskQueryTool taskQueryTool = new TaskQueryTool(taskStorage, taskLockbox, taskMaster);
final WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter = new WorkerTaskRunnerQueryAdapter(taskMaster, null);
// Test Overlord resource stuff
AuditManager auditManager = EasyMock.createNiceMock(AuditManager.class);
overlordResource = new OverlordResource(
taskMaster,
- taskStorageQueryAdapter,
- new IndexerMetadataStorageAdapter(taskStorageQueryAdapter, null),
+ taskQueryTool,
+ new IndexerMetadataStorageAdapter(taskStorage, null),
null,
null,
auditManager,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java
index 3b775ca9491..a25c6aec621 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java
@@ -26,7 +26,7 @@ import com.google.inject.Injector;
import com.sun.jersey.spi.container.ResourceFilter;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.http.OverlordResource;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
@@ -60,7 +60,7 @@ public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper
return ImmutableList.copyOf(
Iterables.concat(
getRequestPaths(OverlordResource.class, ImmutableList.of(
- TaskStorageQueryAdapter.class,
+ TaskQueryTool.class,
AuthorizerMapper.class
)
),
@@ -84,7 +84,7 @@ public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper
private static boolean mockedOnceTsqa;
private static boolean mockedOnceSM;
- private TaskStorageQueryAdapter tsqa;
+ private TaskQueryTool tsqa;
private SupervisorManager supervisorManager;
public OverlordSecurityResourceFilterTest(
@@ -107,7 +107,7 @@ public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper
// Since we are creating the mocked tsqa object only once and getting that object from Guice here therefore
// if the mockedOnce check is not done then we will call EasyMock.expect and EasyMock.replay on the mocked object
// multiple times and it will throw exceptions
- tsqa = injector.getInstance(TaskStorageQueryAdapter.class);
+ tsqa = injector.getInstance(TaskQueryTool.class);
EasyMock.expect(tsqa.getTask(EasyMock.anyString())).andReturn(Optional.of(noopTask)).anyTimes();
EasyMock.replay(tsqa);
mockedOnceTsqa = true;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java
index 59abcaacdf8..8a7684b5ccd 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java
@@ -21,7 +21,7 @@ package org.apache.druid.indexing.overlord.http.security;
import com.google.common.base.Optional;
import com.sun.jersey.spi.container.ContainerRequest;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
import org.apache.druid.server.security.AuthorizerMapper;
import org.easymock.EasyMock;
@@ -42,7 +42,7 @@ import static org.easymock.EasyMock.expect;
public class TaskResourceFilterTest
{
private AuthorizerMapper authorizerMapper;
- private TaskStorageQueryAdapter taskStorageQueryAdapter;
+ private TaskQueryTool taskQueryTool;
private ContainerRequest containerRequest;
private TaskResourceFilter resourceFilter;
@@ -50,9 +50,9 @@ public class TaskResourceFilterTest
public void setup()
{
authorizerMapper = EasyMock.createMock(AuthorizerMapper.class);
- taskStorageQueryAdapter = EasyMock.createMock(TaskStorageQueryAdapter.class);
+ taskQueryTool = EasyMock.createMock(TaskQueryTool.class);
containerRequest = EasyMock.createMock(ContainerRequest.class);
- resourceFilter = new TaskResourceFilter(taskStorageQueryAdapter, authorizerMapper);
+ resourceFilter = new TaskResourceFilter(taskQueryTool, authorizerMapper);
}
@Test
@@ -68,11 +68,11 @@ public class TaskResourceFilterTest
expect(supervisorSpec.getDataSources())
.andReturn(Collections.singletonList(taskId))
.anyTimes();
- expect(taskStorageQueryAdapter.getTask(taskId))
+ expect(taskQueryTool.getTask(taskId))
.andReturn(Optional.absent())
.atLeastOnce();
EasyMock.replay(containerRequest);
- EasyMock.replay(taskStorageQueryAdapter);
+ EasyMock.replay(taskQueryTool);
WebApplicationException expected = null;
try {
@@ -84,7 +84,7 @@ public class TaskResourceFilterTest
Assert.assertNotNull(expected);
Assert.assertEquals(expected.getResponse().getStatus(), Response.Status.NOT_FOUND.getStatusCode());
EasyMock.verify(containerRequest);
- EasyMock.verify(taskStorageQueryAdapter);
+ EasyMock.verify(taskQueryTool);
}
private List getPathSegments(String path)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
index 1a9e5a17e1c..3b5775c22b8 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
@@ -62,7 +62,6 @@ import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
-import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
import org.apache.druid.indexing.common.config.TaskStorageConfig;
@@ -631,9 +630,7 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
objectMapper
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
- taskStorage,
- taskActionToolbox,
- new TaskAuditLogConfig(false)
+ taskActionToolbox
);
final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier()
{
diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
index 45bfde49a8a..4a5a48e8ef7 100644
--- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
+++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
@@ -20,6 +20,8 @@
package org.apache.druid.metadata;
import com.google.common.base.Optional;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.indexer.TaskIdentifier;
import org.apache.druid.indexer.TaskInfo;
import org.apache.druid.metadata.TaskLookup.TaskLookupType;
@@ -31,6 +33,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
+@ExtensionPoint
public interface MetadataStorageActionHandler
{
/**
@@ -161,21 +164,34 @@ public interface MetadataStorageActionHandler getLogs(String entryId);
+ @Deprecated
+ default List getLogs(String entryId)
+ {
+ throw DruidException.defensive()
+ .ofCategory(DruidException.Category.UNSUPPORTED)
+ .build("Task actions are not logged anymore.");
+ }
/**
* Returns the locks for the given entry
@@ -188,7 +204,7 @@ public interface MetadataStorageActionHandler getEntryType();
TypeReference getStatusType();
- TypeReference getLogType();
TypeReference getLockType();
}
diff --git a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java
index 75bbb32431c..6bd4280c5a3 100644
--- a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java
+++ b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java
@@ -43,6 +43,11 @@ public interface TaskLookup
COMPLETE
}
+ static TaskLookup activeTasksOnly()
+ {
+ return ActiveTaskLookup.getInstance();
+ }
+
/**
* Whether this lookup is guaranteed to not return any tasks.
*/
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 a9a18c36d54..57bc1013fc4 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
@@ -126,7 +126,7 @@ public interface Operator
*/
STOP,
/**
- * Inidcates that the downstream processing should pause its pushing of results and instead return a
+ * Indicates that the downstream processing should pause its pushing of results and instead return a
* continuation object that encapsulates whatever state is required to resume processing. When this signal is
* received, Operators that are generating data might choose to exert backpressure or otherwise pause their
* processing efforts until called again with the returned continuation object.
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java
index a4fa74967f6..0e0fc59498c 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java
@@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.query.rowsandcols.RowsAndColumns;
+import java.util.ArrayList;
import java.util.Arrays;
+import java.util.List;
public class ComposingProcessor implements Processor
{
@@ -37,6 +39,16 @@ public class ComposingProcessor implements Processor
this.processors = processors;
}
+ @Override
+ public List getOutputColumnNames()
+ {
+ List outputColumnNames = new ArrayList<>();
+ for (Processor processor : processors) {
+ outputColumnNames.addAll(processor.getOutputColumnNames());
+ }
+ return outputColumnNames;
+ }
+
@JsonProperty("processors")
public Processor[] getProcessors()
{
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java
index fe8d125cbdf..b271d3064ef 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java
@@ -31,6 +31,8 @@ import org.apache.druid.query.operator.window.value.WindowLastProcessor;
import org.apache.druid.query.operator.window.value.WindowOffsetProcessor;
import org.apache.druid.query.rowsandcols.RowsAndColumns;
+import java.util.List;
+
/**
* A Processor is a bit of logic that processes a single RowsAndColumns object to produce a new RowsAndColumns
* object. Generally speaking, it is used to add or alter columns in a batch-oriented fashion.
@@ -80,4 +82,9 @@ public interface Processor
* @return boolean identifying if these processors should be considered equivalent to each other.
*/
boolean validateEquivalent(Processor otherProcessor);
+
+ /**
+ * @return List of output column names for the Processor.
+ */
+ List getOutputColumnNames();
}
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java
index 3545c3740f4..41baced4e61 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java
@@ -27,7 +27,9 @@ import org.apache.druid.query.rowsandcols.semantic.DefaultFramedOnHeapAggregatab
import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable;
import javax.annotation.Nullable;
+import java.util.ArrayList;
import java.util.Arrays;
+import java.util.List;
import java.util.Objects;
public class WindowFramedAggregateProcessor implements Processor
@@ -45,6 +47,16 @@ public class WindowFramedAggregateProcessor implements Processor
private final WindowFrame frame;
private final AggregatorFactory[] aggregations;
+ @Override
+ public List getOutputColumnNames()
+ {
+ List outputColumnNames = new ArrayList<>();
+ for (AggregatorFactory aggregation : aggregations) {
+ outputColumnNames.add(aggregation.getName());
+ }
+ return outputColumnNames;
+ }
+
@JsonCreator
public WindowFramedAggregateProcessor(
@JsonProperty("frame") WindowFrame frame,
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java
index 541c1399e36..b7f77d50969 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java
@@ -28,12 +28,20 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
public class WindowPercentileProcessor implements Processor
{
private final int numBuckets;
private final String outputColumn;
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.singletonList(outputColumn);
+ }
+
@JsonCreator
public WindowPercentileProcessor(
@JsonProperty("outputColumn") String outputColumn,
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
index fb5bedf9519..4e026cbdd3d 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
@@ -27,6 +27,7 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner;
import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner;
+import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.function.Function;
@@ -124,4 +125,9 @@ public abstract class WindowRankingProcessorBase implements Processor
return Objects.equals(groupingCols, other.groupingCols) && Objects.equals(outputColumn, other.outputColumn);
}
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.singletonList(outputColumn);
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java
index 7821e3fd53b..98b09b6f80d 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java
@@ -28,6 +28,9 @@ import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn;
import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
import org.apache.druid.segment.column.ColumnType;
+import java.util.Collections;
+import java.util.List;
+
public class WindowRowNumberProcessor implements Processor
{
private final String outputColumn;
@@ -128,4 +131,10 @@ public class WindowRowNumberProcessor implements Processor
"outputColumn='" + outputColumn + '\'' +
'}';
}
+
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.singletonList(outputColumn);
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java
index 2e084ae983a..93a7ccd9a5b 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java
@@ -26,6 +26,8 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns;
import org.apache.druid.query.rowsandcols.column.Column;
import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
+import java.util.Collections;
+import java.util.List;
import java.util.function.Function;
public abstract class WindowValueProcessorBase implements Processor
@@ -100,4 +102,10 @@ public abstract class WindowValueProcessorBase implements Processor
return "inputColumn=" + inputColumn +
", outputColumn='" + outputColumn + '\'';
}
+
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.singletonList(outputColumn);
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
index 3c6d3cc08c9..9cc87be78e7 100644
--- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
+++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
@@ -41,7 +41,6 @@ import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.HashMap;
-import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@@ -209,7 +208,8 @@ public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryM
myClazz = float.class;
break;
case ARRAY:
- myClazz = List.class;
+ myClazz = Object[].class;
+ break;
default:
throw DruidException.defensive("this class cannot handle type [%s]", columnAccessor.getType());
}
diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java
index ca2cda3e0e1..979d213ee41 100644
--- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java
+++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java
@@ -475,15 +475,15 @@ public class ExpressionSelectors
}
final Class> clazz = selector.classOfObject();
- if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) {
- // Number, String supported as-is.
+ if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz) || Object[].class.isAssignableFrom(clazz)) {
+ // Number, String, Arrays supported as-is.
return selector::getObject;
} else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) {
// Might be Numbers and Strings. Use a selector that double-checks.
return () -> {
final Object val = selector.getObject();
if (val instanceof List) {
- NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue);
+ NonnullPair coerced = ExprEval.coerceListToArray((List>) val, homogenizeMultiValue);
if (coerced == null) {
return null;
}
@@ -496,7 +496,7 @@ public class ExpressionSelectors
return () -> {
final Object val = selector.getObject();
if (val != null) {
- NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue);
+ NonnullPair coerced = ExprEval.coerceListToArray((List>) val, homogenizeMultiValue);
if (coerced == null) {
return null;
}
diff --git a/processing/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java b/processing/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java
new file mode 100644
index 00000000000..41a0a55b284
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.metadata;
+
+import com.google.common.base.Optional;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.indexer.TaskIdentifier;
+import org.apache.druid.indexer.TaskInfo;
+import org.joda.time.DateTime;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests the default methods of the interface {@link MetadataStorageActionHandler}.
+ * Required only for coverage as these methods are already being tested in
+ * {@code SQLMetadataStorageActionHandlerTest}.
+ */
+public class MetadataStorageActionHandlerTest
+{
+
+ private MetadataStorageActionHandler handler;
+
+ @Before
+ public void setup()
+ {
+ this.handler = new MetadataStorageActionHandler()
+ {
+ @Override
+ public void insert(
+ String id,
+ DateTime timestamp,
+ String dataSource,
+ String entry,
+ boolean active,
+ @Nullable String status,
+ String type,
+ String groupId
+ )
+ {
+
+ }
+
+ @Override
+ public boolean setStatus(String entryId, boolean active, String status)
+ {
+ return false;
+ }
+
+ @Override
+ public Optional getEntry(String entryId)
+ {
+ return null;
+ }
+
+ @Override
+ public Optional getStatus(String entryId)
+ {
+ return null;
+ }
+
+ @Nullable
+ @Override
+ public TaskInfo getTaskInfo(String entryId)
+ {
+ return null;
+ }
+
+ @Override
+ public List> getTaskInfos(
+ Map taskLookups,
+ @Nullable String datasource
+ )
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public List> getTaskStatusList(
+ Map taskLookups,
+ @Nullable String datasource
+ )
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public boolean addLock(String entryId, String lock)
+ {
+ return false;
+ }
+
+ @Override
+ public boolean replaceLock(String entryId, long oldLockId, String newLock)
+ {
+ return false;
+ }
+
+ @Override
+ public void removeLock(long lockId)
+ {
+
+ }
+
+ @Override
+ public void removeTasksOlderThan(long timestamp)
+ {
+
+ }
+
+ @Override
+ public Map getLocks(String entryId)
+ {
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public Long getLockId(String entryId, String lock)
+ {
+ return 0L;
+ }
+
+ @Override
+ public void populateTaskTypeAndGroupIdAsync()
+ {
+
+ }
+ };
+ }
+
+ @Test
+ public void testAddLogThrowsUnsupportedException()
+ {
+ Exception exception = Assert.assertThrows(
+ DruidException.class,
+ () -> handler.addLog("abcd", "logentry")
+ );
+ Assert.assertEquals(
+ "Task actions are not logged anymore.",
+ exception.getMessage()
+ );
+ }
+
+ @Test
+ public void testGetLogsThrowsUnsupportedException()
+ {
+ Exception exception = Assert.assertThrows(
+ DruidException.class,
+ () -> handler.getLogs("abcd")
+ );
+ Assert.assertEquals(
+ "Task actions are not logged anymore.",
+ exception.getMessage()
+ );
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java
index 76d90737a5f..fe372558b64 100644
--- a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java
+++ b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java
@@ -130,7 +130,7 @@ public class TaskLookupTest
@Test
public void testGetType()
{
- Assert.assertEquals(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance().getType());
+ Assert.assertEquals(TaskLookupType.ACTIVE, TaskLookup.activeTasksOnly().getType());
}
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java
index 9cce74cb98c..c11a50cf5cb 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java
@@ -27,6 +27,9 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.junit.Assert;
import org.junit.Test;
+import java.util.Collections;
+import java.util.List;
+
public class WindowProcessorOperatorTest
{
@Test
@@ -53,6 +56,12 @@ public class WindowProcessorOperatorTest
{
return true;
}
+
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.emptyList();
+ }
},
InlineScanOperator.make(rac)
);
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java
index 570cba65d92..d8f4599eb1a 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java
@@ -23,6 +23,9 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns;
import org.junit.Assert;
import org.junit.Test;
+import java.util.Collections;
+import java.util.List;
+
public class ComposingProcessorTest
{
@Test
@@ -32,6 +35,7 @@ public class ComposingProcessorTest
final ProcessorForTesting secondProcessor = new ProcessorForTesting();
ComposingProcessor proc = new ComposingProcessor(firstProcessor, secondProcessor);
+ Assert.assertTrue(proc.getOutputColumnNames().isEmpty());
proc.process(null);
Assert.assertEquals(1, firstProcessor.processCounter);
@@ -70,5 +74,11 @@ public class ComposingProcessorTest
++validateCounter;
return validationResult;
}
+
+ @Override
+ public List getOutputColumnNames()
+ {
+ return Collections.emptyList();
+ }
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java
index 88d79c87cdb..5af321b53c8 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.query.operator.window;
+import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.common.config.NullHandling;
@@ -51,6 +52,7 @@ public class WindowFramedAggregateProcessorTest
new DoubleSumAggregatorFactory("cummSum", "doubleCol")
};
WindowFramedAggregateProcessor proc = new WindowFramedAggregateProcessor(theFrame, theAggs);
+ Assert.assertEquals(ImmutableList.of("cummMax", "cummSum"), proc.getOutputColumnNames());
final MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of(
"yay", new IntArrayColumn(new int[]{1, 2, 3})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java
index f5914e4f5db..877c7841549 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java
@@ -25,6 +25,7 @@ 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.IntArrayColumn;
+import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
@@ -42,6 +43,7 @@ public class WindowCumeDistProcessorTest
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
Processor processor = new WindowCumeDistProcessor(Collections.singletonList("vals"), "CumeDist");
+ Assert.assertEquals(Collections.singletonList("CumeDist"), processor.getOutputColumnNames());
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java
index e165f46f074..86580e5bd2f 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java
@@ -25,6 +25,7 @@ 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.IntArrayColumn;
+import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
@@ -42,6 +43,7 @@ public class WindowDenseRankProcessorTest
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
Processor processor = new WindowDenseRankProcessor(Collections.singletonList("vals"), "DenseRank");
+ Assert.assertEquals(Collections.singletonList("DenseRank"), processor.getOutputColumnNames());
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java
index c38cd2a245c..bf5bb727b0a 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.query.operator.window.ranking;
+import com.google.common.collect.ImmutableList;
import org.apache.druid.query.operator.window.ComposingProcessor;
import org.apache.druid.query.operator.window.Processor;
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
@@ -29,6 +30,7 @@ import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
import org.apache.druid.segment.column.ColumnType;
+import org.junit.Assert;
import org.junit.Test;
import java.util.LinkedHashMap;
@@ -63,6 +65,11 @@ public class WindowPercentileProcessorTest
new WindowPercentileProcessor("10292", 10292)
);
+ Assert.assertEquals(
+ ImmutableList.of("1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "10292"),
+ processor.getOutputColumnNames()
+ );
+
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java
index 59c7dd6df36..b7f281c423e 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.query.operator.window.ranking;
+import com.google.common.collect.ImmutableList;
import org.apache.druid.query.operator.window.ComposingProcessor;
import org.apache.druid.query.operator.window.Processor;
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
@@ -26,6 +27,7 @@ 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.IntArrayColumn;
+import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
@@ -49,6 +51,8 @@ public class WindowRankProcessorTest
new WindowRankProcessor(orderingCols, "rankAsPercent", true)
);
+ Assert.assertEquals(ImmutableList.of("rank", "rankAsPercent"), processor.getOutputColumnNames());
+
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
.expectColumn("rank", new int[]{1, 2, 2, 4, 5, 6, 7, 7, 9, 9})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java
index 937fea7c360..f4f9b5bfeee 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java
@@ -28,8 +28,10 @@ import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
import org.apache.druid.segment.column.ColumnType;
+import org.junit.Assert;
import org.junit.Test;
+import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.Map;
@@ -49,6 +51,7 @@ public class WindowRowNumberProcessorTest
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
Processor processor = new WindowRowNumberProcessor("rowRow");
+ Assert.assertEquals(Collections.singletonList("rowRow"), processor.getOutputColumnNames());
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java
index 67242f05503..eb6caa10a0b 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.query.operator.window.value;
+import com.google.common.collect.ImmutableList;
import org.apache.druid.query.operator.window.ComposingProcessor;
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
@@ -28,6 +29,7 @@ import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
import org.apache.druid.segment.column.ColumnType;
+import org.junit.Assert;
import org.junit.Test;
import java.util.LinkedHashMap;
@@ -59,6 +61,11 @@ public class WindowFirstProcessorTest
new WindowFirstProcessor("nullFirstCol", "NullFirstCol")
);
+ Assert.assertEquals(
+ ImmutableList.of("FirstIntCol", "FirstDoubleCol", "FirstObjectCol", "NullFirstCol"),
+ processor.getOutputColumnNames()
+ );
+
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9})
diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java
index 5aa212b6acb..1910401f34a 100644
--- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.query.operator.window.value;
+import com.google.common.collect.ImmutableList;
import org.apache.druid.query.operator.window.ComposingProcessor;
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
@@ -28,6 +29,7 @@ import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
import org.apache.druid.segment.column.ColumnType;
+import org.junit.Assert;
import org.junit.Test;
import java.util.LinkedHashMap;
@@ -58,6 +60,10 @@ public class WindowLastProcessorTest
new WindowLastProcessor("objectCol", "LastObjectCol"),
new WindowLastProcessor("nullLastCol", "NullLastCol")
);
+ Assert.assertEquals(
+ ImmutableList.of("LastIntCol", "LastDoubleCol", "LastObjectCol", "NullLastCol"),
+ processor.getOutputColumnNames()
+ );
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
index 8224f24e895..b6b9713682a 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
@@ -591,7 +591,22 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest
settableSupplier.set(ImmutableList.of("1", "2", "3"));
Assert.assertArrayEquals(new String[]{"1", "2", "3"}, (Object[]) supplier.get());
+ }
+ @Test
+ public void test_supplierFromObjectSelector_onArray()
+ {
+ final SettableSupplier settableSupplier = new SettableSupplier<>();
+ final Supplier supplier = ExpressionSelectors.supplierFromObjectSelector(
+ objectSelectorFromSupplier(settableSupplier, Object[].class),
+ true
+ );
+
+ Assert.assertNotNull(supplier);
+ Assert.assertEquals(null, supplier.get());
+
+ settableSupplier.set(new String[]{"1", "2", "3"});
+ Assert.assertArrayEquals(new String[]{"1", "2", "3"}, (Object[]) supplier.get());
}
@Test
diff --git a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java
index fa39d506b4c..c3265f38829 100644
--- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java
+++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java
@@ -21,7 +21,6 @@ package org.apache.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.druid.java.util.common.StringUtils;
public class DerbyMetadataStorageActionHandler
extends SQLMetadataStorageActionHandler
@@ -46,12 +45,4 @@ public class DerbyMetadataStorageActionHandler
extends SQLMetadataStorageActionHandler
@@ -44,13 +43,4 @@ public class PostgreSQLMetadataStorageActionHandler entryType;
private final TypeReference statusType;
- private final TypeReference logType;
private final TypeReference lockType;
private final String entryTypeName;
private final String entryTable;
- private final String logTable;
private final String lockTable;
private final TaskInfoMapper taskInfoMapper;
@@ -90,7 +88,11 @@ public abstract class SQLMetadataStorageActionHandler taskMigrationCompleteFuture;
- @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName")
+ /**
+ * @deprecated Use the other constructor without {@code logTable} argument
+ * since this argument is now unused.
+ */
+ @Deprecated
public SQLMetadataStorageActionHandler(
final SQLMetadataConnector connector,
final ObjectMapper jsonMapper,
@@ -100,6 +102,19 @@ public abstract class SQLMetadataStorageActionHandler types,
+ final String entryTypeName,
+ final String entryTable,
+ final String lockTable
+ )
{
this.connector = connector;
//fully qualified references required below due to identical package names across project modules.
@@ -108,11 +123,9 @@ public abstract class SQLMetadataStorageActionHandler(jsonMapper, entryType, statusType);
this.taskStatusMapper = new TaskStatusMapper(jsonMapper);
@@ -142,7 +155,7 @@ public abstract class SQLMetadataStorageActionHandler {
- handle.createStatement(getSqlRemoveLogsOlderThan())
- .bind("date_time", dateTime.toString())
- .execute();
+ handle ->
handle.createStatement(
StringUtils.format(
"DELETE FROM %s WHERE created_date < :date_time AND active = false",
entryTable
)
- )
- .bind("date_time", dateTime.toString())
- .execute();
-
- return null;
- }
+ ).bind("date_time", dateTime.toString()).execute()
);
}
@@ -880,78 +885,6 @@ public abstract class SQLMetadataStorageActionHandler()
- {
- @Override
- public Boolean withHandle(Handle handle) throws Exception
- {
- return handle.createStatement(
- StringUtils.format(
- "INSERT INTO %1$s (%2$s_id, log_payload) VALUES (:entryId, :payload)",
- logTable, entryTypeName
- )
- )
- .bind("entryId", entryId)
- .bind("payload", jsonMapper.writeValueAsBytes(log))
- .execute() == 1;
- }
- }
- );
- }
-
- @Override
- public List getLogs(final String entryId)
- {
- return connector.retryWithHandle(
- new HandleCallback>()
- {
- @Override
- public List withHandle(Handle handle)
- {
- return handle
- .createQuery(
- StringUtils.format(
- "SELECT log_payload FROM %1$s WHERE %2$s_id = :entryId",
- logTable, entryTypeName
- )
- )
- .bind("entryId", entryId)
- .map(ByteArrayMapper.FIRST)
- .fold(
- new ArrayList<>(),
- (List list, byte[] bytes, FoldController control, StatementContext ctx) -> {
- try {
- list.add(jsonMapper.readValue(bytes, logType));
- return list;
- }
- catch (IOException e) {
- log.makeAlert(e, "Failed to deserialize log")
- .addData("entryId", entryId)
- .addData("payload", StringUtils.fromUtf8(bytes))
- .emit();
- throw new SQLException(e);
- }
- }
- );
- }
- }
- );
- }
-
- @Deprecated
- public String getSqlRemoveLogsOlderThan()
- {
- return StringUtils.format(
- "DELETE a FROM %s a INNER JOIN %s b ON a.%s_id = b.id "
- + "WHERE b.created_date < :date_time and b.active = false",
- logTable, entryTable, entryTypeName
- );
- }
-
@Override
public Map getLocks(final String entryId)
{
diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java
index e103b428fed..b59af9ef690 100644
--- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java
+++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java
@@ -56,7 +56,6 @@ public class SQLMetadataConnectorSchemaPersistenceTest
tables.add(tablesConfig.getSegmentsTable());
tables.add(tablesConfig.getRulesTable());
tables.add(tablesConfig.getLockTable(entryType));
- tables.add(tablesConfig.getLogTable(entryType));
tables.add(tablesConfig.getEntryTable(entryType));
tables.add(tablesConfig.getAuditTable());
tables.add(tablesConfig.getSupervisorTable());
@@ -67,7 +66,6 @@ public class SQLMetadataConnectorSchemaPersistenceTest
dropSequence.add(tablesConfig.getSegmentSchemasTable());
dropSequence.add(tablesConfig.getRulesTable());
dropSequence.add(tablesConfig.getLockTable(entryType));
- dropSequence.add(tablesConfig.getLogTable(entryType));
dropSequence.add(tablesConfig.getEntryTable(entryType));
dropSequence.add(tablesConfig.getAuditTable());
dropSequence.add(tablesConfig.getSupervisorTable());
diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java
index 484299b5636..a40f95c31b9 100644
--- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java
+++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java
@@ -75,7 +75,6 @@ public class SQLMetadataConnectorTest
tables.add(tablesConfig.getSegmentsTable());
tables.add(tablesConfig.getRulesTable());
tables.add(tablesConfig.getLockTable(entryType));
- tables.add(tablesConfig.getLogTable(entryType));
tables.add(tablesConfig.getEntryTable(entryType));
tables.add(tablesConfig.getAuditTable());
tables.add(tablesConfig.getSupervisorTable());
diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
index 6cddcd5d646..8117d6dcb73 100644
--- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
+++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
@@ -34,7 +34,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup;
import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup;
import org.joda.time.DateTime;
@@ -73,12 +72,10 @@ public class SQLMetadataStorageActionHandlerTest
TestDerbyConnector connector = derbyConnectorRule.getConnector();
final String entryType = "entry";
- final String logTable = "logs";
final String lockTable = "locks";
connector.prepareTaskEntryTable(entryTable);
connector.createLockTable(lockTable, entryType);
- connector.createLogTable(logTable, entryType);
handler = new DerbyMetadataStorageActionHandler<>(
connector,
@@ -101,12 +98,6 @@ public class SQLMetadataStorageActionHandlerTest
};
}
- @Override
- public TypeReference> getLogType()
- {
- return JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING;
- }
-
@Override
public TypeReference> getLockType()
{
@@ -117,7 +108,7 @@ public class SQLMetadataStorageActionHandlerTest
},
entryType,
entryTable,
- logTable,
+ null,
lockTable
);
}
@@ -247,36 +238,30 @@ public class SQLMetadataStorageActionHandlerTest
}
@Test
- public void testLogs()
+ public void testAddLogThrowsUnsupportedException()
{
- final String entryId = "abcd";
- Map entry = ImmutableMap.of("a", 1);
- Map status = ImmutableMap.of("count", 42);
-
- handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group");
-
- Assert.assertEquals(
- ImmutableList.of(),
- handler.getLogs("non_exist_entry")
+ Exception exception = Assert.assertThrows(
+ DruidException.class,
+ () -> handler.addLog("abcd", ImmutableMap.of("logentry", "created"))
);
-
Assert.assertEquals(
- ImmutableMap.of(),
- handler.getLocks(entryId)
- );
-
- final ImmutableMap log1 = ImmutableMap.of("logentry", "created");
- final ImmutableMap log2 = ImmutableMap.of("logentry", "updated");
-
- Assert.assertTrue(handler.addLog(entryId, log1));
- Assert.assertTrue(handler.addLog(entryId, log2));
-
- Assert.assertEquals(
- ImmutableList.of(log1, log2),
- handler.getLogs(entryId)
+ "Task actions are not logged anymore.",
+ exception.getMessage()
);
}
+ @Test
+ public void testGetLogsThrowsUnsupportedException()
+ {
+ Exception exception = Assert.assertThrows(
+ DruidException.class,
+ () -> handler.getLogs("abcd")
+ );
+ Assert.assertEquals(
+ "Task actions are not logged anymore.",
+ exception.getMessage()
+ );
+ }
@Test
public void testLocks()
@@ -388,19 +373,16 @@ public class SQLMetadataStorageActionHandlerTest
Map entry1 = ImmutableMap.of("numericId", 1234);
Map status1 = ImmutableMap.of("count", 42, "temp", 1);
handler.insert(entryId1, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", entry1, false, status1, "type", "group");
- Assert.assertTrue(handler.addLog(entryId1, ImmutableMap.of("logentry", "created")));
final String entryId2 = "ABC123";
Map entry2 = ImmutableMap.of("a", 1);
Map status2 = ImmutableMap.of("count", 42);
handler.insert(entryId2, DateTimes.of("2014-01-01T00:00:00.123"), "test", entry2, true, status2, "type", "group");
- Assert.assertTrue(handler.addLog(entryId2, ImmutableMap.of("logentry", "created")));
final String entryId3 = "DEF5678";
Map entry3 = ImmutableMap.of("numericId", 5678);
Map status3 = ImmutableMap.of("count", 21, "temp", 2);
handler.insert(entryId3, DateTimes.of("2014-01-02T12:00:00.123"), "testDataSource", entry3, false, status3, "type", "group");
- Assert.assertTrue(handler.addLog(entryId3, ImmutableMap.of("logentry", "created")));
Assert.assertEquals(Optional.of(entry1), handler.getEntry(entryId1));
Assert.assertEquals(Optional.of(entry2), handler.getEntry(entryId2));
@@ -438,10 +420,6 @@ public class SQLMetadataStorageActionHandlerTest
.collect(Collectors.toList())
);
- // tasklogs
- Assert.assertEquals(0, handler.getLogs(entryId1).size());
- Assert.assertEquals(1, handler.getLogs(entryId2).size());
- Assert.assertEquals(1, handler.getLogs(entryId3).size());
}
@Test
diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
index 24e98427ce9..4dd23f2faf2 100644
--- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java
+++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
@@ -58,7 +58,6 @@ import org.apache.druid.indexing.common.TaskStorageDirTracker;
import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
-import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskStorageConfig;
import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
@@ -75,9 +74,9 @@ import org.apache.druid.indexing.overlord.MetadataTaskStorage;
import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskQueryTool;
import org.apache.druid.indexing.overlord.TaskRunnerFactory;
import org.apache.druid.indexing.overlord.TaskStorage;
-import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter;
import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningConfig;
import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningStrategy;
import org.apache.druid.indexing.overlord.autoscaling.ProvisioningSchedulerConfig;
@@ -209,7 +208,6 @@ public class CliOverlord extends ServerRunnable
JsonConfigProvider.bind(binder, "druid.indexer.tasklock", TaskLockConfig.class);
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class);
- JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class);
binder.bind(RetryPolicyFactory.class).in(LazySingleton.class);
binder.bind(TaskMaster.class).in(ManageLifecycle.class);
@@ -231,7 +229,7 @@ public class CliOverlord extends ServerRunnable
binder.bind(TaskActionClientFactory.class).to(LocalTaskActionClientFactory.class).in(LazySingleton.class);
binder.bind(TaskActionToolbox.class).in(LazySingleton.class);
binder.bind(TaskLockbox.class).in(LazySingleton.class);
- binder.bind(TaskStorageQueryAdapter.class).in(LazySingleton.class);
+ binder.bind(TaskQueryTool.class).in(LazySingleton.class);
binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class);
binder.bind(SupervisorManager.class).in(LazySingleton.class);
diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java
index f78a763cec4..5edb83d0a3b 100644
--- a/services/src/main/java/org/apache/druid/cli/CliPeon.java
+++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java
@@ -78,7 +78,6 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
-import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskStorageConfig;
import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
@@ -483,7 +482,6 @@ public class CliPeon extends GuiceRunnable
binder.bind(TaskToolboxFactory.class).in(LazySingleton.class);
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
- JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class);
JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class);
configureTaskActionClient(binder);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
index b3d657b148f..ab20925107f 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
@@ -320,6 +320,36 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
.run();
}
+ @Test
+ public void testWithArrayConcat()
+ {
+ testBuilder()
+ .sql("select countryName, cityName, channel, "
+ + "array_concat_agg(ARRAY['abc', channel], 10000) over (partition by cityName order by countryName) as c\n"
+ + "from wikipedia\n"
+ + "where countryName in ('Austria', 'Republic of Korea') "
+ + "and (cityName in ('Vienna', 'Seoul') or cityName is null)\n"
+ + "group by countryName, cityName, channel")
+ .queryContext(ImmutableMap.of(
+ PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
+ QueryContexts.ENABLE_DEBUG, true
+ ))
+ .expectedResults(
+ ResultMatchMode.RELAX_NULLS,
+ ImmutableList.of(
+ new Object[]{"Austria", null, "#de.wikipedia", "[\"abc\",\"#de.wikipedia\"]"},
+ new Object[]{"Republic of Korea", null, "#en.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"},
+ new Object[]{"Republic of Korea", null, "#ja.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"},
+ new Object[]{"Republic of Korea", null, "#ko.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"},
+ new Object[]{"Republic of Korea", "Seoul", "#ko.wikipedia", "[\"abc\",\"#ko.wikipedia\"]"},
+ new Object[]{"Austria", "Vienna", "#de.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"},
+ new Object[]{"Austria", "Vienna", "#es.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"},
+ new Object[]{"Austria", "Vienna", "#tr.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"}
+ )
+ )
+ .run();
+ }
+
private WindowOperatorQuery getWindowOperatorQuery(List> queries)
{
assertEquals(1, queries.size());
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
index cb7bed7e041..4e958383945 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
@@ -7533,4 +7533,78 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
{
windowQueryTest();
}
+
+ /*
+ Druid query tests
+ */
+
+ @DrillTest("druid_queries/same_window_across_columns/wikipedia_query_1")
+ @Test
+ public void test_same_window_wikipedia_query_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/same_window_across_columns/wikipedia_query_1_named_window")
+ @Test
+ public void test_same_window_wikipedia_query_1_named_window()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/multiple_windows/wikipedia_query_1")
+ @Test
+ public void test_multiple_windows_wikipedia_query_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/multiple_windows/wikipedia_query_1_named_windows")
+ @Test
+ public void test_multiple_windows_wikipedia_query_1_named_windows()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/shuffle_columns/wikipedia_query_1")
+ @Test
+ public void test_shuffle_columns_wikipedia_query_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1")
+ @Test
+ public void test_shuffle_columns_wikipedia_query_1_shuffle_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/shuffle_columns/wikipedia_query_2")
+ @Test
+ public void test_shuffle_columns_wikipedia_query_2()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1")
+ @Test
+ public void test_shuffle_columns_wikipedia_query_2_shuffle_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/partition_by_multiple_columns/wikipedia_query_1")
+ @Test
+ public void test_partition_by_multiple_columns_wikipedia_query_1()
+ {
+ windowQueryTest();
+ }
+
+ @DrillTest("druid_queries/partition_by_multiple_columns/wikipedia_query_2")
+ @Test
+ public void test_partition_by_multiple_columns_wikipedia_query_2()
+ {
+ windowQueryTest();
+ }
}
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.e
new file mode 100644
index 00000000000..3625be892e2
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.e
@@ -0,0 +1,13 @@
+null Austria 1 1
+null Republic of Korea 1 2
+null Republic of Korea 2 3
+null Republic of Korea 3 4
+Horsching Austria 2 1
+Jeonju Republic of Korea 4 1
+Seongnam-si Republic of Korea 5 1
+Seoul Republic of Korea 6 1
+Suwon-si Republic of Korea 7 1
+Vienna Austria 3 1
+Vienna Austria 4 2
+Vienna Austria 5 3
+Yongsan-dong Republic of Korea 8 1
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q
new file mode 100644
index 00000000000..d61a33e401f
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q
@@ -0,0 +1,6 @@
+select cityName, countryName,
+row_number() over (partition by countryName order by countryName, cityName, channel) as c1,
+count(channel) over (partition by cityName order by countryName, cityName, channel) as c2
+from wikipedia
+where countryName in ('Austria', 'Republic of Korea')
+group by countryName, cityName, channel
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.e b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.e
new file mode 100644
index 00000000000..3625be892e2
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.e
@@ -0,0 +1,13 @@
+null Austria 1 1
+null Republic of Korea 1 2
+null Republic of Korea 2 3
+null Republic of Korea 3 4
+Horsching Austria 2 1
+Jeonju Republic of Korea 4 1
+Seongnam-si Republic of Korea 5 1
+Seoul Republic of Korea 6 1
+Suwon-si Republic of Korea 7 1
+Vienna Austria 3 1
+Vienna Austria 4 2
+Vienna Austria 5 3
+Yongsan-dong Republic of Korea 8 1
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q
new file mode 100644
index 00000000000..12739d58ceb
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q
@@ -0,0 +1,9 @@
+select cityName, countryName,
+row_number() over w1 as c1,
+count(channel) over w2 as c2
+from wikipedia
+where countryName in ('Austria', 'Republic of Korea')
+group by countryName, cityName, channel
+WINDOW
+ w1 AS (partition by countryName order by countryName, cityName, channel),
+ w2 AS (partition by cityName order by countryName, cityName, channel)
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e
new file mode 100644
index 00000000000..36812a418ae
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e
@@ -0,0 +1,15 @@
+Austria null 94 7
+Austria null 4685 7
+Austria null 14 7
+Austria null 0 7
+Austria null 272 7
+Austria null 0 7
+Austria null 6979 7
+Guatemala null 0 1
+Guatemala El Salvador 1 1
+Guatemala Guatemala City 173 1
+Austria Horsching 0 1
+Austria Vienna 93 4
+Austria Vienna 72 4
+Austria Vienna 0 4
+Austria Vienna 0 4
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q
new file mode 100644
index 00000000000..5d0dd075678
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q
@@ -0,0 +1,7 @@
+SELECT
+countryName,
+cityName,
+added,
+count(added) OVER (PARTITION BY countryName, cityName)
+FROM "wikipedia"
+where countryName in ('Guatemala', 'Austria')
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e
new file mode 100644
index 00000000000..a1b94f5a865
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e
@@ -0,0 +1,15 @@
+Austria null 0 7 12044 1
+Austria null 0 7 12044 2
+Austria null 14 7 12044 1
+Austria null 94 7 12044 1
+Austria null 272 7 12044 1
+Austria null 4685 7 12044 1
+Austria null 6979 7 12044 1
+Guatemala null 0 1 0 1
+Guatemala El Salvador 1 1 1 1
+Guatemala Guatemala City 173 1 173 1
+Austria Horsching 0 1 0 1
+Austria Vienna 0 4 165 1
+Austria Vienna 0 4 165 2
+Austria Vienna 72 4 165 1
+Austria Vienna 93 4 165 1
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q
new file mode 100644
index 00000000000..b1a594beeda
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q
@@ -0,0 +1,9 @@
+SELECT
+countryName,
+cityName,
+added,
+count(added) OVER (PARTITION BY countryName, cityName),
+sum(added) OVER (PARTITION BY countryName, cityName),
+ROW_NUMBER() OVER (PARTITION BY countryName, cityName, added)
+FROM "wikipedia"
+where countryName in ('Guatemala', 'Austria')
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e
new file mode 100644
index 00000000000..0dfb6a832b8
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e
@@ -0,0 +1,15 @@
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Guatemala 167 7 174
+Guatemala 167 7 174
+Guatemala 167 7 174
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q
new file mode 100644
index 00000000000..dcb83c09c23
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q
@@ -0,0 +1,6 @@
+SELECT countryName,
+sum("deleted") OVER (PARTITION BY countryName) as count_c3,
+sum(delta) OVER (PARTITION BY countryName) as count_c1,
+sum(added) OVER (PARTITION BY countryName) as count_c2
+FROM "wikipedia"
+where countryName in ('Guatemala', 'Austria')
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e
new file mode 100644
index 00000000000..0dfb6a832b8
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e
@@ -0,0 +1,15 @@
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Austria 162 12047 12209
+Guatemala 167 7 174
+Guatemala 167 7 174
+Guatemala 167 7 174
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q
new file mode 100644
index 00000000000..adb9287d378
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q
@@ -0,0 +1,7 @@
+SELECT countryName,
+sum("deleted") OVER w as count_c3,
+sum(delta) OVER w as count_c1,
+sum(added) OVER w as count_c2
+FROM "wikipedia"
+where countryName in ('Guatemala', 'Austria')
+WINDOW w AS (PARTITION BY countryName)
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e
new file mode 100644
index 00000000000..e934bc8fc27
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e
@@ -0,0 +1,15 @@
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Austria 1017.4166666666666
+Guatemala 58
+Guatemala 58
+Guatemala 58
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q
new file mode 100644
index 00000000000..f1a7bcb09b1
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q
@@ -0,0 +1,5 @@
+SELECT
+countryName,
+AVG(added) OVER(PARTITION BY countryName)
+FROM wikipedia
+where countryName in ('Guatemala', 'Austria')
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e
new file mode 100644
index 00000000000..e74706be009
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e
@@ -0,0 +1,15 @@
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+1017.4166666666666 Austria
+58 Guatemala
+58 Guatemala
+58 Guatemala
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q
new file mode 100644
index 00000000000..c2dc11546a9
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q
@@ -0,0 +1,5 @@
+SELECT
+AVG(added) OVER(PARTITION BY countryName),
+countryName
+FROM wikipedia
+where countryName in ('Guatemala', 'Austria')
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e
new file mode 100644
index 00000000000..daf6eff61ba
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e
@@ -0,0 +1,16 @@
+Austria null 1 #de.wikipedia 1
+Guatemala null 1 #es.wikipedia 2
+Republic of Korea null 1 #en.wikipedia 3
+Republic of Korea null 2 #ja.wikipedia 4
+Republic of Korea null 3 #ko.wikipedia 5
+Guatemala El Salvador 2 #es.wikipedia 1
+Guatemala Guatemala City 3 #es.wikipedia 1
+Austria Horsching 2 #de.wikipedia 1
+Republic of Korea Jeonju 4 #ko.wikipedia 1
+Republic of Korea Seongnam-si 5 #ko.wikipedia 1
+Republic of Korea Seoul 6 #ko.wikipedia 1
+Republic of Korea Suwon-si 7 #ko.wikipedia 1
+Austria Vienna 3 #de.wikipedia 1
+Austria Vienna 4 #es.wikipedia 2
+Austria Vienna 5 #tr.wikipedia 3
+Republic of Korea Yongsan-dong 8 #ko.wikipedia 1
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q
new file mode 100644
index 00000000000..d3ea2dfc729
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q
@@ -0,0 +1,9 @@
+SELECT
+countryName,
+cityName,
+ROW_NUMBER() OVER(PARTITION BY countryName),
+channel,
+COUNT(channel) over (PARTITION BY cityName order by countryName, cityName, channel)
+FROM wikipedia
+where countryName in ('Guatemala', 'Austria', 'Republic of Korea')
+group by countryName, cityName, channel
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e
new file mode 100644
index 00000000000..813ccdbf6aa
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e
@@ -0,0 +1,16 @@
+1 Austria null 1 #de.wikipedia
+1 Guatemala null 2 #es.wikipedia
+1 Republic of Korea null 3 #en.wikipedia
+2 Republic of Korea null 4 #ja.wikipedia
+3 Republic of Korea null 5 #ko.wikipedia
+2 Guatemala El Salvador 1 #es.wikipedia
+3 Guatemala Guatemala City 1 #es.wikipedia
+2 Austria Horsching 1 #de.wikipedia
+4 Republic of Korea Jeonju 1 #ko.wikipedia
+5 Republic of Korea Seongnam-si 1 #ko.wikipedia
+6 Republic of Korea Seoul 1 #ko.wikipedia
+7 Republic of Korea Suwon-si 1 #ko.wikipedia
+3 Austria Vienna 1 #de.wikipedia
+4 Austria Vienna 2 #es.wikipedia
+5 Austria Vienna 3 #tr.wikipedia
+8 Republic of Korea Yongsan-dong 1 #ko.wikipedia
diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q
new file mode 100644
index 00000000000..779aaf3a86f
--- /dev/null
+++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q
@@ -0,0 +1,9 @@
+SELECT
+ROW_NUMBER() OVER(PARTITION BY countryName),
+countryName,
+cityName,
+COUNT(channel) over (PARTITION BY cityName order by countryName, cityName, channel),
+channel
+FROM wikipedia
+where countryName in ('Guatemala', 'Austria', 'Republic of Korea')
+group by countryName, cityName, channel
diff --git a/web-console/src/components/action-cell/action-cell.spec.tsx b/web-console/src/components/action-cell/action-cell.spec.tsx
index 6349ee1042e..b52c6af4e84 100644
--- a/web-console/src/components/action-cell/action-cell.spec.tsx
+++ b/web-console/src/components/action-cell/action-cell.spec.tsx
@@ -23,7 +23,7 @@ import { ActionCell } from './action-cell';
describe('ActionCell', () => {
it('matches snapshot', () => {
- const actionCell = {}} actions={[]} />;
+ const actionCell = {}} actions={[]} menuTitle="item" />;
const { container } = render(actionCell);
expect(container.firstChild).toMatchSnapshot();
});
diff --git a/web-console/src/components/action-cell/action-cell.tsx b/web-console/src/components/action-cell/action-cell.tsx
index c0e522e68ab..4a20e429977 100644
--- a/web-console/src/components/action-cell/action-cell.tsx
+++ b/web-console/src/components/action-cell/action-cell.tsx
@@ -34,12 +34,13 @@ export const ACTION_COLUMN_WIDTH = 70;
export interface ActionCellProps {
onDetail?: () => void;
disableDetail?: boolean;
- actions?: BasicAction[];
+ actions: BasicAction[];
+ menuTitle: string;
}
export const ActionCell = React.memo(function ActionCell(props: ActionCellProps) {
- const { onDetail, disableDetail, actions } = props;
- const actionsMenu = actions ? basicActionsToMenu(actions) : null;
+ const { onDetail, disableDetail, actions, menuTitle } = props;
+ const actionsMenu = basicActionsToMenu(actions, menuTitle);
return (
diff --git a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap
index c22b1563155..92f290c4a27 100644
--- a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap
+++ b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap
@@ -105,792 +105,800 @@ exports[`RetentionDialog matches snapshot 1`] = `
-
-
-