From 400de143a25626bab5d5ca41ac530b245537fd3b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 20:08:17 +0530 Subject: [PATCH 01/38] Add EmbeddedIndexParallelTaskTest to migrate ITPerfectRollupParallelIndexTest --- .../testing/embedded/indexing/CreateTask.java | 283 ++++++++++++++++++ .../EmbeddedIndexParallelTaskTest.java | 228 ++++++++++++++ .../indexing/EmbeddedIndexTaskTest.java | 22 +- .../testing/embedded/indexing/Resources.java | 53 +--- .../server/EmbeddedHighAvailabilityTest.java | 16 +- .../test/resources/data/json/wikipedia_1.json | 3 + .../test/resources/data/json/wikipedia_2.json | 3 + .../test/resources/data/json/wikipedia_3.json | 4 + .../report/IngestionStatsAndErrors.java | 2 +- .../testing/embedded/EmbeddedClusterApis.java | 19 ++ 10 files changed, 566 insertions(+), 67 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java create mode 100644 embedded-tests/src/test/resources/data/json/wikipedia_1.json create mode 100644 embedded-tests/src/test/resources/data/json/wikipedia_2.json create mode 100644 embedded-tests/src/test/resources/data/json/wikipedia_3.json diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java new file mode 100644 index 000000000000..55e2a8cfb361 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java @@ -0,0 +1,283 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.java.util.common.ISE; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Builder for the raw Map-based payload of a {@code Task}. + *

+ * The builder does not use any defaults and all required fields must be set + * explicitly. + * + * @see #ofType(String) to create a builder + */ +public class CreateTask +{ + private final String type; + + private String dataSource; + private Map inputSource = null; + private Map inputFormat = null; + private Map partitionsSpec = null; + private Map granularitySpec = null; + private Map timestampSpec = null; + private Map dimensionsSpec = null; + private Map splitHintSpec = null; + + private final List> metricsSpec = new ArrayList<>(); + + private Integer maxNumConcurrentSubTasks = null; + private Boolean forceGuaranteedRollup = null; + private Long awaitSegmentAvailabilityTimeoutMillis = null; + + private CreateTask(String type) + { + this.type = type; + } + + /** + * Initializes builder for a new {@code Task} for the given datasource. + */ + public static CreateTask ofType(String type) + { + return new CreateTask(type); + } + + public CreateTask dataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Object build(String taskId) + { + return mapOf( + "id", taskId, + "type", type, + "spec", + mapOf( + "ioConfig", + mapOf( + "type", type, + "inputSource", inputSource, + "inputFormat", inputFormat + ), + "tuningConfig", + mapOf( + "type", type, + "partitionsSpec", partitionsSpec, + "maxNumConcurrentSubTasks", maxNumConcurrentSubTasks, + "forceGuaranteedRollup", forceGuaranteedRollup, + "splitHintSpec", splitHintSpec, + "awaitSegmentAvailabilityTimeoutMillis", awaitSegmentAvailabilityTimeoutMillis + ), + "dataSchema", + mapOf( + "dataSource", dataSource, + "timestampSpec", timestampSpec, + "dimensionsSpec", dimensionsSpec, + "metricsSpec", metricsSpec, + "granularitySpec", granularitySpec + ) + ) + ); + } + + public CreateTask inputSource(Map jsonMap) + { + this.inputSource = jsonMap; + return this; + } + + public CreateTask inlineInputSourceWithData(String data) + { + return inputSource(Map.of("type", "inline", "data", data)); + } + + public CreateTask localInputSourceFromDirWithFilter(String directory, String filter) + { + return inputSource(Map.of("type", "local", "baseDir", directory, "filter", filter)); + } + + public CreateTask druidInputSource(String dataSource, Interval interval) + { + return inputSource(Map.of("type", "druid", "interval", interval, "dataSource", dataSource)); + } + + /** + * Gets the absolute path of the given resource files and sets: + *

+   * "inputSource": {
+   *   "type": "local",
+   *   "files": [<absolute-paths-of-given-resource-files>]
+   * }
+   * 
+ */ + public CreateTask localInputSourceWithFiles(String... files) + { + try { + final List filePaths = new ArrayList<>(); + for (String file : files) { + final URL resourceUrl = getClass().getClassLoader().getResource(file); + if (resourceUrl == null) { + throw new ISE("Could not find file[%s]", file); + } + + filePaths.add(new File(resourceUrl.toURI()).getAbsolutePath()); + } + + return inputSource(Map.of("type", "local", "files", filePaths)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public CreateTask inputFormat(Map jsonMap) + { + this.inputFormat = jsonMap; + return this; + } + + public CreateTask csvInputFormatWithColumns(String... columns) + { + return inputFormat( + Map.of("type", "csv", "findColumnsFromHeader", "false", "columns", List.of(columns)) + ); + } + + public CreateTask partitionsSpec(Map jsonMap) + { + this.partitionsSpec = jsonMap; + return this; + } + + public CreateTask dynamicPartitionsWithMaxRows(int maxRowsPerSegment) + { + return partitionsSpec(Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment)); + } + + public CreateTask granularitySpec(Map jsonMap) + { + this.granularitySpec = jsonMap; + return this; + } + + /** + * Sets {@code "granularitySpec": {"segmentGranularity": }}. + */ + public CreateTask segmentGranularity(String granularity) + { + return granularitySpec(Map.of("segmentGranularity", granularity)); + } + + public CreateTask timestampSpec(Map jsonMap) + { + this.timestampSpec = jsonMap; + return this; + } + + public CreateTask isoTimestampColumn(String timestampColumn) + { + return timestampSpec(Map.of("format", "iso", "column", timestampColumn)); + } + + public CreateTask timestampColumn(String timestampColumn) + { + return timestampSpec(Map.of("column", timestampColumn)); + } + + public CreateTask dimensionsSpec(Map jsonMap) + { + this.dimensionsSpec = jsonMap; + return this; + } + + /** + * Sets {@code "dimensionSpec": {"dimensions": []}}. + */ + public CreateTask dimensions(String... dimensions) + { + return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); + } + + public CreateTask metricAggregate(String column, String type) + { + this.metricsSpec.add(mapOf("type", type, "name", column, "fieldName", column)); + return this; + } + + public CreateTask maxConcurrentSubTasks(int maxNumConcurrentSubTasks) + { + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + return this; + } + + public CreateTask forceGuaranteedRollup(boolean rollup) + { + this.forceGuaranteedRollup = rollup; + return this; + } + + public CreateTask splitHintSpec(Map jsonMap) + { + this.splitHintSpec = jsonMap; + return this; + } + + public CreateTask awaitSegmentAvailabilityTimeoutMillis(long millis) + { + this.awaitSegmentAvailabilityTimeoutMillis = millis; + return this; + } + + /** + * Creates a map using only the non-null key-value pairs. + * + * @param kvPairs key1, value1, key2, value2, ... + * @return null if none of the key-value pairs are non-null. + */ + @Nullable + private static Map mapOf(Object... kvPairs) + { + if (kvPairs.length % 2 > 0) { + throw new ISE("Key and value must be given in pairs."); + } + + final Map map = new HashMap<>(); + for (int i = 0; i < kvPairs.length - 1; i += 2) { + if (kvPairs[i] != null && kvPairs[i + 1] != null) { + map.put(kvPairs[i], kvPairs[i + 1]); + } + } + + return map.isEmpty() ? null : Map.copyOf(map); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java new file mode 100644 index 000000000000..1a42a35636f3 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -0,0 +1,228 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexer.report.IngestionStatsAndErrors; +import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * Makes assertions similar to {@code ITPerfectRollupParallelIndexTest}. + */ +public class EmbeddedIndexParallelTaskTest extends EmbeddedClusterTestBase +{ + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer() + .addProperty("druid.worker.capacity", "20") + .addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.indexer.task.ignoreTimestampSpecForDruidInputSource", "true"); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addServer(coordinator) + .addServer(indexer) + .addServer(overlord) + .addServer(historical) + .addServer(broker) + .addServer(new EmbeddedRouter()); + } + + public static List> getTestParamPartitionsSpec() + { + return List.of( + Map.of("type", "dynamic"), + Map.of("type", "hashed", "numShards", 2), + Map.of("type", "range", "targetRowsPerSegment", 2, "partitionDimensions", List.of("namespace")) + ); + } + + @Test + public void test_theNewSyntax() + { + final String taskId = dataSource + "_" + IdUtils.getRandomId(); + Object task = CreateTask.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .segmentGranularity("DAY") + .dynamicPartitionsWithMaxRows(1000) + .build(taskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + } + + @CsvSource({"true, 5000", "false, 1"}) + @ParameterizedTest(name = "isAvailabilityConfirmed={0}, availabilityTimeout={1}") + public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( + final boolean isSegmentAvailabilityConfirmed, + final long segmentAvailabilityTimeoutMillis + ) + { + final CreateTask indexTask = + CreateTask.ofType("index_parallel") + .dataSource(dataSource) + .partitionsSpec(Map.of("type", "dynamic")) + .timestampColumn("timestamp") + .inputFormat(Map.of("type", "json")) + .localInputSourceWithFiles(Resources.WIKIPEDIA_1_JSON) + .dimensions() + .awaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis); + + final String taskId = runTask(indexTask, dataSource); + + // Get the task report to verify that segment availability has been confirmed + final TaskReport.ReportMap taskReport = cluster.callApi().onLeaderOverlord( + o -> o.taskReportAsMap(taskId) + ); + final Optional statsReportOptional + = taskReport.findReport("ingestionStatsAndErrors"); + Assertions.assertTrue(statsReportOptional.isPresent()); + + final IngestionStatsAndErrors statsAndErrors = statsReportOptional.get().getPayload(); + Assertions.assertEquals( + isSegmentAvailabilityConfirmed, + statsAndErrors.isSegmentAvailabilityConfirmed() + ); + } + + @MethodSource("getTestParamPartitionsSpec") + @ParameterizedTest(name = "partitionsSpec={0}") + public void test_runIndexTask_andReindexIntoAnotherDatasource(Map partitionsSpec) + { + final boolean isRollup = !partitionsSpec.get("type").equals("dynamic"); + + final CreateTask indexTask = + CreateTask.ofType("index_parallel") + .dataSource(dataSource) + .partitionsSpec(partitionsSpec) + .forceGuaranteedRollup(isRollup) + .timestampColumn("timestamp") + .inputFormat(Map.of("type", "json")) + .localInputSourceWithFiles( + Resources.WIKIPEDIA_1_JSON, + Resources.WIKIPEDIA_2_JSON, + Resources.WIKIPEDIA_3_JSON + ) + .segmentGranularity("DAY") + .dimensions("namespace", "page", "language") + .metricAggregate("added", "doubleSum") + .metricAggregate("deleted", "doubleSum") + .metricAggregate("delta", "doubleSum") + .metricAggregate("count", "count") + .maxConcurrentSubTasks(10) + .splitHintSpec(Map.of("type", "maxSize", "maxSplitSize", 1)); + + runTask(indexTask, dataSource); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); + runQueries(dataSource); + + // Re-index into a different datasource, indexing 1 segment per sub-task + final String dataSource2 = EmbeddedClusterApis.createTestDatasourceName(); + final CreateTask reindexTaskSplitBySegment = + CreateTask.ofType("index_parallel") + .dataSource(dataSource2) + .dimensionsSpec(Map.of("dimensionExclusions", List.of("robot", "continent"))) + .isoTimestampColumn("ignored") + .druidInputSource(dataSource, Intervals.ETERNITY) + .maxConcurrentSubTasks(10) + .forceGuaranteedRollup(isRollup) + .partitionsSpec(partitionsSpec) + .splitHintSpec(Map.of("type", "segments", "maxInputSegmentBytesPerTask", 1)); + + runTask(reindexTaskSplitBySegment, dataSource2); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource2, coordinator); + runQueries(dataSource2); + + // Re-index into a different datasource, indexing 1 file per sub-task + final String dataSource3 = EmbeddedClusterApis.createTestDatasourceName(); + final CreateTask reindexTaskSplitByFile = + CreateTask.ofType("index_parallel") + .dataSource(dataSource3) + .partitionsSpec(partitionsSpec) + .forceGuaranteedRollup(isRollup) + .timestampColumn("timestamp") + .druidInputSource(dataSource, Intervals.ETERNITY) + .dimensionsSpec(Map.of("dimensionExclusions", List.of("robot", "continent"))) + .maxConcurrentSubTasks(10) + .splitHintSpec(Map.of("type", "maxSize", "maxNumFiles", 1)); + + runTask(reindexTaskSplitByFile, dataSource3); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource3, coordinator); + runQueries(dataSource3); + } + + private String runTask(CreateTask taskBuilder, String dataSource) + { + final String taskId = dataSource + "_" + IdUtils.getRandomId(); + final Object taskPayload = taskBuilder.build(taskId); + + System.out.println("Running task: " + taskPayload); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskPayload)); + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + + return taskId; + } + + private void runQueries(String dataSource) + { + Assertions.assertEquals( + "10,2013-09-01T12:41:27.000Z,2013-08-31T01:02:33.000Z", + cluster.runSql("SELECT COUNT(*), MAX(__time), MIN(__time) FROM %s", dataSource) + ); + Assertions.assertEquals( + "Crimson Typhoon,1,905.0,9050.0", + cluster.runSql( + "SELECT \"page\", COUNT(*) AS \"rows\", SUM(\"added\"), 10 * SUM(\"added\") AS added_times_ten" + + " FROM %s" + + " WHERE \"language\" = 'zh' AND __time < '2013-09-01'" + + " GROUP BY 1" + + " HAVING added_times_ten > 9000", + dataSource + ) + ); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java index ba3c16b81b56..2fdbf90fc7b3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.DruidMetrics; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; @@ -78,7 +77,7 @@ public void test_runIndexTask_forInlineDatasource() final String taskId = IdUtils.getRandomId(); final Object task = createIndexTaskForInlineData( taskId, - StringUtils.replace(Resources.CSV_DATA_10_DAYS, "\n", "\\n") + Resources.CSV_DATA_10_DAYS ); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); @@ -101,12 +100,7 @@ public void test_runIndexTask_forInlineDatasource() start = start.plusDays(1); } - // Wait for all segments to be loaded and queryable - coordinator.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("segment/loadQueue/success") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(10) - ); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); broker.latchableEmitter().waitForEvent( event -> event.hasDimension(DruidMetrics.DATASOURCE, dataSource) ); @@ -146,10 +140,14 @@ public void test_run100Tasks_concurrently() private Object createIndexTaskForInlineData(String taskId, String inlineDataCsv) { - return EmbeddedClusterApis.createTaskFromPayload( - taskId, - StringUtils.format(Resources.INDEX_TASK_PAYLOAD_WITH_INLINE_DATA, inlineDataCsv, dataSource) - ); + return CreateTask.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(inlineDataCsv) + .segmentGranularity("DAY") + .dimensions() + .build(taskId); } /** diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java index 3e4a7e1f6236..90da3e7a175d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java @@ -24,6 +24,9 @@ */ public class Resources { + /** + * 10 rows (1 row per day) of inline CSV data with 3 columns (time, item, value). + */ public static final String CSV_DATA_10_DAYS = "2025-06-01T00:00:00.000Z,shirt,105" + "\n2025-06-02T00:00:00.000Z,trousers,210" @@ -36,51 +39,7 @@ public class Resources + "\n2025-06-09T00:00:00.000Z,shirt,99" + "\n2025-06-10T00:00:00.000Z,toys,101"; - /** - * Full task payload for an "index" task. The payload has the following format - * arguments: - *
    - *
  1. Data which can be provided as a single CSV string. The data is expected - * to have 3 columns: "time", "item" and "value". (e.g {@link #CSV_DATA_10_DAYS})
  2. - *
  3. Datasource name
  4. - *
- */ - public static final String INDEX_TASK_PAYLOAD_WITH_INLINE_DATA - = "{" - + " \"type\": \"index\"," - + " \"spec\": {" - + " \"ioConfig\": {" - + " \"type\": \"index\"," - + " \"inputSource\": {" - + " \"type\": \"inline\"," - + " \"data\": \"%s\"" - + " },\n" - + " \"inputFormat\": {" - + " \"type\": \"csv\"," - + " \"findColumnsFromHeader\": false," - + " \"columns\": [\"time\",\"item\",\"value\"]" - + " }" - + " }," - + " \"tuningConfig\": {" - + " \"type\": \"index_parallel\"," - + " \"partitionsSpec\": {" - + " \"type\": \"dynamic\"," - + " \"maxRowsPerSegment\": 1000" - + " }" - + " }," - + " \"dataSchema\": {" - + " \"dataSource\": \"%s\"," - + " \"timestampSpec\": {" - + " \"column\": \"time\"," - + " \"format\": \"iso\"" - + " }," - + " \"dimensionsSpec\": {" - + " \"dimensions\": []" - + " }," - + " \"granularitySpec\": {" - + " \"segmentGranularity\": \"DAY\"" - + " }" - + " }" - + " }" - + "}"; + public static final String WIKIPEDIA_1_JSON = "data/json/wikipedia_1.json"; + public static final String WIKIPEDIA_2_JSON = "data/json/wikipedia_2.json"; + public static final String WIKIPEDIA_3_JSON = "data/json/wikipedia_3.json"; } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index 2f5a0cfb173f..63dd3f05738a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -32,13 +32,13 @@ import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedDruidServer; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.CreateTask; import org.apache.druid.testing.embedded.indexing.Resources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -106,13 +106,15 @@ public void test_switchLeader_andVerifyUsingSysTables() { // Ingest some data so that we can query sys tables later final String taskId = dataSource + "_" + IdUtils.getRandomId(); - final String taskPayload = StringUtils.format( - Resources.INDEX_TASK_PAYLOAD_WITH_INLINE_DATA, - StringUtils.replace(Resources.CSV_DATA_10_DAYS, "\n", "\\n"), - dataSource - ); + final Object taskPayload = CreateTask.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .dimensions() + .build(taskId); cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, EmbeddedClusterApis.createTaskFromPayload(taskId, taskPayload)) + o -> o.runTask(taskId, taskPayload) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_1.json b/embedded-tests/src/test/resources/data/json/wikipedia_1.json new file mode 100644 index 000000000000..30fef106aafb --- /dev/null +++ b/embedded-tests/src/test/resources/data/json/wikipedia_1.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "tags": ["t1", "t2"], "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "tags": ["t1", "t2"], "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "tags": ["t3", "t4", "t5"], "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} \ No newline at end of file diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_2.json b/embedded-tests/src/test/resources/data/json/wikipedia_2.json new file mode 100644 index 000000000000..d13b6a37bbcc --- /dev/null +++ b/embedded-tests/src/test/resources/data/json/wikipedia_2.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "tags": ["t6"], "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "tags": ["t1", "t2"], "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} \ No newline at end of file diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_3.json b/embedded-tests/src/test/resources/data/json/wikipedia_3.json new file mode 100644 index 000000000000..a8f5c2ec292e --- /dev/null +++ b/embedded-tests/src/test/resources/data/json/wikipedia_3.json @@ -0,0 +1,4 @@ +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "tags": ["t1", "t2"], "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "tags": ["t3", "t4", "t5"], "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "tags": ["t6"], "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/indexer/report/IngestionStatsAndErrors.java b/processing/src/main/java/org/apache/druid/indexer/report/IngestionStatsAndErrors.java index fe6566947889..f766d0a324cb 100644 --- a/processing/src/main/java/org/apache/druid/indexer/report/IngestionStatsAndErrors.java +++ b/processing/src/main/java/org/apache/druid/indexer/report/IngestionStatsAndErrors.java @@ -175,7 +175,7 @@ public String toString() ", unparseableEvents=" + unparseableEvents + ", rowStats=" + rowStats + ", errorMsg='" + errorMsg + '\'' + - ", segmentAvailabilityConfoirmed=" + segmentAvailabilityConfirmed + + ", segmentAvailabilityConfirmed=" + segmentAvailabilityConfirmed + ", segmentAvailabilityWaitTimeMs=" + segmentAvailabilityWaitTimeMs + ", recordsProcessed=" + recordsProcessed + ", segmentsRead=" + segmentsRead + diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 89267e6bb61c..1ec6c87d8023 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -28,6 +28,7 @@ import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.task.TaskMetrics; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -111,6 +112,24 @@ public void waitForTaskToSucceed(String taskId, EmbeddedOverlord overlord) verifyTaskHasStatus(taskId, TaskStatus.success(taskId)); } + /** + * Waits for all used segments (including overshadowed) of the given datasource + * to be loaded on historicals. + */ + public void waitForAllSegmentsToBeAvailable(String dataSource, EmbeddedCoordinator coordinator) + { + final int numSegments = coordinator + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.INCLUDING_OVERSHADOWED) + .size(); + coordinator.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("segment/loadQueue/success") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(numSegments) + ); + } + /** * Fetches the status of the given task from the cluster and verifies that it * matches the expected status. From 38c87ef1b956f96f7c479cdad3f11474d52dec0c Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 20:29:04 +0530 Subject: [PATCH 02/38] Use deepstore for intermediary storage --- .../EmbeddedIndexParallelTaskTest.java | 23 ++++--------------- .../server/EmbeddedHighAvailabilityTest.java | 16 ++++++------- 2 files changed, 12 insertions(+), 27 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index 1a42a35636f3..a2cabd4e425a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -34,7 +34,6 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; @@ -44,7 +43,8 @@ import java.util.Optional; /** - * Makes assertions similar to {@code ITPerfectRollupParallelIndexTest}. + * Makes assertions similar to {@code ITPerfectRollupParallelIndexTest} and + * {@code ITBestEffortRollupParallelIndexTest}. */ public class EmbeddedIndexParallelTaskTest extends EmbeddedClusterTestBase { @@ -52,7 +52,8 @@ public class EmbeddedIndexParallelTaskTest extends EmbeddedClusterTestBase private final EmbeddedIndexer indexer = new EmbeddedIndexer() .addProperty("druid.worker.capacity", "20") .addProperty("druid.segment.handoff.pollDuration", "PT0.1s") - .addProperty("druid.indexer.task.ignoreTimestampSpecForDruidInputSource", "true"); + .addProperty("druid.indexer.task.ignoreTimestampSpecForDruidInputSource", "true") + .addProperty("druid.processing.intermediaryData.storage.type", "deepstore"); private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedHistorical historical = new EmbeddedHistorical(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); @@ -79,22 +80,6 @@ public static List> getTestParamPartitionsSpec() ); } - @Test - public void test_theNewSyntax() - { - final String taskId = dataSource + "_" + IdUtils.getRandomId(); - Object task = CreateTask.ofType("index") - .dataSource(dataSource) - .csvInputFormatWithColumns("time", "item", "value") - .isoTimestampColumn("time") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) - .segmentGranularity("DAY") - .dynamicPartitionsWithMaxRows(1000) - .build(taskId); - cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); - cluster.callApi().waitForTaskToSucceed(taskId, overlord); - } - @CsvSource({"true, 5000", "false, 1"}) @ParameterizedTest(name = "isAvailabilityConfirmed={0}, availabilityTimeout={1}") public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index 63dd3f05738a..c9965b7c0a8a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -106,15 +106,15 @@ public void test_switchLeader_andVerifyUsingSysTables() { // Ingest some data so that we can query sys tables later final String taskId = dataSource + "_" + IdUtils.getRandomId(); - final Object taskPayload = CreateTask.ofType("index") - .dataSource(dataSource) - .csvInputFormatWithColumns("time", "item", "value") - .isoTimestampColumn("time") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) - .dimensions() - .build(taskId); + final CreateTask taskPayload = + CreateTask.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .dimensions(); cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskPayload) + o -> o.runTask(taskId, taskPayload.build(taskId)) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); From 52b2e71983d9c5d149ea20d5edcb4cd11f2931e7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 21:12:25 +0530 Subject: [PATCH 03/38] Remove old tests --- .github/workflows/cron-job-its.yml | 6 +- .github/workflows/standard-its.yml | 22 +- .../EmbeddedIndexParallelTaskTest.java | 6 +- .../ITBestEffortRollupParallelIndexTest.java | 250 ----------------- .../org/apache/druid/tests/TestNGGroup.java | 4 - .../ITBestEffortRollupParallelIndexTest.java | 261 ------------------ .../ITPerfectRollupParallelIndexTest.java | 129 --------- 7 files changed, 8 insertions(+), 670 deletions(-) delete mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index 6084577b9d22..101639becf77 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -60,7 +60,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction] + testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction] uses: ./.github/workflows/reusable-standard-its.yml needs: build with: @@ -74,7 +74,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [ input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion, compaction ] + testing_group: [ input-source, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion, compaction ] uses: ./.github/workflows/reusable-standard-its.yml needs: build with: @@ -109,7 +109,7 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,custom-coordinator-duties use_indexer: ${{ matrix.indexer }} group: other diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index adf5190177a2..df1c3323e74d 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -47,7 +47,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-metadata-query-disabled] + testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-metadata-query-disabled] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -63,7 +63,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [input-source, perfect-rollup-parallel-batch-index, kafka-index, append-ingestion, compaction] + testing_group: [input-source, kafka-index, append-ingestion, compaction] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -102,22 +102,6 @@ jobs: override_config_path: ./environment-configs/test-groups/prepopulated-data group: query - integration-shuffle-deep-store-tests: - needs: changes - strategy: - fail-fast: false - matrix: - indexer: [indexer, middleManager] - uses: ./.github/workflows/reusable-standard-its.yml - if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} - with: - build_jdk: 17 - runtime_jdk: 17 - testing_groups: -Dgroups=shuffle-deep-store - use_indexer: ${{ matrix.indexer }} - override_config_path: ./environment-configs/test-groups/shuffle-deep-store - group: shuffle deep store - integration-custom-coordinator-duties-tests: needs: changes uses: ./.github/workflows/reusable-standard-its.yml @@ -196,6 +180,6 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled use_indexer: ${{ matrix.indexer }} group: other diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index a2cabd4e425a..81061f6357fb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -43,8 +43,7 @@ import java.util.Optional; /** - * Makes assertions similar to {@code ITPerfectRollupParallelIndexTest} and - * {@code ITBestEffortRollupParallelIndexTest}. + * Runs tasks of "index_parallel" type using Indexers. */ public class EmbeddedIndexParallelTaskTest extends EmbeddedClusterTestBase { @@ -80,7 +79,7 @@ public static List> getTestParamPartitionsSpec() ); } - @CsvSource({"true, 5000", "false, 1"}) + @CsvSource({"true, 5000", "false, 1", "false, 0"}) @ParameterizedTest(name = "isAvailabilityConfirmed={0}, availabilityTimeout={1}") public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( final boolean isSegmentAvailabilityConfirmed, @@ -185,7 +184,6 @@ private String runTask(CreateTask taskBuilder, String dataSource) final String taskId = dataSource + "_" + IdUtils.getRandomId(); final Object taskPayload = taskBuilder.build(taskId); - System.out.println("Running task: " + taskPayload); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskPayload)); cluster.callApi().waitForTaskToSucceed(taskId, overlord); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java deleted file mode 100644 index 20bdd9767a77..000000000000 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java +++ /dev/null @@ -1,250 +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.testsEx.indexer; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.inject.Inject; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; -import org.apache.druid.testing.clients.CoordinatorResourceTestClient; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.testsEx.categories.BatchIndex; -import org.apache.druid.testsEx.config.DruidTestRunner; -import org.junit.Assert; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import java.io.Closeable; -import java.util.function.Function; - -@RunWith(DruidTestRunner.class) -@Category(BatchIndex.class) -public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest -{ - // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle - // maxSplitSize of 1 properly. - private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; - private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json"; - private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_reindex_queries.json"; - private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; - private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test"; - private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json"; - private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test"; - private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json"; - - private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = - CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); - private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = - CoordinatorDynamicConfig.builder().build(); - - @Inject - CoordinatorResourceTestClient coordinatorClient; - - @Test - public void testIndexData() throws Exception - { - PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix()) - ) { - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - - // Index again, this time only choosing the second data file, and without explicit intervals chosen. - // The second datafile covers both day segments, so this should replace them, as reflected in the queries. - doIndexTest( - INDEX_DATASOURCE, - REINDEX_TASK, - rollupTransform, - REINDEX_QUERIES_RESOURCE, - true, - true, - true, - new Pair<>(false, false) - ); - - doReindexTest( - INDEX_DATASOURCE, - INDEX_INGEST_SEGMENT_DATASOURCE, - rollupTransform, - INDEX_INGEST_SEGMENT_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - - // with DruidInputSource - doReindexTest( - INDEX_DATASOURCE, - INDEX_DRUID_INPUT_SOURCE_DATASOURCE, - rollupTransform, - INDEX_DRUID_INPUT_SOURCE_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - } - } - - /** - * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. This will confirm that the report for the task - * indicates segments were confirmed to be available on the cluster before finishing the ingestion job. - */ - @Test - public void testIndexDataVerifySegmentAvailability() throws Exception - { - PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("600000") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(true, true) - ); - } - } - - /** - * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. Setting the config value to 1 millis - * and pausing coordination to confirm that the task will still succeed even if the job was not able to confirm the - * segments were loaded by the time the timeout occurs. - */ - @Test - public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception - { - PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("1") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - false, - false, - new Pair<>(true, false) - ); - coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" - ); - } - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index 9ba40a8d1d89..b9629091f621 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -47,8 +47,6 @@ public class TestNGGroup public static final String APPEND_INGESTION = "append-ingestion"; - public static final String PERFECT_ROLLUP_PARALLEL_BATCH_INDEX = "perfect-rollup-parallel-batch-index"; - /** * This group can only be run individually using -Dgroups=query since it requires specific test data setup. */ @@ -152,8 +150,6 @@ public class TestNGGroup public static final String HIGH_AVAILABILTY = "high-availability"; - public static final String SHUFFLE_DEEP_STORE = "shuffle-deep-store"; - public static final String CUSTOM_COORDINATOR_DUTIES = "custom-coordinator-duties"; public static final String HTTP_ENDPOINT = "http-endpoint"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java deleted file mode 100644 index 53deb831a2db..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java +++ /dev/null @@ -1,261 +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.tests.indexer; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.inject.Inject; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; -import org.apache.druid.testing.clients.CoordinatorResourceTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.testng.Assert; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.util.function.Function; - -@Test(groups = {TestNGGroup.BATCH_INDEX}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest -{ - // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle - // maxSplitSize of 1 properly. - private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; - private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json"; - private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_reindex_queries.json"; - private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; - private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test"; - private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json"; - private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test"; - private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json"; - - private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = - CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); - private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = - CoordinatorDynamicConfig.builder().build(); - - @Inject - CoordinatorResourceTestClient coordinatorClient; - - @DataProvider - public static Object[][] resources() - { - return new Object[][]{ - {new DynamicPartitionsSpec(null, null)} - }; - } - - @Test(dataProvider = "resources") - public void testIndexData(PartitionsSpec partitionsSpec) throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix()) - ) { - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse(forceGuaranteedRollup, "parititionSpec does not support best-effort rollup"); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - - // Index again, this time only choosing the second data file, and without explicit intervals chosen. - // The second datafile covers both day segments, so this should replace them, as reflected in the queries. - doIndexTest( - INDEX_DATASOURCE, - REINDEX_TASK, - rollupTransform, - REINDEX_QUERIES_RESOURCE, - true, - true, - true, - new Pair<>(false, false) - ); - - doReindexTest( - INDEX_DATASOURCE, - INDEX_INGEST_SEGMENT_DATASOURCE, - rollupTransform, - INDEX_INGEST_SEGMENT_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - - // with DruidInputSource - doReindexTest( - INDEX_DATASOURCE, - INDEX_DRUID_INPUT_SOURCE_DATASOURCE, - rollupTransform, - INDEX_DRUID_INPUT_SOURCE_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - } - } - - /** - * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. This will confirm that the report for the task - * indicates segments were confirmed to be available on the cluster before finishing the ingestion job. - * - * @param partitionsSpec - * @throws Exception - */ - @Test(dataProvider = "resources") - public void testIndexDataVerifySegmentAvailability(PartitionsSpec partitionsSpec) throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse(forceGuaranteedRollup, "parititionSpec does not support best-effort rollup"); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("600000") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(true, true) - ); - } - } - - /** - * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. Setting the config value to 1 millis - * and pausing coordination to confirm that the task will still succeed even if the job was not able to confirm the - * segments were loaded by the time the timeout occurs. - * - * @param partitionsSpec - * @throws Exception - */ - @Test(dataProvider = "resources") - public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds(PartitionsSpec partitionsSpec) throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertFalse(forceGuaranteedRollup, "parititionSpec does not support best-effort rollup"); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(false) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("1") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - false, - false, - new Pair<>(true, false) - ); - coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" - ); - } - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java deleted file mode 100644 index 78823477434b..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java +++ /dev/null @@ -1,129 +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.tests.indexer; - -import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.timeline.partition.HashPartitionFunction; -import org.testng.Assert; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.util.function.Function; - -@Test(groups = {TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX, TestNGGroup.SHUFFLE_DEEP_STORE}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest -{ - // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle - // maxSplitSize of 1 properly. - private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; - private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; - private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test"; - private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json"; - private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test"; - private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json"; - - @DataProvider - public static Object[][] resources() - { - return new Object[][]{ - {new HashedPartitionsSpec(null, 2, null, HashPartitionFunction.MURMUR3_32_ABS)}, - {new HashedPartitionsSpec(null, 2, null, null)}, - {new SingleDimensionPartitionsSpec(2, null, "namespace", false)} - }; - } - - @Test(dataProvider = "resources") - public void testIndexData(PartitionsSpec partitionsSpec) throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix()) - ) { - boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - Assert.assertTrue(forceGuaranteedRollup, "parititionSpec does not support perfect rollup"); - - final Function rollupTransform = spec -> { - try { - spec = StringUtils.replace( - spec, - "%%FORCE_GUARANTEED_ROLLUP%%", - Boolean.toString(true) - ); - spec = StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - return StringUtils.replace( - spec, - "%%PARTITIONS_SPEC%%", - jsonMapper.writeValueAsString(partitionsSpec) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - rollupTransform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - - doReindexTest( - INDEX_DATASOURCE, - INDEX_INGEST_SEGMENT_DATASOURCE, - rollupTransform, - INDEX_INGEST_SEGMENT_TASK, - INDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - - ); - - // with DruidInputSource - doReindexTest( - INDEX_DATASOURCE, - INDEX_DRUID_INPUT_SOURCE_DATASOURCE, - rollupTransform, - INDEX_DRUID_INPUT_SOURCE_TASK, - INDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - } - } -} From c43205c007b184c02e3378af8ae4310a424dc544 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 21:17:44 +0530 Subject: [PATCH 04/38] Remove 2 unused ITs --- .../org/apache/druid/tests/TestNGGroup.java | 2 - .../api/ITOverlordResourceNotFoundTest.java | 141 ------------------ .../tests/api/ITOverlordResourceTest.java | 72 --------- .../api/overlord-resource-test-task.json | 87 ----------- 4 files changed, 302 deletions(-) delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceNotFoundTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceTest.java delete mode 100644 integration-tests/src/test/resources/api/overlord-resource-test-task.json diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index b9629091f621..9bf79a185e82 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -152,8 +152,6 @@ public class TestNGGroup public static final String CUSTOM_COORDINATOR_DUTIES = "custom-coordinator-duties"; - public static final String HTTP_ENDPOINT = "http-endpoint"; - public static final String CENTRALIZED_DATASOURCE_SCHEMA = "centralized-datasource-schema"; public static final String CDS_TASK_SCHEMA_PUBLISH_DISABLED = "cds-task-schema-publish-disabled"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceNotFoundTest.java b/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceNotFoundTest.java deleted file mode 100644 index 8959c7744fe0..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceNotFoundTest.java +++ /dev/null @@ -1,141 +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.tests.api; - -import com.google.inject.Inject; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.testing.clients.OverlordResourceTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.testng.Assert; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.util.function.Consumer; - -@Test(groups = TestNGGroup.HTTP_ENDPOINT) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITOverlordResourceNotFoundTest -{ - @Inject - protected OverlordResourceTestClient indexer; - - @Test - public void testGetSupervisorStatusNotFound() - { - callAndCheckNotFound(indexer::getSupervisorStatus); - } - - @Test - public void testGetSupervisorHistoryNotFound() - { - callAndCheckNotFound(indexer::getSupervisorHistory); - } - - @Test - public void testResumeSupervisorNotFound() - { - callAndCheckNotFound(indexer::resumeSupervisor); - } - - @Test - public void testSuspendSupervisorNotFound() - { - callAndCheckNotFound(indexer::suspendSupervisor); - } - - @Test - public void testShutdownSupervisorNotFound() - { - callAndCheckNotFound(indexer::shutdownSupervisor); - } - - @Test - public void testTerminateSupervisorNotFound() - { - callAndCheckNotFound(indexer::terminateSupervisor); - } - - @Test - public void testGetSupervisorHealthNotFound() - { - callAndCheckNotFound(indexer::getSupervisorHealth); - } - - @Test - public void testStatsSupervisorNotFound() - { - callAndCheckNotFound(indexer::statsSupervisor); - } - - @Test - public void testResetSupervisorNotFound() - { - callAndCheckNotFound(indexer::resetSupervisor); - } - - @Test - public void testGetTaskStatusNotFound() - { - callAndCheckNotFound(indexer::getTaskStatus); - } - - @Test - public void testShutdownTaskNotFound() - { - callAndCheckNotFound(indexer::shutdownTask); - } - - @Test - public void testGetTaskLogNotFound() - { - callAndCheckNotFound(indexer::getTaskLog); - } - - @Test - public void testGetTaskReportNotFound() - { - callAndCheckNotFound(indexer::getTaskReport); - } - - @Test - public void testGetTaskPayLoadNotFound() - { - callAndCheckNotFound(indexer::getTaskPayload); - } - - private void callAndCheckNotFound(Consumer runnable) - { - String supervisorId = "not_exist_id"; - try { - runnable.accept(supervisorId); - } - catch (ISE e) { - // OverlordResourceTestClient turns all non-200 response into ISE exception - // So we catch ISE and check if the message in this exception matches expected message - Assert.assertTrue( - e.getMessage().contains("[404 Not Found") && e.getMessage().contains(supervisorId), - "Unexpected exception. Message does not match expected. " + e.getMessage() - ); - return; - } - Assert.fail("Should not go to here"); - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceTest.java deleted file mode 100644 index eb59afee0f76..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/api/ITOverlordResourceTest.java +++ /dev/null @@ -1,72 +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.tests.api; - -import com.google.inject.Inject; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.testing.clients.OverlordResourceTestClient; -import org.apache.druid.testing.clients.TaskResponseObject; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractIndexerTest; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.IOException; -import java.util.List; - -@Test(groups = TestNGGroup.HTTP_ENDPOINT) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITOverlordResourceTest -{ - private static final String INGESTION_SPEC = "/api/overlord-resource-test-task.json"; - - @Inject - protected OverlordResourceTestClient indexer; - - @Test - public void testGetAllTasks() throws IOException - { - final String taskSpec = AbstractIndexerTest.getResourceAsString(INGESTION_SPEC); - final String taskId = indexer.submitTask(taskSpec); - - ITRetryUtil.retryUntil( - () -> { - final List tasks = indexer.getAllTasks(); - final TaskResponseObject taskStatus = tasks - .stream() - .filter(task -> taskId.equals(task.getId())) - .findAny() - .orElseThrow(() -> new ISE("Cannot find task[%s]", taskId)); - TaskState status = taskStatus.getStatus(); - if (status == TaskState.FAILED) { - throw new ISE("Task[%s] FAILED", taskId); - } - return status == TaskState.SUCCESS; - }, - true, - ITRetryUtil.DEFAULT_RETRY_SLEEP, - ITRetryUtil.DEFAULT_RETRY_COUNT, - taskId - ); - } -} diff --git a/integration-tests/src/test/resources/api/overlord-resource-test-task.json b/integration-tests/src/test/resources/api/overlord-resource-test-task.json deleted file mode 100644 index a3f99ad62543..000000000000 --- a/integration-tests/src/test/resources/api/overlord-resource-test-task.json +++ /dev/null @@ -1,87 +0,0 @@ -{ - "type": "index_parallel", - "spec": { - "dataSchema": { - "dataSource": "it-overlord-resource-test", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - }, - { - "name": "thetaSketch", - "type": "thetaSketch", - "fieldName": "user" - }, - { - "name": "quantilesDoublesSketch", - "type": "quantilesDoublesSketch", - "fieldName": "delta" - }, - { - "name": "HLLSketchBuild", - "type": "HLLSketchBuild", - "fieldName": "user" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-02" ] - } - }, - "ioConfig": { - "type": "index_parallel", - "inputSource": { - "type": "local", - "filter" : "*.json", - "baseDir": "/resources/data/batch_index/json" - }, - "inputFormat": { - "type": "json" - } - }, - "tuningConfig": { - "type": "index_parallel", - "maxNumConcurrentSubTasks": 1, - "splitHintSpec": { - "type": "maxSize", - "maxNumFiles": 1 - } - } - } -} \ No newline at end of file From 91f092ea7c13be60ce62967213b4a938181fc896 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 22:52:44 +0530 Subject: [PATCH 05/38] Fix deps --- embedded-tests/pom.xml | 5 + .../EmbeddedIndexParallelTaskTest.java | 96 +++++++++---------- .../indexing/EmbeddedIndexTaskTest.java | 16 ++-- .../{CreateTask.java => TaskPayload.java} | 72 +++++++------- .../server/EmbeddedHighAvailabilityTest.java | 18 ++-- 5 files changed, 104 insertions(+), 103 deletions(-) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/{CreateTask.java => TaskPayload.java} (75%) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 6458dc01dc45..68ab27b4635d 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -150,6 +150,11 @@ org.junit.jupiter junit-jupiter-api + + org.junit.jupiter + junit-jupiter-params + test + junit junit diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index 81061f6357fb..3878c05a3348 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -86,15 +86,15 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( final long segmentAvailabilityTimeoutMillis ) { - final CreateTask indexTask = - CreateTask.ofType("index_parallel") - .dataSource(dataSource) - .partitionsSpec(Map.of("type", "dynamic")) - .timestampColumn("timestamp") - .inputFormat(Map.of("type", "json")) - .localInputSourceWithFiles(Resources.WIKIPEDIA_1_JSON) - .dimensions() - .awaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis); + final TaskPayload indexTask = + TaskPayload.ofType("index_parallel") + .dataSource(dataSource) + .partitionsSpec(Map.of("type", "dynamic")) + .timestampColumn("timestamp") + .inputFormat(Map.of("type", "json")) + .localInputSourceWithFiles(Resources.WIKIPEDIA_1_JSON) + .dimensions() + .awaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis); final String taskId = runTask(indexTask, dataSource); @@ -119,26 +119,26 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(Map o.runTask(taskId, taskPayload)); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.withId(taskId))); cluster.callApi().waitForTaskToSucceed(taskId, overlord); return taskId; diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java index 2fdbf90fc7b3..910699c7213b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java @@ -140,14 +140,14 @@ public void test_run100Tasks_concurrently() private Object createIndexTaskForInlineData(String taskId, String inlineDataCsv) { - return CreateTask.ofType("index") - .dataSource(dataSource) - .csvInputFormatWithColumns("time", "item", "value") - .isoTimestampColumn("time") - .inlineInputSourceWithData(inlineDataCsv) - .segmentGranularity("DAY") - .dimensions() - .build(taskId); + return TaskPayload.ofType("index") + .dataSource(dataSource) + .isoTimestampColumn("time") + .csvInputFormatWithColumns("time", "item", "value") + .inlineInputSourceWithData(inlineDataCsv) + .segmentGranularity("DAY") + .dimensions() + .withId(taskId); } /** diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java similarity index 75% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java index 55e2a8cfb361..48b694af89a7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/CreateTask.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java @@ -21,8 +21,8 @@ import org.apache.druid.java.util.common.ISE; import org.joda.time.Interval; +import org.junit.platform.commons.util.Preconditions; -import javax.annotation.Nullable; import java.io.File; import java.net.URL; import java.util.ArrayList; @@ -38,7 +38,7 @@ * * @see #ofType(String) to create a builder */ -public class CreateTask +public class TaskPayload { private final String type; @@ -57,7 +57,7 @@ public class CreateTask private Boolean forceGuaranteedRollup = null; private Long awaitSegmentAvailabilityTimeoutMillis = null; - private CreateTask(String type) + private TaskPayload(String type) { this.type = type; } @@ -65,19 +65,28 @@ private CreateTask(String type) /** * Initializes builder for a new {@code Task} for the given datasource. */ - public static CreateTask ofType(String type) + public static TaskPayload ofType(String type) { - return new CreateTask(type); + return new TaskPayload(type); } - public CreateTask dataSource(String dataSource) + public TaskPayload dataSource(String dataSource) { this.dataSource = dataSource; return this; } - public Object build(String taskId) + /** + * Creates a raw Map-based payload for a {@code Task} that may be submitted to + * the Overlord using {@code OverlordClient.runTask()}. + */ + public Object withId(String taskId) { + Preconditions.notNull(taskId, "Task ID must not be null"); + Preconditions.notNull(type, "Task type must be specified"); + Preconditions.notNull(inputSource, "'inputSource' must be specified"); + Preconditions.notNull(dataSource, "'dataSource' must be specified"); + return mapOf( "id", taskId, "type", type, @@ -110,23 +119,18 @@ public Object build(String taskId) ); } - public CreateTask inputSource(Map jsonMap) + public TaskPayload inputSource(Map jsonMap) { this.inputSource = jsonMap; return this; } - public CreateTask inlineInputSourceWithData(String data) + public TaskPayload inlineInputSourceWithData(String data) { return inputSource(Map.of("type", "inline", "data", data)); } - public CreateTask localInputSourceFromDirWithFilter(String directory, String filter) - { - return inputSource(Map.of("type", "local", "baseDir", directory, "filter", filter)); - } - - public CreateTask druidInputSource(String dataSource, Interval interval) + public TaskPayload druidInputSource(String dataSource, Interval interval) { return inputSource(Map.of("type", "druid", "interval", interval, "dataSource", dataSource)); } @@ -140,7 +144,7 @@ public CreateTask druidInputSource(String dataSource, Interval interval) * } * */ - public CreateTask localInputSourceWithFiles(String... files) + public TaskPayload localInputSourceWithFiles(String... files) { try { final List filePaths = new ArrayList<>(); @@ -160,31 +164,26 @@ public CreateTask localInputSourceWithFiles(String... files) } } - public CreateTask inputFormat(Map jsonMap) + public TaskPayload inputFormat(Map jsonMap) { this.inputFormat = jsonMap; return this; } - public CreateTask csvInputFormatWithColumns(String... columns) + public TaskPayload csvInputFormatWithColumns(String... columns) { return inputFormat( Map.of("type", "csv", "findColumnsFromHeader", "false", "columns", List.of(columns)) ); } - public CreateTask partitionsSpec(Map jsonMap) + public TaskPayload partitionsSpec(Map jsonMap) { this.partitionsSpec = jsonMap; return this; } - public CreateTask dynamicPartitionsWithMaxRows(int maxRowsPerSegment) - { - return partitionsSpec(Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment)); - } - - public CreateTask granularitySpec(Map jsonMap) + public TaskPayload granularitySpec(Map jsonMap) { this.granularitySpec = jsonMap; return this; @@ -193,28 +192,28 @@ public CreateTask granularitySpec(Map jsonMap) /** * Sets {@code "granularitySpec": {"segmentGranularity": }}. */ - public CreateTask segmentGranularity(String granularity) + public TaskPayload segmentGranularity(String granularity) { return granularitySpec(Map.of("segmentGranularity", granularity)); } - public CreateTask timestampSpec(Map jsonMap) + public TaskPayload timestampSpec(Map jsonMap) { this.timestampSpec = jsonMap; return this; } - public CreateTask isoTimestampColumn(String timestampColumn) + public TaskPayload isoTimestampColumn(String timestampColumn) { return timestampSpec(Map.of("format", "iso", "column", timestampColumn)); } - public CreateTask timestampColumn(String timestampColumn) + public TaskPayload timestampColumn(String timestampColumn) { return timestampSpec(Map.of("column", timestampColumn)); } - public CreateTask dimensionsSpec(Map jsonMap) + public TaskPayload dimensionsSpec(Map jsonMap) { this.dimensionsSpec = jsonMap; return this; @@ -223,36 +222,36 @@ public CreateTask dimensionsSpec(Map jsonMap) /** * Sets {@code "dimensionSpec": {"dimensions": []}}. */ - public CreateTask dimensions(String... dimensions) + public TaskPayload dimensions(String... dimensions) { return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); } - public CreateTask metricAggregate(String column, String type) + public TaskPayload metricAggregate(String column, String type) { this.metricsSpec.add(mapOf("type", type, "name", column, "fieldName", column)); return this; } - public CreateTask maxConcurrentSubTasks(int maxNumConcurrentSubTasks) + public TaskPayload maxConcurrentSubTasks(int maxNumConcurrentSubTasks) { this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; return this; } - public CreateTask forceGuaranteedRollup(boolean rollup) + public TaskPayload forceGuaranteedRollup(boolean rollup) { this.forceGuaranteedRollup = rollup; return this; } - public CreateTask splitHintSpec(Map jsonMap) + public TaskPayload splitHintSpec(Map jsonMap) { this.splitHintSpec = jsonMap; return this; } - public CreateTask awaitSegmentAvailabilityTimeoutMillis(long millis) + public TaskPayload awaitSegmentAvailabilityTimeoutMillis(long millis) { this.awaitSegmentAvailabilityTimeoutMillis = millis; return this; @@ -264,7 +263,6 @@ public CreateTask awaitSegmentAvailabilityTimeoutMillis(long millis) * @param kvPairs key1, value1, key2, value2, ... * @return null if none of the key-value pairs are non-null. */ - @Nullable private static Map mapOf(Object... kvPairs) { if (kvPairs.length % 2 > 0) { diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index c9965b7c0a8a..451f4009b9b4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -38,8 +38,8 @@ import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; -import org.apache.druid.testing.embedded.indexing.CreateTask; import org.apache.druid.testing.embedded.indexing.Resources; +import org.apache.druid.testing.embedded.indexing.TaskPayload; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -106,15 +106,15 @@ public void test_switchLeader_andVerifyUsingSysTables() { // Ingest some data so that we can query sys tables later final String taskId = dataSource + "_" + IdUtils.getRandomId(); - final CreateTask taskPayload = - CreateTask.ofType("index") - .dataSource(dataSource) - .csvInputFormatWithColumns("time", "item", "value") - .isoTimestampColumn("time") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) - .dimensions(); + final TaskPayload taskPayload = + TaskPayload.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .dimensions(); cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskPayload.build(taskId)) + o -> o.runTask(taskId, taskPayload.withId(taskId)) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); From 5c3a2de31275484f850517b096b260030aa80ee5 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 4 Jul 2025 23:08:34 +0530 Subject: [PATCH 06/38] Minor cleanup --- .../EmbeddedIndexParallelTaskTest.java | 3 +-- .../indexing/EmbeddedIndexTaskTest.java | 11 +++++++---- .../server/EmbeddedHighAvailabilityTest.java | 18 +++++++++--------- .../testing/embedded/EmbeddedClusterApis.java | 10 +++------- 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index 3878c05a3348..cac98693e43e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -19,7 +19,6 @@ package org.apache.druid.testing.embedded.indexing; -import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexer.report.TaskReport; @@ -181,7 +180,7 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(Map o.runTask(taskId, taskBuilder.withId(taskId))); cluster.callApi().waitForTaskToSucceed(taskId, overlord); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java index 910699c7213b..3fa4126a445f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java @@ -19,13 +19,13 @@ package org.apache.druid.testing.embedded.indexing; -import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.DruidMetrics; import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; @@ -74,7 +74,7 @@ public EmbeddedDruidCluster createCluster() @Timeout(20) public void test_runIndexTask_forInlineDatasource() { - final String taskId = IdUtils.getRandomId(); + final String taskId = EmbeddedClusterApis.newTaskId(dataSource); final Object task = createIndexTaskForInlineData( taskId, Resources.CSV_DATA_10_DAYS @@ -104,7 +104,10 @@ public void test_runIndexTask_forInlineDatasource() broker.latchableEmitter().waitForEvent( event -> event.hasDimension(DruidMetrics.DATASOURCE, dataSource) ); - Assertions.assertEquals(Resources.CSV_DATA_10_DAYS, cluster.runSql("SELECT * FROM %s", dataSource)); + Assertions.assertEquals( + Resources.CSV_DATA_10_DAYS, + cluster.runSql("SELECT * FROM %s", dataSource) + ); Assertions.assertEquals("10", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); } @@ -159,7 +162,7 @@ private void runTasksConcurrently(int count) final DateTime jan1 = DateTimes.of("2025-01-01"); final List taskIds = IntStream.range(0, count).mapToObj( - i -> dataSource + "_" + IdUtils.getRandomId() + i -> EmbeddedClusterApis.newTaskId(dataSource) ).collect(Collectors.toList()); int index = 0; diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index 451f4009b9b4..f21a0d2f9feb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -19,7 +19,6 @@ package org.apache.druid.testing.embedded.server; -import org.apache.druid.common.utils.IdUtils; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.discovery.DruidNodeDiscovery; @@ -32,6 +31,7 @@ import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedDruidServer; @@ -85,10 +85,10 @@ protected EmbeddedDruidCluster createCluster() @Test public void test_allNodesHaveDiscoveredEachOther() { - final List allServers + final List> allServers = List.of(coordinator1, coordinator2, overlord1, overlord2, indexer, broker, router); - for (EmbeddedDruidServer server : allServers) { + for (EmbeddedDruidServer server : allServers) { final DruidNodeDiscoveryProvider discoveryProvider = server.bindings().nodeDiscovery(); final HttpClient httpClient = server.bindings().escalatedHttpClient(); @@ -105,7 +105,7 @@ public void test_allNodesHaveDiscoveredEachOther() public void test_switchLeader_andVerifyUsingSysTables() { // Ingest some data so that we can query sys tables later - final String taskId = dataSource + "_" + IdUtils.getRandomId(); + final String taskId = EmbeddedClusterApis.newTaskId(dataSource); final TaskPayload taskPayload = TaskPayload.ofType("index") .dataSource(dataSource) @@ -184,7 +184,7 @@ private void verifySelfDiscoveredStatusReturnsOk( * Restarts the current leader in the server pair to force the other server to * gain leadership. Returns the updated server pair. */ - private ServerPair switchAndVerifyLeader(ServerPair serverPair) + private > ServerPair switchAndVerifyLeader(ServerPair serverPair) { try { // Restart the current leader @@ -206,7 +206,7 @@ private ServerPair switchAndVerifyLeader(Serv } } - private ServerPair createServerPair(S serverA, S serverB) + private > ServerPair createServerPair(S serverA, S serverB) { final boolean aIsLeader; if (serverA instanceof EmbeddedOverlord) { @@ -222,9 +222,9 @@ private ServerPair createServerPair(S serverA * Verifies that exactly one of the servers in the pair is a leader and that * other servers know it to be the leader. */ - private void verifyOnlyOneInPairIsLeader( + private > void verifyOnlyOneInPairIsLeader( ServerPair serverPair, - Function getLeaderSelector + Function, DruidLeaderSelector> getLeaderSelector ) { final String leaderUri = serverPair.leader.bindings().selfNode().getUriToUse().toString(); @@ -271,7 +271,7 @@ private void verifyOnlyOneInPairIsLeader( /** * A pair of highly available Coordinator or Overlord nodes where one is leader. */ - private static class ServerPair + private static class ServerPair> { private final S leader; private final S notLeader; diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 1ec6c87d8023..a61bd7a0ab38 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -178,15 +178,11 @@ public static String createTestDatasourceName() } /** - * Deserializes the given String payload into a Map-based object that may be - * submitted directly to the Overlord using - * {@code cluster.callApi().onLeaderOverlord(o -> o.runTask(...))}. + * Creates a random task ID prefixed with the {@code dataSource}. */ - public static Object createTaskFromPayload(String taskId, String payload) + public static String newTaskId(String dataSource) { - final Map task = deserializeJsonToMap(payload); - task.put("id", taskId); - return task; + return dataSource + "_" + IdUtils.getRandomId(); } /** From b0e1026e0267c508594b461243977588e9b25822 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 5 Jul 2025 08:38:42 +0530 Subject: [PATCH 07/38] Fix dependencies --- .../druid/testing/embedded/indexing/TaskPayload.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java index 48b694af89a7..5c3fdf8a352a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java @@ -19,9 +19,9 @@ package org.apache.druid.testing.embedded.indexing; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.ISE; import org.joda.time.Interval; -import org.junit.platform.commons.util.Preconditions; import java.io.File; import java.net.URL; @@ -82,10 +82,10 @@ public TaskPayload dataSource(String dataSource) */ public Object withId(String taskId) { - Preconditions.notNull(taskId, "Task ID must not be null"); - Preconditions.notNull(type, "Task type must be specified"); - Preconditions.notNull(inputSource, "'inputSource' must be specified"); - Preconditions.notNull(dataSource, "'dataSource' must be specified"); + Preconditions.checkNotNull(taskId, "Task ID must not be null"); + Preconditions.checkNotNull(type, "Task type must be specified"); + Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); + Preconditions.checkNotNull(dataSource, "'dataSource' must be specified"); return mapOf( "id", taskId, From 627963072b050c5891bade81fc75b0415c061aab Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 12:46:58 +0530 Subject: [PATCH 08/38] Convert ITAutoCompactionTest to EmbeddedAutoCompactionTest --- embedded-tests/pom.xml | 11 + .../CompactionResourceTestClient.java | 72 +- .../compact/EmbeddedAutoCompactionTest.java | 1043 ++++++++--------- .../EmbeddedAutoCompactionUpgradeTest.java | 88 +- .../EmbeddedKafkaClusterMetricsTest.java | 79 ++ .../testing/embedded/indexing/Resources.java | 23 + .../embedded/indexing/TaskPayload.java | 49 +- .../ITAutoCompactionLockContentionTest.java | 361 ------ 8 files changed, 774 insertions(+), 952 deletions(-) rename {integration-tests/src/main/java/org/apache/druid/testing/clients => embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact}/CompactionResourceTestClient.java (83%) rename integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java => embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java (66%) rename integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java => embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java (52%) delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 68ab27b4635d..d06617835b8a 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -121,6 +121,12 @@ ${project.parent.version} test + + org.apache.druid.extensions + druid-datasketches + ${project.parent.version} + test + org.apache.druid @@ -193,6 +199,11 @@ ${mariadb.version} test + + org.hamcrest + hamcrest-all + test + diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java similarity index 83% rename from integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java index e05e72b12ffa..cdbb395721ca 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java @@ -17,11 +17,10 @@ * under the License. */ -package org.apache.druid.testing.clients; +package org.apache.druid.testing.embedded.compact; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; import org.apache.druid.indexing.overlord.http.CompactionConfigsResponse; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -30,14 +29,18 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.segment.TestHelper; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -50,36 +53,45 @@ public class CompactionResourceTestClient private static final Logger log = new Logger(CompactionResourceTestClient.class); private final ObjectMapper jsonMapper; - private final HttpClient httpClient; - private final String coordinator; - private final String overlord; + private final EmbeddedOverlord overlord; + private final EmbeddedCoordinator coordinator; private final StatusResponseHandler responseHandler; - @Inject CompactionResourceTestClient( - ObjectMapper jsonMapper, - @TestClient HttpClient httpClient, - IntegrationTestingConfig config + EmbeddedCoordinator coordinator, + EmbeddedOverlord overlord ) { - this.jsonMapper = jsonMapper; - this.httpClient = httpClient; - this.coordinator = config.getCoordinatorUrl(); - this.overlord = config.getOverlordUrl(); + this.jsonMapper = TestHelper.JSON_MAPPER + .registerModules(new SketchModule().getJacksonModules()) + .registerModules(new HllSketchModule().getJacksonModules()) + .registerModules(new DoublesSketchModule().getJacksonModules()); + this.overlord = overlord; + this.coordinator = coordinator; this.responseHandler = StatusResponseHandler.getInstance(); } + private HttpClient httpClient() + { + return overlord.bindings().escalatedHttpClient(); + } + private String getCoordinatorURL() { return StringUtils.format( - "%s/druid/coordinator/v1/", - coordinator + "http://%s:%s/druid/coordinator/v1/", + coordinator.bindings().selfNode().getHost(), + coordinator.bindings().selfNode().getPlaintextPort() ); } private String getOverlordURL() { - return StringUtils.format("%s/druid/indexer/v1", overlord); + return StringUtils.format( + "http://%s:%s/druid/indexer/v1", + overlord.bindings().selfNode().getHost(), + overlord.bindings().selfNode().getPlaintextPort() + ); } public void submitCompactionConfig(final DataSourceCompactionConfig dataSourceCompactionConfig) throws Exception @@ -89,7 +101,7 @@ public void submitCompactionConfig(final DataSourceCompactionConfig dataSourceCo "%s/compaction/config/datasources/%s", getOverlordURL(), StringUtils.urlEncode(dataSource) ); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", jsonMapper.writeValueAsBytes(dataSourceCompactionConfig) @@ -116,7 +128,7 @@ public void deleteDataSourceCompactionConfig(final String dataSource) throws Exc "%s/compaction/config/datasources/%s", getOverlordURL(), StringUtils.urlEncode(dataSource) ); - StatusResponseHolder response = httpClient.go(new Request(HttpMethod.DELETE, new URL(url)), responseHandler).get(); + StatusResponseHolder response = httpClient().go(new Request(HttpMethod.DELETE, new URL(url)), responseHandler).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE( @@ -135,7 +147,7 @@ public void deleteDataSourceCompactionConfig(final String dataSource) throws Exc public DruidCompactionConfig getCoordinatorCompactionConfig() throws Exception { String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -151,7 +163,7 @@ public DruidCompactionConfig getCoordinatorCompactionConfig() throws Exception public List getAllCompactionConfigs() throws Exception { String url = StringUtils.format("%s/compaction/config/datasources", getOverlordURL()); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -174,7 +186,7 @@ public DataSourceCompactionConfig getDataSourceCompactionConfig(String dataSourc "%s/compaction/config/datasources/%s", getOverlordURL(), StringUtils.urlEncode(dataSource) ); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -206,7 +218,7 @@ public void forceTriggerAutoCompaction() throws Exception ); String url = StringUtils.format("%scompaction/compact", getCoordinatorURL()); - StatusResponseHolder response = httpClient.go(new Request(HttpMethod.POST, new URL(url)), responseHandler).get(); + StatusResponseHolder response = httpClient().go(new Request(HttpMethod.POST, new URL(url)), responseHandler).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE( "Error while force trigger auto compaction status[%s] content[%s]", @@ -222,7 +234,7 @@ public void updateClusterConfig(ClusterCompactionConfig config) throws Exception "%s/compaction/config/cluster", getOverlordURL() ); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", jsonMapper.writeValueAsBytes(config) @@ -241,7 +253,7 @@ public void updateClusterConfig(ClusterCompactionConfig config) throws Exception public ClusterCompactionConfig getClusterConfig() throws Exception { String url = StringUtils.format("%s/compaction/config/cluster", getOverlordURL()); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -267,7 +279,7 @@ private void updateCompactionTaskSlot(Double compactionTaskSlotRatio, Integer ma StringUtils.urlEncode(compactionTaskSlotRatio.toString()), StringUtils.urlEncode(maxCompactionTaskSlots.toString()) ); - StatusResponseHolder response = httpClient.go(new Request(HttpMethod.POST, new URL(url)), responseHandler).get(); + StatusResponseHolder response = httpClient().go(new Request(HttpMethod.POST, new URL(url)), responseHandler).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE( "Error while updating compaction task slot status[%s] content[%s]", @@ -280,7 +292,7 @@ private void updateCompactionTaskSlot(Double compactionTaskSlotRatio, Integer ma public Map getCompactionProgress(String dataSource) throws Exception { String url = StringUtils.format("%scompaction/progress?dataSource=%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -296,7 +308,7 @@ public Map getCompactionProgress(String dataSource) throws Excep public AutoCompactionSnapshot getCompactionStatus(String dataSource) throws Exception { String url = StringUtils.format("%scompaction/status?dataSource=%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { @@ -317,7 +329,7 @@ public CompactionSimulateResult simulateRunOnCoordinator() throws Exception final ClusterCompactionConfig clusterConfig = getClusterConfig(); final String url = StringUtils.format("%scompaction/simulate", getCoordinatorURL()); - StatusResponseHolder response = httpClient.go( + StatusResponseHolder response = httpClient().go( new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", jsonMapper.writeValueAsBytes(clusterConfig) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java similarity index 66% rename from integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index cdafb7fa48eb..3eae5802201f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -17,36 +17,35 @@ * under the License. */ -package org.apache.druid.tests.coordinator.duty; +package org.apache.druid.testing.embedded.compact; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; -import com.google.inject.Inject; -import org.apache.commons.io.IOUtils; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexing.common.task.CompactionIntervalSpec; -import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.overlord.http.TaskPayloadResponse; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -69,14 +68,17 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.clients.CompactionResourceTestClient; -import org.apache.druid.testing.clients.TaskResponseObject; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; -import org.apache.druid.tests.indexer.AbstractIndexerTest; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.Resources; +import org.apache.druid.testing.embedded.indexing.TaskPayload; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.apache.druid.timeline.DataSegment; import org.hamcrest.Matcher; import org.hamcrest.MatcherAssert; @@ -85,68 +87,174 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.joda.time.chrono.ISOChronology; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashSet; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.UUID; +import java.util.TreeSet; +import java.util.function.Supplier; import java.util.stream.Collectors; -@Test(groups = {TestNGGroup.COMPACTION}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITAutoCompactionTest extends AbstractIndexerTest +/** + * Embedded mode of integration-tests originally present in {@code ITAutoCompactionTest}. + */ +public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase { - private static final Logger LOG = new Logger(ITAutoCompactionTest.class); - private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; - private static final String INDEX_TASK_WITH_GRANULARITY_SPEC = "/indexer/wikipedia_index_task_with_granularity_spec.json"; - private static final String INDEX_TASK_WITH_DIMENSION_SPEC = "/indexer/wikipedia_index_task_with_dimension_spec.json"; - private static final String INDEX_ROLLUP_QUERIES_RESOURCE = "/indexer/wikipedia_index_rollup_queries.json"; - private static final String INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE = "/indexer/wikipedia_index_sketch_queries.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - private static final String INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = "/indexer/wikipedia_index_rollup_preserve_metric.json"; - private static final String INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = "/indexer/wikipedia_index_no_rollup_preserve_metric.json"; + private static final Logger LOG = new Logger(EmbeddedAutoCompactionTest.class); + private static final Supplier INDEX_TASK = + () -> TaskPayload + .ofType("index") + .jsonInputFormat() + .localInputSourceWithFiles( + Resources.WIKIPEDIA_1_JSON, + Resources.WIKIPEDIA_2_JSON, + Resources.WIKIPEDIA_3_JSON + ) + .timestampColumn("timestamp") + .dimensions( + "page", + "language", "tags", "user", "unpatrolled", "newPage", "robot", + "anonymous", "namespace", "continent", "country", "region", "city" + ) + .metricAggregate("ingested_events", "count") + .metricAggregate("added", "doubleSum") + .metricAggregate("deleted", "doubleSum") + .metricAggregate("delta", "doubleSum") + .metricAggregate("thetaSketch", "thetaSketch", "user") + .metricAggregate("HLLSketchBuild", "HLLSketchBuild", "user") + .metricAggregate("quantilesDoublesSketch", "quantilesDoublesSketch", "delta") + .dynamicPartitionWithMaxRows(3) + .granularitySpec("DAY", "SECOND", true) + .appendToExisting(false); + + private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = + () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); + private static final Supplier INDEX_TASK_WITH_DIMENSION_SPEC = + () -> INDEX_TASK.get().granularitySpec("DAY", "DAY", true); + + private static final String SELECT_APPROX_COUNT_DISTINCT = + "SELECT" + + " APPROX_COUNT_DISTINCT_DS_THETA(\"thetaSketch\")," + + " APPROX_COUNT_DISTINCT_DS_HLL(\"HLLSketchBuild\")" + + " FROM %s"; + private static final List> INDEX_QUERIES_RESOURCE = List.of( + Pair.of( + "SELECT MIN(__time), MAX(__time) FROM %s", + "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z" + ), + Pair.of(SELECT_APPROX_COUNT_DISTINCT, "5,5"), + Pair.of( + "SELECT EARLIEST(\"user\"), LATEST(\"user\") FROM %s WHERE __time < '2013-09-01'", + "nuclear,stringer" + ), + Pair.of( + "SELECT \"page\", COUNT(*) AS \"rows\", SUM(\"added\"), 10 * SUM(\"added\") AS added_times_ten" + + " FROM %s" + + " WHERE \"language\" = 'zh' AND __time < '2013-09-01'" + + " GROUP BY 1" + + " HAVING added_times_ten > 9000", + "Crimson Typhoon,1,905.0,9050.0" + ) + ); + private static final Supplier INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = + () -> TaskPayload + .ofType("index_parallel") + .jsonInputFormat() + .inlineInputSourceWithData(Resources.JSON_DATA_2_ROWS) + .isoTimestampColumn("timestamp") + .appendToExisting(true) + .granularitySpec("DAY", "HOUR", true) + .metricAggregate("ingested_events", "count") + .metricAggregate("thetaSketch", "thetaSketch", "user") + .metricAggregate("HLLSketchBuild", "HLLSketchBuild", "user") + .metricAggregate("quantilesDoublesSketch", "quantilesDoublesSketch", "delta") + .metricAggregate("sum_added", "longSum", "added") + .metricAggregate("sum_deleted", "longSum", "deleted") + .metricAggregate("sum_delta", "longSum", "delta") + .metricAggregate("sum_deltaBucket", "longSum", "deltaBucket") + .metricAggregate("sum_commentLength", "longSum", "commentLength") + .dimensions( + "isRobot", + "language", "flags", "isUnpatrolled", "page", "diffUrl", "comment", + "isNew", "isMinor", "isAnonymous", "namespace" + ); + + private static final Supplier INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = + () -> TaskPayload + .ofType("index_parallel") + .jsonInputFormat() + .inlineInputSourceWithData(Resources.JSON_DATA_1_ROW) + .isoTimestampColumn("timestamp") + .granularitySpec("DAY", "HOUR", false) + .appendToExisting(true) + .dimensionsSpec( + Map.of( + "dimensions", + List.of( + "isRobot", + "language", "flags", "isUnpatrolled", "page", "diffUrl", + Map.of("type", "long", "name", "added"), + "comment", + Map.of("type", "long", "name", "commentLength"), + "isNew", "isMinor", + Map.of("type", "long", "name", "delta"), + "isAnonymous", "user", + Map.of("type", "long", "name", "deltaBucket"), + Map.of("type", "long", "name", "deleted"), + "namespace", "cityName", "countryName", "regionIsoCode", + "metroCode", "countryIsoCode", "regionName" + ) + ) + ); private static final int MAX_ROWS_PER_SEGMENT_COMPACTED = 10000; private static final Period NO_SKIP_OFFSET = Period.seconds(0); private static final FixedIntervalOrderPolicy COMPACT_NOTHING_POLICY = new FixedIntervalOrderPolicy(List.of()); - @DataProvider(name = "engine") - public static Object[][] engine() + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() + .addProperty("druid.manager.segments.useIncrementalCache", "always"); + private final EmbeddedBroker broker = new EmbeddedBroker() + .addProperty("druid.sql.planner.metadataRefreshPeriod", "PT0.1s"); + + public static List getEngine() { - return new Object[][]{{CompactionEngine.NATIVE}}; + return List.of(CompactionEngine.NATIVE); } - - @DataProvider(name = "useSupervisors") - public static Object[][] useSupervisors() + + @Override + protected EmbeddedDruidCluster createCluster() { - return new Object[][]{{true}, {false}}; + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addServer(coordinator) + .addServer(overlord) + .addServer(broker) + .addServer(new EmbeddedIndexer().addProperty("druid.worker.capacity", "10")) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()); } - @Inject - protected CompactionResourceTestClient compactionResource; - - @Inject - private IntegrationTestingConfig config; + protected final CompactionResourceTestClient compactionResource = + new CompactionResourceTestClient(coordinator, overlord); private String fullDatasourceName; - @BeforeMethod - public void setup() throws Exception + @BeforeEach + public void resetCompactionTaskSlots() throws Exception { // Set compaction slot to 5 updateCompactionTaskSlot(0.5, 10); - fullDatasourceName = "wikipedia_index_test_" + UUID.randomUUID() + config.getExtraDatasourceNameSuffix(); + fullDatasourceName = dataSource; } @Test @@ -157,36 +265,15 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 2 segments across 1 days... verifySegmentsCount(2); - ArrayList nullList = new ArrayList<>(); - nullList.add(null); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList)), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2))), ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62))), ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%QUANTILESRESULT%%", 2, - "%%THETARESULT%%", 2.0, - "%%HLLRESULT%%", 2 - ); - verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "...||31"); + verifyScanResult("ingested_events", "2||..."); + verifyScanResult("sum_added", "62||..."); + verifyScanResult("COUNT(*)", "2"); + verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "2,2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -195,7 +282,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[]{ - new CountAggregatorFactory("count"), + new CountAggregatorFactory("ingested_events"), // FloatSumAggregator combine method takes in two Float but return Double new FloatSumAggregatorFactory("sum_added", "added"), new SketchMergeAggregatorFactory("thetaSketch", "user", 16384, true, false, null), @@ -209,39 +296,20 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = null, count = 3, sum_added = 93.0 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(3)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(93.0f)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%QUANTILESRESULT%%", 3, - "%%THETARESULT%%", 3.0, - "%%HLLRESULT%%", 3 - ); - verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "..."); + verifyScanResult("ingested_events", "3"); + verifyScanResult("sum_added", "93.0"); + verifyScanResult("COUNT(*)", "1"); + verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); checkCompactionIntervals(intervalsBeforeCompaction); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(1); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } @@ -253,36 +321,16 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 2 segments across 1 days... verifySegmentsCount(2); - ArrayList nullList = new ArrayList<>(); - nullList.add(null); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList)), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2))), ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62))), ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%QUANTILESRESULT%%", 2, - "%%THETARESULT%%", 2.0, - "%%HLLRESULT%%", 2 - ); - verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields); + + verifyScanResult("COUNT(*)", "2"); + verifyScanResult("added", "...||31"); + verifyScanResult("ingested_events", "2||..."); + verifyScanResult("sum_added", "62||..."); + verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "2,2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -291,7 +339,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[]{ - new CountAggregatorFactory("count"), + new CountAggregatorFactory("ingested_events"), new LongSumAggregatorFactory("sum_added", "added"), new SketchMergeAggregatorFactory("thetaSketch", "user", 16384, true, false, null), new HllSketchBuildAggregatorFactory( @@ -312,39 +360,20 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = null, count = 3, sum_added = 93 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(3)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(93)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%QUANTILESRESULT%%", 3, - "%%THETARESULT%%", 3.0, - "%%HLLRESULT%%", 3 - ); - verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "..."); + verifyScanResult("ingested_events", "3"); + verifyScanResult("sum_added", "93"); + verifyScanResult("COUNT(*)", "1"); + verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); checkCompactionIntervals(intervalsBeforeCompaction); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(1); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } @@ -356,18 +385,12 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 2 segments across 1 days... verifySegmentsCount(2); - ArrayList nullList = new ArrayList<>(); - nullList.add(null); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31))), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "31||31"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -375,7 +398,7 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws new UserCompactionTaskGranularityConfig(null, null, true), new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, - new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, + new AggregatorFactory[] {new CountAggregatorFactory("ingested_events"), new LongSumAggregatorFactory("sum_added", "added")}, false, CompactionEngine.NATIVE ); @@ -383,37 +406,24 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws // added = null, count = 2, sum_added = 62 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "..."); + verifyScanResult("ingested_events", "2"); + verifyScanResult("sum_added", "62"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); checkCompactionIntervals(intervalsBeforeCompaction); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(1); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInputWithMetrics(CompactionEngine engine) throws Exception { @@ -425,16 +435,12 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput updateCompactionTaskSlot(0.1, 2); } try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 2 segments across 1 days... verifySegmentsCount(2); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31))), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "31||31"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -450,21 +456,17 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput // added = 62 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "62"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); checkCompactionIntervals(intervalsBeforeCompaction); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(1); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } @@ -476,22 +478,13 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() // added = null, count = 2, sum_added = 62 loadData(INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 2 segments across 1 days... verifySegmentsCount(2); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2))), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62))), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("ingested_events", "2||2"); + verifyScanResult("sum_added", "62||62"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -499,7 +492,7 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() new UserCompactionTaskGranularityConfig(null, null, true), new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, - new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, + new AggregatorFactory[] {new CountAggregatorFactory("ingested_events"), new LongSumAggregatorFactory("sum_added", "added")}, false, CompactionEngine.NATIVE ); @@ -507,36 +500,28 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() // added = null, count = 4, sum_added = 124 forceTriggerAutoCompaction(1); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "count", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(4)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "sum_added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(124)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("ingested_events", "4"); + verifyScanResult("sum_added", "124"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); checkCompactionIntervals(intervalsBeforeCompaction); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(1); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @ParameterizedTest(name = "compactionEngine={0}") + @MethodSource("getEngine") public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total) verifySegmentsCount(4); @@ -567,12 +552,13 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionRollsUpMultiValueDimensionsWithoutUnnest(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total) verifySegmentsCount(4); @@ -597,17 +583,8 @@ public void testAutoCompactionRollsUpMultiValueDimensionsWithoutUnnest(Compactio ); // Compacted into 1 segment for the entire year. forceTriggerAutoCompaction(1); - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", - "added", - "%%EXPECTED_COUNT_RESULT%%", - 1, - "%%EXPECTED_SCAN_RESULT%%", - ImmutableList.of( - ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(516))) - ) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("COUNT(*)", "1"); + verifyScanResult("added", "516.0"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); verifySegmentsCompactedDimensionSchema(dimensionSchemas); } @@ -618,7 +595,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -663,12 +640,13 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -728,12 +706,13 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -747,7 +726,7 @@ public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine eng verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(0, null); // Auto compaction stats should be deleted as compacation config was deleted - Assert.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); + Assertions.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); checkCompactionIntervals(intervalsBeforeCompaction); } } @@ -759,7 +738,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception updateCompactionTaskSlot(0, 0); loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -771,7 +750,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(0, null); checkCompactionIntervals(intervalsBeforeCompaction); - Assert.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); + Assertions.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); // Update compaction slots to be 1 updateCompactionTaskSlot(1, 1); // One day compacted (1 new segment) and one day remains uncompacted. (3 total) @@ -816,7 +795,8 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, @@ -842,7 +822,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -892,7 +872,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = - coordinator.getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(fullDatasourceName); expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsAfterYEARCompactionButBeforeMONTHCompaction) { for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { @@ -935,15 +915,16 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( checkCompactionIntervals(expectedIntervalAfterCompaction); // verify that autocompaction completed before - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueThenFalse(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, @@ -969,7 +950,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1019,7 +1000,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = - coordinator.getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(fullDatasourceName); expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsAfterYEARCompactionButBeforeMONTHCompaction) { for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { @@ -1059,10 +1040,10 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT checkCompactionIntervals(expectedIntervalAfterCompaction); // verify that autocompaction completed before - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } @@ -1071,7 +1052,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1130,12 +1111,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1168,12 +1150,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(Compacti } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveSameSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1185,7 +1168,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec @@ -1195,17 +1178,18 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); // should be no new compaction task as segmentGranularity is already DAY - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveDifferentSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1217,7 +1201,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec @@ -1228,17 +1212,18 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); // There should be new compaction tasks since SegmentGranularity changed from DAY to YEAR - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertTrue(compactTasksAfter.size() > compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertTrue(compactTasksAfter.size() > compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranularityCoveringMultipleSegmentsInTimelineAndDropExistingTrue(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1273,7 +1258,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula newGranularity = Granularities.MONTH; final List intervalsAfterYEARButBeforeMONTHCompaction = - coordinator.getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(fullDatasourceName); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); @@ -1302,7 +1287,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1378,16 +1363,11 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegment() throws Exception { updateCompactionTaskSlot(1, 1); - final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); - Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.MONTH, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); + Map specs = Map.of("segmentGranularity", "MONTH", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, @@ -1404,41 +1384,36 @@ public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegm // does not have data on every week on the month forceTriggerAutoCompaction(3); // Make sure that no data is lost after compaction - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - List tasks = indexer.getCompleteTasksForDataSource(fullDatasourceName); - TaskResponseObject compactTask = null; - for (TaskResponseObject task : tasks) { + List tasks = getCompleteTasksForDataSource(fullDatasourceName); + TaskStatusPlus compactTask = null; + for (TaskStatusPlus task : tasks) { if (task.getType().equals("compact")) { compactTask = task; } } - Assert.assertNotNull(compactTask); - TaskPayloadResponse task = indexer.getTaskPayload(compactTask.getId()); + Assertions.assertNotNull(compactTask); + TaskPayloadResponse task = getTaskPayload(compactTask.getId()); // Verify that compaction task interval is adjusted to align with segmentGranularity - Assert.assertEquals(Intervals.of("2013-08-26T00:00:00.000Z/2013-10-07T00:00:00.000Z"), ((CompactionIntervalSpec) ((CompactionTask) task.getPayload()).getIoConfig().getInputSpec()).getInterval()); + Assertions.assertEquals( + Intervals.of("2013-08-26T00:00:00.000Z/2013-10-07T00:00:00.000Z"), + (((ClientCompactionTaskQuery) task.getPayload()).getIoConfig().getInputSpec()).getInterval() + ); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment(CompactionEngine engine) throws Exception { updateCompactionTaskSlot(1, 1); - final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); - Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.WEEK, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); + Map specs = Map.of("segmentGranularity", "WEEK", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, @@ -1452,40 +1427,34 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe // we expect the compaction task's interval to align with the MONTH segmentGranularity (2013-08-01 to 2013-10-01) forceTriggerAutoCompaction(2); // Make sure that no data is lost after compaction - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List tasks = indexer.getCompleteTasksForDataSource(fullDatasourceName); - TaskResponseObject compactTask = null; - for (TaskResponseObject task : tasks) { + List tasks = getCompleteTasksForDataSource(fullDatasourceName); + TaskStatusPlus compactTask = null; + for (TaskStatusPlus task : tasks) { if (task.getType().equals("compact")) { compactTask = task; } } - Assert.assertNotNull(compactTask); - TaskPayloadResponse task = indexer.getTaskPayload(compactTask.getId()); + Assertions.assertNotNull(compactTask); + TaskPayloadResponse task = getTaskPayload(compactTask.getId()); // Verify that compaction task interval is adjusted to align with segmentGranularity - Assert.assertEquals(Intervals.of("2013-08-01T00:00:00.000Z/2013-10-01T00:00:00.000Z"), ((CompactionIntervalSpec) ((CompactionTask) task.getPayload()).getIoConfig().getInputSpec()).getInterval()); + Assertions.assertEquals( + Intervals.of("2013-08-01T00:00:00.000Z/2013-10-01T00:00:00.000Z"), + (((ClientCompactionTaskQuery) task.getPayload()).getIoConfig().getInputSpec()).getInterval() + ); } } @Test() public void testAutoCompactionDutyWithRollup() throws Exception { - final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); - Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); + Map specs = Map.of("segmentGranularity", "DAY", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, @@ -1494,35 +1463,27 @@ public void testAutoCompactionDutyWithRollup() throws Exception CompactionEngine.NATIVE ); forceTriggerAutoCompaction(2); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(516.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "516.0"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @ParameterizedTest(name = "compactionEngine={0}") + @MethodSource("getEngine") public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) throws Exception { - final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); - Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, true, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); + Map specs = Map.of("segmentGranularity", "DAY", "queryGranularity", "NONE", "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, @@ -1531,41 +1492,34 @@ public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) engine ); forceTriggerAutoCompaction(2); - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(516.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "516.0"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "engine") + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) throws Exception { // Index data with dimensions "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", // "namespace", "continent", "country", "region", "city" loadData(INDEX_TASK_WITH_DIMENSION_SPEC); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); // Result is not rollup - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); // Compact and change dimension to only "language" submitCompactionConfig( @@ -1581,42 +1535,35 @@ public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) th forceTriggerAutoCompaction(2); // Result should rollup on language dimension - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(516.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "516.0"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify compacted segments does not get compacted again forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "useSupervisors") + @ValueSource(booleans = {false}) + @ParameterizedTest(name = "useSupervisors={0}") public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exception { updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); - loadData(INDEX_TASK_WITH_DIMENSION_SPEC); + loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(Ordering.natural().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); // Result is not rollup // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); // Compact and filter with selector on dim "page" and value "Striker Eureka" submitCompactionConfig( @@ -1632,41 +1579,34 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); // For dim "page", result should only contain value "Striker Eureka" - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 1, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "459.0"); + verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify compacted segments does not get compacted again forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test(dataProvider = "useSupervisors") + @ValueSource(booleans = {false}) + @ParameterizedTest(name = "useSupervisors={0}") public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws Exception { updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); - loadData(INDEX_TASK_WITH_DIMENSION_SPEC); + loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); intervalsBeforeCompaction.sort(Ordering.natural().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); // Compact and add longSum and doubleSum metrics submitCompactionConfig( @@ -1683,39 +1623,27 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws // Result should be the same with the addition of new metrics, "double_sum_added" and "long_sum_added". // These new metrics should have the same value as the input field "added" - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "double_sum_added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); - - queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "long_sum_added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57), ImmutableList.of(459)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("double_sum_added", "57.0||459.0"); + verifyScanResult("long_sum_added", "57||459"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify compacted segments does not get compacted again forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } @Test public void testAutoCompactionDutyWithOverlappingInterval() throws Exception { - final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); - Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.WEEK, Granularities.NONE, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); // Create WEEK segment with 2013-08-26 to 2013-09-02 + Map specs = Map.of("segmentGranularity", "WEEK", "queryGranularity", "NONE", "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); - specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.MONTH, Granularities.NONE, false, ImmutableList.of(new Interval("2013-09-01/2013-09-02", chrono)))); // Create MONTH segment with 2013-09-01 to 2013-10-01 + specs = Map.of("segmentGranularity", "MONTH", "queryGranularity", "NONE", "intervals", List.of("2013-09-01T-07/2013-09-02T-07")); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1723,12 +1651,8 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception // Result is not rollup // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - Map queryAndResultFields = ImmutableMap.of( - "%%FIELD_TO_QUERY%%", "added", - "%%EXPECTED_COUNT_RESULT%%", 2, - "%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(57.0), ImmutableList.of(459.0)))) - ); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -1742,84 +1666,85 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception ); // Compact the MONTH segment forceTriggerAutoCompaction(2); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); // Compact the WEEK segment forceTriggerAutoCompaction(2); - verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields); + verifyScanResult("added", "57.0||459.0"); + verifyScanResult("COUNT(*)", "2"); // Verify all task succeed - List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); - for (TaskResponseObject taskResponseObject : compactTasksBefore) { - Assert.assertEquals(TaskState.SUCCESS, taskResponseObject.getStatus()); + List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); + for (TaskStatusPlus taskResponseObject : compactTasksBefore) { + Assertions.assertEquals(TaskState.SUCCESS, taskResponseObject.getStatus()); } // Verify compacted segments does not get compacted again forceTriggerAutoCompaction(2); - List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); - Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); + List compactTasksAfter = getCompleteTasksForDataSource(fullDatasourceName); + Assertions.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - private void loadData(String indexTask) throws Exception + private void loadData(Supplier updatePayload) { - loadData(indexTask, ImmutableMap.of()); + loadData(updatePayload, Map.of()); } - private void loadData(String indexTask, Map specs) throws Exception + private void loadData(Supplier taskPayloadSupplier, Map granularitySpec) { - String taskSpec = getResourceAsString(indexTask); - taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName); - taskSpec = StringUtils.replace( - taskSpec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - for (Map.Entry entry : specs.entrySet()) { - taskSpec = StringUtils.replace( - taskSpec, - entry.getKey(), - jsonMapper.writeValueAsString(entry.getValue()) - ); + final TaskPayload taskPayload = taskPayloadSupplier.get().dataSource(fullDatasourceName); + if (!granularitySpec.isEmpty()) { + taskPayload.granularitySpec(granularitySpec); } - final String taskId = indexer.submitTask(taskSpec); + + final String taskId = EmbeddedClusterApis.newTaskId(fullDatasourceName); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskPayload.withId(taskId))); LOG.info("Submitted task[%s] to load data", taskId); - indexer.waitUntilTaskCompletes(taskId); + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(fullDatasourceName, coordinator); + } - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segments are loaded" + private void verifyQuery(List> queries) + { + queries.forEach( + query -> verifyQuery(query.lhs, query.rhs) ); } - private void verifyQuery(String queryResource) throws Exception + private void verifyQuery(String query, String result) { - verifyQuery(queryResource, ImmutableMap.of()); + Assertions.assertEquals( + result, + cluster.runSql(query, dataSource), + StringUtils.format("Query[%s] failed", query) + ); } - private void verifyQuery(String queryResource, Map keyValueToReplace) throws Exception + /** + * Verifies the result of a SELECT query + * + * @param field Field to select + * @param result CSV result with special strings {@code ||} to represent + * new-lines and {@code ...} to represent an empty string. + */ + private void verifyScanResult(String field, String result) { - String queryResponseTemplate; - try { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryResource); - queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", queryResource); - } - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName + final String sql = StringUtils.format( + "SELECT %s FROM %s WHERE \"language\" = 'en' AND __time < '2013-09-01'", + field, dataSource + ); + + // replace empty placeholder with empty string + result = StringUtils.replace(result, "...", "\"\""); + result = StringUtils.replace(result, "||", "\n"); + + Assertions.assertEquals( + result, + cluster.runSql(sql), + StringUtils.format("Query[%s] failed", sql) ); - for (Map.Entry entry : keyValueToReplace.entrySet()) { - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - entry.getKey(), - jsonMapper.writeValueAsString(entry.getValue()) - ); - } - queryHelper.testQueriesFromString(queryResponseTemplate); } private void updateClusterConfig(ClusterCompactionConfig clusterConfig) throws Exception @@ -1942,16 +1867,13 @@ private void submitCompactionConfig( .build(); compactionResource.submitCompactionConfig(dataSourceCompactionConfig); - // Wait for compaction config to persist - Thread.sleep(2000); - // Verify that the compaction config is updated correctly. DataSourceCompactionConfig foundDataSourceCompactionConfig = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); - Assert.assertNotNull(foundDataSourceCompactionConfig); - Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); - Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); - Assert.assertEquals(foundDataSourceCompactionConfig.getSkipOffsetFromLatest(), skipOffsetFromLatest); + Assertions.assertNotNull(foundDataSourceCompactionConfig); + Assertions.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); + Assertions.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); + Assertions.assertEquals(foundDataSourceCompactionConfig.getSkipOffsetFromLatest(), skipOffsetFromLatest); } private void deleteCompactionConfig() throws Exception @@ -1963,7 +1885,7 @@ private void deleteCompactionConfig() throws Exception .empty().withDatasourceConfigs(compactionResource.getAllCompactionConfigs()); DataSourceCompactionConfig foundDataSourceCompactionConfig = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); - Assert.assertNull(foundDataSourceCompactionConfig); + Assertions.assertNull(foundDataSourceCompactionConfig); } /** @@ -1989,6 +1911,7 @@ private void forceTriggerAutoCompaction( ); // Wait for scheduler to pick up the compaction job + // TODO: make this latch-based Thread.sleep(30_000); waitForCompactionToFinish(numExpectedSegmentsAfterCompaction); @@ -2009,30 +1932,37 @@ private void forceTriggerAutoCompaction(int numExpectedSegmentsAfterCompaction) private void waitForCompactionToFinish(int numExpectedSegmentsAfterCompaction) { - waitForAllTasksToCompleteForDataSource(fullDatasourceName); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segments are loaded" - ); + final Set taskIds = getTaskIdsForState(null, dataSource); + for (String taskId : taskIds) { + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + } + + cluster.callApi().waitForAllSegmentsToBeAvailable(fullDatasourceName, coordinator); verifySegmentsCount(numExpectedSegmentsAfterCompaction); } private void verifySegmentsCount(int numExpectedSegments) { - ITRetryUtil.retryUntilEquals( - () -> coordinator.getSegments(fullDatasourceName).size(), - numExpectedSegments, - "Segment count" + int segmentCount = getFullSegmentsMetadata(dataSource).size(); + Assertions.assertEquals(numExpectedSegments, segmentCount, "Segment count mismatch"); + Assertions.assertEquals( + String.valueOf(segmentCount), + cluster.runSql( + "SELECT COUNT(*) FROM sys.segments" + + " WHERE datasource='%s' AND is_overshadowed = 0 AND is_available = 1", + dataSource + ), + "Segment count mismatch in sys table" ); } private void checkCompactionIntervals(List expectedIntervals) { - final Set expectedIntervalsSet = new HashSet<>(expectedIntervals); - ITRetryUtil.retryUntilEquals( - () -> Set.copyOf(coordinator.getSegmentIntervals(fullDatasourceName)), - expectedIntervalsSet, - "Segment intervals" + // TODO: is waiting really needed here? + // If we have waited for all segments to be loaded, we can just move on from here + Assertions.assertEquals( + Set.copyOf(expectedIntervals), + Set.copyOf(getSegmentIntervals(dataSource)) ); } @@ -2046,37 +1976,37 @@ private void verifySegmentsCompacted(int expectedCompactedSegmentCount, Integer private void verifyTombstones(int expectedCompactedTombstoneCount) { - List segments = coordinator.getFullSegmentsMetadata(fullDatasourceName); + Set segments = getFullSegmentsMetadata(dataSource); int actualTombstoneCount = 0; for (DataSegment segment : segments) { if (segment.isTombstone()) { actualTombstoneCount++; } } - Assert.assertEquals(actualTombstoneCount, expectedCompactedTombstoneCount); + Assertions.assertEquals(actualTombstoneCount, expectedCompactedTombstoneCount); } private void verifySegmentsCompacted(PartitionsSpec partitionsSpec, int expectedCompactedSegmentCount) { - List segments = coordinator.getFullSegmentsMetadata(fullDatasourceName); + Set segments = getFullSegmentsMetadata(dataSource); List foundCompactedSegments = new ArrayList<>(); for (DataSegment segment : segments) { if (segment.getLastCompactionState() != null) { foundCompactedSegments.add(segment); } } - Assert.assertEquals(foundCompactedSegments.size(), expectedCompactedSegmentCount); + Assertions.assertEquals(foundCompactedSegments.size(), expectedCompactedSegmentCount); for (DataSegment compactedSegment : foundCompactedSegments) { - Assert.assertNotNull(compactedSegment.getLastCompactionState()); - Assert.assertNotNull(compactedSegment.getLastCompactionState().getPartitionsSpec()); - Assert.assertEquals(compactedSegment.getLastCompactionState().getPartitionsSpec(), partitionsSpec); + Assertions.assertNotNull(compactedSegment.getLastCompactionState()); + Assertions.assertNotNull(compactedSegment.getLastCompactionState().getPartitionsSpec()); + Assertions.assertEquals(compactedSegment.getLastCompactionState().getPartitionsSpec(), partitionsSpec); } } private void verifySegmentsCompactedDimensionSchema(List dimensionSchemas) { - List segments = coordinator.getFullSegmentsMetadata(fullDatasourceName); + Set segments = getFullSegmentsMetadata(dataSource); List foundCompactedSegments = new ArrayList<>(); for (DataSegment segment : segments) { if (segment.getLastCompactionState() != null) { @@ -2110,8 +2040,8 @@ private void updateCompactionTaskSlot(double compactionTaskSlotRatio, int maxCom // Verify that the compaction config is updated correctly final ClusterCompactionConfig updatedConfig = compactionResource.getClusterConfig(); - Assert.assertEquals(updatedConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); - Assert.assertEquals(updatedConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); + Assertions.assertEquals(updatedConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); + Assertions.assertEquals(updatedConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); LOG.info( "Updated compactionTaskSlotRatio[%s] and maxCompactionTaskSlots[%d]", compactionTaskSlotRatio, maxCompactionTaskSlots @@ -2133,16 +2063,71 @@ private void getAndAssertCompactionStatus( ) throws Exception { AutoCompactionSnapshot actualStatus = compactionResource.getCompactionStatus(fullDatasourceName); - Assert.assertNotNull(actualStatus); - Assert.assertEquals(actualStatus.getScheduleStatus(), scheduleStatus); + Assertions.assertNotNull(actualStatus); + Assertions.assertEquals(actualStatus.getScheduleStatus(), scheduleStatus); MatcherAssert.assertThat(actualStatus.getBytesAwaitingCompaction(), bytesAwaitingCompactionMatcher); MatcherAssert.assertThat(actualStatus.getBytesCompacted(), bytesCompactedMatcher); MatcherAssert.assertThat(actualStatus.getBytesSkipped(), bytesSkippedMatcher); - Assert.assertEquals(actualStatus.getSegmentCountAwaitingCompaction(), segmentCountAwaitingCompaction); - Assert.assertEquals(actualStatus.getSegmentCountCompacted(), segmentCountCompacted); - Assert.assertEquals(actualStatus.getSegmentCountSkipped(), segmentCountSkipped); - Assert.assertEquals(actualStatus.getIntervalCountAwaitingCompaction(), intervalCountAwaitingCompaction); - Assert.assertEquals(actualStatus.getIntervalCountCompacted(), intervalCountCompacted); - Assert.assertEquals(actualStatus.getIntervalCountSkipped(), intervalCountSkipped); + Assertions.assertEquals(actualStatus.getSegmentCountAwaitingCompaction(), segmentCountAwaitingCompaction); + Assertions.assertEquals(actualStatus.getSegmentCountCompacted(), segmentCountCompacted); + Assertions.assertEquals(actualStatus.getSegmentCountSkipped(), segmentCountSkipped); + Assertions.assertEquals(actualStatus.getIntervalCountAwaitingCompaction(), intervalCountAwaitingCompaction); + Assertions.assertEquals(actualStatus.getIntervalCountCompacted(), intervalCountCompacted); + Assertions.assertEquals(actualStatus.getIntervalCountSkipped(), intervalCountSkipped); + } + + private Set getFullSegmentsMetadata(String dataSource) + { + return overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE); + } + + private List getSegmentIntervals(String dataSource) + { + final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); + final Set sortedIntervals = new TreeSet<>(comparator); + + final Set allUsedSegments = getFullSegmentsMetadata(dataSource); + for (DataSegment segment : allUsedSegments) { + sortedIntervals.add(segment.getInterval()); + } + + return sortedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + } + + private List getCompleteTasksForDataSource(String dataSource) + { + return ImmutableList.copyOf( + (CloseableIterator) cluster.callApi().onLeaderOverlord( + o -> o.taskStatuses("complete", dataSource, 100) + ) + ); + } + + private TaskPayloadResponse getTaskPayload(String taskId) + { + return cluster.callApi().onLeaderOverlord( + o -> o.taskPayload(taskId) + ); + } + + private Set getTaskIdsForState(String state, String dataSource) + { + return ImmutableList.copyOf( + (CloseableIterator) cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(state, dataSource, 0)) + ).stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); + } + + /** + * Deletes all the data for the given datasource so that compaction tasks for + * this datasource do not take up task slots unnecessarily. + */ + private Closeable unloader(String dataSource) + { + return () -> { + overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); + }; } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java similarity index 52% rename from integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java index 505e261b2687..11703d83c965 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java @@ -17,51 +17,61 @@ * under the License. */ -package org.apache.druid.tests.coordinator.duty; +package org.apache.druid.testing.embedded.compact; -import com.google.inject.Inject; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.clients.CompactionResourceTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractIndexerTest; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.joda.time.Period; -import org.testng.Assert; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; -@Test(groups = {TestNGGroup.UPGRADE}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITAutoCompactionUpgradeTest extends AbstractIndexerTest +import java.nio.charset.StandardCharsets; + +public class EmbeddedAutoCompactionUpgradeTest extends EmbeddedClusterTestBase { - private static final String UPGRADE_DATASOURCE_NAME = "upgradeTest"; + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() + .addProperty("druid.manager.segments.useIncrementalCache", "always"); - @Inject - protected CompactionResourceTestClient compactionResource; + protected CompactionResourceTestClient compactionResource = + new CompactionResourceTestClient(coordinator, overlord); - @Inject - private IntegrationTestingConfig config; + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .addServer(coordinator); + } @Test - public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVersionAlreadyExist() throws Exception + public void test_configCanBeUpdated_afterVersionUpgrades() throws Exception { + // Insert a minimal compaction config manually into the DB + // Then start the Overlord to ensure that the Overlord has the latest config cached + insertMinimalCompactionConfig((TestDerbyConnector) coordinator.bindings().sqlMetadataConnector()); + cluster.addServer(overlord); + overlord.start(); + // Verify that compaction config already exist. This config was inserted manually into the database using SQL script. - // This auto compaction configuration payload is from Druid 0.21.0 DruidCompactionConfig coordinatorCompactionConfig = DruidCompactionConfig.empty() .withDatasourceConfigs(compactionResource.getAllCompactionConfigs()); DataSourceCompactionConfig foundDataSourceCompactionConfig - = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); - Assert.assertNotNull(foundDataSourceCompactionConfig); + = coordinatorCompactionConfig.findConfigForDatasource(dataSource).orNull(); + Assertions.assertNotNull(foundDataSourceCompactionConfig); // Now submit a new auto compaction configuration PartitionsSpec newPartitionsSpec = new DynamicPartitionsSpec(4000, null); @@ -69,7 +79,7 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() - .forDataSource(UPGRADE_DATASOURCE_NAME) + .forDataSource(dataSource) .withSkipOffsetFromLatest(newSkipOffset) .withTuningConfig( new UserCompactionTaskQueryTuningConfig( @@ -103,10 +113,32 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers // Verify that compaction was successfully updated foundDataSourceCompactionConfig - = compactionResource.getDataSourceCompactionConfig(UPGRADE_DATASOURCE_NAME); - Assert.assertNotNull(foundDataSourceCompactionConfig); - Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); - Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), newPartitionsSpec); - Assert.assertEquals(foundDataSourceCompactionConfig.getSkipOffsetFromLatest(), newSkipOffset); + = compactionResource.getDataSourceCompactionConfig(dataSource); + Assertions.assertNotNull(foundDataSourceCompactionConfig); + Assertions.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); + Assertions.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), newPartitionsSpec); + Assertions.assertEquals(foundDataSourceCompactionConfig.getSkipOffsetFromLatest(), newSkipOffset); + } + + /** + * Inserts a bare-bones compaction config directly into the druid_config + * metadata table. + */ + private void insertMinimalCompactionConfig(TestDerbyConnector sqlConnector) + { + final String configJson = StringUtils.format( + "{\"compactionConfigs\":[{\"dataSource\":\"%s\"}]}", + dataSource + ); + + sqlConnector.retryWithHandle( + handle -> handle.insert( + StringUtils.format( + "INSERT INTO %s (name, payload) VALUES ('coordinator.compaction.config',?)", + sqlConnector.getMetadataTablesConfig().getConfigTable() + ), + configJson.getBytes(StandardCharsets.UTF_8) + ) + ); } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java index 8d10bae6e4c1..463d17b43e27 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java @@ -307,6 +307,85 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip ); } + @Test + @Timeout(120) + public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() + { + final int maxRowsPerSegment = 500; + final int compactedMaxRowsPerSegment = 5000; + + final int taskCount = 2; + final int taskDurationMillis = 500; + final int taskCompletionTimeoutMillis = 5_000; + + // Submit and start a supervisor + final String supervisorId = dataSource + "_supe"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( + supervisorId, + taskCount, + taskDurationMillis, + taskCompletionTimeoutMillis, + maxRowsPerSegment + ); + cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(kafkaSupervisorSpec) + ); + + // Wait for some segments to be published + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("segment/txn/success") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + ); + + // Enable compaction supervisors on the Overlord + final ClusterCompactionConfig originalCompactionConfig = cluster.callApi().onLeaderOverlord( + OverlordClient::getClusterCompactionConfig + ); + + final ClusterCompactionConfig updatedCompactionConfig + = new ClusterCompactionConfig(1.0, 10, null, true, null); + final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( + o -> o.updateClusterCompactionConfig(updatedCompactionConfig) + ); + Assertions.assertTrue(updateResponse.isSuccess()); + + // Submit a compaction supervisor for this datasource + final CompactionSupervisorSpec compactionSupervisorSpec = new CompactionSupervisorSpec( + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withMaxRowsPerSegment(compactedMaxRowsPerSegment) + .withTaskContext(Map.of("useConcurrentLocks", false)) + .build(), + false, + null + ); + cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(compactionSupervisorSpec) + ); + + // Wait until some skipped metrics have been emitted + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("interval/skipCompact/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(1) + ); + + // Revert the cluster compaction config + cluster.callApi().onLeaderOverlord( + o -> o.updateClusterCompactionConfig(originalCompactionConfig) + ); + + // Suspend the supervisors + cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(compactionSupervisorSpec.createSuspendedSpec()) + ); + cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()) + ); + } + /** * SELECTs the total count of the given metric in the {@link #dataSource} and * verifies it against the metrics actually emitted by the server. diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java index 90da3e7a175d..1f0fb99990ec 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java @@ -39,6 +39,29 @@ public class Resources + "\n2025-06-09T00:00:00.000Z,shirt,99" + "\n2025-06-10T00:00:00.000Z,toys,101"; + public static final String JSON_DATA_2_ROWS = + "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," + + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," + + "\"isAnonymous\":false,\"user\":\"maytas1\",\"deltaBucket\":0.0,\"deleted\":0," + + "\"namespace\":\"Main\"}" + + "\n{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\",\"commentLength\":35," + + "\"isNew\":true,\"isMinor\":false,\"delta\":11,\"isAnonymous\":false,\"user\":\"maytas2\"," + + "\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; + + public static final String JSON_DATA_1_ROW = + "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," + + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," + + "\"isAnonymous\":false,\"user\":\"maytas3\",\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; + public static final String WIKIPEDIA_1_JSON = "data/json/wikipedia_1.json"; public static final String WIKIPEDIA_2_JSON = "data/json/wikipedia_2.json"; public static final String WIKIPEDIA_3_JSON = "data/json/wikipedia_3.json"; diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java index 5c3fdf8a352a..92b1dbe86649 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java @@ -56,6 +56,7 @@ public class TaskPayload private Integer maxNumConcurrentSubTasks = null; private Boolean forceGuaranteedRollup = null; private Long awaitSegmentAvailabilityTimeoutMillis = null; + private Boolean appendToExisting = null; private TaskPayload(String type) { @@ -96,7 +97,8 @@ public Object withId(String taskId) mapOf( "type", type, "inputSource", inputSource, - "inputFormat", inputFormat + "inputFormat", inputFormat, + "appendToExisting", appendToExisting ), "tuningConfig", mapOf( @@ -112,7 +114,7 @@ public Object withId(String taskId) "dataSource", dataSource, "timestampSpec", timestampSpec, "dimensionsSpec", dimensionsSpec, - "metricsSpec", metricsSpec, + "metricsSpec", metricsSpec.isEmpty() ? null : metricsSpec, "granularitySpec", granularitySpec ) ) @@ -170,6 +172,12 @@ public TaskPayload inputFormat(Map jsonMap) return this; } + public TaskPayload jsonInputFormat() + { + this.inputFormat = Map.of("type", "json"); + return this; + } + public TaskPayload csvInputFormatWithColumns(String... columns) { return inputFormat( @@ -177,18 +185,40 @@ public TaskPayload csvInputFormatWithColumns(String... columns) ); } + public TaskPayload appendToExisting(boolean append) + { + this.appendToExisting = append; + return this; + } + public TaskPayload partitionsSpec(Map jsonMap) { this.partitionsSpec = jsonMap; return this; } + public TaskPayload dynamicPartitionWithMaxRows(int maxRowsPerSegment) + { + this.partitionsSpec = Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment); + return this; + } + public TaskPayload granularitySpec(Map jsonMap) { this.granularitySpec = jsonMap; return this; } + public TaskPayload granularitySpec(String segmentGranularity, String queryGranularity, boolean rollup) + { + this.granularitySpec = Map.of( + "segmentGranularity", segmentGranularity, + "queryGranularity", queryGranularity, + "rollup", rollup + ); + return this; + } + /** * Sets {@code "granularitySpec": {"segmentGranularity": }}. */ @@ -227,9 +257,20 @@ public TaskPayload dimensions(String... dimensions) return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); } - public TaskPayload metricAggregate(String column, String type) + public TaskPayload metricAggregate(String name, String type) + { + return metricAggregate(name, type, name); + } + + public TaskPayload metricAggregate(String name, String type, String fieldName) + { + this.metricsSpec.add(mapOf("type", type, "name", name, "fieldName", fieldName)); + return this; + } + + public TaskPayload metricAggregate(Map jsonMap) { - this.metricsSpec.add(mapOf("type", type, "name", column, "fieldName", column)); + this.metricsSpec.add(Map.copyOf(jsonMap)); return this; } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java deleted file mode 100644 index 11e4ec6cad30..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java +++ /dev/null @@ -1,361 +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.tests.coordinator.duty; - -import com.google.inject.Inject; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.LockFilterPolicy; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.server.coordinator.ClusterCompactionConfig; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; -import org.apache.druid.testing.clients.CompactionResourceTestClient; -import org.apache.druid.testing.clients.TaskResponseObject; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.EventSerializer; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.testing.utils.KafkaUtil; -import org.apache.druid.testing.utils.StreamEventWriter; -import org.apache.druid.testing.utils.StreamGenerator; -import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest; -import org.apache.druid.tests.indexer.AbstractStreamIndexingTest; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; -import org.joda.time.Period; -import org.testng.Assert; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * Integration Test to verify behaviour when there is a lock contention between - * compaction tasks and on-going stream ingestion tasks. - */ -@Test(groups = {TestNGGroup.COMPACTION}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITAutoCompactionLockContentionTest extends AbstractKafkaIndexingServiceTest -{ - private static final Logger LOG = new Logger(ITAutoCompactionLockContentionTest.class); - - @Inject - private CompactionResourceTestClient compactionResource; - - private GeneratedTestConfig generatedTestConfig; - private StreamGenerator streamGenerator; - - private String fullDatasourceName; - - @DataProvider - public static Object[] getParameters() - { - return new Object[]{false, true}; - } - - @BeforeClass - public void setupClass() throws Exception - { - doBeforeClass(); - } - - @BeforeMethod - public void setup() throws Exception - { - generatedTestConfig = new GeneratedTestConfig( - Specs.PARSER_TYPE, - getResourceAsString(Specs.INPUT_FORMAT_PATH) - ); - fullDatasourceName = generatedTestConfig.getFullDatasourceName(); - final EventSerializer serializer = jsonMapper.readValue( - getResourceAsStream(Specs.SERIALIZER_PATH), - EventSerializer.class - ); - streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 100); - } - - @Override - public String getTestNamePrefix() - { - return "autocompact_lock_contention"; - } - - @Test(dataProvider = "getParameters") - public void testAutoCompactionSkipsLockedIntervals(boolean transactionEnabled) throws Exception - { - if (shouldSkipTest(transactionEnabled)) { - return; - } - - try ( - final Closeable closer = createResourceCloser(generatedTestConfig); - final StreamEventWriter streamEventWriter = createStreamEventWriter(config, transactionEnabled) - ) { - // Start supervisor - final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform() - .apply(getResourceAsString(SUPERVISOR_SPEC_TEMPLATE_PATH)); - generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec)); - LOG.info("supervisorSpec: [%s]", taskSpec); - - // Generate data for minutes 1, 2 and 3 - final Interval minute1 = Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z"); - final long rowsForMinute1 = generateData(minute1, streamEventWriter); - - final Interval minute2 = Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z"); - long rowsForMinute2 = generateData(minute2, streamEventWriter); - - final Interval minute3 = Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z"); - final long rowsForMinute3 = generateData(minute3, streamEventWriter); - - // Wait for data to be ingested for all the minutes - ensureRowCount(rowsForMinute1 + rowsForMinute2 + rowsForMinute3); - - // Wait for the segments to be loaded and interval locks to be released - ensureLockedIntervals(); - ensureSegmentsLoaded(); - - // 2 segments for each minute, total 6 - ensureSegmentsCount(6); - - // Generate more data for minute2 so that it gets locked - rowsForMinute2 += generateData(minute2, streamEventWriter); - ensureLockedIntervals(minute2); - - // Trigger auto compaction - submitAndVerifyCompactionConfig(); - compactionResource.forceTriggerAutoCompaction(); - - // Wait for segments to be loaded - ensureRowCount(rowsForMinute1 + rowsForMinute2 + rowsForMinute3); - ensureLockedIntervals(); - ensureSegmentsLoaded(); - - // Verify that minute1 and minute3 have been compacted - ensureCompactionTaskCount(2); - verifyCompactedIntervals(minute1, minute3); - - // Trigger auto compaction again - compactionResource.forceTriggerAutoCompaction(); - - // Verify that all the segments are now compacted - ensureCompactionTaskCount(3); - ensureSegmentsLoaded(); - verifyCompactedIntervals(minute1, minute2, minute3); - ensureSegmentsCount(3); - } - } - - /** - * Retries until the segment count is as expected. - */ - private void ensureSegmentsCount(int numExpectedSegments) - { - ITRetryUtil.retryUntilEquals( - () -> coordinator.getFullSegmentsMetadata(fullDatasourceName).size(), - numExpectedSegments, - "Segment count" - ); - } - - /** - * Verifies that the given intervals have been compacted. - */ - private void verifyCompactedIntervals(Interval... compactedIntervals) - { - List segments = coordinator.getFullSegmentsMetadata(fullDatasourceName); - List observedCompactedSegments = new ArrayList<>(); - Set observedCompactedIntervals = new HashSet<>(); - for (DataSegment segment : segments) { - if (segment.getLastCompactionState() != null) { - observedCompactedSegments.add(segment); - observedCompactedIntervals.add(segment.getInterval()); - } - } - - Set expectedCompactedIntervals = new HashSet<>(Arrays.asList(compactedIntervals)); - Assert.assertEquals(observedCompactedIntervals, expectedCompactedIntervals); - - DynamicPartitionsSpec expectedPartitionSpec = new DynamicPartitionsSpec( - Specs.MAX_ROWS_PER_SEGMENT, - Long.MAX_VALUE - ); - for (DataSegment compactedSegment : observedCompactedSegments) { - Assert.assertNotNull(compactedSegment.getLastCompactionState()); - Assert.assertEquals( - compactedSegment.getLastCompactionState().getPartitionsSpec(), - expectedPartitionSpec - ); - } - } - - /** - * Generates data points for the specified interval. - * - * @return Number of rows generated. - */ - private long generateData(Interval interval, StreamEventWriter streamEventWriter) - { - long rowCount = streamGenerator.run( - generatedTestConfig.getStreamName(), - streamEventWriter, - 10, - interval.getStart() - ); - LOG.info("Generated %d Rows for Interval [%s]", rowCount, interval); - - return rowCount; - } - - /** - * Retries until segments have been loaded. - */ - private void ensureSegmentsLoaded() - { - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segments are loaded" - ); - } - - /** - * Retries until the specified Intervals are locked for the current datasource. - * If no interval has been specified, retries until no interval is locked - */ - private void ensureLockedIntervals(Interval... intervals) - { - final LockFilterPolicy lockFilterPolicy = new LockFilterPolicy(fullDatasourceName, 0, null, null); - final Set expectedLockedIntervals = Arrays.stream(intervals).collect(Collectors.toSet()); - ITRetryUtil.retryUntilEquals( - () -> Set.copyOf( - indexer.getLockedIntervals(List.of(lockFilterPolicy)) - .getOrDefault(fullDatasourceName, List.of()) - ), - expectedLockedIntervals, - "Locked intervals" - ); - } - - /** - * Checks if a test should be skipped based on whether transaction is enabled or not. - */ - private boolean shouldSkipTest(boolean testEnableTransaction) - { - Map kafkaTestProps = KafkaUtil - .getAdditionalKafkaTestConfigFromProperties(config); - boolean configEnableTransaction = Boolean.parseBoolean( - kafkaTestProps.getOrDefault(KafkaUtil.TEST_CONFIG_TRANSACTION_ENABLED, "false") - ); - - return configEnableTransaction != testEnableTransaction; - } - - /** - * Submits a compaction config for the current datasource. - */ - private void submitAndVerifyCompactionConfig() throws Exception - { - final DataSourceCompactionConfig dataSourceCompactionConfig = InlineSchemaDataSourceCompactionConfig - .builder() - .forDataSource(fullDatasourceName) - .withSkipOffsetFromLatest(Period.ZERO) - .withMaxRowsPerSegment(Specs.MAX_ROWS_PER_SEGMENT) - .build(); - compactionResource.updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, null, null)); - compactionResource.submitCompactionConfig(dataSourceCompactionConfig); - - // Verify that the compaction config is updated correctly. - DataSourceCompactionConfig observedCompactionConfig - = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); - Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); - } - - /** - * Checks if the given TaskResponseObject represents a Compaction Task. - */ - private boolean isCompactionTask(TaskResponseObject taskResponse) - { - return "compact".equalsIgnoreCase(taskResponse.getType()); - } - - /** - * Retries until the total number of complete compaction tasks is as expected. - */ - private void ensureCompactionTaskCount(int expectedCount) - { - ITRetryUtil.retryUntilEquals( - this::getNumberOfCompletedCompactionTasks, - expectedCount, - "Number of completed compaction tasks" - ); - } - - /** - * Gets the number of complete compaction tasks. - */ - private int getNumberOfCompletedCompactionTasks() - { - List completeTasks = indexer - .getCompleteTasksForDataSource(fullDatasourceName); - - return (int) completeTasks.stream().filter(this::isCompactionTask).count(); - } - - /** - * Retries until the total row count is as expected. - */ - private void ensureRowCount(long totalRows) - { - ITRetryUtil.retryUntilEquals( - () -> queryHelper.countRows( - fullDatasourceName, - Intervals.ETERNITY, - name -> new LongSumAggregatorFactory(name, "count") - ), - totalRows, - "Total row count in datasource" - ); - } - - /** - * Constants for test specs. - */ - private static class Specs - { - static final String SERIALIZER_PATH = DATA_RESOURCE_ROOT + "/csv/serializer/serializer.json"; - static final String INPUT_FORMAT_PATH = DATA_RESOURCE_ROOT + "/csv/input_format/input_format.json"; - static final String PARSER_TYPE = AbstractStreamIndexingTest.INPUT_FORMAT; - - static final int MAX_ROWS_PER_SEGMENT = 10000; - } - -} From 0b98f643af100461ca0a43e6dfcf300f6a10f7f6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 12:50:22 +0530 Subject: [PATCH 09/38] Fix pom --- embedded-tests/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 52e2e169521e..733406e1b292 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -249,7 +249,6 @@ test -<<<<<<< HEAD org.testcontainers minio ${testcontainers.version} @@ -268,10 +267,11 @@ com.amazonaws aws-java-sdk-core -======= + test + + org.hamcrest hamcrest-all ->>>>>>> 627963072b050c5891bade81fc75b0415c061aab test From 60200d8c1c33fe29aeb71df70660ce5eee8ef7df Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 13:16:18 +0530 Subject: [PATCH 10/38] Add necessary extensions to embedded cluster test --- .../embedded/compact/EmbeddedAutoCompactionTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index 3eae5802201f..434bce6b5c5f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -52,8 +52,11 @@ import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule; import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.column.ColumnType; @@ -236,6 +239,9 @@ protected EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .addExtension(SketchModule.class) + .addExtension(HllSketchModule.class) + .addExtension(DoublesSketchModule.class) .addServer(coordinator) .addServer(overlord) .addServer(broker) From f2f2d3a8b0ac3d5a5b0d64305ed4ff944d411d08 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 13:19:20 +0530 Subject: [PATCH 11/38] Do not use deprecated APIs --- .../testing/embedded/compact/EmbeddedAutoCompactionTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index 434bce6b5c5f..1c1c79661657 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -1683,7 +1683,7 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception // Verify all task succeed List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); for (TaskStatusPlus taskResponseObject : compactTasksBefore) { - Assertions.assertEquals(TaskState.SUCCESS, taskResponseObject.getStatus()); + Assertions.assertEquals(TaskState.SUCCESS, taskResponseObject.getStatusCode()); } // Verify compacted segments does not get compacted again From 1c1959fb1b1d01c37b20d9adeda955abb884cf6e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 13:31:51 +0530 Subject: [PATCH 12/38] Remove upgrade TestNGGroup --- .github/workflows/cron-job-its.yml | 2 +- .github/workflows/standard-its.yml | 2 +- .../src/test/java/org/apache/druid/tests/TestNGGroup.java | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index 101639becf77..7328a5a0dedc 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -109,7 +109,7 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,custom-coordinator-duties + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,custom-coordinator-duties use_indexer: ${{ matrix.indexer }} group: other diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index df1c3323e74d..60983088f0c7 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -180,6 +180,6 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled use_indexer: ${{ matrix.indexer }} group: other diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index 9bf79a185e82..6112af682e3a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -43,8 +43,6 @@ public class TestNGGroup public static final String COMPACTION = "compaction"; - public static final String UPGRADE = "upgrade"; - public static final String APPEND_INGESTION = "append-ingestion"; /** From 7ee8b38e9db7b1da363cbba3b6ae6e6ef94f8d88 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 17:45:22 +0530 Subject: [PATCH 13/38] Address failures --- embedded-tests/pom.xml | 5 +++++ .../embedded/server/EmbeddedHighAvailabilityTest.java | 1 + 2 files changed, 6 insertions(+) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 733406e1b292..30dc8130d649 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -269,6 +269,11 @@ aws-java-sdk-core test + + org.hamcrest + hamcrest-core + test + org.hamcrest hamcrest-all diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index f21a0d2f9feb..b301496724b1 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -117,6 +117,7 @@ public void test_switchLeader_andVerifyUsingSysTables() o -> o.runTask(taskId, taskPayload.withId(taskId)) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator1); // Run sys queries, switch leaders, repeat ServerPair overlordPair = createServerPair(overlord1, overlord2); From 700459257448cf916ac44608f65093143efb14ef Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 19:04:27 +0530 Subject: [PATCH 14/38] Add embedded test for concurrent append and replace --- .../EmbeddedConcurrentAppendReplaceTest.java | 119 ++++++++++++++++++ .../embedded/indexing/TaskPayload.java | 8 ++ 2 files changed, 127 insertions(+) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java new file mode 100644 index 000000000000..0817c138f269 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java @@ -0,0 +1,119 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import java.util.List; + +public class EmbeddedConcurrentAppendReplaceTest extends EmbeddedClusterTestBase +{ + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addServer(coordinator) + .addServer(overlord) + .addServer(new EmbeddedIndexer()) + .addServer(new EmbeddedBroker()) + .addServer(new EmbeddedHistorical()); + } + + @Test + @Disabled("Enable when concurrent append to interval with unused segment is fixed") + public void test_concurrentAppend_toIntervalWithUnusedSegment() + { + // Run an APPEND task to ingest data into an interval + final String data1Row = "2013-01-01T00:00:00.000Z,shirt,100"; + final String task1 = EmbeddedClusterApis.newTaskId(dataSource); + final TaskPayload taskPayload = + TaskPayload.ofType("index") + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(data1Row) + .appendToExisting(true) + .context("useConcurrentLocks", true) + .dimensions(); + cluster.callApi().onLeaderOverlord( + o -> o.runTask(task1, taskPayload.withId(task1)) + ); + cluster.callApi().waitForTaskToSucceed(task1, overlord); + + List usedSegments = getAllUsedSegments(); + Assertions.assertEquals(1, usedSegments.size()); + + final SegmentId segmentId1 = usedSegments.get(0).getId(); + Assertions.assertEquals("1970-01-01T00:00:00.000Z", segmentId1.getVersion()); + Assertions.assertEquals(0, segmentId1.getPartitionNum()); + + // Mark all segments as unused and verify that the interval is now empty + overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); + usedSegments = getAllUsedSegments(); + Assertions.assertTrue(usedSegments.isEmpty()); + + // Run the APPEND task again with a different taskId + final String task2 = EmbeddedClusterApis.newTaskId(dataSource); + cluster.callApi().onLeaderOverlord( + o -> o.runTask(task2, taskPayload.withId(task2)) + ); + cluster.callApi().waitForTaskToSucceed(task2, overlord); + + // Verify that the new segment gets appended with the same version but a different ID + usedSegments = getAllUsedSegments(); + Assertions.assertEquals(1, usedSegments.size()); + + final SegmentId segmentId2 = usedSegments.get(0).getId(); + Assertions.assertEquals("1970-01-01T00:00:00.000Z", segmentId2.getVersion()); + Assertions.assertEquals(1, segmentId2.getPartitionNum()); + + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); + Assertions.assertEquals( + data1Row, + cluster.runSql("SELECT * FROM %s", dataSource) + ); + } + + private List getAllUsedSegments() + { + return List.copyOf( + overlord.bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + ); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java index 92b1dbe86649..4340b03cef8f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java @@ -51,6 +51,7 @@ public class TaskPayload private Map dimensionsSpec = null; private Map splitHintSpec = null; + private final Map context = new HashMap<>(); private final List> metricsSpec = new ArrayList<>(); private Integer maxNumConcurrentSubTasks = null; @@ -91,6 +92,7 @@ public Object withId(String taskId) return mapOf( "id", taskId, "type", type, + "context", context, "spec", mapOf( "ioConfig", @@ -298,6 +300,12 @@ public TaskPayload awaitSegmentAvailabilityTimeoutMillis(long millis) return this; } + public TaskPayload context(String key, Object value) + { + this.context.put(key, value); + return this; + } + /** * Creates a map using only the non-null key-value pairs. * From bf465e55396871e7e9f5f19a605d6b3259509cc2 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 20:27:00 +0530 Subject: [PATCH 15/38] Fix deps and tests --- embedded-tests/pom.xml | 6 ++++++ .../embedded/server/EmbeddedHighAvailabilityTest.java | 1 - 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 30dc8130d649..9bea1abf1199 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -164,6 +164,12 @@ + + org.apache.datasketches + datasketches-java + ${datasketches.version} + test + org.scala-lang scala-library diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index b301496724b1..f21a0d2f9feb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -117,7 +117,6 @@ public void test_switchLeader_andVerifyUsingSysTables() o -> o.runTask(taskId, taskPayload.withId(taskId)) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); - cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator1); // Run sys queries, switch leaders, repeat ServerPair overlordPair = createServerPair(overlord1, overlord2); From 0caf10e0a32f720b967e6248cd571de497bf0a92 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 22:47:22 +0530 Subject: [PATCH 16/38] Enable test that validates fix for concurrent append --- .../embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java index 0817c138f269..6c41538393e7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java @@ -31,7 +31,6 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.List; @@ -54,7 +53,6 @@ protected EmbeddedDruidCluster createCluster() } @Test - @Disabled("Enable when concurrent append to interval with unused segment is fixed") public void test_concurrentAppend_toIntervalWithUnusedSegment() { // Run an APPEND task to ingest data into an interval From f4d0017a294478b87a5d78136722321fcfa28a70 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 8 Jul 2025 22:48:37 +0530 Subject: [PATCH 17/38] Fix test name --- .../embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java index 6c41538393e7..50592def9732 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java @@ -53,7 +53,7 @@ protected EmbeddedDruidCluster createCluster() } @Test - public void test_concurrentAppend_toIntervalWithUnusedSegment() + public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() { // Run an APPEND task to ingest data into an interval final String data1Row = "2013-01-01T00:00:00.000Z,shirt,100"; From 3bc246d9ce29954337e97c8a4a11055ee96297a4 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 11 Jul 2025 15:13:19 +0530 Subject: [PATCH 18/38] wip: temp changes --- .../indexing/common/task/TaskPayload.java | 330 ++++++++++++++++++ 1 file changed, 330 insertions(+) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java new file mode 100644 index 000000000000..7d2d69cbce98 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java @@ -0,0 +1,330 @@ +/* + * 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.task; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.ISE; +import org.joda.time.Interval; + +import java.io.File; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Builder for the raw Map-based payload of a {@code Task}. + *

+ * The builder does not use any defaults and all required fields must be set + * explicitly. + * + * @see #ofType(String) to create a builder + */ +public class TaskPayload +{ + private final String type; + + private String dataSource; + private Map inputSource = null; + private Map inputFormat = null; + private Map partitionsSpec = null; + private Map granularitySpec = null; + private Map timestampSpec = null; + private Map dimensionsSpec = null; + private Map splitHintSpec = null; + + private final Map context = new HashMap<>(); + private final List> metricsSpec = new ArrayList<>(); + + private Integer maxNumConcurrentSubTasks = null; + private Boolean forceGuaranteedRollup = null; + private Long awaitSegmentAvailabilityTimeoutMillis = null; + private Boolean appendToExisting = null; + + private TaskPayload(String type) + { + this.type = type; + } + + /** + * Initializes builder for a new {@code Task} for the given datasource. + */ + public static TaskPayload ofType(String type) + { + return new TaskPayload(type); + } + + public TaskPayload dataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + /** + * Creates a raw Map-based payload for a {@code Task} that may be submitted to + * the Overlord using {@code OverlordClient.runTask()}. + */ + public Object withId(String taskId) + { + Preconditions.checkNotNull(taskId, "Task ID must not be null"); + Preconditions.checkNotNull(type, "Task type must be specified"); + Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); + Preconditions.checkNotNull(dataSource, "'dataSource' must be specified"); + + return mapOf( + "id", taskId, + "type", type, + "context", context, + "spec", + mapOf( + "ioConfig", + mapOf( + "type", type, + "inputSource", inputSource, + "inputFormat", inputFormat, + "appendToExisting", appendToExisting + ), + "tuningConfig", + mapOf( + "type", type, + "partitionsSpec", partitionsSpec, + "maxNumConcurrentSubTasks", maxNumConcurrentSubTasks, + "forceGuaranteedRollup", forceGuaranteedRollup, + "splitHintSpec", splitHintSpec, + "awaitSegmentAvailabilityTimeoutMillis", awaitSegmentAvailabilityTimeoutMillis + ), + "dataSchema", + mapOf( + "dataSource", dataSource, + "timestampSpec", timestampSpec, + "dimensionsSpec", dimensionsSpec, + "metricsSpec", metricsSpec.isEmpty() ? null : metricsSpec, + "granularitySpec", granularitySpec + ) + ) + ); + } + + public TaskPayload inputSource(Map jsonMap) + { + this.inputSource = jsonMap; + return this; + } + + public TaskPayload inlineInputSourceWithData(String data) + { + return inputSource(Map.of("type", "inline", "data", data)); + } + + public TaskPayload druidInputSource(String dataSource, Interval interval) + { + return inputSource(Map.of("type", "druid", "interval", interval, "dataSource", dataSource)); + } + + /** + * Gets the absolute path of the given resource files and sets: + *

+   * "inputSource": {
+   *   "type": "local",
+   *   "files": [<absolute-paths-of-given-resource-files>]
+   * }
+   * 
+ */ + public TaskPayload localInputSourceWithFiles(String... files) + { + try { + final List filePaths = new ArrayList<>(); + for (String file : files) { + final URL resourceUrl = getClass().getClassLoader().getResource(file); + if (resourceUrl == null) { + throw new ISE("Could not find file[%s]", file); + } + + filePaths.add(new File(resourceUrl.toURI()).getAbsolutePath()); + } + + return inputSource(Map.of("type", "local", "files", filePaths)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public TaskPayload inputFormat(Map jsonMap) + { + this.inputFormat = jsonMap; + return this; + } + + public TaskPayload jsonInputFormat() + { + this.inputFormat = Map.of("type", "json"); + return this; + } + + public TaskPayload csvInputFormatWithColumns(String... columns) + { + return inputFormat( + Map.of("type", "csv", "findColumnsFromHeader", "false", "columns", List.of(columns)) + ); + } + + public TaskPayload appendToExisting(boolean append) + { + this.appendToExisting = append; + return this; + } + + public TaskPayload partitionsSpec(Map jsonMap) + { + this.partitionsSpec = jsonMap; + return this; + } + + public TaskPayload dynamicPartitionWithMaxRows(int maxRowsPerSegment) + { + this.partitionsSpec = Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment); + return this; + } + + public TaskPayload granularitySpec(Map jsonMap) + { + this.granularitySpec = jsonMap; + return this; + } + + public TaskPayload granularitySpec(String segmentGranularity, String queryGranularity, boolean rollup) + { + this.granularitySpec = Map.of( + "segmentGranularity", segmentGranularity, + "queryGranularity", queryGranularity, + "rollup", rollup + ); + return this; + } + + /** + * Sets {@code "granularitySpec": {"segmentGranularity": }}. + */ + public TaskPayload segmentGranularity(String granularity) + { + return granularitySpec(Map.of("segmentGranularity", granularity)); + } + + public TaskPayload timestampSpec(Map jsonMap) + { + this.timestampSpec = jsonMap; + return this; + } + + public TaskPayload isoTimestampColumn(String timestampColumn) + { + return timestampSpec(Map.of("format", "iso", "column", timestampColumn)); + } + + public TaskPayload timestampColumn(String timestampColumn) + { + return timestampSpec(Map.of("column", timestampColumn)); + } + + public TaskPayload dimensionsSpec(Map jsonMap) + { + this.dimensionsSpec = jsonMap; + return this; + } + + /** + * Sets {@code "dimensionSpec": {"dimensions": []}}. + */ + public TaskPayload dimensions(String... dimensions) + { + return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); + } + + public TaskPayload metricAggregate(String name, String type) + { + return metricAggregate(name, type, name); + } + + public TaskPayload metricAggregate(String name, String type, String fieldName) + { + this.metricsSpec.add(mapOf("type", type, "name", name, "fieldName", fieldName)); + return this; + } + + public TaskPayload metricAggregate(Map jsonMap) + { + this.metricsSpec.add(Map.copyOf(jsonMap)); + return this; + } + + public TaskPayload maxConcurrentSubTasks(int maxNumConcurrentSubTasks) + { + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + return this; + } + + public TaskPayload forceGuaranteedRollup(boolean rollup) + { + this.forceGuaranteedRollup = rollup; + return this; + } + + public TaskPayload splitHintSpec(Map jsonMap) + { + this.splitHintSpec = jsonMap; + return this; + } + + public TaskPayload awaitSegmentAvailabilityTimeoutMillis(long millis) + { + this.awaitSegmentAvailabilityTimeoutMillis = millis; + return this; + } + + public TaskPayload context(String key, Object value) + { + this.context.put(key, value); + return this; + } + + /** + * Creates a map using only the non-null key-value pairs. + * + * @param kvPairs key1, value1, key2, value2, ... + * @return null if none of the key-value pairs are non-null. + */ + private static Map mapOf(Object... kvPairs) + { + if (kvPairs.length % 2 > 0) { + throw new ISE("Key and value must be given in pairs."); + } + + final Map map = new HashMap<>(); + for (int i = 0; i < kvPairs.length - 1; i += 2) { + if (kvPairs[i] != null && kvPairs[i + 1] != null) { + map.put(kvPairs[i], kvPairs[i + 1]); + } + } + + return map.isEmpty() ? null : Map.copyOf(map); + } +} From 2aae4b7a03bb4f661b2bc0406df06f6d9fb3943a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 11 Jul 2025 20:15:35 +0530 Subject: [PATCH 19/38] Convert TaskPayload to TaskBuilder --- embedded-tests/pom.xml | 6 + .../compact/EmbeddedAutoCompactionTest.java | 141 ++++--- .../EmbeddedConcurrentAppendReplaceTest.java | 12 +- .../EmbeddedIndexParallelTaskTest.java | 119 +++--- .../indexing/EmbeddedIndexTaskTest.java | 5 +- .../embedded/indexing/TaskPayload.java | 330 ---------------- .../server/EmbeddedCoordinatorClientTest.java | 55 +-- .../server/EmbeddedHighAvailabilityTest.java | 26 +- .../indexing/common/task/TaskBuilder.java | 363 ++++++++++++++++++ .../indexing/common/task/TaskPayload.java | 330 ---------------- 10 files changed, 563 insertions(+), 824 deletions(-) delete mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 65aac4f05db0..8deebafdcaa8 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -124,6 +124,12 @@ ${project.parent.version} test-jar
+ + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + org.apache.druid.extensions druid-kafka-indexing-service diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index 1c1c79661657..8b49912bbdaf 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -28,15 +28,20 @@ import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -80,7 +85,6 @@ import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.Resources; -import org.apache.druid.testing.embedded.indexing.TaskPayload; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.apache.druid.timeline.DataSegment; import org.hamcrest.Matcher; @@ -102,7 +106,6 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.function.Supplier; @@ -114,9 +117,9 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase { private static final Logger LOG = new Logger(EmbeddedAutoCompactionTest.class); - private static final Supplier INDEX_TASK = - () -> TaskPayload - .ofType("index") + private static final Supplier> INDEX_TASK = + () -> TaskBuilder + .ofTypeIndex() .jsonInputFormat() .localInputSourceWithFiles( Resources.WIKIPEDIA_1_JSON, @@ -129,20 +132,22 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase "language", "tags", "user", "unpatrolled", "newPage", "robot", "anonymous", "namespace", "continent", "country", "region", "city" ) - .metricAggregate("ingested_events", "count") - .metricAggregate("added", "doubleSum") - .metricAggregate("deleted", "doubleSum") - .metricAggregate("delta", "doubleSum") - .metricAggregate("thetaSketch", "thetaSketch", "user") - .metricAggregate("HLLSketchBuild", "HLLSketchBuild", "user") - .metricAggregate("quantilesDoublesSketch", "quantilesDoublesSketch", "delta") + .metricAggregates( + new CountAggregatorFactory("ingested_events"), + new DoubleSumAggregatorFactory("added", "added"), + new DoubleSumAggregatorFactory("deleted", "deleted"), + new DoubleSumAggregatorFactory("delta", "delta"), + new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), + new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), + new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null) + ) .dynamicPartitionWithMaxRows(3) .granularitySpec("DAY", "SECOND", true) .appendToExisting(false); - private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = + private static final Supplier> INDEX_TASK_WITH_GRANULARITY_SPEC = () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); - private static final Supplier INDEX_TASK_WITH_DIMENSION_SPEC = + private static final Supplier> INDEX_TASK_WITH_DIMENSION_SPEC = () -> INDEX_TASK.get().granularitySpec("DAY", "DAY", true); private static final String SELECT_APPROX_COUNT_DISTINCT = @@ -169,56 +174,67 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase "Crimson Typhoon,1,905.0,9050.0" ) ); - private static final Supplier INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = - () -> TaskPayload - .ofType("index_parallel") + private static final Supplier> INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = + () -> TaskBuilder + .ofTypeIndexParallel() .jsonInputFormat() .inlineInputSourceWithData(Resources.JSON_DATA_2_ROWS) .isoTimestampColumn("timestamp") .appendToExisting(true) .granularitySpec("DAY", "HOUR", true) - .metricAggregate("ingested_events", "count") - .metricAggregate("thetaSketch", "thetaSketch", "user") - .metricAggregate("HLLSketchBuild", "HLLSketchBuild", "user") - .metricAggregate("quantilesDoublesSketch", "quantilesDoublesSketch", "delta") - .metricAggregate("sum_added", "longSum", "added") - .metricAggregate("sum_deleted", "longSum", "deleted") - .metricAggregate("sum_delta", "longSum", "delta") - .metricAggregate("sum_deltaBucket", "longSum", "deltaBucket") - .metricAggregate("sum_commentLength", "longSum", "commentLength") + .metricAggregates( + new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), + new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), + new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null), + new CountAggregatorFactory("ingested_events"), + new LongSumAggregatorFactory("sum_added", "added"), + new LongSumAggregatorFactory("sum_deleted", "deleted"), + new LongSumAggregatorFactory("sum_delta", "delta"), + new LongSumAggregatorFactory("sum_deltaBucket", "deltaBucket"), + new LongSumAggregatorFactory("sum_commentLength", "commentLength") + ) .dimensions( "isRobot", "language", "flags", "isUnpatrolled", "page", "diffUrl", "comment", "isNew", "isMinor", "isAnonymous", "namespace" ); - private static final Supplier INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = - () -> TaskPayload - .ofType("index_parallel") + private static final Supplier> INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = + () -> TaskBuilder + .ofTypeIndexParallel() .jsonInputFormat() .inlineInputSourceWithData(Resources.JSON_DATA_1_ROW) .isoTimestampColumn("timestamp") .granularitySpec("DAY", "HOUR", false) .appendToExisting(true) - .dimensionsSpec( - Map.of( - "dimensions", - List.of( - "isRobot", - "language", "flags", "isUnpatrolled", "page", "diffUrl", - Map.of("type", "long", "name", "added"), - "comment", - Map.of("type", "long", "name", "commentLength"), - "isNew", "isMinor", - Map.of("type", "long", "name", "delta"), - "isAnonymous", "user", - Map.of("type", "long", "name", "deltaBucket"), - Map.of("type", "long", "name", "deleted"), - "namespace", "cityName", "countryName", "regionIsoCode", - "metroCode", "countryIsoCode", "regionName" - ) + .dataSchema( + d -> d.withDimensions( + new StringDimensionSchema("isRobot"), + new StringDimensionSchema("language"), + new StringDimensionSchema("flags"), + new StringDimensionSchema("isUnpatrolled"), + new StringDimensionSchema("page"), + new StringDimensionSchema("diffUrl"), + new LongDimensionSchema("added"), + new StringDimensionSchema("comment"), + new LongDimensionSchema("commentLength"), + new StringDimensionSchema("isNew"), + new StringDimensionSchema("isMinor"), + new LongDimensionSchema("delta"), + new StringDimensionSchema("isAnonymous"), + new StringDimensionSchema("user"), + new LongDimensionSchema("deltaBucket"), + new LongDimensionSchema("deleted"), + new StringDimensionSchema("namespace"), + new StringDimensionSchema("cityName"), + new StringDimensionSchema("countryName"), + new StringDimensionSchema("regionIsoCode"), + new StringDimensionSchema("metroCode"), + new StringDimensionSchema("countryIsoCode"), + new StringDimensionSchema("regionName") ) ); + private static final int MAX_ROWS_PER_SEGMENT_COMPACTED = 10000; private static final Period NO_SKIP_OFFSET = Period.seconds(0); private static final FixedIntervalOrderPolicy COMPACT_NOTHING_POLICY = new FixedIntervalOrderPolicy(List.of()); @@ -242,8 +258,8 @@ protected EmbeddedDruidCluster createCluster() .addExtension(SketchModule.class) .addExtension(HllSketchModule.class) .addExtension(DoublesSketchModule.class) - .addServer(coordinator) .addServer(overlord) + .addServer(coordinator) .addServer(broker) .addServer(new EmbeddedIndexer().addProperty("druid.worker.capacity", "10")) .addServer(new EmbeddedHistorical()) @@ -1369,7 +1385,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegment() throws Exception { updateCompactionTaskSlot(1, 1); - Map specs = Map.of("segmentGranularity", "MONTH", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); + final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); + GranularitySpec specs = new UniformGranularitySpec(Granularities.MONTH, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { verifyScanResult("added", "57.0||459.0"); @@ -1415,7 +1432,8 @@ public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegm public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment(CompactionEngine engine) throws Exception { updateCompactionTaskSlot(1, 1); - Map specs = Map.of("segmentGranularity", "WEEK", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); + final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); + GranularitySpec specs = new UniformGranularitySpec(Granularities.WEEK, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { verifyScanResult("added", "57.0||459.0"); @@ -1456,7 +1474,8 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe @Test() public void testAutoCompactionDutyWithRollup() throws Exception { - Map specs = Map.of("segmentGranularity", "DAY", "queryGranularity", "DAY", "rollup", false, "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); + final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); + GranularitySpec specs = new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { verifyScanResult("added", "57.0||459.0"); @@ -1485,7 +1504,8 @@ public void testAutoCompactionDutyWithRollup() throws Exception @MethodSource("getEngine") public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) throws Exception { - Map specs = Map.of("segmentGranularity", "DAY", "queryGranularity", "NONE", "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); + final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); + GranularitySpec specs = new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, true, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { verifyScanResult("added", "57.0||459.0"); @@ -1645,11 +1665,12 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws @Test public void testAutoCompactionDutyWithOverlappingInterval() throws Exception { + final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); // Create WEEK segment with 2013-08-26 to 2013-09-02 - Map specs = Map.of("segmentGranularity", "WEEK", "queryGranularity", "NONE", "intervals", List.of("2013-08-31T-07/2013-09-02T-07")); + GranularitySpec specs = new UniformGranularitySpec(Granularities.WEEK, Granularities.NONE, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); // Create MONTH segment with 2013-09-01 to 2013-10-01 - specs = Map.of("segmentGranularity", "MONTH", "queryGranularity", "NONE", "intervals", List.of("2013-09-01T-07/2013-09-02T-07")); + specs = new UniformGranularitySpec(Granularities.MONTH, Granularities.NONE, false, List.of(new Interval("2013-09-01/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1693,20 +1714,20 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception } } - private void loadData(Supplier updatePayload) + private void loadData(Supplier> updatePayload) { - loadData(updatePayload, Map.of()); + loadData(updatePayload, null); } - private void loadData(Supplier taskPayloadSupplier, Map granularitySpec) + private void loadData(Supplier> taskPayloadSupplier, GranularitySpec granularitySpec) { - final TaskPayload taskPayload = taskPayloadSupplier.get().dataSource(fullDatasourceName); - if (!granularitySpec.isEmpty()) { - taskPayload.granularitySpec(granularitySpec); + final TaskBuilder taskBuilder = taskPayloadSupplier.get().dataSource(fullDatasourceName); + if (granularitySpec != null) { + taskBuilder.granularitySpec(granularitySpec); } final String taskId = EmbeddedClusterApis.newTaskId(fullDatasourceName); - cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskPayload.withId(taskId))); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.withId(taskId))); LOG.info("Submitted task[%s] to load data", taskId); cluster.callApi().waitForTaskToSucceed(taskId, overlord); cluster.callApi().waitForAllSegmentsToBeAvailable(fullDatasourceName, coordinator); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java index 50592def9732..ee75eefd8445 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java @@ -19,6 +19,8 @@ package org.apache.druid.testing.embedded.indexing; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.testing.embedded.EmbeddedBroker; import org.apache.druid.testing.embedded.EmbeddedClusterApis; @@ -45,8 +47,8 @@ protected EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() - .addServer(coordinator) .addServer(overlord) + .addServer(coordinator) .addServer(new EmbeddedIndexer()) .addServer(new EmbeddedBroker()) .addServer(new EmbeddedHistorical()); @@ -58,8 +60,8 @@ public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() // Run an APPEND task to ingest data into an interval final String data1Row = "2013-01-01T00:00:00.000Z,shirt,100"; final String task1 = EmbeddedClusterApis.newTaskId(dataSource); - final TaskPayload taskPayload = - TaskPayload.ofType("index") + final TaskBuilder taskBuilder = + TaskBuilder.ofTypeIndex() .dataSource(dataSource) .csvInputFormatWithColumns("time", "item", "value") .isoTimestampColumn("time") @@ -68,7 +70,7 @@ public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() .context("useConcurrentLocks", true) .dimensions(); cluster.callApi().onLeaderOverlord( - o -> o.runTask(task1, taskPayload.withId(task1)) + o -> o.runTask(task1, taskBuilder.withId(task1)) ); cluster.callApi().waitForTaskToSucceed(task1, overlord); @@ -87,7 +89,7 @@ public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() // Run the APPEND task again with a different taskId final String task2 = EmbeddedClusterApis.newTaskId(dataSource); cluster.callApi().onLeaderOverlord( - o -> o.runTask(task2, taskPayload.withId(task2)) + o -> o.runTask(task2, taskBuilder.withId(task2)) ); cluster.callApi().waitForTaskToSucceed(task2, overlord); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index cac98693e43e..aa6c69e928c6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -19,10 +19,21 @@ package org.apache.druid.testing.embedded.indexing; +import org.apache.druid.data.input.MaxSizeSplitHintSpec; +import org.apache.druid.data.input.SegmentsSplitHintSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.testing.embedded.EmbeddedBroker; import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; @@ -38,7 +49,6 @@ import org.junit.jupiter.params.provider.MethodSource; import java.util.List; -import java.util.Map; import java.util.Optional; /** @@ -61,20 +71,20 @@ protected EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .addServer(overlord) .addServer(coordinator) .addServer(indexer) - .addServer(overlord) .addServer(historical) .addServer(broker) .addServer(new EmbeddedRouter()); } - public static List> getTestParamPartitionsSpec() + public static List getTestParamPartitionsSpec() { return List.of( - Map.of("type", "dynamic"), - Map.of("type", "hashed", "numShards", 2), - Map.of("type", "range", "targetRowsPerSegment", 2, "partitionDimensions", List.of("namespace")) + new DynamicPartitionsSpec(null, null), + new HashedPartitionsSpec(null, 2, null, null), + new SingleDimensionPartitionsSpec(2, null, "namespace", false) ); } @@ -85,15 +95,16 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( final long segmentAvailabilityTimeoutMillis ) { - final TaskPayload indexTask = - TaskPayload.ofType("index_parallel") + final TaskBuilder indexTask = + TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource) - .partitionsSpec(Map.of("type", "dynamic")) .timestampColumn("timestamp") - .inputFormat(Map.of("type", "json")) + .jsonInputFormat() .localInputSourceWithFiles(Resources.WIKIPEDIA_1_JSON) .dimensions() - .awaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis); + .tuningConfig( + t -> t.withAwaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis) + ); final String taskId = runTask(indexTask, dataSource); @@ -114,30 +125,34 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( @MethodSource("getTestParamPartitionsSpec") @ParameterizedTest(name = "partitionsSpec={0}") - public void test_runIndexTask_andReindexIntoAnotherDatasource(Map partitionsSpec) + public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec partitionsSpec) { - final boolean isRollup = !partitionsSpec.get("type").equals("dynamic"); + final boolean isRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - final TaskPayload indexTask = - TaskPayload.ofType("index_parallel") + final TaskBuilder indexTask = + TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource) - .partitionsSpec(partitionsSpec) - .forceGuaranteedRollup(isRollup) .timestampColumn("timestamp") - .inputFormat(Map.of("type", "json")) + .jsonInputFormat() .localInputSourceWithFiles( - Resources.WIKIPEDIA_1_JSON, - Resources.WIKIPEDIA_2_JSON, - Resources.WIKIPEDIA_3_JSON - ) + Resources.WIKIPEDIA_1_JSON, + Resources.WIKIPEDIA_2_JSON, + Resources.WIKIPEDIA_3_JSON + ) .segmentGranularity("DAY") .dimensions("namespace", "page", "language") - .metricAggregate("added", "doubleSum") - .metricAggregate("deleted", "doubleSum") - .metricAggregate("delta", "doubleSum") - .metricAggregate("count", "count") - .maxConcurrentSubTasks(10) - .splitHintSpec(Map.of("type", "maxSize", "maxSplitSize", 1)); + .metricAggregates( + new DoubleSumAggregatorFactory("added", "added"), + new DoubleSumAggregatorFactory("deleted", "deleted"), + new DoubleSumAggregatorFactory("delta", "delta"), + new CountAggregatorFactory("count") + ) + .tuningConfig( + t -> t.withPartitionsSpec(partitionsSpec) + .withForceGuaranteedRollup(isRollup) + .withMaxNumConcurrentSubTasks(10) + .withSplitHintSpec(new MaxSizeSplitHintSpec(1, null)) + ); runTask(indexTask, dataSource); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); @@ -145,16 +160,25 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(Map reindexTaskSplitBySegment = + TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource2) - .dimensionsSpec(Map.of("dimensionExclusions", List.of("robot", "continent"))) .isoTimestampColumn("ignored") .druidInputSource(dataSource, Intervals.ETERNITY) - .maxConcurrentSubTasks(10) - .forceGuaranteedRollup(isRollup) - .partitionsSpec(partitionsSpec) - .splitHintSpec(Map.of("type", "segments", "maxInputSegmentBytesPerTask", 1)); + .tuningConfig( + t -> t.withPartitionsSpec(partitionsSpec) + .withMaxNumConcurrentSubTasks(10) + .withForceGuaranteedRollup(isRollup) + .withSplitHintSpec(new SegmentsSplitHintSpec(HumanReadableBytes.valueOf(1), null)) + ) + .dataSchema( + d -> d.withDimensions( + DimensionsSpec + .builder() + .setDimensionExclusions(List.of("robot", "continent")) + .build() + ) + ); runTask(reindexTaskSplitBySegment, dataSource2); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource2, coordinator); @@ -162,23 +186,32 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(Map reindexTaskSplitByFile = + TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource3) - .partitionsSpec(partitionsSpec) - .forceGuaranteedRollup(isRollup) .timestampColumn("timestamp") .druidInputSource(dataSource, Intervals.ETERNITY) - .dimensionsSpec(Map.of("dimensionExclusions", List.of("robot", "continent"))) - .maxConcurrentSubTasks(10) - .splitHintSpec(Map.of("type", "maxSize", "maxNumFiles", 1)); + .dataSchema( + d -> d.withDimensions( + DimensionsSpec + .builder() + .setDimensionExclusions(List.of("robot", "continent")) + .build() + ) + ) + .tuningConfig( + t -> t.withPartitionsSpec(partitionsSpec) + .withMaxNumConcurrentSubTasks(10) + .withForceGuaranteedRollup(isRollup) + .withSplitHintSpec(new MaxSizeSplitHintSpec(null, 1)) + ); runTask(reindexTaskSplitByFile, dataSource3); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource3, coordinator); runQueries(dataSource3); } - private String runTask(TaskPayload taskBuilder, String dataSource) + private String runTask(TaskBuilder taskBuilder, String dataSource) { final String taskId = EmbeddedClusterApis.newTaskId(dataSource); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.withId(taskId))); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java index 3fa4126a445f..fcd6c7ebd7f0 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java @@ -20,6 +20,7 @@ package org.apache.druid.testing.embedded.indexing; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; @@ -62,9 +63,9 @@ public EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .addServer(overlord) .addServer(coordinator) .addServer(indexer) - .addServer(overlord) .addServer(historical) .addServer(broker) .addServer(new EmbeddedRouter()); @@ -143,7 +144,7 @@ public void test_run100Tasks_concurrently() private Object createIndexTaskForInlineData(String taskId, String inlineDataCsv) { - return TaskPayload.ofType("index") + return TaskBuilder.ofTypeIndex() .dataSource(dataSource) .isoTimestampColumn("time") .csvInputFormatWithColumns("time", "item", "value") diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java deleted file mode 100644 index 4340b03cef8f..000000000000 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskPayload.java +++ /dev/null @@ -1,330 +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.testing.embedded.indexing; - -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.ISE; -import org.joda.time.Interval; - -import java.io.File; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Builder for the raw Map-based payload of a {@code Task}. - *

- * The builder does not use any defaults and all required fields must be set - * explicitly. - * - * @see #ofType(String) to create a builder - */ -public class TaskPayload -{ - private final String type; - - private String dataSource; - private Map inputSource = null; - private Map inputFormat = null; - private Map partitionsSpec = null; - private Map granularitySpec = null; - private Map timestampSpec = null; - private Map dimensionsSpec = null; - private Map splitHintSpec = null; - - private final Map context = new HashMap<>(); - private final List> metricsSpec = new ArrayList<>(); - - private Integer maxNumConcurrentSubTasks = null; - private Boolean forceGuaranteedRollup = null; - private Long awaitSegmentAvailabilityTimeoutMillis = null; - private Boolean appendToExisting = null; - - private TaskPayload(String type) - { - this.type = type; - } - - /** - * Initializes builder for a new {@code Task} for the given datasource. - */ - public static TaskPayload ofType(String type) - { - return new TaskPayload(type); - } - - public TaskPayload dataSource(String dataSource) - { - this.dataSource = dataSource; - return this; - } - - /** - * Creates a raw Map-based payload for a {@code Task} that may be submitted to - * the Overlord using {@code OverlordClient.runTask()}. - */ - public Object withId(String taskId) - { - Preconditions.checkNotNull(taskId, "Task ID must not be null"); - Preconditions.checkNotNull(type, "Task type must be specified"); - Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); - Preconditions.checkNotNull(dataSource, "'dataSource' must be specified"); - - return mapOf( - "id", taskId, - "type", type, - "context", context, - "spec", - mapOf( - "ioConfig", - mapOf( - "type", type, - "inputSource", inputSource, - "inputFormat", inputFormat, - "appendToExisting", appendToExisting - ), - "tuningConfig", - mapOf( - "type", type, - "partitionsSpec", partitionsSpec, - "maxNumConcurrentSubTasks", maxNumConcurrentSubTasks, - "forceGuaranteedRollup", forceGuaranteedRollup, - "splitHintSpec", splitHintSpec, - "awaitSegmentAvailabilityTimeoutMillis", awaitSegmentAvailabilityTimeoutMillis - ), - "dataSchema", - mapOf( - "dataSource", dataSource, - "timestampSpec", timestampSpec, - "dimensionsSpec", dimensionsSpec, - "metricsSpec", metricsSpec.isEmpty() ? null : metricsSpec, - "granularitySpec", granularitySpec - ) - ) - ); - } - - public TaskPayload inputSource(Map jsonMap) - { - this.inputSource = jsonMap; - return this; - } - - public TaskPayload inlineInputSourceWithData(String data) - { - return inputSource(Map.of("type", "inline", "data", data)); - } - - public TaskPayload druidInputSource(String dataSource, Interval interval) - { - return inputSource(Map.of("type", "druid", "interval", interval, "dataSource", dataSource)); - } - - /** - * Gets the absolute path of the given resource files and sets: - *

-   * "inputSource": {
-   *   "type": "local",
-   *   "files": [<absolute-paths-of-given-resource-files>]
-   * }
-   * 
- */ - public TaskPayload localInputSourceWithFiles(String... files) - { - try { - final List filePaths = new ArrayList<>(); - for (String file : files) { - final URL resourceUrl = getClass().getClassLoader().getResource(file); - if (resourceUrl == null) { - throw new ISE("Could not find file[%s]", file); - } - - filePaths.add(new File(resourceUrl.toURI()).getAbsolutePath()); - } - - return inputSource(Map.of("type", "local", "files", filePaths)); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - public TaskPayload inputFormat(Map jsonMap) - { - this.inputFormat = jsonMap; - return this; - } - - public TaskPayload jsonInputFormat() - { - this.inputFormat = Map.of("type", "json"); - return this; - } - - public TaskPayload csvInputFormatWithColumns(String... columns) - { - return inputFormat( - Map.of("type", "csv", "findColumnsFromHeader", "false", "columns", List.of(columns)) - ); - } - - public TaskPayload appendToExisting(boolean append) - { - this.appendToExisting = append; - return this; - } - - public TaskPayload partitionsSpec(Map jsonMap) - { - this.partitionsSpec = jsonMap; - return this; - } - - public TaskPayload dynamicPartitionWithMaxRows(int maxRowsPerSegment) - { - this.partitionsSpec = Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment); - return this; - } - - public TaskPayload granularitySpec(Map jsonMap) - { - this.granularitySpec = jsonMap; - return this; - } - - public TaskPayload granularitySpec(String segmentGranularity, String queryGranularity, boolean rollup) - { - this.granularitySpec = Map.of( - "segmentGranularity", segmentGranularity, - "queryGranularity", queryGranularity, - "rollup", rollup - ); - return this; - } - - /** - * Sets {@code "granularitySpec": {"segmentGranularity": }}. - */ - public TaskPayload segmentGranularity(String granularity) - { - return granularitySpec(Map.of("segmentGranularity", granularity)); - } - - public TaskPayload timestampSpec(Map jsonMap) - { - this.timestampSpec = jsonMap; - return this; - } - - public TaskPayload isoTimestampColumn(String timestampColumn) - { - return timestampSpec(Map.of("format", "iso", "column", timestampColumn)); - } - - public TaskPayload timestampColumn(String timestampColumn) - { - return timestampSpec(Map.of("column", timestampColumn)); - } - - public TaskPayload dimensionsSpec(Map jsonMap) - { - this.dimensionsSpec = jsonMap; - return this; - } - - /** - * Sets {@code "dimensionSpec": {"dimensions": []}}. - */ - public TaskPayload dimensions(String... dimensions) - { - return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); - } - - public TaskPayload metricAggregate(String name, String type) - { - return metricAggregate(name, type, name); - } - - public TaskPayload metricAggregate(String name, String type, String fieldName) - { - this.metricsSpec.add(mapOf("type", type, "name", name, "fieldName", fieldName)); - return this; - } - - public TaskPayload metricAggregate(Map jsonMap) - { - this.metricsSpec.add(Map.copyOf(jsonMap)); - return this; - } - - public TaskPayload maxConcurrentSubTasks(int maxNumConcurrentSubTasks) - { - this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; - return this; - } - - public TaskPayload forceGuaranteedRollup(boolean rollup) - { - this.forceGuaranteedRollup = rollup; - return this; - } - - public TaskPayload splitHintSpec(Map jsonMap) - { - this.splitHintSpec = jsonMap; - return this; - } - - public TaskPayload awaitSegmentAvailabilityTimeoutMillis(long millis) - { - this.awaitSegmentAvailabilityTimeoutMillis = millis; - return this; - } - - public TaskPayload context(String key, Object value) - { - this.context.put(key, value); - return this; - } - - /** - * Creates a map using only the non-null key-value pairs. - * - * @param kvPairs key1, value1, key2, value2, ... - * @return null if none of the key-value pairs are non-null. - */ - private static Map mapOf(Object... kvPairs) - { - if (kvPairs.length % 2 > 0) { - throw new ISE("Key and value must be given in pairs."); - } - - final Map map = new HashMap<>(); - for (int i = 0; i < kvPairs.length - 1; i += 2) { - if (kvPairs[i] != null && kvPairs[i + 1] != null) { - map.put(kvPairs[i], kvPairs[i + 1]); - } - } - - return map.isEmpty() ? null : Map.copyOf(map); - } -} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java index 218fe092c131..1317d381bf9c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java @@ -22,15 +22,14 @@ import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; @@ -41,9 +40,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentStatusInCluster; import org.joda.time.Interval; -import org.junit.Assert; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -70,9 +67,9 @@ protected EmbeddedDruidCluster createCluster() return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .addServer(overlord) .addServer(coordinator) .addServer(indexer) - .addServer(overlord) .addServer(historical) .addServer(broker); } @@ -89,11 +86,6 @@ public void test_findCurrentLeader() public void test_isHandoffComplete() { runIndexTask(); - coordinator.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("segment/loadQueue/success") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(1) - ); final List segments = new ArrayList<>( overlord.bindings().segmentsMetadataStorage().retrieveAllUsedSegments(dataSource, null) ); @@ -108,7 +100,6 @@ public void test_isHandoffComplete() } @Test - @Disabled("This test is flaky due to Coordinator not refreshing its metadata") @Timeout(20) public void test_fetchSegment() { @@ -124,7 +115,7 @@ public void test_fetchSegment() true ) ); - Assert.assertEquals(firstSegment, result); + Assertions.assertEquals(firstSegment, result); } @Test @@ -132,11 +123,6 @@ public void test_fetchSegment() public void test_fetchServerViewSegments() { runIndexTask(); - coordinator.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("segment/loadQueue/success") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(1) - ); final List segments = new ArrayList<>( overlord.bindings().segmentsMetadataStorage().retrieveAllUsedSegments(dataSource, null) @@ -155,11 +141,6 @@ public void test_fetchServerViewSegments() public void test_fetchUsedSegments() { runIndexTask(); - coordinator.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("segment/loadQueue/success") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(1) - ); final List segments = new ArrayList<>( overlord.bindings().segmentsMetadataStorage().retrieveAllUsedSegments(dataSource, null) @@ -176,11 +157,6 @@ public void test_fetchUsedSegments() public void test_fetchAllUsedSegmentsWithOvershadowedStatus() throws IOException { runIndexTask(); - coordinator.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("segment/loadQueue/success") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(1) - ); try (CloseableIterator iterator = cluster.callApi().onLeaderCoordinator( c -> c.fetchAllUsedSegmentsWithOvershadowedStatus(Set.of(dataSource), true)) @@ -200,27 +176,24 @@ public void test_loadRules() c -> c.updateRulesForDatasource(dataSource, List.of(broadcastRule)) ); Map> rules = cluster.callApi().onLeaderCoordinator(CoordinatorClient::getRulesForAllDatasources); - Assertions.assertTrue(!rules.isEmpty()); + Assertions.assertFalse(rules.isEmpty()); Assertions.assertEquals(List.of(broadcastRule), rules.get(dataSource)); } private void runIndexTask() { final String taskId = IdUtils.getRandomId(); - final Object task = EmbeddedClusterApis.createTaskFromPayload( - taskId, - StringUtils.format( - Resources.INDEX_TASK_PAYLOAD_WITH_INLINE_DATA, - StringUtils.replace(Resources.CSV_DATA_10_DAYS, "\n", "\\n"), - dataSource - ) - ); + final Task task = TaskBuilder.ofTypeIndex() + .dataSource(dataSource) + .isoTimestampColumn("time") + .csvInputFormatWithColumns("time", "item", "value") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .segmentGranularity("DAY") + .dimensions() + .withId(taskId); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); cluster.callApi().waitForTaskToSucceed(taskId, overlord); - indexer.latchableEmitter().waitForEvent( - e -> e.hasMetricName("ingest/handoff/count") - .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)) - ); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index f21a0d2f9feb..2097455758dc 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -24,6 +24,8 @@ import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.HttpClient; @@ -39,7 +41,6 @@ import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.Resources; -import org.apache.druid.testing.embedded.indexing.TaskPayload; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -73,10 +74,10 @@ protected EmbeddedDruidCluster createCluster() return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() - .addServer(coordinator1) - .addServer(coordinator2) .addServer(overlord1) .addServer(overlord2) + .addServer(coordinator1) + .addServer(coordinator2) .addServer(indexer) .addServer(broker) .addServer(router); @@ -106,16 +107,15 @@ public void test_switchLeader_andVerifyUsingSysTables() { // Ingest some data so that we can query sys tables later final String taskId = EmbeddedClusterApis.newTaskId(dataSource); - final TaskPayload taskPayload = - TaskPayload.ofType("index") - .dataSource(dataSource) - .csvInputFormatWithColumns("time", "item", "value") - .isoTimestampColumn("time") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) - .dimensions(); - cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskPayload.withId(taskId)) - ); + final Task task = TaskBuilder + .ofTypeIndex() + .dataSource(dataSource) + .csvInputFormatWithColumns("time", "item", "value") + .isoTimestampColumn("time") + .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .dimensions() + .withId(taskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); cluster.callApi().waitForTaskToSucceed(taskId, overlord1); // Run sys queries, switch leaders, repeat diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java new file mode 100644 index 000000000000..d8bd82044c35 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java @@ -0,0 +1,363 @@ +/* + * 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.task; + +import com.google.common.base.Preconditions; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.config.TaskConfigBuilder; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.indexing.DataSchema; +import org.joda.time.Interval; + +import java.io.File; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Builder for the {@link Task} objects. + *

+ * The builder does not use any defaults and all required fields must be set + * explicitly. + * + * @param Type of task created by this builder. + * @param Type of tuning config used by this builder. + * @see #ofTypeIndex() + * @see #dataSchema(Consumer) to specify the {@code dataSchema}. + * @see #tuningConfig(Consumer) to specify the {@code tuningConfig}. + */ +public class TaskBuilder +{ + private InputSource inputSource = null; + private InputFormat inputFormat = null; + + private final Map context = new HashMap<>(); + + private Boolean appendToExisting = null; + + private final BuilderType type; + private final DataSchema.Builder dataSchema; + private final TuningConfigBuilder tuningConfig; + + private TaskBuilder(BuilderType type) + { + this.type = type; + this.dataSchema = DataSchema.builder(); + this.tuningConfig = type.tuningConfigBuilder(); + } + + /** + * Initializes builder for a new {@link IndexTask}. + */ + public static TaskBuilder ofTypeIndex() + { + return new TaskBuilder<>(new Index()); + } + + /** + * Initializes builder for a new {@link ParallelIndexSupervisorTask}. + */ + public static TaskBuilder ofTypeIndexParallel() + { + return new TaskBuilder<>(new ParallelIndex()); + } + + public TaskBuilder dataSource(String dataSource) + { + dataSchema.withDataSource(dataSource); + return this; + } + + /** + * Creates a raw Map-based payload for a {@code Task} that may be submitted to + * the Overlord using {@code OverlordClient.runTask()}. + */ + public T withId(String taskId) + { + Preconditions.checkNotNull(taskId, "Task ID must not be null"); + Preconditions.checkNotNull(type, "Task type must be specified"); + Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); + + return type.buildTask(taskId, this); + } + + public TaskBuilder inputSource(InputSource inputSource) + { + this.inputSource = inputSource; + return this; + } + + public TaskBuilder inlineInputSourceWithData(String data) + { + return inputSource(new InlineInputSource(data)); + } + + public TaskBuilder druidInputSource(String dataSource, Interval interval) + { + return inputSource( + new DruidInputSource( + dataSource, + interval, + null, + null, + null, + null, + TestHelper.getTestIndexIO(), + new NoopCoordinatorClient(), + new SegmentCacheManagerFactory(TestHelper.getTestIndexIO(), TestHelper.JSON_MAPPER), + new TaskConfigBuilder().build() + ) + ); + } + + /** + * Gets the absolute path of the given resource files and sets: + *

+   * "inputSource": {
+   *   "type": "local",
+   *   "files": [<absolute-paths-of-given-resource-files>]
+   * }
+   * 
+ */ + public TaskBuilder localInputSourceWithFiles(String... resources) + { + try { + final List files = new ArrayList<>(); + for (String file : resources) { + final URL resourceUrl = getClass().getClassLoader().getResource(file); + if (resourceUrl == null) { + throw new ISE("Could not find file[%s]", file); + } + + files.add(new File(resourceUrl.toURI())); + } + + return inputSource( + new LocalInputSource(null, null, files, null) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public TaskBuilder inputFormat(InputFormat inputFormat) + { + this.inputFormat = inputFormat; + return this; + } + + public TaskBuilder jsonInputFormat() + { + return inputFormat( + new JsonInputFormat(null, null, null, null, null) + ); + } + + public TaskBuilder csvInputFormatWithColumns(String... columns) + { + return inputFormat( + new CsvInputFormat(List.of(columns), null, null, false, 0, null) + ); + } + + public TaskBuilder appendToExisting(boolean append) + { + this.appendToExisting = append; + return this; + } + + public TaskBuilder dynamicPartitionWithMaxRows(int maxRowsPerSegment) + { + tuningConfig.withPartitionsSpec(new DynamicPartitionsSpec(maxRowsPerSegment, null)); + return this; + } + + public TaskBuilder granularitySpec(GranularitySpec granularitySpec) + { + dataSchema.withGranularity(granularitySpec); + return this; + } + + public TaskBuilder granularitySpec(String segmentGranularity, String queryGranularity, Boolean rollup) + { + dataSchema.withGranularity( + new UniformGranularitySpec( + Granularity.fromString(segmentGranularity), + queryGranularity == null ? null : Granularity.fromString(queryGranularity), + rollup, + null + ) + ); + return this; + } + + /** + * Sets {@code "granularitySpec": {"segmentGranularity": }}. + */ + public TaskBuilder segmentGranularity(String granularity) + { + return granularitySpec(granularity, null, null); + } + + public TaskBuilder isoTimestampColumn(String timestampColumn) + { + dataSchema.withTimestamp(new TimestampSpec(timestampColumn, "iso", null)); + return this; + } + + public TaskBuilder timestampColumn(String timestampColumn) + { + dataSchema.withTimestamp(new TimestampSpec(timestampColumn, null, null)); + return this; + } + + /** + * Sets the given dimensions as string dimensions in the {@link DataSchema}. + * + * @see #dataSchema(Consumer) for more options + */ + public TaskBuilder dimensions(String... dimensions) + { + dataSchema.withDimensions( + Stream.of(dimensions).map(StringDimensionSchema::new).collect(Collectors.toList()) + ); + return this; + } + + public TaskBuilder metricAggregates(AggregatorFactory... aggregators) + { + dataSchema.withAggregators(aggregators); + return this; + } + + public TaskBuilder tuningConfig(Consumer> updateTuningConfig) + { + updateTuningConfig.accept(tuningConfig); + return this; + } + + public TaskBuilder dataSchema(Consumer updateDataSchema) + { + updateDataSchema.accept(dataSchema); + return this; + } + + public TaskBuilder context(String key, Object value) + { + this.context.put(key, value); + return this; + } + + public interface BuilderType + { + T buildTask(String taskId, TaskBuilder builder); + + TuningConfigBuilder tuningConfigBuilder(); + } + + private static class Index implements BuilderType + { + @Override + public TuningConfigBuilder tuningConfigBuilder() + { + return TuningConfigBuilder.forIndexTask(); + } + + @Override + public IndexTask buildTask(String taskId, TaskBuilder builder) + { + return new IndexTask( + taskId, + null, + new IndexTask.IndexIngestionSpec( + builder.dataSchema.build(), + new IndexTask.IndexIOConfig( + builder.inputSource, + builder.inputFormat, + builder.appendToExisting, + null + ), + builder.tuningConfig.build() + ), + builder.context + ); + } + } + + private static class ParallelIndex implements BuilderType + { + + @Override + public ParallelIndexSupervisorTask buildTask( + String taskId, + TaskBuilder builder + ) + { + return new ParallelIndexSupervisorTask( + taskId, + null, + null, + new ParallelIndexIngestionSpec( + builder.dataSchema.build(), + new ParallelIndexIOConfig( + builder.inputSource, + builder.inputFormat, + builder.appendToExisting, + null + ), + builder.tuningConfig.build() + ), + builder.context + ); + } + + @Override + public TuningConfigBuilder tuningConfigBuilder() + { + return TuningConfigBuilder.forParallelIndexTask(); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java deleted file mode 100644 index 7d2d69cbce98..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskPayload.java +++ /dev/null @@ -1,330 +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.task; - -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.ISE; -import org.joda.time.Interval; - -import java.io.File; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Builder for the raw Map-based payload of a {@code Task}. - *

- * The builder does not use any defaults and all required fields must be set - * explicitly. - * - * @see #ofType(String) to create a builder - */ -public class TaskPayload -{ - private final String type; - - private String dataSource; - private Map inputSource = null; - private Map inputFormat = null; - private Map partitionsSpec = null; - private Map granularitySpec = null; - private Map timestampSpec = null; - private Map dimensionsSpec = null; - private Map splitHintSpec = null; - - private final Map context = new HashMap<>(); - private final List> metricsSpec = new ArrayList<>(); - - private Integer maxNumConcurrentSubTasks = null; - private Boolean forceGuaranteedRollup = null; - private Long awaitSegmentAvailabilityTimeoutMillis = null; - private Boolean appendToExisting = null; - - private TaskPayload(String type) - { - this.type = type; - } - - /** - * Initializes builder for a new {@code Task} for the given datasource. - */ - public static TaskPayload ofType(String type) - { - return new TaskPayload(type); - } - - public TaskPayload dataSource(String dataSource) - { - this.dataSource = dataSource; - return this; - } - - /** - * Creates a raw Map-based payload for a {@code Task} that may be submitted to - * the Overlord using {@code OverlordClient.runTask()}. - */ - public Object withId(String taskId) - { - Preconditions.checkNotNull(taskId, "Task ID must not be null"); - Preconditions.checkNotNull(type, "Task type must be specified"); - Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); - Preconditions.checkNotNull(dataSource, "'dataSource' must be specified"); - - return mapOf( - "id", taskId, - "type", type, - "context", context, - "spec", - mapOf( - "ioConfig", - mapOf( - "type", type, - "inputSource", inputSource, - "inputFormat", inputFormat, - "appendToExisting", appendToExisting - ), - "tuningConfig", - mapOf( - "type", type, - "partitionsSpec", partitionsSpec, - "maxNumConcurrentSubTasks", maxNumConcurrentSubTasks, - "forceGuaranteedRollup", forceGuaranteedRollup, - "splitHintSpec", splitHintSpec, - "awaitSegmentAvailabilityTimeoutMillis", awaitSegmentAvailabilityTimeoutMillis - ), - "dataSchema", - mapOf( - "dataSource", dataSource, - "timestampSpec", timestampSpec, - "dimensionsSpec", dimensionsSpec, - "metricsSpec", metricsSpec.isEmpty() ? null : metricsSpec, - "granularitySpec", granularitySpec - ) - ) - ); - } - - public TaskPayload inputSource(Map jsonMap) - { - this.inputSource = jsonMap; - return this; - } - - public TaskPayload inlineInputSourceWithData(String data) - { - return inputSource(Map.of("type", "inline", "data", data)); - } - - public TaskPayload druidInputSource(String dataSource, Interval interval) - { - return inputSource(Map.of("type", "druid", "interval", interval, "dataSource", dataSource)); - } - - /** - * Gets the absolute path of the given resource files and sets: - *

-   * "inputSource": {
-   *   "type": "local",
-   *   "files": [<absolute-paths-of-given-resource-files>]
-   * }
-   * 
- */ - public TaskPayload localInputSourceWithFiles(String... files) - { - try { - final List filePaths = new ArrayList<>(); - for (String file : files) { - final URL resourceUrl = getClass().getClassLoader().getResource(file); - if (resourceUrl == null) { - throw new ISE("Could not find file[%s]", file); - } - - filePaths.add(new File(resourceUrl.toURI()).getAbsolutePath()); - } - - return inputSource(Map.of("type", "local", "files", filePaths)); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - public TaskPayload inputFormat(Map jsonMap) - { - this.inputFormat = jsonMap; - return this; - } - - public TaskPayload jsonInputFormat() - { - this.inputFormat = Map.of("type", "json"); - return this; - } - - public TaskPayload csvInputFormatWithColumns(String... columns) - { - return inputFormat( - Map.of("type", "csv", "findColumnsFromHeader", "false", "columns", List.of(columns)) - ); - } - - public TaskPayload appendToExisting(boolean append) - { - this.appendToExisting = append; - return this; - } - - public TaskPayload partitionsSpec(Map jsonMap) - { - this.partitionsSpec = jsonMap; - return this; - } - - public TaskPayload dynamicPartitionWithMaxRows(int maxRowsPerSegment) - { - this.partitionsSpec = Map.of("type", "dynamic", "maxRowsPerSegment", maxRowsPerSegment); - return this; - } - - public TaskPayload granularitySpec(Map jsonMap) - { - this.granularitySpec = jsonMap; - return this; - } - - public TaskPayload granularitySpec(String segmentGranularity, String queryGranularity, boolean rollup) - { - this.granularitySpec = Map.of( - "segmentGranularity", segmentGranularity, - "queryGranularity", queryGranularity, - "rollup", rollup - ); - return this; - } - - /** - * Sets {@code "granularitySpec": {"segmentGranularity": }}. - */ - public TaskPayload segmentGranularity(String granularity) - { - return granularitySpec(Map.of("segmentGranularity", granularity)); - } - - public TaskPayload timestampSpec(Map jsonMap) - { - this.timestampSpec = jsonMap; - return this; - } - - public TaskPayload isoTimestampColumn(String timestampColumn) - { - return timestampSpec(Map.of("format", "iso", "column", timestampColumn)); - } - - public TaskPayload timestampColumn(String timestampColumn) - { - return timestampSpec(Map.of("column", timestampColumn)); - } - - public TaskPayload dimensionsSpec(Map jsonMap) - { - this.dimensionsSpec = jsonMap; - return this; - } - - /** - * Sets {@code "dimensionSpec": {"dimensions": []}}. - */ - public TaskPayload dimensions(String... dimensions) - { - return dimensionsSpec(Map.of("dimensions", List.of(dimensions))); - } - - public TaskPayload metricAggregate(String name, String type) - { - return metricAggregate(name, type, name); - } - - public TaskPayload metricAggregate(String name, String type, String fieldName) - { - this.metricsSpec.add(mapOf("type", type, "name", name, "fieldName", fieldName)); - return this; - } - - public TaskPayload metricAggregate(Map jsonMap) - { - this.metricsSpec.add(Map.copyOf(jsonMap)); - return this; - } - - public TaskPayload maxConcurrentSubTasks(int maxNumConcurrentSubTasks) - { - this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; - return this; - } - - public TaskPayload forceGuaranteedRollup(boolean rollup) - { - this.forceGuaranteedRollup = rollup; - return this; - } - - public TaskPayload splitHintSpec(Map jsonMap) - { - this.splitHintSpec = jsonMap; - return this; - } - - public TaskPayload awaitSegmentAvailabilityTimeoutMillis(long millis) - { - this.awaitSegmentAvailabilityTimeoutMillis = millis; - return this; - } - - public TaskPayload context(String key, Object value) - { - this.context.put(key, value); - return this; - } - - /** - * Creates a map using only the non-null key-value pairs. - * - * @param kvPairs key1, value1, key2, value2, ... - * @return null if none of the key-value pairs are non-null. - */ - private static Map mapOf(Object... kvPairs) - { - if (kvPairs.length % 2 > 0) { - throw new ISE("Key and value must be given in pairs."); - } - - final Map map = new HashMap<>(); - for (int i = 0; i < kvPairs.length - 1; i += 2) { - if (kvPairs[i] != null && kvPairs[i + 1] != null) { - map.put(kvPairs[i], kvPairs[i + 1]); - } - } - - return map.isEmpty() ? null : Map.copyOf(map); - } -} From ce69567a602505e99f5fb2b24c33195e8ef440e7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 11 Jul 2025 20:22:09 +0530 Subject: [PATCH 20/38] Rename data files --- .../embedded/compact/EmbeddedAutoCompactionTest.java | 6 +++--- .../embedded/indexing/EmbeddedIndexParallelTaskTest.java | 8 ++++---- .../apache/druid/testing/embedded/indexing/Resources.java | 6 +++--- .../data/json/{wikipedia_1.json => tiny_wiki_1.json} | 0 .../data/json/{wikipedia_2.json => tiny_wiki_2.json} | 0 .../data/json/{wikipedia_3.json => tiny_wiki_3.json} | 0 6 files changed, 10 insertions(+), 10 deletions(-) rename embedded-tests/src/test/resources/data/json/{wikipedia_1.json => tiny_wiki_1.json} (100%) rename embedded-tests/src/test/resources/data/json/{wikipedia_2.json => tiny_wiki_2.json} (100%) rename embedded-tests/src/test/resources/data/json/{wikipedia_3.json => tiny_wiki_3.json} (100%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index 8b49912bbdaf..c68fe5882f1a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -122,9 +122,9 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase .ofTypeIndex() .jsonInputFormat() .localInputSourceWithFiles( - Resources.WIKIPEDIA_1_JSON, - Resources.WIKIPEDIA_2_JSON, - Resources.WIKIPEDIA_3_JSON + Resources.TINY_WIKI_1_JSON, + Resources.TINY_WIKI_2_JSON, + Resources.TINY_WIKI_JSON ) .timestampColumn("timestamp") .dimensions( diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index aa6c69e928c6..7aac0eb34c81 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -100,7 +100,7 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( .dataSource(dataSource) .timestampColumn("timestamp") .jsonInputFormat() - .localInputSourceWithFiles(Resources.WIKIPEDIA_1_JSON) + .localInputSourceWithFiles(Resources.TINY_WIKI_1_JSON) .dimensions() .tuningConfig( t -> t.withAwaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis) @@ -135,9 +135,9 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par .timestampColumn("timestamp") .jsonInputFormat() .localInputSourceWithFiles( - Resources.WIKIPEDIA_1_JSON, - Resources.WIKIPEDIA_2_JSON, - Resources.WIKIPEDIA_3_JSON + Resources.TINY_WIKI_1_JSON, + Resources.TINY_WIKI_2_JSON, + Resources.TINY_WIKI_JSON ) .segmentGranularity("DAY") .dimensions("namespace", "page", "language") diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java index 1f0fb99990ec..849d239dce7b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java @@ -62,7 +62,7 @@ public class Resources + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," + "\"isAnonymous\":false,\"user\":\"maytas3\",\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; - public static final String WIKIPEDIA_1_JSON = "data/json/wikipedia_1.json"; - public static final String WIKIPEDIA_2_JSON = "data/json/wikipedia_2.json"; - public static final String WIKIPEDIA_3_JSON = "data/json/wikipedia_3.json"; + public static final String TINY_WIKI_1_JSON = "data/json/tiny_wiki_1.json"; + public static final String TINY_WIKI_2_JSON = "data/json/tiny_wiki_2.json"; + public static final String TINY_WIKI_JSON = "data/json/tiny_wiki_3.json"; } diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_1.json b/embedded-tests/src/test/resources/data/json/tiny_wiki_1.json similarity index 100% rename from embedded-tests/src/test/resources/data/json/wikipedia_1.json rename to embedded-tests/src/test/resources/data/json/tiny_wiki_1.json diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_2.json b/embedded-tests/src/test/resources/data/json/tiny_wiki_2.json similarity index 100% rename from embedded-tests/src/test/resources/data/json/wikipedia_2.json rename to embedded-tests/src/test/resources/data/json/tiny_wiki_2.json diff --git a/embedded-tests/src/test/resources/data/json/wikipedia_3.json b/embedded-tests/src/test/resources/data/json/tiny_wiki_3.json similarity index 100% rename from embedded-tests/src/test/resources/data/json/wikipedia_3.json rename to embedded-tests/src/test/resources/data/json/tiny_wiki_3.json From b126ad3c6a1ccff3976fc8f16f6c8cfca296e8f8 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 12 Jul 2025 22:47:19 +0530 Subject: [PATCH 21/38] Add EmbeddedCompactionSparseColumnTest --- .../EmbeddedCompactionSparseColumnTest.java | 311 ++++++++++++++++++ .../indexing/common/task/TaskBuilder.java | 73 +++- .../indexer/ITCompactionSparseColumnTest.java | 254 -------------- .../druid/client/broker/BrokerClient.java | 6 + .../druid/client/broker/BrokerClientImpl.java | 14 + 5 files changed, 398 insertions(+), 260 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java new file mode 100644 index 000000000000..d143c94d420e --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java @@ -0,0 +1,311 @@ +/* + * 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.testing.embedded.compact; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.Closeable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase +{ + private static final Supplier> INDEX_TASK = + () -> TaskBuilder + .ofTypeIndexParallel() + .jsonInputFormat() + .isoTimestampColumn("time") + .granularitySpec("HOUR", "HOUR", true) + .dimensions("dimB", "dimA", "dimC", "dimD", "dimE", "dimF") + .metricAggregates( + new CountAggregatorFactory("ingested_events"), + new LongSumAggregatorFactory("sum_metA", "metA") + ) + .tuningConfig(t -> t.withPartitionsSpec(new DynamicPartitionsSpec(3, 3L))) + .inlineInputSourceWithData( + "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}" + + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" + ); + + private static final Supplier> COMPACTION_TASK = + () -> TaskBuilder + .ofTypeCompact() + .interval(Intervals.of("2010-10-29T05:00:00Z/2030-10-29T06:00:00Z")) + .tuningConfig( + t -> t.withMaxRowsInMemory(3) + .withMaxRowsPerSegment(3) + .withMaxNumConcurrentSubTasks(2) + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new HashedPartitionsSpec(null, 1, null)) + ); + + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addServer(overlord) + .addServer(coordinator) + .addServer(new EmbeddedIndexer()) + .addServer(new EmbeddedBroker()) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()); + } + + @Test + public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception + { + try (final Closeable ignored = unloader(dataSource)) { + // Load and verify initial data + loadAndVerifyDataWithSparseColumn(); + // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up + runTask(COMPACTION_TASK.get()); + + // Verify compacted data. + // Compacted data only have one segments. First segment have the following rows: + // The ordering of the columns will be "dimB", "dimA", "dimC", "dimD", "dimE", "dimF" + // (This is the same as the ordering in the initial ingestion task). + List> segmentRows = ImmutableList.of( + Arrays.asList(1442016000000L, "F", "C", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "J", "C", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "R", "J", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "S", "Z", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "T", "H", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "X", null, "A", null, null, null, 1, 1), + Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 3, 3), + Arrays.asList(1442016000000L, "Z", "H", null, null, null, null, 1, 1) + ); + verifyCompactedData(segmentRows); + } + } + + @Test + public void testCompactionPerfectRollUpWithLexicographicDimensionSpec() throws Exception + { + try (final Closeable ignored = unloader(dataSource)) { + // Load and verify initial data + loadAndVerifyDataWithSparseColumn(); + // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up + runTask(COMPACTION_TASK.get().dimensions("dimA", "dimB", "dimC")); + + // Verify compacted data. + // Compacted data only have one segments. First segment have the following rows: + // The ordering of the columns will be "dimA", "dimB", "dimC" + List> segmentRows = ImmutableList.of( + Arrays.asList(1442016000000L, null, "X", "A", 1, 1), + Arrays.asList(1442016000000L, "C", "F", null, 1, 1), + Arrays.asList(1442016000000L, "C", "J", null, 1, 1), + Arrays.asList(1442016000000L, "H", "T", null, 1, 1), + Arrays.asList(1442016000000L, "H", "X", null, 3, 3), + Arrays.asList(1442016000000L, "H", "Z", null, 1, 1), + Arrays.asList(1442016000000L, "J", "R", null, 1, 1), + Arrays.asList(1442016000000L, "Z", "S", null, 1, 1) + ); + verifyCompactedData(segmentRows); + } + } + + @Test + public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throws Exception + { + try (final Closeable ignored = unloader(dataSource)) { + // Load and verify initial data + loadAndVerifyDataWithSparseColumn(); + // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up + runTask(COMPACTION_TASK.get().dimensions("dimC", "dimB", "dimA")); + + // Verify compacted data. + // Compacted data only have one segments. First segment have the following rows: + // The ordering of the columns will be "dimC", "dimB", "dimA" + List> segment1Rows = ImmutableList.of( + Arrays.asList(1442016000000L, null, "F", "C", 1, 1), + Arrays.asList(1442016000000L, null, "J", "C", 1, 1), + Arrays.asList(1442016000000L, null, "R", "J", 1, 1), + Arrays.asList(1442016000000L, null, "S", "Z", 1, 1), + Arrays.asList(1442016000000L, null, "T", "H", 1, 1), + Arrays.asList(1442016000000L, null, "X", "H", 3, 3), + Arrays.asList(1442016000000L, null, "Z", "H", 1, 1), + Arrays.asList(1442016000000L, "A", "X", null, 1, 1) + ); + verifyCompactedData(segment1Rows); + } + } + + private void loadAndVerifyDataWithSparseColumn() + { + runTask(INDEX_TASK.get()); + List>>> expectedResultBeforeCompaction = new ArrayList<>(); + // First segments have the following rows: + List> segment1Rows = ImmutableList.of( + Arrays.asList(1442016000000L, "F", "C", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "J", "C", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1) + ); + expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment1Rows)); + // Second segments have the following rows: + List> segment2Rows = ImmutableList.of( + Arrays.asList(1442016000000L, "S", "Z", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "Z", "H", null, null, null, null, 1, 1) + ); + expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment2Rows)); + // Third segments have the following rows: + List> segment3Rows = ImmutableList.of( + Arrays.asList(1442016000000L, "R", "J", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "T", "H", null, null, null, null, 1, 1), + Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1) + ); + expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment3Rows)); + // Fourth segments have the following rows: + List> segment4Rows = ImmutableList.of( + Arrays.asList(1442016000000L, "X", null, "A", null, null, null, 1, 1) + ); + expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment4Rows)); + verifyQueryResult(expectedResultBeforeCompaction, 10, 10, 1); + } + + private void verifyCompactedData(List> segmentRows) + { + List>>> expectedResultAfterCompaction = new ArrayList<>(); + expectedResultAfterCompaction.add(ImmutableMap.of("events", segmentRows)); + verifyQueryResult(expectedResultAfterCompaction, 8, 10, 0.8); + } + + private void verifyQueryResult( + List>>> expectedScanResult, + int expectedNumRoll, + int expectedSumCount, + double expectedRollupRatio + ) + { + Assertions.assertEquals( + "2015-09-12T00:00:00.000Z,2015-09-12T00:00:00.000Z", + cluster.runSql("SELECT MIN(__time), MAX(__time) FROM %s", dataSource) + ); + Assertions.assertEquals( + StringUtils.format("%d,%d,%s", expectedNumRoll, expectedSumCount, expectedRollupRatio), + cluster.runSql( + "SELECT COUNT(*) as num_rows, SUM(ingested_events) as total_events," + + " (COUNT(*) * 1.0) / SUM(ingested_events)" + + " FROM %s", + dataSource + ) + ); + final ScanQuery scanQuery = Druids + .newScanQueryBuilder() + .dataSource(dataSource) + .eternityInterval() + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build(); + Assertions.assertEquals(toJson(expectedScanResult), getScanEvents(scanQuery)); + } + + /** + * Runs the given scan query and extracts the "events" field from it. + */ + private String getScanEvents(ScanQuery scanQuery) + { + final String resultAsJson = + (String) FutureUtils.getUnchecked(cluster.anyBroker().submitNativeQuery(scanQuery), true); + final List> resultList = JacksonUtils.readValue( + TestHelper.JSON_MAPPER, + resultAsJson.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() {} + ); + + final List> trimmedResult = resultList + .stream() + .map(map -> Map.of("events", map.getOrDefault("events", ""))) + .collect(Collectors.toList()); + return toJson(trimmedResult); + } + + private String toJson(Object stuff) + { + try { + return TestHelper.JSON_MAPPER.writeValueAsString(stuff); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Deletes all the data for the given datasource so that compaction tasks for + * this datasource do not take up task slots unnecessarily. + */ + private Closeable unloader(String dataSource) + { + return () -> { + overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); + }; + } + + private void runTask(TaskBuilder taskBuilder) + { + final String taskId = EmbeddedClusterApis.newTaskId(dataSource); + cluster.callApi().onLeaderOverlord( + o -> o.runTask(taskId, taskBuilder.dataSource(dataSource).withId(taskId)) + ); + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java index d8bd82044c35..697af13b990b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java @@ -24,6 +24,8 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; @@ -77,6 +79,10 @@ public class TaskBuilder private Boolean appendToExisting = null; + private String dataSource; + private Interval interval; + private DimensionsSpec dimensionsSpec; + private final BuilderType type; private final DataSchema.Builder dataSchema; private final TuningConfigBuilder tuningConfig; @@ -104,9 +110,15 @@ public static TaskBuilder(new ParallelIndex()); } + public static TaskBuilder ofTypeCompact() + { + return new TaskBuilder<>(new Compact()); + } + public TaskBuilder dataSource(String dataSource) { - dataSchema.withDataSource(dataSource); + this.dataSource = dataSource; + this.dataSchema.withDataSource(dataSource); return this; } @@ -118,11 +130,20 @@ public T withId(String taskId) { Preconditions.checkNotNull(taskId, "Task ID must not be null"); Preconditions.checkNotNull(type, "Task type must be specified"); - Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); + Preconditions.checkNotNull(dataSource, "Datasource must not be null"); return type.buildTask(taskId, this); } + /** + * Used by {@link CompactionTask}. + */ + public TaskBuilder interval(Interval interval) + { + this.interval = interval; + return this; + } + public TaskBuilder inputSource(InputSource inputSource) { this.inputSource = inputSource; @@ -261,9 +282,12 @@ public TaskBuilder timestampColumn(String timestampColumn) */ public TaskBuilder dimensions(String... dimensions) { - dataSchema.withDimensions( - Stream.of(dimensions).map(StringDimensionSchema::new).collect(Collectors.toList()) - ); + List dimensionList = Stream.of(dimensions) + .map(StringDimensionSchema::new) + .collect(Collectors.toList()); + dataSchema.withDimensions(dimensionList); + dimensionsSpec = new DimensionsSpec(dimensionList); + return this; } @@ -309,6 +333,7 @@ public TuningConfigBuilder tuningConfigBuilder() @Override public IndexTask buildTask(String taskId, TaskBuilder builder) { + Preconditions.checkNotNull(builder.inputSource, "'inputSource' must be specified"); return new IndexTask( taskId, null, @@ -329,13 +354,13 @@ public IndexTask buildTask(String taskId, TaskBuilder { - @Override public ParallelIndexSupervisorTask buildTask( String taskId, TaskBuilder builder ) { + Preconditions.checkNotNull(builder.inputSource, "'inputSource' must be specified"); return new ParallelIndexSupervisorTask( taskId, null, @@ -360,4 +385,40 @@ public TuningConfigBuilder tuningConfigBuilder() return TuningConfigBuilder.forParallelIndexTask(); } } + + private static class Compact implements BuilderType + { + @Override + public CompactionTask buildTask( + String taskId, + TaskBuilder builder + ) + { + return new CompactionTask( + taskId, + null, + builder.dataSource, + builder.interval, + null, + null, + builder.dimensionsSpec, + null, + null, + null, + null, + null, + null, + builder.tuningConfig.build(), + null, + null, + null + ); + } + + @Override + public TuningConfigBuilder tuningConfigBuilder() + { + return TuningConfigBuilder.forCompactionTask(); + } + } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java deleted file mode 100644 index 58dc7c43ae7b..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java +++ /dev/null @@ -1,254 +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.tests.indexer; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.inject.Inject; -import org.apache.commons.io.IOUtils; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.io.InputStream; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITCompactionSparseColumnTest extends AbstractIndexerTest -{ - private static final String INDEX_DATASOURCE = "sparse_column_index_test"; - - private static final String INDEX_TASK = "/indexer/sparse_column_index_task.json"; - private static final String COMPACTION_QUERIES_RESOURCE = "/indexer/sparse_column_index_queries.json"; - - private static final String COMPACTION_TASK_WITHOUT_DIMENSION = "/indexer/sparse_column_without_dim_compaction_task.json"; - private static final String COMPACTION_TASK_WITH_DIMENSION = "/indexer/sparse_column_with_dim_compaction_task.json"; - - @Inject - private IntegrationTestingConfig config; - - private String fullDatasourceName; - - @BeforeMethod - public void setFullDatasourceName(Method method) - { - fullDatasourceName = INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + "-" + method.getName(); - } - - @Test - public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - // Load and verify initial data - loadAndVerifyDataWithSparseColumn(fullDatasourceName); - // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - String template = getResourceAsString(COMPACTION_TASK_WITHOUT_DIMENSION); - template = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); - final String taskID = indexer.submitTask(template); - indexer.waitUntilTaskCompletes(taskID); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segment Compaction" - ); - // Verify compacted data. - // Compacted data only have one segments. First segment have the following rows: - // The ordering of the columns will be "dimB", "dimA", "dimC", "dimD", "dimE", "dimF" - // (This is the same as the ordering in the initial ingestion task). - List> segmentRows = ImmutableList.of( - Arrays.asList(1442016000000L, "F", "C", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "J", "C", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "R", "J", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "S", "Z", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "T", "H", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "X", null, "A", null, null, null, 1, 1), - Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 3, 3), - Arrays.asList(1442016000000L, "Z", "H", null, null, null, null, 1, 1) - ); - verifyCompactedData(segmentRows); - } - } - - @Test - public void testCompactionPerfectRollUpWithLexicographicDimensionSpec() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - // Load and verify initial data - loadAndVerifyDataWithSparseColumn(fullDatasourceName); - // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - String template = getResourceAsString(COMPACTION_TASK_WITH_DIMENSION); - template = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); - // - List dimensionsOrder = ImmutableList.of("dimA", "dimB", "dimC"); - template = StringUtils.replace( - template, - "%%DIMENSION_NAMES%%", - jsonMapper.writeValueAsString(dimensionsOrder) - ); - final String taskID = indexer.submitTask(template); - indexer.waitUntilTaskCompletes(taskID); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segment Compaction" - ); - // Verify compacted data. - // Compacted data only have one segments. First segment have the following rows: - // The ordering of the columns will be "dimA", "dimB", "dimC" - List> segmentRows = ImmutableList.of( - Arrays.asList(1442016000000L, null, "X", "A", 1, 1), - Arrays.asList(1442016000000L, "C", "F", null, 1, 1), - Arrays.asList(1442016000000L, "C", "J", null, 1, 1), - Arrays.asList(1442016000000L, "H", "T", null, 1, 1), - Arrays.asList(1442016000000L, "H", "X", null, 3, 3), - Arrays.asList(1442016000000L, "H", "Z", null, 1, 1), - Arrays.asList(1442016000000L, "J", "R", null, 1, 1), - Arrays.asList(1442016000000L, "Z", "S", null, 1, 1) - ); - verifyCompactedData(segmentRows); - } - } - - @Test - public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - // Load and verify initial data - loadAndVerifyDataWithSparseColumn(fullDatasourceName); - // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - String template = getResourceAsString(COMPACTION_TASK_WITH_DIMENSION); - template = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); - // - List dimensionsOrder = ImmutableList.of("dimC", "dimB", "dimA"); - template = StringUtils.replace( - template, - "%%DIMENSION_NAMES%%", - jsonMapper.writeValueAsString(dimensionsOrder) - ); - final String taskID = indexer.submitTask(template); - indexer.waitUntilTaskCompletes(taskID); - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segment Compaction" - ); - // Verify compacted data. - // Compacted data only have one segments. First segment have the following rows: - // The ordering of the columns will be "dimC", "dimB", "dimA" - List> segment1Rows = ImmutableList.of( - Arrays.asList(1442016000000L, null, "F", "C", 1, 1), - Arrays.asList(1442016000000L, null, "J", "C", 1, 1), - Arrays.asList(1442016000000L, null, "R", "J", 1, 1), - Arrays.asList(1442016000000L, null, "S", "Z", 1, 1), - Arrays.asList(1442016000000L, null, "T", "H", 1, 1), - Arrays.asList(1442016000000L, null, "X", "H", 3, 3), - Arrays.asList(1442016000000L, null, "Z", "H", 1, 1), - Arrays.asList(1442016000000L, "A", "X", null, 1, 1) - ); - verifyCompactedData(segment1Rows); - } - } - - private void loadAndVerifyDataWithSparseColumn(String fullDatasourceName) throws Exception - { - loadData(INDEX_TASK, fullDatasourceName); - List>>> expectedResultBeforeCompaction = new ArrayList<>(); - // First segments have the following rows: - List> segment1Rows = ImmutableList.of( - Arrays.asList(1442016000000L, "F", "C", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "J", "C", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1) - ); - expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment1Rows)); - // Second segments have the following rows: - List> segment2Rows = ImmutableList.of( - Arrays.asList(1442016000000L, "S", "Z", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "Z", "H", null, null, null, null, 1, 1) - ); - expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment2Rows)); - // Third segments have the following rows: - List> segment3Rows = ImmutableList.of( - Arrays.asList(1442016000000L, "R", "J", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "T", "H", null, null, null, null, 1, 1), - Arrays.asList(1442016000000L, "X", "H", null, null, null, null, 1, 1) - ); - expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment3Rows)); - // Fourth segments have the following rows: - List> segment4Rows = ImmutableList.of( - Arrays.asList(1442016000000L, "X", null, "A", null, null, null, 1, 1) - ); - expectedResultBeforeCompaction.add(ImmutableMap.of("events", segment4Rows)); - verifyQueryResult(expectedResultBeforeCompaction, 10, 10, 1); - } - - private void verifyCompactedData(List> segmentRows) throws Exception - { - List>>> expectedResultAfterCompaction = new ArrayList<>(); - expectedResultAfterCompaction.add(ImmutableMap.of("events", segmentRows)); - verifyQueryResult(expectedResultAfterCompaction, 8, 10, 0.8); - } - - private void verifyQueryResult( - List>>> expectedScanResult, - int expectedNumRoll, - int expectedSumCount, - double expectedRollupRatio - ) throws Exception - { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(COMPACTION_QUERIES_RESOURCE); - String queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%EXPECTED_SCAN_RESULT%%", - jsonMapper.writeValueAsString(expectedScanResult) - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%EXPECTED_SUM_COUNT%%", - jsonMapper.writeValueAsString(expectedSumCount) - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%EXPECTED_ROLLUP_RATIO%%", - jsonMapper.writeValueAsString(expectedRollupRatio) - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%EXPECTED_NUM_ROW%%", - jsonMapper.writeValueAsString(expectedNumRoll) - ); - queryHelper.testQueriesFromString(queryResponseTemplate); - } -} diff --git a/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java b/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java index 30bfa22937d9..c5ad710c1a03 100644 --- a/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java @@ -20,6 +20,7 @@ package org.apache.druid.client.broker; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.query.Query; import org.apache.druid.query.explain.ExplainPlan; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; @@ -38,6 +39,11 @@ */ public interface BrokerClient { + /** + * Submits the given query to the {@code /druid/v2} endpoint of a Broker. + */ + ListenableFuture submitNativeQuery(Query query); + /** * Submit the given {@code sqlQuery} to the Broker's SQL query endpoint, {@code /druid/v2/sql/}. */ diff --git a/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java b/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java index 3bce514c9801..641133774dc3 100644 --- a/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; +import org.apache.druid.query.Query; import org.apache.druid.query.explain.ExplainPlan; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; @@ -51,6 +52,19 @@ public BrokerClientImpl(final ServiceClient client, final ObjectMapper jsonMappe this.jsonMapper = jsonMapper; } + @Override + public ListenableFuture submitNativeQuery(Query query) + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, "/druid/v2") + .jsonContent(jsonMapper, query), + new StringFullResponseHandler(StandardCharsets.UTF_8) + ), + FullResponseHolder::getContent + ); + } + @Override public ListenableFuture submitSqlQuery(final ClientSqlQuery sqlQuery) { From 3470f9349ece3c58ec76a3695f2d2986d228dc30 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 06:31:51 +0530 Subject: [PATCH 22/38] Clean up TaskBuilder --- .../compact/EmbeddedAutoCompactionTest.java | 20 +- .../EmbeddedCompactionSparseColumnTest.java | 16 +- .../EmbeddedConcurrentAppendReplaceTest.java | 3 +- .../EmbeddedIndexParallelTaskTest.java | 10 +- .../indexing/common/task/TaskBuilder.java | 331 +++++++++--------- .../indexer/sparse_column_index_queries.json | 80 ----- .../indexer/sparse_column_index_task.json | 57 --- ...parse_column_with_dim_compaction_task.json | 19 - ...se_column_without_dim_compaction_task.json | 16 - 9 files changed, 194 insertions(+), 358 deletions(-) delete mode 100644 integration-tests/src/test/resources/indexer/sparse_column_index_queries.json delete mode 100644 integration-tests/src/test/resources/indexer/sparse_column_index_task.json delete mode 100644 integration-tests/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json delete mode 100644 integration-tests/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index c68fe5882f1a..f8c690017f33 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -117,7 +117,7 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase { private static final Logger LOG = new Logger(EmbeddedAutoCompactionTest.class); - private static final Supplier> INDEX_TASK = + private static final Supplier INDEX_TASK = () -> TaskBuilder .ofTypeIndex() .jsonInputFormat() @@ -145,9 +145,9 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase .granularitySpec("DAY", "SECOND", true) .appendToExisting(false); - private static final Supplier> INDEX_TASK_WITH_GRANULARITY_SPEC = + private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); - private static final Supplier> INDEX_TASK_WITH_DIMENSION_SPEC = + private static final Supplier INDEX_TASK_WITH_DIMENSION_SPEC = () -> INDEX_TASK.get().granularitySpec("DAY", "DAY", true); private static final String SELECT_APPROX_COUNT_DISTINCT = @@ -174,7 +174,7 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase "Crimson Typhoon,1,905.0,9050.0" ) ); - private static final Supplier> INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = + private static final Supplier INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = () -> TaskBuilder .ofTypeIndexParallel() .jsonInputFormat() @@ -199,7 +199,7 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase "isNew", "isMinor", "isAnonymous", "namespace" ); - private static final Supplier> INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = + private static final Supplier INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = () -> TaskBuilder .ofTypeIndexParallel() .jsonInputFormat() @@ -1714,14 +1714,18 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception } } - private void loadData(Supplier> updatePayload) + private > void loadData(Supplier updatePayload) { loadData(updatePayload, null); } - private void loadData(Supplier> taskPayloadSupplier, GranularitySpec granularitySpec) + private > void loadData( + Supplier taskPayloadSupplier, + GranularitySpec granularitySpec + ) { - final TaskBuilder taskBuilder = taskPayloadSupplier.get().dataSource(fullDatasourceName); + final TaskBuilder.IndexCommon taskBuilder = taskPayloadSupplier.get(); + taskBuilder.dataSource(fullDatasourceName); if (granularitySpec != null) { taskBuilder.granularitySpec(granularitySpec); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java index d143c94d420e..903372c579a6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java @@ -57,7 +57,7 @@ public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase { - private static final Supplier> INDEX_TASK = + private static final Supplier INDEX_TASK = () -> TaskBuilder .ofTypeIndexParallel() .jsonInputFormat() @@ -82,7 +82,7 @@ public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase + "\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" ); - private static final Supplier> COMPACTION_TASK = + private static final Supplier COMPACTION_TASK = () -> TaskBuilder .ofTypeCompact() .interval(Intervals.of("2010-10-29T05:00:00Z/2030-10-29T06:00:00Z")) @@ -117,7 +117,7 @@ public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get()); + runTask(COMPACTION_TASK.get().dataSource(dataSource)); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -144,7 +144,7 @@ public void testCompactionPerfectRollUpWithLexicographicDimensionSpec() throws E // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dimensions("dimA", "dimB", "dimC")); + runTask(COMPACTION_TASK.get().dataSource(dataSource).dimensions("dimA", "dimB", "dimC")); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -170,7 +170,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dimensions("dimC", "dimB", "dimA")); + runTask(COMPACTION_TASK.get().dataSource(dataSource).dimensions("dimC", "dimB", "dimA")); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -191,7 +191,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw private void loadAndVerifyDataWithSparseColumn() { - runTask(INDEX_TASK.get()); + runTask(INDEX_TASK.get().dataSource(dataSource)); List>>> expectedResultBeforeCompaction = new ArrayList<>(); // First segments have the following rows: List> segment1Rows = ImmutableList.of( @@ -299,11 +299,11 @@ private Closeable unloader(String dataSource) }; } - private void runTask(TaskBuilder taskBuilder) + private void runTask(TaskBuilder taskBuilder) { final String taskId = EmbeddedClusterApis.newTaskId(dataSource); cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskBuilder.dataSource(dataSource).withId(taskId)) + o -> o.runTask(taskId, taskBuilder.withId(taskId)) ); cluster.callApi().waitForTaskToSucceed(taskId, overlord); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java index ee75eefd8445..1dfb9ca9da1e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java @@ -19,7 +19,6 @@ package org.apache.druid.testing.embedded.indexing; -import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.testing.embedded.EmbeddedBroker; @@ -60,7 +59,7 @@ public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() // Run an APPEND task to ingest data into an interval final String data1Row = "2013-01-01T00:00:00.000Z,shirt,100"; final String task1 = EmbeddedClusterApis.newTaskId(dataSource); - final TaskBuilder taskBuilder = + final TaskBuilder.Index taskBuilder = TaskBuilder.ofTypeIndex() .dataSource(dataSource) .csvInputFormatWithColumns("time", "item", "value") diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index 7aac0eb34c81..f990d0962035 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -95,7 +95,7 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( final long segmentAvailabilityTimeoutMillis ) { - final TaskBuilder indexTask = + final TaskBuilder.IndexParallel indexTask = TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource) .timestampColumn("timestamp") @@ -129,7 +129,7 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par { final boolean isRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); - final TaskBuilder indexTask = + final TaskBuilder.IndexParallel indexTask = TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource) .timestampColumn("timestamp") @@ -160,7 +160,7 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par // Re-index into a different datasource, indexing 1 segment per sub-task final String dataSource2 = EmbeddedClusterApis.createTestDatasourceName(); - final TaskBuilder reindexTaskSplitBySegment = + final TaskBuilder.IndexParallel reindexTaskSplitBySegment = TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource2) .isoTimestampColumn("ignored") @@ -186,7 +186,7 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par // Re-index into a different datasource, indexing 1 file per sub-task final String dataSource3 = EmbeddedClusterApis.createTestDatasourceName(); - final TaskBuilder reindexTaskSplitByFile = + final TaskBuilder.IndexParallel reindexTaskSplitByFile = TaskBuilder.ofTypeIndexParallel() .dataSource(dataSource3) .timestampColumn("timestamp") @@ -211,7 +211,7 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par runQueries(dataSource3); } - private String runTask(TaskBuilder taskBuilder, String dataSource) + private String runTask(TaskBuilder.IndexParallel taskBuilder, String dataSource) { final String taskId = EmbeddedClusterApis.newTaskId(dataSource); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.withId(taskId))); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java index 697af13b990b..46b11d41c301 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java @@ -24,7 +24,6 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.CsvInputFormat; -import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.JsonInputFormat; @@ -67,95 +66,67 @@ * @param Type of task created by this builder. * @param Type of tuning config used by this builder. * @see #ofTypeIndex() - * @see #dataSchema(Consumer) to specify the {@code dataSchema}. * @see #tuningConfig(Consumer) to specify the {@code tuningConfig}. */ -public class TaskBuilder +@SuppressWarnings("unchecked") +public abstract class TaskBuilder, C, T extends Task> { - private InputSource inputSource = null; - private InputFormat inputFormat = null; + // Fields are package-protected to allow access by subclasses like Index, Compact + InputSource inputSource = null; + InputFormat inputFormat = null; - private final Map context = new HashMap<>(); + final Map context = new HashMap<>(); - private Boolean appendToExisting = null; + Boolean appendToExisting = null; - private String dataSource; - private Interval interval; - private DimensionsSpec dimensionsSpec; + final TuningConfigBuilder tuningConfig; - private final BuilderType type; - private final DataSchema.Builder dataSchema; - private final TuningConfigBuilder tuningConfig; - - private TaskBuilder(BuilderType type) + private TaskBuilder() { - this.type = type; - this.dataSchema = DataSchema.builder(); - this.tuningConfig = type.tuningConfigBuilder(); + this.tuningConfig = tuningConfigBuilder(); } /** - * Initializes builder for a new {@link IndexTask}. + * Creates a raw Map-based payload for a {@code Task} that may be submitted to + * the Overlord using {@code OverlordClient.runTask()}. */ - public static TaskBuilder ofTypeIndex() - { - return new TaskBuilder<>(new Index()); - } + public abstract T withId(String taskId); + + abstract TuningConfigBuilder tuningConfigBuilder(); /** - * Initializes builder for a new {@link ParallelIndexSupervisorTask}. + * Initializes builder for a new {@link IndexTask}. */ - public static TaskBuilder ofTypeIndexParallel() - { - return new TaskBuilder<>(new ParallelIndex()); - } - - public static TaskBuilder ofTypeCompact() - { - return new TaskBuilder<>(new Compact()); - } - - public TaskBuilder dataSource(String dataSource) + public static Index ofTypeIndex() { - this.dataSource = dataSource; - this.dataSchema.withDataSource(dataSource); - return this; + return new Index(); } /** - * Creates a raw Map-based payload for a {@code Task} that may be submitted to - * the Overlord using {@code OverlordClient.runTask()}. + * Initializes builder for a new {@link ParallelIndexSupervisorTask}. */ - public T withId(String taskId) + public static IndexParallel ofTypeIndexParallel() { - Preconditions.checkNotNull(taskId, "Task ID must not be null"); - Preconditions.checkNotNull(type, "Task type must be specified"); - Preconditions.checkNotNull(dataSource, "Datasource must not be null"); - - return type.buildTask(taskId, this); + return new IndexParallel(); } - /** - * Used by {@link CompactionTask}. - */ - public TaskBuilder interval(Interval interval) + public static Compact ofTypeCompact() { - this.interval = interval; - return this; + return new Compact(); } - public TaskBuilder inputSource(InputSource inputSource) + public B inputSource(InputSource inputSource) { this.inputSource = inputSource; - return this; + return (B) this; } - public TaskBuilder inlineInputSourceWithData(String data) + public B inlineInputSourceWithData(String data) { return inputSource(new InlineInputSource(data)); } - public TaskBuilder druidInputSource(String dataSource, Interval interval) + public B druidInputSource(String dataSource, Interval interval) { return inputSource( new DruidInputSource( @@ -182,7 +153,7 @@ public TaskBuilder druidInputSource(String dataSource, Interval interval) * } * */ - public TaskBuilder localInputSourceWithFiles(String... resources) + public B localInputSourceWithFiles(String... resources) { try { final List files = new ArrayList<>(); @@ -204,125 +175,132 @@ public TaskBuilder localInputSourceWithFiles(String... resources) } } - public TaskBuilder inputFormat(InputFormat inputFormat) + public B inputFormat(InputFormat inputFormat) { this.inputFormat = inputFormat; - return this; + return (B) this; } - public TaskBuilder jsonInputFormat() + public B jsonInputFormat() { return inputFormat( new JsonInputFormat(null, null, null, null, null) ); } - public TaskBuilder csvInputFormatWithColumns(String... columns) + public B csvInputFormatWithColumns(String... columns) { return inputFormat( new CsvInputFormat(List.of(columns), null, null, false, 0, null) ); } - public TaskBuilder appendToExisting(boolean append) + public B appendToExisting(boolean append) { this.appendToExisting = append; - return this; + return (B) this; } - public TaskBuilder dynamicPartitionWithMaxRows(int maxRowsPerSegment) + public B dynamicPartitionWithMaxRows(int maxRowsPerSegment) { tuningConfig.withPartitionsSpec(new DynamicPartitionsSpec(maxRowsPerSegment, null)); - return this; + return (B) this; } - public TaskBuilder granularitySpec(GranularitySpec granularitySpec) + public B tuningConfig(Consumer> updateTuningConfig) { - dataSchema.withGranularity(granularitySpec); - return this; - } - - public TaskBuilder granularitySpec(String segmentGranularity, String queryGranularity, Boolean rollup) - { - dataSchema.withGranularity( - new UniformGranularitySpec( - Granularity.fromString(segmentGranularity), - queryGranularity == null ? null : Granularity.fromString(queryGranularity), - rollup, - null - ) - ); - return this; + updateTuningConfig.accept(tuningConfig); + return (B) this; } - /** - * Sets {@code "granularitySpec": {"segmentGranularity": }}. - */ - public TaskBuilder segmentGranularity(String granularity) + public B context(String key, Object value) { - return granularitySpec(granularity, null, null); + this.context.put(key, value); + return (B) this; } - public TaskBuilder isoTimestampColumn(String timestampColumn) + public abstract static class IndexCommon, C, T extends Task> + extends TaskBuilder { - dataSchema.withTimestamp(new TimestampSpec(timestampColumn, "iso", null)); - return this; - } + final DataSchema.Builder dataSchema = DataSchema.builder(); - public TaskBuilder timestampColumn(String timestampColumn) - { - dataSchema.withTimestamp(new TimestampSpec(timestampColumn, null, null)); - return this; - } + public B dataSource(String dataSource) + { + dataSchema.withDataSource(dataSource); + return (B) this; + } - /** - * Sets the given dimensions as string dimensions in the {@link DataSchema}. - * - * @see #dataSchema(Consumer) for more options - */ - public TaskBuilder dimensions(String... dimensions) - { - List dimensionList = Stream.of(dimensions) - .map(StringDimensionSchema::new) - .collect(Collectors.toList()); - dataSchema.withDimensions(dimensionList); - dimensionsSpec = new DimensionsSpec(dimensionList); + public B dataSchema(Consumer updateDataSchema) + { + updateDataSchema.accept(dataSchema); + return (B) this; + } - return this; - } + public B isoTimestampColumn(String timestampColumn) + { + dataSchema.withTimestamp(new TimestampSpec(timestampColumn, "iso", null)); + return (B) this; + } - public TaskBuilder metricAggregates(AggregatorFactory... aggregators) - { - dataSchema.withAggregators(aggregators); - return this; - } + public B timestampColumn(String timestampColumn) + { + dataSchema.withTimestamp(new TimestampSpec(timestampColumn, null, null)); + return (B) this; + } - public TaskBuilder tuningConfig(Consumer> updateTuningConfig) - { - updateTuningConfig.accept(tuningConfig); - return this; - } + public B granularitySpec(GranularitySpec granularitySpec) + { + dataSchema.withGranularity(granularitySpec); + return (B) this; + } - public TaskBuilder dataSchema(Consumer updateDataSchema) - { - updateDataSchema.accept(dataSchema); - return this; - } + public B granularitySpec(String segmentGranularity, String queryGranularity, Boolean rollup) + { + dataSchema.withGranularity( + new UniformGranularitySpec( + Granularity.fromString(segmentGranularity), + queryGranularity == null ? null : Granularity.fromString(queryGranularity), + rollup, + null + ) + ); + return (B) this; + } - public TaskBuilder context(String key, Object value) - { - this.context.put(key, value); - return this; - } + /** + * Sets {@code "granularitySpec": {"segmentGranularity": }}. + */ + public B segmentGranularity(String granularity) + { + return granularitySpec(granularity, null, null); + } - public interface BuilderType - { - T buildTask(String taskId, TaskBuilder builder); + /** + * Sets the given dimensions as string dimensions in the {@link DataSchema}. + * + * @see #dataSchema(Consumer) for more options + */ + public B dimensions(String... dimensions) + { + dataSchema.withDimensions( + Stream.of(dimensions) + .map(StringDimensionSchema::new) + .collect(Collectors.toList()) + ); + return (B) this; + } - TuningConfigBuilder tuningConfigBuilder(); + public B metricAggregates(AggregatorFactory... aggregators) + { + dataSchema.withAggregators(aggregators); + return (B) this; + } } - private static class Index implements BuilderType + /** + * Builder for {@link IndexTask} that uses a {@link IndexTask.IndexTuningConfig}. + */ + public static class Index extends IndexCommon { @Override public TuningConfigBuilder tuningConfigBuilder() @@ -331,51 +309,52 @@ public TuningConfigBuilder tuningConfigBuilder() } @Override - public IndexTask buildTask(String taskId, TaskBuilder builder) + public IndexTask withId(String taskId) { - Preconditions.checkNotNull(builder.inputSource, "'inputSource' must be specified"); + Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); + return new IndexTask( taskId, null, new IndexTask.IndexIngestionSpec( - builder.dataSchema.build(), + dataSchema.build(), new IndexTask.IndexIOConfig( - builder.inputSource, - builder.inputFormat, - builder.appendToExisting, + inputSource, + inputFormat, + appendToExisting, null ), - builder.tuningConfig.build() + tuningConfig.build() ), - builder.context + context ); } } - private static class ParallelIndex implements BuilderType + /** + * Builder for {@link ParallelIndexSupervisorTask} which uses a {@link ParallelIndexTuningConfig}. + */ + public static class IndexParallel extends IndexCommon { @Override - public ParallelIndexSupervisorTask buildTask( - String taskId, - TaskBuilder builder - ) + public ParallelIndexSupervisorTask withId(String taskId) { - Preconditions.checkNotNull(builder.inputSource, "'inputSource' must be specified"); + Preconditions.checkNotNull(inputSource, "'inputSource' must be specified"); return new ParallelIndexSupervisorTask( taskId, null, null, new ParallelIndexIngestionSpec( - builder.dataSchema.build(), + dataSchema.build(), new ParallelIndexIOConfig( - builder.inputSource, - builder.inputFormat, - builder.appendToExisting, + inputSource, + inputFormat, + appendToExisting, null ), - builder.tuningConfig.build() + tuningConfig.build() ), - builder.context + context ); } @@ -386,29 +365,55 @@ public TuningConfigBuilder tuningConfigBuilder() } } - private static class Compact implements BuilderType + /** + * Builder for a {@link CompactionTask} which uses a {@link CompactionTask.CompactionTuningConfig}. + */ + public static class Compact extends TaskBuilder { + private String dataSource; + private Interval interval; + private DimensionsSpec dimensionsSpec; + + public Compact dataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Compact interval(Interval interval) + { + this.interval = interval; + return this; + } + + public Compact dimensions(String... dimensions) + { + dimensionsSpec = new DimensionsSpec( + Stream.of(dimensions) + .map(StringDimensionSchema::new) + .collect(Collectors.toList()) + ); + return this; + } + @Override - public CompactionTask buildTask( - String taskId, - TaskBuilder builder - ) + public CompactionTask withId(String taskId) { return new CompactionTask( taskId, null, - builder.dataSource, - builder.interval, + dataSource, + interval, null, null, - builder.dimensionsSpec, + dimensionsSpec, null, null, null, null, null, null, - builder.tuningConfig.build(), + tuningConfig.build(), null, null, null diff --git a/integration-tests/src/test/resources/indexer/sparse_column_index_queries.json b/integration-tests/src/test/resources/indexer/sparse_column_index_queries.json deleted file mode 100644 index 193e69fbc9ca..000000000000 --- a/integration-tests/src/test/resources/indexer/sparse_column_index_queries.json +++ /dev/null @@ -1,80 +0,0 @@ -[ - { - "description": "timeseries, 1 agg, all", - "query":{ - "queryType" : "timeBoundary", - "dataSource": "%%DATASOURCE%%" - }, - "expectedResults":[ - { - "timestamp" : "2015-09-12T00:00:00.000Z", - "result" : { - "minTime" : "2015-09-12T00:00:00.000Z", - "maxTime" : "2015-09-12T00:00:00.000Z" - } - } - ] - }, - { - "description": "scan, all", - "query": { - "queryType": "scan", - "dataSource": "%%DATASOURCE%%", - "intervals": [ - "2013-01-01/2020-01-02" - ], - "resultFormat":"compactedList" - }, - "expectedResults": %%EXPECTED_SCAN_RESULT%%, - "fieldsToTest": ["events"] - }, - { - "description": "roll up ratio", - "query": { - "queryType":"timeseries", - "dataSource":{ - "type":"table", - "name":"%%DATASOURCE%%" - }, - "intervals":{ - "type":"intervals", - "intervals":[ - "2013-01-01/2020-01-02" - ] - }, - "granularity":{ - "type":"all" - }, - "aggregations":[ - { - "type":"count", - "name":"a0" - }, - { - "type":"longSum", - "name":"a1", - "fieldName":"count", - "expression":null - } - ], - "postAggregations":[ - { - "type":"expression", - "name":"p0", - "expression":"((\"a0\" * 1.00) / \"a1\")", - "ordering":null - } - ] - }, - "expectedResults": [ - { - "timestamp" : "2015-09-12T00:00:00.000Z", - "result" : { - "a1" : %%EXPECTED_SUM_COUNT%%, - "p0" : %%EXPECTED_ROLLUP_RATIO%%, - "a0" : %%EXPECTED_NUM_ROW%% - } - } - ] - } -] \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/sparse_column_index_task.json b/integration-tests/src/test/resources/indexer/sparse_column_index_task.json deleted file mode 100644 index 3a21a856ac6f..000000000000 --- a/integration-tests/src/test/resources/indexer/sparse_column_index_task.json +++ /dev/null @@ -1,57 +0,0 @@ -{ - "type": "index_parallel", - "spec": { - "ioConfig": { - "type": "index_parallel", - "inputSource": { - "type": "inline", - "data": "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" - }, - "inputFormat": { - "type": "json" - } - }, - "tuningConfig": { - "type": "index_parallel", - "partitionsSpec": { - "type": "dynamic", - "maxRowsPerSegment": 3, - "maxTotalRows": 3 - }, - "maxRowsInMemory": 3 - }, - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "timestampSpec": { - "column": "time", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "dimB", - "dimA", - "dimC", - "dimD", - "dimE", - "dimF" - ] - }, - "granularitySpec": { - "queryGranularity": "hour", - "rollup": true, - "segmentGranularity": "hour" - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" - }, - { - "name": "sum_metA", - "type": "longSum", - "fieldName": "metA" - } - ] - } - } -} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json b/integration-tests/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json deleted file mode 100644 index 9416a3f6bdad..000000000000 --- a/integration-tests/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "compact", - "dataSource": "%%DATASOURCE%%", - "dimensionsSpec": { - "dimensions": %%DIMENSION_NAMES%% - }, - "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", - "tuningConfig": { - "type": "index_parallel", - "maxRowsPerSegment": 3, - "maxRowsInMemory": 3, - "maxNumConcurrentSubTasks": 2, - "partitionsSpec": { - "type": "hashed", - "numShards": 1 - }, - "forceGuaranteedRollup": true - } -} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json b/integration-tests/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json deleted file mode 100644 index a149d7a25127..000000000000 --- a/integration-tests/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json +++ /dev/null @@ -1,16 +0,0 @@ -{ - "type": "compact", - "dataSource": "%%DATASOURCE%%", - "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", - "tuningConfig": { - "type": "index_parallel", - "maxRowsPerSegment": 3, - "maxRowsInMemory": 3, - "maxNumConcurrentSubTasks": 2, - "partitionsSpec": { - "type": "hashed", - "numShards": 1 - }, - "forceGuaranteedRollup": true - } -} \ No newline at end of file From bc18de804fce3fa2a4dcb76a780d2d5afee54b77 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 06:37:34 +0530 Subject: [PATCH 23/38] Add test for BrokerClient.submitNativeQuery --- .../EmbeddedCompactionSparseColumnTest.java | 2 +- .../druid/client/broker/BrokerClient.java | 2 +- .../druid/client/broker/BrokerClientImpl.java | 2 +- .../client/broker/BrokerClientImplTest.java | 28 +++++++++++++++++++ 4 files changed, 31 insertions(+), 3 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java index 903372c579a6..8b8bb831360b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java @@ -264,7 +264,7 @@ private void verifyQueryResult( private String getScanEvents(ScanQuery scanQuery) { final String resultAsJson = - (String) FutureUtils.getUnchecked(cluster.anyBroker().submitNativeQuery(scanQuery), true); + FutureUtils.getUnchecked(cluster.anyBroker().submitNativeQuery(scanQuery), true); final List> resultList = JacksonUtils.readValue( TestHelper.JSON_MAPPER, resultAsJson.getBytes(StandardCharsets.UTF_8), diff --git a/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java b/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java index c5ad710c1a03..377866c48ee4 100644 --- a/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/client/broker/BrokerClient.java @@ -42,7 +42,7 @@ public interface BrokerClient /** * Submits the given query to the {@code /druid/v2} endpoint of a Broker. */ - ListenableFuture submitNativeQuery(Query query); + ListenableFuture submitNativeQuery(Query query); /** * Submit the given {@code sqlQuery} to the Broker's SQL query endpoint, {@code /druid/v2/sql/}. diff --git a/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java b/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java index 641133774dc3..52f7c3184d56 100644 --- a/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/broker/BrokerClientImpl.java @@ -53,7 +53,7 @@ public BrokerClientImpl(final ServiceClient client, final ObjectMapper jsonMappe } @Override - public ListenableFuture submitNativeQuery(Query query) + public ListenableFuture submitNativeQuery(Query query) { return FutureUtils.transform( client.asyncRequest( diff --git a/server/src/test/java/org/apache/druid/client/broker/BrokerClientImplTest.java b/server/src/test/java/org/apache/druid/client/broker/BrokerClientImplTest.java index 90d1b49534d3..782191732e26 100644 --- a/server/src/test/java/org/apache/druid/client/broker/BrokerClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/broker/BrokerClientImplTest.java @@ -26,12 +26,15 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.Druids; import org.apache.druid.query.explain.ExplainAttributes; import org.apache.druid.query.explain.ExplainPlan; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.segment.TestDataSource; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.junit.After; @@ -65,6 +68,31 @@ public void tearDown() serviceClient.verify(); } + @Test + public void test_submitNativeQuery() throws Exception + { + final ScanQuery scanQuery = Druids + .newScanQueryBuilder() + .dataSource(TestDataSource.WIKI) + .eternityInterval() + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build(); + + final Map response = Map.of("events", List.of()); + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/druid/v2") + .jsonContent(jsonMapper, scanQuery), + HttpResponseStatus.OK, + Map.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(response) + ); + + assertEquals( + jsonMapper.writeValueAsString(response), + brokerClient.submitNativeQuery(scanQuery).get() + ); + } + @Test public void testSubmitSqlTask() throws Exception { From 272c2b73636d8389f06ab22bdf8f21f57b6c8eda Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 09:25:18 +0530 Subject: [PATCH 24/38] Remove unused files --- .../indexer/sparse_column_index_queries.json | 80 ------------------- .../indexer/sparse_column_index_task.json | 57 ------------- ...parse_column_with_dim_compaction_task.json | 19 ----- ...se_column_without_dim_compaction_task.json | 16 ---- 4 files changed, 172 deletions(-) delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json deleted file mode 100644 index 193e69fbc9ca..000000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json +++ /dev/null @@ -1,80 +0,0 @@ -[ - { - "description": "timeseries, 1 agg, all", - "query":{ - "queryType" : "timeBoundary", - "dataSource": "%%DATASOURCE%%" - }, - "expectedResults":[ - { - "timestamp" : "2015-09-12T00:00:00.000Z", - "result" : { - "minTime" : "2015-09-12T00:00:00.000Z", - "maxTime" : "2015-09-12T00:00:00.000Z" - } - } - ] - }, - { - "description": "scan, all", - "query": { - "queryType": "scan", - "dataSource": "%%DATASOURCE%%", - "intervals": [ - "2013-01-01/2020-01-02" - ], - "resultFormat":"compactedList" - }, - "expectedResults": %%EXPECTED_SCAN_RESULT%%, - "fieldsToTest": ["events"] - }, - { - "description": "roll up ratio", - "query": { - "queryType":"timeseries", - "dataSource":{ - "type":"table", - "name":"%%DATASOURCE%%" - }, - "intervals":{ - "type":"intervals", - "intervals":[ - "2013-01-01/2020-01-02" - ] - }, - "granularity":{ - "type":"all" - }, - "aggregations":[ - { - "type":"count", - "name":"a0" - }, - { - "type":"longSum", - "name":"a1", - "fieldName":"count", - "expression":null - } - ], - "postAggregations":[ - { - "type":"expression", - "name":"p0", - "expression":"((\"a0\" * 1.00) / \"a1\")", - "ordering":null - } - ] - }, - "expectedResults": [ - { - "timestamp" : "2015-09-12T00:00:00.000Z", - "result" : { - "a1" : %%EXPECTED_SUM_COUNT%%, - "p0" : %%EXPECTED_ROLLUP_RATIO%%, - "a0" : %%EXPECTED_NUM_ROW%% - } - } - ] - } -] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json deleted file mode 100644 index 3a21a856ac6f..000000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json +++ /dev/null @@ -1,57 +0,0 @@ -{ - "type": "index_parallel", - "spec": { - "ioConfig": { - "type": "index_parallel", - "inputSource": { - "type": "inline", - "data": "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" - }, - "inputFormat": { - "type": "json" - } - }, - "tuningConfig": { - "type": "index_parallel", - "partitionsSpec": { - "type": "dynamic", - "maxRowsPerSegment": 3, - "maxTotalRows": 3 - }, - "maxRowsInMemory": 3 - }, - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "timestampSpec": { - "column": "time", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "dimB", - "dimA", - "dimC", - "dimD", - "dimE", - "dimF" - ] - }, - "granularitySpec": { - "queryGranularity": "hour", - "rollup": true, - "segmentGranularity": "hour" - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" - }, - { - "name": "sum_metA", - "type": "longSum", - "fieldName": "metA" - } - ] - } - } -} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json deleted file mode 100644 index 9416a3f6bdad..000000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json +++ /dev/null @@ -1,19 +0,0 @@ -{ - "type": "compact", - "dataSource": "%%DATASOURCE%%", - "dimensionsSpec": { - "dimensions": %%DIMENSION_NAMES%% - }, - "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", - "tuningConfig": { - "type": "index_parallel", - "maxRowsPerSegment": 3, - "maxRowsInMemory": 3, - "maxNumConcurrentSubTasks": 2, - "partitionsSpec": { - "type": "hashed", - "numShards": 1 - }, - "forceGuaranteedRollup": true - } -} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json deleted file mode 100644 index a149d7a25127..000000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json +++ /dev/null @@ -1,16 +0,0 @@ -{ - "type": "compact", - "dataSource": "%%DATASOURCE%%", - "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", - "tuningConfig": { - "type": "index_parallel", - "maxRowsPerSegment": 3, - "maxRowsInMemory": 3, - "maxNumConcurrentSubTasks": 2, - "partitionsSpec": { - "type": "hashed", - "numShards": 1 - }, - "forceGuaranteedRollup": true - } -} \ No newline at end of file From 8e35b81942f84d45ad4bbbf4ac4498b2ade996f7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 09:27:29 +0530 Subject: [PATCH 25/38] Rename the file to simplify the patch --- ...nSparseColumnTest.java => ITCompactionSparseColumnTest.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedCompactionSparseColumnTest.java => ITCompactionSparseColumnTest.java} (99%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java index 8b8bb831360b..07c289a9ed60 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java @@ -55,7 +55,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase +public class ITCompactionSparseColumnTest extends EmbeddedClusterTestBase { private static final Supplier INDEX_TASK = () -> TaskBuilder From 41dc765101a55f5d5fe33dfd809a468a3fd0e406 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 09:28:59 +0530 Subject: [PATCH 26/38] Move back the file --- .../druid/tests/indexer}/ITCompactionSparseColumnTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename {embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact => integration-tests/src/test/java/org/apache/druid/tests/indexer}/ITCompactionSparseColumnTest.java (99%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java rename to integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java index 07c289a9ed60..f4c3f6f5656c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/ITCompactionSparseColumnTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.testing.embedded.compact; +package org.apache.druid.tests.indexer; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableList; From d59f4e759396260eb5189f3920dbf9138f453c97 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 10:33:08 +0530 Subject: [PATCH 27/38] Add centralized schema variant of test --- .../EmbeddedCompactionSparseColumnTest.java | 4 +- ...edCentralizedSchemaPublishFailureTest.java | 53 +++++++++++++++++++ .../embedded/derby/InMemoryDerbyModule.java | 10 +++- 3 files changed, 64 insertions(+), 3 deletions(-) rename integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java => embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java (99%) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java similarity index 99% rename from integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java index f4c3f6f5656c..8b8bb831360b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.tests.indexer; +package org.apache.druid.testing.embedded.compact; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableList; @@ -55,7 +55,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class ITCompactionSparseColumnTest extends EmbeddedClusterTestBase +public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase { private static final Supplier INDEX_TASK = () -> TaskBuilder diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java new file mode 100644 index 000000000000..a2534e1ba7fc --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java @@ -0,0 +1,53 @@ +/* + * 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.testing.embedded.schema; + +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.compact.EmbeddedCompactionSparseColumnTest; +import org.junit.jupiter.api.Nested; + +/** + * Re-runs various tests with {@code druid.centralizedDatasourceSchema.taskSchemaPublishDisbled} + * set to true. This is a test-only config used to verify that schema is populated + * correctly even when tasks fail to publish it. + */ +public class EmbeddedCentralizedSchemaPublishFailureTest +{ + private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluster) + { + cluster.addCommonProperty("druid.centralizedDatasourceSchema.enabled", "true") + .addCommonProperty("druid.centralizedDatasourceSchema.taskSchemaPublishDisabled", "true") + .addCommonProperty("druid.centralizedDatasourceSchema.backFillEnabled", "true") + .addCommonProperty("druid.centralizedDatasourceSchema.backFillPeriod", "500") + .addCommonProperty("druid.coordinator.segmentMetadata.metadataRefreshPeriod", "PT0.1s"); + + return cluster; + } + + @Nested + public class CompactionSparseColumn extends EmbeddedCompactionSparseColumnTest + { + @Override + protected EmbeddedDruidCluster createCluster() + { + return configureCluster(super.createCluster()); + } + } +} diff --git a/services/src/test/java/org/apache/druid/testing/embedded/derby/InMemoryDerbyModule.java b/services/src/test/java/org/apache/druid/testing/embedded/derby/InMemoryDerbyModule.java index 1ad787706076..addc2c1919aa 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/derby/InMemoryDerbyModule.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/derby/InMemoryDerbyModule.java @@ -74,7 +74,7 @@ public void configure(Binder binder) MetadataStorageConnectorConfig.create(connectURI, null, null, null), MetadataStorageTablesConfig.fromBase(properties.getProperty("druid.metadata.storage.tables.base")), connectURI, - CentralizedDatasourceSchemaConfig.create() + createSchemaConfig() ); PolyBind.optionBinder(binder, Key.get(MetadataStorageProvider.class)) @@ -95,4 +95,12 @@ public void configure(Binder binder) .to(DerbyMetadataStorageActionHandlerFactory.class) .in(LazySingleton.class); } + + private CentralizedDatasourceSchemaConfig createSchemaConfig() + { + final Boolean isEnabled = Boolean.parseBoolean( + properties.getProperty("druid.centralizedDatasourceSchema.enabled") + ); + return new CentralizedDatasourceSchemaConfig(isEnabled, null, null, null); + } } From 25e4f008d12a85e2f237014e7129efbf37f23e1b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 13 Jul 2025 14:05:30 +0530 Subject: [PATCH 28/38] Completely remove `compaction` IT group --- .github/workflows/cron-job-its.yml | 6 +- .github/workflows/standard-its.yml | 6 +- .../compact/EmbeddedAutoCompactionTest.java | 135 +----- .../EmbeddedCompactionSparseColumnTest.java | 56 +-- .../compact/EmbeddedCompactionTaskTest.java | 379 ++++++++++++++++ .../compact/EmbeddedCompactionTestBase.java | 134 ++++++ .../EmbeddedIndexParallelTaskTest.java | 8 +- .../indexing/EmbeddedIndexTaskTest.java | 4 +- .../testing/embedded/indexing/Resources.java | 148 +++++-- ...alizedSchemaMetadataQueryDisabledTest.java | 65 +++ ...edCentralizedSchemaPublishFailureTest.java | 11 + .../server/EmbeddedCoordinatorClientTest.java | 2 +- .../server/EmbeddedHighAvailabilityTest.java | 2 +- .../indexing/common/task/TaskBuilder.java | 34 +- .../org/apache/druid/tests/TestNGGroup.java | 2 - .../tests/indexer/AbstractIndexerTest.java | 2 +- .../tests/indexer/ITCompactionTaskTest.java | 405 ------------------ ..._index_queries_hour_query_granularity.json | 150 ------- ..._index_queries_year_query_granularity.json | 150 ------- 19 files changed, 759 insertions(+), 940 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index 7328a5a0dedc..b3a40f1decf9 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -60,7 +60,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction] + testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion] uses: ./.github/workflows/reusable-standard-its.yml needs: build with: @@ -74,7 +74,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [ input-source, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion, compaction ] + testing_group: [ input-source, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion ] uses: ./.github/workflows/reusable-standard-its.yml needs: build with: @@ -109,7 +109,7 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,custom-coordinator-duties + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,high-availability,custom-coordinator-duties use_indexer: ${{ matrix.indexer }} group: other diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index 60983088f0c7..25932d0f2c5d 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -47,7 +47,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-metadata-query-disabled] + testing_group: [batch-index, input-format, input-source, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, cds-task-schema-publish-disabled, cds-coordinator-metadata-query-disabled] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -63,7 +63,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [input-source, kafka-index, append-ingestion, compaction] + testing_group: [input-source, kafka-index, append-ingestion] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -180,6 +180,6 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,high-availability,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled use_indexer: ${{ matrix.indexer }} group: other diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java index f8c690017f33..f75ee367a080 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java @@ -40,7 +40,6 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.TaskBuilder; -import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -48,7 +47,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -77,15 +75,11 @@ import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; -import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; -import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.Resources; -import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.apache.druid.timeline.DataSegment; import org.hamcrest.Matcher; import org.hamcrest.MatcherAssert; @@ -104,81 +98,36 @@ import java.io.Closeable; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; import java.util.Set; -import java.util.TreeSet; import java.util.function.Supplier; import java.util.stream.Collectors; /** * Embedded mode of integration-tests originally present in {@code ITAutoCompactionTest}. */ -public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase +public class EmbeddedAutoCompactionTest extends EmbeddedCompactionTestBase { private static final Logger LOG = new Logger(EmbeddedAutoCompactionTest.class); - private static final Supplier INDEX_TASK = - () -> TaskBuilder - .ofTypeIndex() - .jsonInputFormat() - .localInputSourceWithFiles( - Resources.TINY_WIKI_1_JSON, - Resources.TINY_WIKI_2_JSON, - Resources.TINY_WIKI_JSON - ) - .timestampColumn("timestamp") - .dimensions( - "page", - "language", "tags", "user", "unpatrolled", "newPage", "robot", - "anonymous", "namespace", "continent", "country", "region", "city" - ) - .metricAggregates( - new CountAggregatorFactory("ingested_events"), - new DoubleSumAggregatorFactory("added", "added"), - new DoubleSumAggregatorFactory("deleted", "deleted"), - new DoubleSumAggregatorFactory("delta", "delta"), - new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), - new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), - new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null) - ) - .dynamicPartitionWithMaxRows(3) - .granularitySpec("DAY", "SECOND", true) - .appendToExisting(false); + private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); private static final Supplier INDEX_TASK_WITH_DIMENSION_SPEC = () -> INDEX_TASK.get().granularitySpec("DAY", "DAY", true); - private static final String SELECT_APPROX_COUNT_DISTINCT = - "SELECT" - + " APPROX_COUNT_DISTINCT_DS_THETA(\"thetaSketch\")," - + " APPROX_COUNT_DISTINCT_DS_HLL(\"HLLSketchBuild\")" - + " FROM %s"; + private static final String SELECT_APPROX_COUNT_DISTINCT = Resources.Query.SELECT_APPROX_COUNT_DISTINCT; private static final List> INDEX_QUERIES_RESOURCE = List.of( - Pair.of( - "SELECT MIN(__time), MAX(__time) FROM %s", - "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z" - ), - Pair.of(SELECT_APPROX_COUNT_DISTINCT, "5,5"), - Pair.of( - "SELECT EARLIEST(\"user\"), LATEST(\"user\") FROM %s WHERE __time < '2013-09-01'", - "nuclear,stringer" - ), - Pair.of( - "SELECT \"page\", COUNT(*) AS \"rows\", SUM(\"added\"), 10 * SUM(\"added\") AS added_times_ten" - + " FROM %s" - + " WHERE \"language\" = 'zh' AND __time < '2013-09-01'" - + " GROUP BY 1" - + " HAVING added_times_ten > 9000", - "Crimson Typhoon,1,905.0,9050.0" - ) + Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z"), + Pair.of(Resources.Query.SELECT_APPROX_COUNT_DISTINCT, "5,5"), + Pair.of(Resources.Query.SELECT_EARLIEST_LATEST_USER, "nuclear,stringer"), + Pair.of(Resources.Query.SELECT_COUNT_OF_CHINESE_PAGES, "Crimson Typhoon,1,905.0,9050.0") ); private static final Supplier INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS = () -> TaskBuilder .ofTypeIndexParallel() .jsonInputFormat() - .inlineInputSourceWithData(Resources.JSON_DATA_2_ROWS) + .inlineInputSourceWithData(Resources.InlineData.JSON_2_ROWS) .isoTimestampColumn("timestamp") .appendToExisting(true) .granularitySpec("DAY", "HOUR", true) @@ -203,7 +152,7 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase () -> TaskBuilder .ofTypeIndexParallel() .jsonInputFormat() - .inlineInputSourceWithData(Resources.JSON_DATA_1_ROW) + .inlineInputSourceWithData(Resources.InlineData.JSON_1_ROW) .isoTimestampColumn("timestamp") .granularitySpec("DAY", "HOUR", false) .appendToExisting(true) @@ -239,9 +188,6 @@ public class EmbeddedAutoCompactionTest extends EmbeddedClusterTestBase private static final Period NO_SKIP_OFFSET = Period.seconds(0); private static final FixedIntervalOrderPolicy COMPACT_NOTHING_POLICY = new FixedIntervalOrderPolicy(List.of()); - private final EmbeddedOverlord overlord = new EmbeddedOverlord(); - private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() - .addProperty("druid.manager.segments.useIncrementalCache", "always"); private final EmbeddedBroker broker = new EmbeddedBroker() .addProperty("druid.sql.planner.metadataRefreshPeriod", "PT0.1s"); @@ -1730,11 +1676,7 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception taskBuilder.granularitySpec(granularitySpec); } - final String taskId = EmbeddedClusterApis.newTaskId(fullDatasourceName); - cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.withId(taskId))); - LOG.info("Submitted task[%s] to load data", taskId); - cluster.callApi().waitForTaskToSucceed(taskId, overlord); - cluster.callApi().waitForAllSegmentsToBeAvailable(fullDatasourceName, coordinator); + runTask(taskBuilder, fullDatasourceName); } private void verifyQuery(List> queries) @@ -1972,31 +1914,6 @@ private void waitForCompactionToFinish(int numExpectedSegmentsAfterCompaction) verifySegmentsCount(numExpectedSegmentsAfterCompaction); } - private void verifySegmentsCount(int numExpectedSegments) - { - int segmentCount = getFullSegmentsMetadata(dataSource).size(); - Assertions.assertEquals(numExpectedSegments, segmentCount, "Segment count mismatch"); - Assertions.assertEquals( - String.valueOf(segmentCount), - cluster.runSql( - "SELECT COUNT(*) FROM sys.segments" - + " WHERE datasource='%s' AND is_overshadowed = 0 AND is_available = 1", - dataSource - ), - "Segment count mismatch in sys table" - ); - } - - private void checkCompactionIntervals(List expectedIntervals) - { - // TODO: is waiting really needed here? - // If we have waited for all segments to be loaded, we can just move on from here - Assertions.assertEquals( - Set.copyOf(expectedIntervals), - Set.copyOf(getSegmentIntervals(dataSource)) - ); - } - private void verifySegmentsCompacted(int expectedCompactedSegmentCount, Integer expectedMaxRowsPerSegment) { verifySegmentsCompacted( @@ -2106,27 +2023,6 @@ private void getAndAssertCompactionStatus( Assertions.assertEquals(actualStatus.getIntervalCountCompacted(), intervalCountCompacted); Assertions.assertEquals(actualStatus.getIntervalCountSkipped(), intervalCountSkipped); } - - private Set getFullSegmentsMetadata(String dataSource) - { - return overlord - .bindings() - .segmentsMetadataStorage() - .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE); - } - - private List getSegmentIntervals(String dataSource) - { - final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); - final Set sortedIntervals = new TreeSet<>(comparator); - - final Set allUsedSegments = getFullSegmentsMetadata(dataSource); - for (DataSegment segment : allUsedSegments) { - sortedIntervals.add(segment.getInterval()); - } - - return sortedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); - } private List getCompleteTasksForDataSource(String dataSource) { @@ -2150,15 +2046,4 @@ private Set getTaskIdsForState(String state, String dataSource) (CloseableIterator) cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(state, dataSource, 0)) ).stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); } - - /** - * Deletes all the data for the given datasource so that compaction tasks for - * this datasource do not take up task slots unnecessarily. - */ - private Closeable unloader(String dataSource) - { - return () -> { - overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); - }; - } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java index 8b8bb831360b..98c0837f5dfb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java @@ -34,15 +34,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.TestHelper; -import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; -import org.apache.druid.testing.embedded.EmbeddedCoordinator; -import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -import org.apache.druid.testing.embedded.EmbeddedHistorical; -import org.apache.druid.testing.embedded.EmbeddedIndexer; -import org.apache.druid.testing.embedded.EmbeddedOverlord; -import org.apache.druid.testing.embedded.EmbeddedRouter; -import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -55,7 +46,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase +public class EmbeddedCompactionSparseColumnTest extends EmbeddedCompactionTestBase { private static final Supplier INDEX_TASK = () -> TaskBuilder @@ -94,22 +85,6 @@ public class EmbeddedCompactionSparseColumnTest extends EmbeddedClusterTestBase .withPartitionsSpec(new HashedPartitionsSpec(null, 1, null)) ); - private final EmbeddedOverlord overlord = new EmbeddedOverlord(); - private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); - - @Override - protected EmbeddedDruidCluster createCluster() - { - return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() - .useLatchableEmitter() - .addServer(overlord) - .addServer(coordinator) - .addServer(new EmbeddedIndexer()) - .addServer(new EmbeddedBroker()) - .addServer(new EmbeddedHistorical()) - .addServer(new EmbeddedRouter()); - } - @Test public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception { @@ -117,7 +92,7 @@ public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dataSource(dataSource)); + runTask(COMPACTION_TASK.get(), dataSource); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -144,7 +119,7 @@ public void testCompactionPerfectRollUpWithLexicographicDimensionSpec() throws E // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dataSource(dataSource).dimensions("dimA", "dimB", "dimC")); + runTask(COMPACTION_TASK.get().dimensions("dimA", "dimB", "dimC"), dataSource); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -170,7 +145,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dataSource(dataSource).dimensions("dimC", "dimB", "dimA")); + runTask(COMPACTION_TASK.get().dimensions("dimC", "dimB", "dimA"), dataSource); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -191,7 +166,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw private void loadAndVerifyDataWithSparseColumn() { - runTask(INDEX_TASK.get().dataSource(dataSource)); + runTask(INDEX_TASK.get(), dataSource); List>>> expectedResultBeforeCompaction = new ArrayList<>(); // First segments have the following rows: List> segment1Rows = ImmutableList.of( @@ -287,25 +262,4 @@ private String toJson(Object stuff) throw new RuntimeException(e); } } - - /** - * Deletes all the data for the given datasource so that compaction tasks for - * this datasource do not take up task slots unnecessarily. - */ - private Closeable unloader(String dataSource) - { - return () -> { - overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); - }; - } - - private void runTask(TaskBuilder taskBuilder) - { - final String taskId = EmbeddedClusterApis.newTaskId(dataSource); - cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskBuilder.withId(taskId)) - ); - cluster.callApi().waitForTaskToSucceed(taskId, overlord); - cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); - } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java new file mode 100644 index 000000000000..b470cd71f664 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java @@ -0,0 +1,379 @@ +/* + * 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.testing.embedded.compact; + +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.report.IngestionStatsAndErrors; +import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.common.task.CompactionIntervalSpec; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.Resources; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.Closeable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class EmbeddedCompactionTaskTest extends EmbeddedCompactionTestBase +{ + private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; + + private static final List> INDEX_QUERIES_RESOURCE = List.of( + Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z"), + Pair.of(Resources.Query.SELECT_APPROX_COUNT_DISTINCT, "5,5"), + Pair.of(Resources.Query.SELECT_EARLIEST_LATEST_USER, "nuclear,stringer"), + Pair.of(Resources.Query.SELECT_COUNT_OF_CHINESE_PAGES, "Crimson Typhoon,1,905.0,9050.0") + ); + + private static final List> INDEX_QUERIES_YEAR_RESOURCE = List.of( + Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-01-01T00:00:00.000Z,2013-01-01T00:00:00.000Z"), + Pair.of(Resources.Query.SELECT_APPROX_COUNT_DISTINCT, "5,5"), + Pair.of(Resources.Query.SELECT_EARLIEST_LATEST_USER, "masterYi,speed"), + Pair.of(Resources.Query.SELECT_COUNT_OF_CHINESE_PAGES, "Crimson Typhoon,1,1810.0,18100.0") + ); + + private static final List> INDEX_QUERIES_HOUR_RESOURCE = List.of( + Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:00:00.000Z,2013-09-01T12:00:00.000Z"), + Pair.of(Resources.Query.SELECT_APPROX_COUNT_DISTINCT, "5,5"), + Pair.of(Resources.Query.SELECT_EARLIEST_LATEST_USER, "nuclear,stringer"), + Pair.of(Resources.Query.SELECT_COUNT_OF_CHINESE_PAGES, "Crimson Typhoon,1,905.0,9050.0") + ); + + private static final Supplier COMPACTION_TASK = + () -> TaskBuilder + .ofTypeCompact() + .context("storeCompactionState", true) + .ioConfig(new CompactionIntervalSpec(Intervals.of("2013-08-31/2013-09-02"), null), false); + private static final Supplier PARALLEL_COMPACTION_TASK = + () -> COMPACTION_TASK.get().tuningConfig( + t -> t.withPartitionsSpec(new HashedPartitionsSpec(null, null, null)) + .withMaxNumConcurrentSubTasks(3) + .withForceGuaranteedRollup(true) + ); + private static final Supplier COMPACTION_TASK_ALLOW_NON_ALIGNED = + () -> TaskBuilder + .ofTypeCompact() + .context("storeCompactionState", true) + .ioConfig(new CompactionIntervalSpec(Intervals.of("2013-08-31/2013-09-02"), null), true); + + private static final Supplier INDEX_TASK_WITH_TIMESTAMP = + () -> Resources.Task.BASIC_INDEX.get().dimensions( + "page", + "language", "user", "unpatrolled", "newPage", "robot", "anonymous", + "namespace", "continent", "country", "region", "city", "timestamp" + ); + + private String fullDatasourceName; + private final EmbeddedBroker broker = new EmbeddedBroker() + .addProperty("druid.sql.planner.metadataRefreshPeriod", "PT0.1s"); + + @BeforeEach + public void setFullDatasourceName() + { + fullDatasourceName = dataSource; + } + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addExtension(SketchModule.class) + .addExtension(HllSketchModule.class) + .addExtension(DoublesSketchModule.class) + .addServer(overlord) + .addServer(coordinator) + .addServer(broker) + .addServer(new EmbeddedIndexer().addProperty("druid.worker.capacity", "10")) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()); + } + + @Test + public void testCompaction() throws Exception + { + loadDataAndCompact(INDEX_TASK.get(), COMPACTION_TASK.get(), null); + } + + @Test + public void testCompactionWithSegmentGranularity() throws Exception + { + loadDataAndCompact(INDEX_TASK.get(), COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), Granularities.MONTH); + } + + @Test + public void testCompactionWithSegmentGranularityInGranularitySpec() throws Exception + { + loadDataAndCompact(INDEX_TASK.get(), COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), Granularities.MONTH); + } + + @Test + public void testCompactionWithQueryGranularityInGranularitySpec() throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + runTask(INDEX_TASK.get(), fullDatasourceName); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + verifySegmentsHaveQueryGranularity("SECOND", 4); + runQueries(INDEX_QUERIES_RESOURCE); + + // QueryGranularity was SECOND, now we will change it to HOUR (QueryGranularity changed to coarser) + compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), null, Granularities.HOUR); + + // The original 4 segments should be compacted into 2 new segments since data only has 2 days and the compaction + // segmentGranularity is DAY + checkNumberOfSegments(2); + runQueries(INDEX_QUERIES_HOUR_RESOURCE); + verifySegmentsHaveQueryGranularity("HOUR", 2); + checkCompactionIntervals(expectedIntervalAfterCompaction); + + // QueryGranularity was HOUR, now we will change it to MINUTE (QueryGranularity changed to finer) + compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), null, Granularities.MINUTE); + + // There will be no change in number of segments as compaction segmentGranularity is the same and data interval + // is the same. Since QueryGranularity is changed to finer qranularity, the data will remains the same. (data + // will just be bucketed to a finer qranularity but roll up will not be different + // i.e. 2020-10-29T05:00 will just be bucketed to 2020-10-29T05:00:00) + checkNumberOfSegments(2); + runQueries(INDEX_QUERIES_HOUR_RESOURCE); + verifySegmentsHaveQueryGranularity("MINUTE", 2); + checkCompactionIntervals(expectedIntervalAfterCompaction); + } + } + + @Test + public void testParallelHashedCompaction() throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + runTask(INDEX_TASK.get(), fullDatasourceName); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + verifySegmentsHaveQueryGranularity("SECOND", 4); + runQueries(INDEX_QUERIES_RESOURCE); + String taskId = compactData(PARALLEL_COMPACTION_TASK.get(), null, null); + + // The original 4 segments should be compacted into 2 new segments + checkNumberOfSegments(2); + runQueries(INDEX_QUERIES_RESOURCE); + verifySegmentsHaveQueryGranularity("SECOND", 2); + + + checkCompactionIntervals(expectedIntervalAfterCompaction); + + Map reports = cluster.callApi().onLeaderOverlord(o -> o.taskReportAsMap(taskId)); + Assertions.assertTrue(reports != null && !reports.isEmpty()); + + Assertions.assertEquals( + 2, + reports.values() + .stream() + .filter(r -> r instanceof IngestionStatsAndErrorsTaskReport) + .mapToLong(r -> ((IngestionStatsAndErrors) r.getPayload()).getSegmentsPublished()) + .sum() + ); + Assertions.assertEquals( + 4, + reports.values() + .stream() + .filter(r -> r instanceof IngestionStatsAndErrorsTaskReport) + .mapToLong(r -> ((IngestionStatsAndErrors) r.getPayload()).getSegmentsRead()) + .sum() + ); + } + } + + @Test + public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularitySpec() throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + runTask(INDEX_TASK.get(), fullDatasourceName); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + verifySegmentsHaveQueryGranularity("SECOND", 4); + runQueries(INDEX_QUERIES_RESOURCE); + compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), Granularities.YEAR, Granularities.YEAR); + + // The original 4 segments should be compacted into 1 new segment + checkNumberOfSegments(1); + runQueries(INDEX_QUERIES_YEAR_RESOURCE); + verifySegmentsHaveQueryGranularity("YEAR", 1); + + List newIntervals = new ArrayList<>(); + for (String interval : expectedIntervalAfterCompaction) { + for (Interval newinterval : Granularities.YEAR.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { + newIntervals.add(newinterval.toString()); + } + } + expectedIntervalAfterCompaction = newIntervals; + checkCompactionIntervals(expectedIntervalAfterCompaction); + } + } + + @Test + public void testCompactionWithTimestampDimension() throws Exception + { + loadDataAndCompact(INDEX_TASK_WITH_TIMESTAMP.get(), COMPACTION_TASK.get(), null); + } + + private void loadDataAndCompact( + TaskBuilder.Index indexTask, + TaskBuilder.Compact compactionResource, + Granularity newSegmentGranularity + ) throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + runTask(indexTask, fullDatasourceName); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + verifySegmentsHaveQueryGranularity("SECOND", 4); + runQueries(INDEX_QUERIES_RESOURCE); + + compactData(compactionResource, newSegmentGranularity, null); + + // The original 4 segments should be compacted into 2 new segments + checkNumberOfSegments(2); + runQueries(INDEX_QUERIES_RESOURCE); + verifySegmentsHaveQueryGranularity("SECOND", 2); + + if (newSegmentGranularity != null) { + List newIntervals = new ArrayList<>(); + for (String interval : expectedIntervalAfterCompaction) { + for (Interval newinterval : newSegmentGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { + newIntervals.add(newinterval.toString()); + } + } + expectedIntervalAfterCompaction = newIntervals; + } + checkCompactionIntervals(expectedIntervalAfterCompaction); + } + } + + private String compactData( + TaskBuilder.Compact template, + Granularity newSegmentGranularity, + Granularity newQueryGranularity + ) + { + template.dataSource(fullDatasourceName); + // For the new granularitySpec map + template.granularitySpec( + new ClientCompactionTaskGranularitySpec(newSegmentGranularity, newQueryGranularity, null) + ); + // For the deprecated segment granularity field + if (newSegmentGranularity != null) { + template.segmentGranularity(newSegmentGranularity); + } + return runTask(template, fullDatasourceName); + } + + private void verifySegmentsHaveQueryGranularity(String expectedQueryGranularity, int segmentCount) + { + final SegmentMetadataQuery query = new Druids.SegmentMetadataQueryBuilder() + .dataSource(fullDatasourceName) + .analysisTypes(SegmentMetadataQuery.AnalysisType.QUERYGRANULARITY) + .intervals("2013-08-31/2013-09-02") + .build(); + + final String resultAsJson = FutureUtils.getUnchecked( + cluster.anyBroker().submitNativeQuery(query), + true + ); + + // Trim the result so that it contains only the `queryGranularity` fields + final List> resultList = JacksonUtils.readValue( + TestHelper.JSON_MAPPER, + resultAsJson.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() {} + ); + final List> trimmedResult = resultList + .stream() + .map(map -> Map.of("queryGranularity", map.getOrDefault("queryGranularity", ""))) + .collect(Collectors.toList()); + + final List> expectedResults = new ArrayList<>(); + for (int i = 0; i < segmentCount; i++) { + expectedResults.add( + Map.of("queryGranularity", expectedQueryGranularity) + ); + } + + Assertions.assertEquals(expectedResults, trimmedResult); + } + + private void checkNumberOfSegments(int numExpectedSegments) + { + verifySegmentsCount(numExpectedSegments); + } + + private void runQueries(List> queries) + { + if (queries == null) { + return; + } + for (Pair query : queries) { + Assertions.assertEquals( + query.rhs, + cluster.runSql(query.lhs, dataSource), + StringUtils.format("Query[%s] failed", query.lhs) + ); + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java new file mode 100644 index 000000000000..e014106b5972 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java @@ -0,0 +1,134 @@ +/* + * 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.testing.embedded.compact; + +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; +import org.junit.jupiter.api.Assertions; + +import java.io.Closeable; +import java.util.Comparator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; + +public abstract class EmbeddedCompactionTestBase extends EmbeddedClusterTestBase +{ + protected final EmbeddedOverlord overlord = new EmbeddedOverlord(); + protected final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addServer(overlord) + .addServer(coordinator) + .addServer(new EmbeddedIndexer()) + .addServer(new EmbeddedBroker()) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()); + } + + /** + * Deletes all the data for the given datasource so that compaction tasks for + * this datasource do not take up task slots unnecessarily. + */ + protected Closeable unloader(String dataSource) + { + return () -> { + overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); + }; + } + + /** + * Creates a Task using the given builder and runs it. + * + * @return ID of the task. + */ + protected String runTask(TaskBuilder taskBuilder, String dataSource) + { + final String taskId = EmbeddedClusterApis.newTaskId(dataSource); + cluster.callApi().onLeaderOverlord( + o -> o.runTask(taskId, taskBuilder.dataSource(dataSource).withId(taskId)) + ); + cluster.callApi().waitForTaskToSucceed(taskId, overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); + + return taskId; + } + + protected void checkCompactionIntervals(List expectedIntervals) + { + Assertions.assertEquals( + Set.copyOf(expectedIntervals), + Set.copyOf(getSegmentIntervals(dataSource)) + ); + } + + protected Set getFullSegmentsMetadata(String dataSource) + { + return overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE); + } + + protected List getSegmentIntervals(String dataSource) + { + final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); + final Set sortedIntervals = new TreeSet<>(comparator); + + final Set allUsedSegments = getFullSegmentsMetadata(dataSource); + for (DataSegment segment : allUsedSegments) { + sortedIntervals.add(segment.getInterval()); + } + + return sortedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + } + + protected void verifySegmentsCount(int numExpectedSegments) + { + int segmentCount = getFullSegmentsMetadata(dataSource).size(); + Assertions.assertEquals(numExpectedSegments, segmentCount, "Segment count mismatch"); + Assertions.assertEquals( + String.valueOf(segmentCount), + cluster.runSql( + "SELECT COUNT(*) FROM sys.segments" + + " WHERE datasource='%s' AND is_overshadowed = 0 AND is_available = 1", + dataSource + ), + "Segment count mismatch in sys table" + ); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java index f990d0962035..ea96129319ff 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java @@ -100,7 +100,7 @@ public void test_segmentAvailabilityIsConfirmed_whenTaskWaits5secondsForHandoff( .dataSource(dataSource) .timestampColumn("timestamp") .jsonInputFormat() - .localInputSourceWithFiles(Resources.TINY_WIKI_1_JSON) + .localInputSourceWithFiles(Resources.DataFile.TINY_WIKI_1_JSON) .dimensions() .tuningConfig( t -> t.withAwaitSegmentAvailabilityTimeoutMillis(segmentAvailabilityTimeoutMillis) @@ -135,9 +135,9 @@ public void test_runIndexTask_andReindexIntoAnotherDatasource(PartitionsSpec par .timestampColumn("timestamp") .jsonInputFormat() .localInputSourceWithFiles( - Resources.TINY_WIKI_1_JSON, - Resources.TINY_WIKI_2_JSON, - Resources.TINY_WIKI_JSON + Resources.DataFile.TINY_WIKI_1_JSON, + Resources.DataFile.TINY_WIKI_2_JSON, + Resources.DataFile.TINY_WIKI_3_JSON ) .segmentGranularity("DAY") .dimensions("namespace", "page", "language") diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java index fcd6c7ebd7f0..4cb90fdacc74 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java @@ -78,7 +78,7 @@ public void test_runIndexTask_forInlineDatasource() final String taskId = EmbeddedClusterApis.newTaskId(dataSource); final Object task = createIndexTaskForInlineData( taskId, - Resources.CSV_DATA_10_DAYS + Resources.InlineData.CSV_10_DAYS ); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); @@ -106,7 +106,7 @@ public void test_runIndexTask_forInlineDatasource() event -> event.hasDimension(DruidMetrics.DATASOURCE, dataSource) ); Assertions.assertEquals( - Resources.CSV_DATA_10_DAYS, + Resources.InlineData.CSV_10_DAYS, cluster.runSql("SELECT * FROM %s", dataSource) ); Assertions.assertEquals("10", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java index 849d239dce7b..48ddd6534145 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java @@ -19,50 +19,122 @@ package org.apache.druid.testing.embedded.indexing; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; + +import java.util.function.Supplier; + /** * Constants and utility methods used in embedded cluster tests. */ public class Resources { - /** - * 10 rows (1 row per day) of inline CSV data with 3 columns (time, item, value). - */ - public static final String CSV_DATA_10_DAYS = - "2025-06-01T00:00:00.000Z,shirt,105" - + "\n2025-06-02T00:00:00.000Z,trousers,210" - + "\n2025-06-03T00:00:00.000Z,jeans,150" - + "\n2025-06-04T00:00:00.000Z,t-shirt,53" - + "\n2025-06-05T00:00:00.000Z,microwave,1099" - + "\n2025-06-06T00:00:00.000Z,spoon,11" - + "\n2025-06-07T00:00:00.000Z,television,1100" - + "\n2025-06-08T00:00:00.000Z,plant pots,75" - + "\n2025-06-09T00:00:00.000Z,shirt,99" - + "\n2025-06-10T00:00:00.000Z,toys,101"; + public static class InlineData + { + /** + * 10 rows (1 row per day) of inline CSV data with 3 columns (time, item, value). + */ + public static final String CSV_10_DAYS = + "2025-06-01T00:00:00.000Z,shirt,105" + + "\n2025-06-02T00:00:00.000Z,trousers,210" + + "\n2025-06-03T00:00:00.000Z,jeans,150" + + "\n2025-06-04T00:00:00.000Z,t-shirt,53" + + "\n2025-06-05T00:00:00.000Z,microwave,1099" + + "\n2025-06-06T00:00:00.000Z,spoon,11" + + "\n2025-06-07T00:00:00.000Z,television,1100" + + "\n2025-06-08T00:00:00.000Z,plant pots,75" + + "\n2025-06-09T00:00:00.000Z,shirt,99" + + "\n2025-06-10T00:00:00.000Z,toys,101"; + + public static final String JSON_2_ROWS = + "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," + + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," + + "\"isAnonymous\":false,\"user\":\"maytas1\",\"deltaBucket\":0.0,\"deleted\":0," + + "\"namespace\":\"Main\"}" + + "\n{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\",\"commentLength\":35," + + "\"isNew\":true,\"isMinor\":false,\"delta\":11,\"isAnonymous\":false,\"user\":\"maytas2\"," + + "\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; - public static final String JSON_DATA_2_ROWS = - "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," - + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," - + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," - + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," - + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," - + "\"isAnonymous\":false,\"user\":\"maytas1\",\"deltaBucket\":0.0,\"deleted\":0," - + "\"namespace\":\"Main\"}" - + "\n{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," - + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," - + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," - + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\",\"commentLength\":35," - + "\"isNew\":true,\"isMinor\":false,\"delta\":11,\"isAnonymous\":false,\"user\":\"maytas2\"," - + "\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; + public static final String JSON_1_ROW = + "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," + + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," + + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," + + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," + + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," + + "\"isAnonymous\":false,\"user\":\"maytas3\",\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; + } - public static final String JSON_DATA_1_ROW = - "{\"isRobot\":true,\"language\":\"en\",\"timestamp\":\"2013-08-31T00:00:11.080Z\"," - + "\"flags\":\"NB\",\"isUnpatrolled\":false,\"page\":\"Salo Toraut\"," - + "\"diffUrl\":\"https://sv.wikipedia.org/w/index.php?oldid=36099284&rcid=89369918\"," - + "\"added\":31,\"comment\":\"Botskapande Indonesien omdirigering\"," - + "\"commentLength\":35,\"isNew\":true,\"isMinor\":false,\"delta\":31," - + "\"isAnonymous\":false,\"user\":\"maytas3\",\"deltaBucket\":0.0,\"deleted\":0,\"namespace\":\"Main\"}\n"; + public static class DataFile + { + public static final String TINY_WIKI_1_JSON = "data/json/tiny_wiki_1.json"; + public static final String TINY_WIKI_2_JSON = "data/json/tiny_wiki_2.json"; + public static final String TINY_WIKI_3_JSON = "data/json/tiny_wiki_3.json"; + } - public static final String TINY_WIKI_1_JSON = "data/json/tiny_wiki_1.json"; - public static final String TINY_WIKI_2_JSON = "data/json/tiny_wiki_2.json"; - public static final String TINY_WIKI_JSON = "data/json/tiny_wiki_3.json"; + /** + * Contains queries (that can be run with {@code cluster.runSql(...)}) and + * their corresponding results. + */ + public static class Query + { + // Queries used with Task.BASIC_INDEX + public static final String SELECT_MIN_MAX_TIME = "SELECT MIN(__time), MAX(__time) FROM %s"; + public static final String SELECT_APPROX_COUNT_DISTINCT = + "SELECT" + + " APPROX_COUNT_DISTINCT_DS_THETA(\"thetaSketch\")," + + " APPROX_COUNT_DISTINCT_DS_HLL(\"HLLSketchBuild\")" + + " FROM %s"; + public static final String SELECT_EARLIEST_LATEST_USER = + "SELECT EARLIEST(\"user\"), LATEST(\"user\") FROM %s WHERE __time < '2013-09-01'"; + public static final String SELECT_COUNT_OF_CHINESE_PAGES = + "SELECT \"page\", COUNT(*) AS \"rows\", SUM(\"added\"), 10 * SUM(\"added\") AS added_times_ten" + + " FROM %s" + + " WHERE \"language\" = 'zh' AND __time < '2013-09-01'" + + " GROUP BY 1" + + " HAVING added_times_ten > 9000"; + } + + /** + * Task payload builders. + */ + public static class Task + { + public static final Supplier BASIC_INDEX = + () -> TaskBuilder + .ofTypeIndex() + .jsonInputFormat() + .localInputSourceWithFiles( + DataFile.TINY_WIKI_1_JSON, + DataFile.TINY_WIKI_2_JSON, + DataFile.TINY_WIKI_3_JSON + ) + .timestampColumn("timestamp") + .dimensions( + "page", + "language", "tags", "user", "unpatrolled", "newPage", "robot", + "anonymous", "namespace", "continent", "country", "region", "city" + ) + .metricAggregates( + new CountAggregatorFactory("ingested_events"), + new DoubleSumAggregatorFactory("added", "added"), + new DoubleSumAggregatorFactory("deleted", "deleted"), + new DoubleSumAggregatorFactory("delta", "delta"), + new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), + new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), + new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null) + ) + .dynamicPartitionWithMaxRows(3) + .granularitySpec("DAY", "SECOND", true) + .appendToExisting(false); + } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java new file mode 100644 index 000000000000..d0ace406f108 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java @@ -0,0 +1,65 @@ +/* + * 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.testing.embedded.schema; + +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.compact.EmbeddedCompactionSparseColumnTest; +import org.apache.druid.testing.embedded.compact.EmbeddedCompactionTaskTest; +import org.junit.jupiter.api.Nested; + +/** + * Re-runs various tests with {@code druid.coordinator.segmentMetadata.disableSegmentMetadataQueries} + * set to true. This is a test-only config used to verify that schema is populated + * correctly even when metadata queries fail to fetch schema from Historicals. + */ +public class EmbeddedCentralizedSchemaMetadataQueryDisabledTest +{ + private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluster) + { + cluster.addCommonProperty("druid.centralizedDatasourceSchema.enabled", "true") + .addCommonProperty("druid.centralizedDatasourceSchema.backFillEnabled", "true") + .addCommonProperty("druid.centralizedDatasourceSchema.backFillPeriod", "500") + .addCommonProperty("druid.coordinator.segmentMetadata.disableSegmentMetadataQueries", "true") + .addCommonProperty("druid.coordinator.segmentMetadata.metadataRefreshPeriod", "PT0.1s") + .addCommonProperty("druid.manager.segments.useIncrementalCache", "always"); + + return cluster; + } + + @Nested + public class CompactionSparseColumn extends EmbeddedCompactionSparseColumnTest + { + @Override + protected EmbeddedDruidCluster createCluster() + { + return configureCluster(super.createCluster()); + } + } + + @Nested + public class CompactionTask extends EmbeddedCompactionTaskTest + { + @Override + protected EmbeddedDruidCluster createCluster() + { + return configureCluster(super.createCluster()); + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java index a2534e1ba7fc..b6dbc8582894 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java @@ -21,6 +21,7 @@ import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.compact.EmbeddedCompactionSparseColumnTest; +import org.apache.druid.testing.embedded.compact.EmbeddedCompactionTaskTest; import org.junit.jupiter.api.Nested; /** @@ -50,4 +51,14 @@ protected EmbeddedDruidCluster createCluster() return configureCluster(super.createCluster()); } } + + @Nested + public class CompactionTask extends EmbeddedCompactionTaskTest + { + @Override + protected EmbeddedDruidCluster createCluster() + { + return configureCluster(super.createCluster()); + } + } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java index 1317d381bf9c..699432e75227 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java @@ -187,7 +187,7 @@ private void runIndexTask() .dataSource(dataSource) .isoTimestampColumn("time") .csvInputFormatWithColumns("time", "item", "value") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .inlineInputSourceWithData(Resources.InlineData.CSV_10_DAYS) .segmentGranularity("DAY") .dimensions() .withId(taskId); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java index 2097455758dc..fc4566953b5c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java @@ -112,7 +112,7 @@ public void test_switchLeader_andVerifyUsingSysTables() .dataSource(dataSource) .csvInputFormatWithColumns("time", "item", "value") .isoTimestampColumn("time") - .inlineInputSourceWithData(Resources.CSV_DATA_10_DAYS) + .inlineInputSourceWithData(Resources.InlineData.CSV_10_DAYS) .dimensions() .withId(taskId); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java index 46b11d41c301..e616471034c0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskBuilder.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.client.coordinator.NoopCoordinatorClient; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.CsvInputFormat; @@ -92,7 +93,9 @@ private TaskBuilder() */ public abstract T withId(String taskId); - abstract TuningConfigBuilder tuningConfigBuilder(); + public abstract B dataSource(String dataSource); + + protected abstract TuningConfigBuilder tuningConfigBuilder(); /** * Initializes builder for a new {@link IndexTask}. @@ -224,6 +227,7 @@ public abstract static class IndexCommon, C, T ex { final DataSchema.Builder dataSchema = DataSchema.builder(); + @Override public B dataSource(String dataSource) { dataSchema.withDataSource(dataSource); @@ -373,7 +377,11 @@ public static class Compact extends TaskBuilder listResources(String dir) throws IOException diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java deleted file mode 100644 index b974c7d20e97..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java +++ /dev/null @@ -1,405 +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.tests.indexer; - -import com.google.inject.Inject; -import org.apache.commons.io.IOUtils; -import org.apache.druid.indexer.report.IngestionStatsAndErrors; -import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; -import org.apache.druid.indexer.report.TaskReport; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.GranularityType; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.joda.time.Interval; -import org.joda.time.chrono.ISOChronology; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITCompactionTaskTest extends AbstractIndexerTest -{ - private static final Logger LOG = new Logger(ITCompactionTaskTest.class); - private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - - private static final String INDEX_QUERIES_YEAR_RESOURCE = "/indexer/wikipedia_index_queries_year_query_granularity.json"; - private static final String INDEX_QUERIES_HOUR_RESOURCE = "/indexer/wikipedia_index_queries_hour_query_granularity.json"; - - private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - - private static final String SEGMENT_METADATA_QUERY_RESOURCE = "/indexer/segment_metadata_query.json"; - - private static final String COMPACTION_TASK = "/indexer/wikipedia_compaction_task.json"; - private static final String PARALLEL_COMPACTION_TASK = "/indexer/wikipedia_compaction_task_parallel.json"; - private static final String COMPACTION_TASK_WITH_SEGMENT_GRANULARITY = "/indexer/wikipedia_compaction_task_with_segment_granularity.json"; - private static final String COMPACTION_TASK_WITH_GRANULARITY_SPEC = "/indexer/wikipedia_compaction_task_with_granularity_spec.json"; - - private static final String INDEX_TASK_WITH_TIMESTAMP = "/indexer/wikipedia_with_timestamp_index_task.json"; - - @Inject - private IntegrationTestingConfig config; - - private String fullDatasourceName; - - @BeforeMethod - public void setFullDatasourceName(Method method) - { - fullDatasourceName = INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + "-" + method.getName(); - } - - @Test - public void testCompaction() throws Exception - { - loadDataAndCompact(INDEX_TASK, INDEX_QUERIES_RESOURCE, COMPACTION_TASK, null); - } - - @Test - public void testCompactionWithSegmentGranularity() throws Exception - { - loadDataAndCompact(INDEX_TASK, INDEX_QUERIES_RESOURCE, COMPACTION_TASK_WITH_SEGMENT_GRANULARITY, GranularityType.MONTH); - } - - @Test - public void testCompactionWithSegmentGranularityInGranularitySpec() throws Exception - { - loadDataAndCompact(INDEX_TASK, INDEX_QUERIES_RESOURCE, COMPACTION_TASK_WITH_GRANULARITY_SPEC, GranularityType.MONTH); - } - - @Test - public void testCompactionWithQueryGranularityInGranularitySpec() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - loadData(INDEX_TASK, fullDatasourceName); - // 4 segments across 2 days - checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); - - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); - String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); - queryHelper.testQueriesFromString(queryResponseTemplate); - // QueryGranularity was SECOND, now we will change it to HOUR (QueryGranularity changed to coarser) - compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, null, GranularityType.HOUR); - - // The original 4 segments should be compacted into 2 new segments since data only has 2 days and the compaction - // segmentGranularity is DAY - checkNumberOfSegments(2); - queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_HOUR_RESOURCE); - queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.HOUR.name(), 2); - checkCompactionIntervals(expectedIntervalAfterCompaction); - - // QueryGranularity was HOUR, now we will change it to MINUTE (QueryGranularity changed to finer) - compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, null, GranularityType.MINUTE); - - // There will be no change in number of segments as compaction segmentGranularity is the same and data interval - // is the same. Since QueryGranularity is changed to finer qranularity, the data will remains the same. (data - // will just be bucketed to a finer qranularity but roll up will not be different - // i.e. 2020-10-29T05:00 will just be bucketed to 2020-10-29T05:00:00) - checkNumberOfSegments(2); - queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_HOUR_RESOURCE); - queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.MINUTE.name(), 2); - checkCompactionIntervals(expectedIntervalAfterCompaction); - } - } - - @Test - public void testParallelHashedCompaction() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - loadData(INDEX_TASK, fullDatasourceName); - // 4 segments across 2 days - checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); - - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); - String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); - - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - - queryHelper.testQueriesFromString(queryResponseTemplate); - String taskId = compactData(PARALLEL_COMPACTION_TASK, null, null); - - // The original 4 segments should be compacted into 2 new segments - checkNumberOfSegments(2); - queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 2); - - - checkCompactionIntervals(expectedIntervalAfterCompaction); - - Map reports = indexer.getTaskReport(taskId); - Assert.assertTrue(reports != null && reports.size() > 0); - - Assert.assertEquals( - 2, - reports.values() - .stream() - .filter(r -> r instanceof IngestionStatsAndErrorsTaskReport) - .mapToLong(r -> ((IngestionStatsAndErrors) r.getPayload()).getSegmentsPublished()) - .sum() - ); - Assert.assertEquals( - 4, - reports.values() - .stream() - .filter(r -> r instanceof IngestionStatsAndErrorsTaskReport) - .mapToLong(r -> ((IngestionStatsAndErrors) r.getPayload()).getSegmentsRead()) - .sum() - ); - } - } - - @Test - public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularitySpec() throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - loadData(INDEX_TASK, fullDatasourceName); - // 4 segments across 2 days - checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); - - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); - String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); - queryHelper.testQueriesFromString(queryResponseTemplate); - compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, GranularityType.YEAR, GranularityType.YEAR); - - // The original 4 segments should be compacted into 1 new segment - checkNumberOfSegments(1); - queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_YEAR_RESOURCE); - queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.YEAR.name(), 1); - - List newIntervals = new ArrayList<>(); - for (String interval : expectedIntervalAfterCompaction) { - for (Interval newinterval : GranularityType.YEAR.getDefaultGranularity().getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - newIntervals.add(newinterval.toString()); - } - } - expectedIntervalAfterCompaction = newIntervals; - checkCompactionIntervals(expectedIntervalAfterCompaction); - } - } - - @Test - public void testCompactionWithTimestampDimension() throws Exception - { - loadDataAndCompact(INDEX_TASK_WITH_TIMESTAMP, INDEX_QUERIES_RESOURCE, COMPACTION_TASK, null); - } - - private void loadDataAndCompact( - String indexTask, - String queriesResource, - String compactionResource, - GranularityType newSegmentGranularity - ) throws Exception - { - try (final Closeable ignored = unloader(fullDatasourceName)) { - loadData(indexTask, fullDatasourceName); - // 4 segments across 2 days - checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); - - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); - String queryResponseTemplate = getQueryResponseTemplate(queriesResource); - - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - - queryHelper.testQueriesFromString(queryResponseTemplate); - String taskId = compactData(compactionResource, newSegmentGranularity, null); - - // The original 4 segments should be compacted into 2 new segments - checkNumberOfSegments(2); - queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 2); - - if (newSegmentGranularity != null) { - List newIntervals = new ArrayList<>(); - for (String interval : expectedIntervalAfterCompaction) { - for (Interval newinterval : newSegmentGranularity.getDefaultGranularity().getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - newIntervals.add(newinterval.toString()); - } - } - expectedIntervalAfterCompaction = newIntervals; - } - checkCompactionIntervals(expectedIntervalAfterCompaction); - - Map reports = indexer.getTaskReport(taskId); - Assert.assertTrue(reports != null && reports.size() > 0); - } - } - - private String compactData( - String compactionResource, - GranularityType newSegmentGranularity, - GranularityType newQueryGranularity - ) throws Exception - { - String template = getResourceAsString(compactionResource); - template = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); - // For the new granularitySpec map - Map granularityMap = new HashMap<>(); - if (newSegmentGranularity != null) { - granularityMap.put("segmentGranularity", newSegmentGranularity.name()); - } - if (newQueryGranularity != null) { - granularityMap.put("queryGranularity", newQueryGranularity.name()); - } - template = StringUtils.replace( - template, - "%%GRANULARITY_SPEC%%", - jsonMapper.writeValueAsString(granularityMap) - ); - // For the deprecated segment granularity field - if (newSegmentGranularity != null) { - template = StringUtils.replace( - template, - "%%SEGMENT_GRANULARITY%%", - newSegmentGranularity.name() - ); - } - final String taskID = indexer.submitTask(template); - LOG.info("TaskID for compaction task %s", taskID); - indexer.waitUntilTaskCompletes(taskID); - - ITRetryUtil.retryUntilTrue( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - "Segment Compaction" - ); - - return taskID; - } - - private void checkQueryGranularity(String queryResource, String expectedQueryGranularity, int segmentCount) throws Exception - { - String queryResponseTemplate; - try { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryResource); - queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", queryResource); - } - - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%ANALYSIS_TYPE%%", - "queryGranularity" - ); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%INTERVALS%%", - "2013-08-31/2013-09-02" - ); - List> expectedResults = new ArrayList<>(); - for (int i = 0; i < segmentCount; i++) { - Map result = new HashMap<>(); - result.put("queryGranularity", expectedQueryGranularity); - expectedResults.add(result); - } - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%EXPECTED_QUERY_GRANULARITY%%", - jsonMapper.writeValueAsString(expectedResults) - ); - queryHelper.testQueriesFromString(queryResponseTemplate); - } - - private void checkNumberOfSegments(int numExpectedSegments) - { - ITRetryUtil.retryUntilTrue( - () -> { - int metadataSegmentCount = coordinator.getSegments(fullDatasourceName).size(); - LOG.info("Current metadata segment count: %d, expected: %d", metadataSegmentCount, numExpectedSegments); - return metadataSegmentCount == numExpectedSegments; - }, - "Segment count check" - ); - } - - private void checkCompactionIntervals(List expectedIntervals) - { - Set expectedIntervalsSet = new HashSet<>(expectedIntervals); - ITRetryUtil.retryUntilTrue( - () -> { - final Set intervalsAfterCompaction = new HashSet<>(coordinator.getSegmentIntervals(fullDatasourceName)); - System.out.println("ACTUAL: " + intervalsAfterCompaction); - System.out.println("EXPECTED: " + expectedIntervalsSet); - return intervalsAfterCompaction.equals(expectedIntervalsSet); - }, - "Compaction interval check" - ); - } - - private String getQueryResponseTemplate(String queryResourcePath) - { - String queryResponseTemplate; - try { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryResourcePath); - queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName - ); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", queryResourcePath); - } - return queryResponseTemplate; - } -} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json deleted file mode 100644 index 6ed715b89eb5..000000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json +++ /dev/null @@ -1,150 +0,0 @@ -[ - { - "description": "timeseries, 1 agg, all", - "query":{ - "queryType" : "timeBoundary", - "dataSource": "%%DATASOURCE%%" - }, - "expectedResults":[ - { - "timestamp" : "2013-08-31T01:00:00.000Z", - "result" : { - "minTime" : "2013-08-31T01:00:00.000Z", - "maxTime" : "2013-09-01T12:00:00.000Z" - } - } - ] - }, - { - "description": "timeseries, datasketch aggs, all", - "query":{ - "queryType" : "timeseries", - "dataSource": "%%DATASOURCE%%", - "granularity":"day", - "intervals":[ - "2013-08-31T00:00/2013-09-01T00:00" - ], - "filter":null, - "aggregations":[ - { - "type": "HLLSketchMerge", - "name": "approxCountHLL", - "fieldName": "HLLSketchBuild", - "lgK": 12, - "tgtHllType": "HLL_4", - "round": true - }, - { - "type":"thetaSketch", - "name":"approxCountTheta", - "fieldName":"thetaSketch", - "size":16384, - "shouldFinalize":true, - "isInputThetaSketch":false, - "errorBoundsStdDev":null - }, - { - "type":"quantilesDoublesSketch", - "name":"quantilesSketch", - "fieldName":"quantilesDoublesSketch", - "k":128 - } - ] - }, - "expectedResults":[ - { - "timestamp" : "2013-08-31T00:00:00.000Z", - "result" : { - "quantilesSketch":5, - "approxCountTheta":5.0, - "approxCountHLL":5 - } - } - ] - }, - { - "description":"having spec on post aggregation", - "query":{ - "queryType":"groupBy", - "dataSource":"%%DATASOURCE%%", - "granularity":"day", - "dimensions":[ - "page" - ], - "filter":{ - "type":"selector", - "dimension":"language", - "value":"zh" - }, - "aggregations":[ - { - "type":"count", - "name":"rows" - }, - { - "type":"longSum", - "fieldName":"added", - "name":"added_count" - } - ], - "postAggregations": [ - { - "type":"arithmetic", - "name":"added_count_times_ten", - "fn":"*", - "fields":[ - {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, - {"type":"constant", "name":"const", "value":10} - ] - } - ], - "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, - "intervals":[ - "2013-08-31T00:00/2013-09-01T00:00" - ] - }, - "expectedResults":[ { - "version" : "v1", - "timestamp" : "2013-08-31T00:00:00.000Z", - "event" : { - "added_count_times_ten" : 9050.0, - "page" : "Crimson Typhoon", - "added_count" : 905, - "rows" : 1 - } - } ] - }, - { - "description": "timeseries, stringFirst/stringLast aggs, all", - "query":{ - "queryType" : "timeseries", - "dataSource": "%%DATASOURCE%%", - "granularity":"day", - "intervals":[ - "2013-08-31T00:00/2013-09-01T00:00" - ], - "filter":null, - "aggregations":[ - { - "type": "stringFirst", - "name": "first_user", - "fieldName": "user" - }, - { - "type":"stringLast", - "name":"last_user", - "fieldName":"user" - } - ] - }, - "expectedResults":[ - { - "timestamp" : "2013-08-31T00:00:00.000Z", - "result" : { - "first_user":"nuclear", - "last_user":"stringer" - } - } - ] - } -] \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json deleted file mode 100644 index 25f9203ac1b8..000000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json +++ /dev/null @@ -1,150 +0,0 @@ -[ - { - "description": "timeseries, 1 agg, all", - "query":{ - "queryType" : "timeBoundary", - "dataSource": "%%DATASOURCE%%" - }, - "expectedResults":[ - { - "timestamp" : "2013-01-01T00:00:00.000Z", - "result" : { - "minTime" : "2013-01-01T00:00:00.000Z", - "maxTime" : "2013-01-01T00:00:00.000Z" - } - } - ] - }, - { - "description": "timeseries, datasketch aggs, all", - "query":{ - "queryType" : "timeseries", - "dataSource": "%%DATASOURCE%%", - "granularity":"day", - "intervals":[ - "2013-01-01T00:00/2014-01-01T00:00" - ], - "filter":null, - "aggregations":[ - { - "type": "HLLSketchMerge", - "name": "approxCountHLL", - "fieldName": "HLLSketchBuild", - "lgK": 12, - "tgtHllType": "HLL_4", - "round": true - }, - { - "type":"thetaSketch", - "name":"approxCountTheta", - "fieldName":"thetaSketch", - "size":16384, - "shouldFinalize":true, - "isInputThetaSketch":false, - "errorBoundsStdDev":null - }, - { - "type":"quantilesDoublesSketch", - "name":"quantilesSketch", - "fieldName":"quantilesDoublesSketch", - "k":128 - } - ] - }, - "expectedResults":[ - { - "timestamp" : "2013-01-01T00:00:00.000Z", - "result" : { - "quantilesSketch":10, - "approxCountTheta":5.0, - "approxCountHLL":5 - } - } - ] - }, - { - "description":"having spec on post aggregation", - "query":{ - "queryType":"groupBy", - "dataSource":"%%DATASOURCE%%", - "granularity":"day", - "dimensions":[ - "page" - ], - "filter":{ - "type":"selector", - "dimension":"language", - "value":"zh" - }, - "aggregations":[ - { - "type":"count", - "name":"rows" - }, - { - "type":"longSum", - "fieldName":"added", - "name":"added_count" - } - ], - "postAggregations": [ - { - "type":"arithmetic", - "name":"added_count_times_ten", - "fn":"*", - "fields":[ - {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, - {"type":"constant", "name":"const", "value":10} - ] - } - ], - "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, - "intervals":[ - "2013-01-01T00:00/2014-01-01T00:00" - ] - }, - "expectedResults":[ { - "version" : "v1", - "timestamp" : "2013-01-01T00:00:00.000Z", - "event" : { - "added_count_times_ten" : 18100.0, - "page" : "Crimson Typhoon", - "added_count" : 1810, - "rows" : 1 - } - } ] - }, - { - "description": "timeseries, stringFirst/stringLast aggs, all", - "query":{ - "queryType" : "timeseries", - "dataSource": "%%DATASOURCE%%", - "granularity":"day", - "intervals":[ - "2013-01-01T00:00/2014-01-01T00:00" - ], - "filter":null, - "aggregations":[ - { - "type": "stringFirst", - "name": "first_user", - "fieldName": "user" - }, - { - "type":"stringLast", - "name":"last_user", - "fieldName":"user" - } - ] - }, - "expectedResults":[ - { - "timestamp" : "2013-01-01T00:00:00.000Z", - "result" : { - "first_user":"masterYi", - "last_user":"speed" - } - } - ] - } -] \ No newline at end of file From 97f4430bb1c22047b862759ec5c3d770d9586014 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 14 Jul 2025 08:27:57 +0530 Subject: [PATCH 29/38] Rename some tests to follow a simpler nomenclature --- ...AutoCompactionTest.java => AutoCompactionTest.java} | 4 ++-- ...UpgradeTest.java => AutoCompactionUpgradeTest.java} | 2 +- ...ColumnTest.java => CompactionSparseColumnTest.java} | 2 +- ...CompactionTaskTest.java => CompactionTaskTest.java} | 2 +- ...CompactionTestBase.java => CompactionTestBase.java} | 2 +- ...placeTest.java => ConcurrentAppendReplaceTest.java} | 2 +- ...arallelTaskTest.java => IndexParallelTaskTest.java} | 2 +- .../{EmbeddedIndexTaskTest.java => IndexTaskTest.java} | 2 +- ...erMetricsTest.java => KafkaClusterMetricsTest.java} | 2 +- ...EmbeddedJdbcLookupTest.java => JdbcLookupTest.java} | 2 +- .../mariadb/EmbeddedMariaDBMetadataStoreTest.java | 6 +++--- .../embedded/minio/EmbeddedMinIOStorageTest.java | 6 +++--- ...=> CentralizedSchemaMetadataQueryDisabledTest.java} | 10 +++++----- ...t.java => CentralizedSchemaPublishFailureTest.java} | 10 +++++----- ...natorClientTest.java => CoordinatorClientTest.java} | 2 +- ...AvailabilityTest.java => HighAvailabilityTest.java} | 2 +- ...OverlordClientTest.java => OverlordClientTest.java} | 2 +- 17 files changed, 30 insertions(+), 30 deletions(-) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedAutoCompactionTest.java => AutoCompactionTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedAutoCompactionUpgradeTest.java => AutoCompactionUpgradeTest.java} (98%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedCompactionSparseColumnTest.java => CompactionSparseColumnTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedCompactionTaskTest.java => CompactionTaskTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/{EmbeddedCompactionTestBase.java => CompactionTestBase.java} (98%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/{EmbeddedConcurrentAppendReplaceTest.java => ConcurrentAppendReplaceTest.java} (98%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/{EmbeddedIndexParallelTaskTest.java => IndexParallelTaskTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/{EmbeddedIndexTaskTest.java => IndexTaskTest.java} (98%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/{EmbeddedKafkaClusterMetricsTest.java => KafkaClusterMetricsTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/{EmbeddedJdbcLookupTest.java => JdbcLookupTest.java} (98%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/{EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java => CentralizedSchemaMetadataQueryDisabledTest.java} (85%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/{EmbeddedCentralizedSchemaPublishFailureTest.java => CentralizedSchemaPublishFailureTest.java} (85%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/{EmbeddedCoordinatorClientTest.java => CoordinatorClientTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/{EmbeddedHighAvailabilityTest.java => HighAvailabilityTest.java} (99%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/{EmbeddedOverlordClientTest.java => OverlordClientTest.java} (99%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index f75ee367a080..77cf7356a2ee 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -106,9 +106,9 @@ /** * Embedded mode of integration-tests originally present in {@code ITAutoCompactionTest}. */ -public class EmbeddedAutoCompactionTest extends EmbeddedCompactionTestBase +public class AutoCompactionTest extends CompactionTestBase { - private static final Logger LOG = new Logger(EmbeddedAutoCompactionTest.class); + private static final Logger LOG = new Logger(AutoCompactionTest.class); private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionUpgradeTest.java similarity index 98% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionUpgradeTest.java index 11703d83c965..22155c6745e6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedAutoCompactionUpgradeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionUpgradeTest.java @@ -41,7 +41,7 @@ import java.nio.charset.StandardCharsets; -public class EmbeddedAutoCompactionUpgradeTest extends EmbeddedClusterTestBase +public class AutoCompactionUpgradeTest extends EmbeddedClusterTestBase { private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java index 98c0837f5dfb..364bcac077ba 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java @@ -46,7 +46,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class EmbeddedCompactionSparseColumnTest extends EmbeddedCompactionTestBase +public class CompactionSparseColumnTest extends CompactionTestBase { private static final Supplier INDEX_TASK = () -> TaskBuilder diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java index b470cd71f664..05cf12afeb9d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java @@ -60,7 +60,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class EmbeddedCompactionTaskTest extends EmbeddedCompactionTestBase +public class CompactionTaskTest extends CompactionTestBase { private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java similarity index 98% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java index e014106b5972..6d500d4d3823 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/EmbeddedCompactionTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java @@ -42,7 +42,7 @@ import java.util.TreeSet; import java.util.stream.Collectors; -public abstract class EmbeddedCompactionTestBase extends EmbeddedClusterTestBase +public abstract class CompactionTestBase extends EmbeddedClusterTestBase { protected final EmbeddedOverlord overlord = new EmbeddedOverlord(); protected final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java similarity index 98% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java index 1dfb9ca9da1e..3a1ff08554b4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java @@ -36,7 +36,7 @@ import java.util.List; -public class EmbeddedConcurrentAppendReplaceTest extends EmbeddedClusterTestBase +public class ConcurrentAppendReplaceTest extends EmbeddedClusterTestBase { private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexParallelTaskTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexParallelTaskTest.java index ea96129319ff..6725c23b923c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexParallelTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexParallelTaskTest.java @@ -54,7 +54,7 @@ /** * Runs tasks of "index_parallel" type using Indexers. */ -public class EmbeddedIndexParallelTaskTest extends EmbeddedClusterTestBase +public class IndexParallelTaskTest extends EmbeddedClusterTestBase { private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java similarity index 98% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java index 4cb90fdacc74..28fb7bdb5015 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedIndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java @@ -50,7 +50,7 @@ /** * Simulation tests for batch {@link IndexTask} using inline datasources. */ -public class EmbeddedIndexTaskTest extends EmbeddedClusterTestBase +public class IndexTaskTest extends EmbeddedClusterTestBase { protected final EmbeddedBroker broker = new EmbeddedBroker(); protected final EmbeddedIndexer indexer = new EmbeddedIndexer().addProperty("druid.worker.capacity", "25"); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 463d17b43e27..c5de6ebd95f3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -68,7 +68,7 @@ * ingest them back into the cluster with a {@code KafkaSupervisor}. */ @SuppressWarnings("resource") -public class EmbeddedKafkaClusterMetricsTest extends EmbeddedClusterTestBase +public class KafkaClusterMetricsTest extends EmbeddedClusterTestBase { private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/EmbeddedJdbcLookupTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/JdbcLookupTest.java similarity index 98% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/EmbeddedJdbcLookupTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/JdbcLookupTest.java index 81b5e7dab1d6..13c6a4c8d449 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/EmbeddedJdbcLookupTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/lookup/JdbcLookupTest.java @@ -36,7 +36,7 @@ /** * Embedded test to verify JDBC lookups. */ -public class EmbeddedJdbcLookupTest extends EmbeddedClusterTestBase +public class JdbcLookupTest extends EmbeddedClusterTestBase { private static final String JDBC_LOOKUP_TABLE = "embedded_lookups"; private static final String BULK_UPDATE_LOOKUP_PAYLOAD diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java index e1a8b0ca160e..61bfc20bbe40 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java @@ -21,12 +21,12 @@ import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedRouter; -import org.apache.druid.testing.embedded.indexing.EmbeddedIndexTaskTest; +import org.apache.druid.testing.embedded.indexing.IndexTaskTest; /** - * Same as {@link EmbeddedIndexTaskTest}, but using a MariaDB metadata store instead of Derby. + * Same as {@link IndexTaskTest}, but using a MariaDB metadata store instead of Derby. */ -public class EmbeddedMariaDBMetadataStoreTest extends EmbeddedIndexTaskTest +public class EmbeddedMariaDBMetadataStoreTest extends IndexTaskTest { @Override public EmbeddedDruidCluster createCluster() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java index 4bc5aa4737a8..acc6fb0a8a50 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java @@ -21,12 +21,12 @@ import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedRouter; -import org.apache.druid.testing.embedded.indexing.EmbeddedIndexTaskTest; +import org.apache.druid.testing.embedded.indexing.IndexTaskTest; /** - * Same as {@link EmbeddedIndexTaskTest}, but using a MinIO metadata store through the S3 extension. + * Same as {@link IndexTaskTest}, but using a MinIO metadata store through the S3 extension. */ -public class EmbeddedMinIOStorageTest extends EmbeddedIndexTaskTest +public class EmbeddedMinIOStorageTest extends IndexTaskTest { @Override public EmbeddedDruidCluster createCluster() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaMetadataQueryDisabledTest.java similarity index 85% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaMetadataQueryDisabledTest.java index d0ace406f108..aebc70b6d7c7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaMetadataQueryDisabledTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaMetadataQueryDisabledTest.java @@ -20,8 +20,8 @@ package org.apache.druid.testing.embedded.schema; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -import org.apache.druid.testing.embedded.compact.EmbeddedCompactionSparseColumnTest; -import org.apache.druid.testing.embedded.compact.EmbeddedCompactionTaskTest; +import org.apache.druid.testing.embedded.compact.CompactionSparseColumnTest; +import org.apache.druid.testing.embedded.compact.CompactionTaskTest; import org.junit.jupiter.api.Nested; /** @@ -29,7 +29,7 @@ * set to true. This is a test-only config used to verify that schema is populated * correctly even when metadata queries fail to fetch schema from Historicals. */ -public class EmbeddedCentralizedSchemaMetadataQueryDisabledTest +public class CentralizedSchemaMetadataQueryDisabledTest { private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluster) { @@ -44,7 +44,7 @@ private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluste } @Nested - public class CompactionSparseColumn extends EmbeddedCompactionSparseColumnTest + public class CompactionSparseColumn extends CompactionSparseColumnTest { @Override protected EmbeddedDruidCluster createCluster() @@ -54,7 +54,7 @@ protected EmbeddedDruidCluster createCluster() } @Nested - public class CompactionTask extends EmbeddedCompactionTaskTest + public class CompactionTask extends CompactionTaskTest { @Override protected EmbeddedDruidCluster createCluster() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaPublishFailureTest.java similarity index 85% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaPublishFailureTest.java index b6dbc8582894..7b489a244180 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/EmbeddedCentralizedSchemaPublishFailureTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/schema/CentralizedSchemaPublishFailureTest.java @@ -20,8 +20,8 @@ package org.apache.druid.testing.embedded.schema; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -import org.apache.druid.testing.embedded.compact.EmbeddedCompactionSparseColumnTest; -import org.apache.druid.testing.embedded.compact.EmbeddedCompactionTaskTest; +import org.apache.druid.testing.embedded.compact.CompactionSparseColumnTest; +import org.apache.druid.testing.embedded.compact.CompactionTaskTest; import org.junit.jupiter.api.Nested; /** @@ -29,7 +29,7 @@ * set to true. This is a test-only config used to verify that schema is populated * correctly even when tasks fail to publish it. */ -public class EmbeddedCentralizedSchemaPublishFailureTest +public class CentralizedSchemaPublishFailureTest { private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluster) { @@ -43,7 +43,7 @@ private static EmbeddedDruidCluster configureCluster(EmbeddedDruidCluster cluste } @Nested - public class CompactionSparseColumn extends EmbeddedCompactionSparseColumnTest + public class CompactionSparseColumn extends CompactionSparseColumnTest { @Override protected EmbeddedDruidCluster createCluster() @@ -53,7 +53,7 @@ protected EmbeddedDruidCluster createCluster() } @Nested - public class CompactionTask extends EmbeddedCompactionTaskTest + public class CompactionTask extends CompactionTaskTest { @Override protected EmbeddedDruidCluster createCluster() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/CoordinatorClientTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/CoordinatorClientTest.java index 699432e75227..90a82b3eaec0 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedCoordinatorClientTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/CoordinatorClientTest.java @@ -51,7 +51,7 @@ import java.util.Map; import java.util.Set; -public class EmbeddedCoordinatorClientTest extends EmbeddedClusterTestBase +public class CoordinatorClientTest extends EmbeddedClusterTestBase { private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); private final EmbeddedBroker broker = new EmbeddedBroker(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java index fc4566953b5c..ba158e365f30 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedHighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java @@ -55,7 +55,7 @@ * Embedded cluster test to verify leadership changes in Coordinator and Overlord. * Makes assertions similar to {@code ITHighAvailabilityTest}. */ -public class EmbeddedHighAvailabilityTest extends EmbeddedClusterTestBase +public class HighAvailabilityTest extends EmbeddedClusterTestBase { private final EmbeddedOverlord overlord1 = new EmbeddedOverlord(); private final EmbeddedOverlord overlord2 = new EmbeddedOverlord(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedOverlordClientTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java similarity index 99% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedOverlordClientTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java index f36ad3a87556..25261553190a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedOverlordClientTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java @@ -66,7 +66,7 @@ * Tests all the REST APIs exposed by the Overlord using the * {@link OverlordClient}. */ -public class EmbeddedOverlordClientTest extends EmbeddedClusterTestBase +public class OverlordClientTest extends EmbeddedClusterTestBase { private static final String UNKNOWN_TASK_ID = IdUtils.newTaskId("sim_test_noop", "dummy", null); From 69a5a2829c28fab5d219a616b3a47c225d465350 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 14 Jul 2025 10:30:51 +0530 Subject: [PATCH 30/38] Fix tests --- .../embedded/indexing/ConcurrentAppendReplaceTest.java | 4 ++-- ...DBMetadataStoreTest.java => MariaDBMetadataStoreTest.java} | 4 ++-- .../{EmbeddedMinIOStorageTest.java => MinIOStorageTest.java} | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/{EmbeddedMariaDBMetadataStoreTest.java => MariaDBMetadataStoreTest.java} (96%) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/{EmbeddedMinIOStorageTest.java => MinIOStorageTest.java} (96%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java index 3a1ff08554b4..560481c8caa2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ConcurrentAppendReplaceTest.java @@ -97,8 +97,8 @@ public void test_concurrentAppend_toIntervalWithUnusedSegment_usesNewSegmentId() Assertions.assertEquals(1, usedSegments.size()); final SegmentId segmentId2 = usedSegments.get(0).getId(); - Assertions.assertEquals("1970-01-01T00:00:00.000Z", segmentId2.getVersion()); - Assertions.assertEquals(1, segmentId2.getPartitionNum()); + Assertions.assertEquals("1970-01-01T00:00:00.000ZS", segmentId2.getVersion()); + Assertions.assertEquals(0, segmentId2.getPartitionNum()); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); Assertions.assertEquals( diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/MariaDBMetadataStoreTest.java similarity index 96% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/MariaDBMetadataStoreTest.java index 61bfc20bbe40..c79f69e8a0e3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/EmbeddedMariaDBMetadataStoreTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/mariadb/MariaDBMetadataStoreTest.java @@ -26,7 +26,7 @@ /** * Same as {@link IndexTaskTest}, but using a MariaDB metadata store instead of Derby. */ -public class EmbeddedMariaDBMetadataStoreTest extends IndexTaskTest +public class MariaDBMetadataStoreTest extends IndexTaskTest { @Override public EmbeddedDruidCluster createCluster() @@ -34,9 +34,9 @@ public EmbeddedDruidCluster createCluster() return EmbeddedDruidCluster.withZookeeper() .useLatchableEmitter() .addResource(new MariaDBMetadataResource()) + .addServer(overlord) .addServer(coordinator) .addServer(indexer) - .addServer(overlord) .addServer(historical) .addServer(broker) .addServer(new EmbeddedRouter()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageTest.java similarity index 96% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageTest.java index acc6fb0a8a50..199ddaec9c74 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/EmbeddedMinIOStorageTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageTest.java @@ -26,7 +26,7 @@ /** * Same as {@link IndexTaskTest}, but using a MinIO metadata store through the S3 extension. */ -public class EmbeddedMinIOStorageTest extends IndexTaskTest +public class MinIOStorageTest extends IndexTaskTest { @Override public EmbeddedDruidCluster createCluster() @@ -34,9 +34,9 @@ public EmbeddedDruidCluster createCluster() return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() .addResource(new MinIOStorageResource()) + .addServer(overlord) .addServer(coordinator) .addServer(indexer) - .addServer(overlord) .addServer(historical) .addServer(broker) .addServer(new EmbeddedRouter()); From 9a3e25ed4fe5303dbcf6a5d59004e984e0a32b17 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 14 Jul 2025 10:32:44 +0530 Subject: [PATCH 31/38] Fix pom --- embedded-tests/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 8deebafdcaa8..b1cd4a1053b3 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -296,11 +296,6 @@ hamcrest-core test - - org.hamcrest - hamcrest-all - test - From 5d960b1620fe513aa167efce4ccf3124f03967a9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 14 Jul 2025 10:33:26 +0530 Subject: [PATCH 32/38] Remove duplicate dependency --- embedded-tests/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index b1cd4a1053b3..bea6a43ca00d 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -291,11 +291,6 @@ aws-java-sdk-core test - - org.hamcrest - hamcrest-core - test - From aaae9e24b6d85c41de27c86db986c51326d808c2 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 23 Jul 2025 20:35:58 +0530 Subject: [PATCH 33/38] Fix compile --- embedded-tests/pom.xml | 6 ---- .../testing/embedded/indexing/Resources.java | 0 .../server/HistoricalCloningTest.java | 22 ++++++------- extensions-core/druid-catalog/pom.xml | 10 +++++- .../compact/CatalogCompactionTest.java | 33 +++++++++++-------- 5 files changed, 39 insertions(+), 32 deletions(-) rename {services => embedded-tests}/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java (100%) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index fb034fc9d6a1..a69976e1f9e9 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -165,12 +165,6 @@ ${project.parent.version} test - - org.apache.druid.extensions - druid-catalog - ${project.parent.version} - test - org.apache.druid diff --git a/services/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java similarity index 100% rename from services/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HistoricalCloningTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HistoricalCloningTest.java index 691a8279d447..db22b015c8c0 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HistoricalCloningTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HistoricalCloningTest.java @@ -20,12 +20,11 @@ package org.apache.druid.testing.embedded.server; import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.query.DruidMetrics; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; @@ -129,20 +128,21 @@ public void test_cloneHistoricals_inTurboMode_duringCoordinatorLeaderSwitch() th private void runIngestion() { final String taskId = IdUtils.getRandomId(); - final Object task = createIndexTaskForInlineData( - taskId, - StringUtils.replace(Resources.CSV_DATA_10_DAYS, "\n", "\\n") - ); + final Object task = createIndexTaskForInlineData(taskId); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); cluster.callApi().waitForTaskToSucceed(taskId, overlord); } - private Object createIndexTaskForInlineData(String taskId, String inlineDataCsv) + private Object createIndexTaskForInlineData(String taskId) { - return EmbeddedClusterApis.createTaskFromPayload( - taskId, - StringUtils.format(Resources.INDEX_TASK_PAYLOAD_WITH_INLINE_DATA, inlineDataCsv, dataSource) - ); + return TaskBuilder.ofTypeIndex() + .dataSource(dataSource) + .isoTimestampColumn("time") + .csvInputFormatWithColumns("time", "item", "value") + .inlineInputSourceWithData(Resources.InlineData.CSV_10_DAYS) + .segmentGranularity("DAY") + .dimensions() + .withId(taskId); } } diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml index 9bd320c24a7f..80d64d7b46e9 100644 --- a/extensions-core/druid-catalog/pom.xml +++ b/extensions-core/druid-catalog/pom.xml @@ -217,7 +217,15 @@ druid-indexing-service ${project.parent.version} test - + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + test + + org.apache.druid druid-services ${project.parent.version} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java index d44f3ad6593f..6c04d2eebbf2 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java @@ -26,6 +26,7 @@ import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.sync.CatalogClient; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.StringUtils; @@ -34,13 +35,11 @@ import org.apache.druid.server.coordinator.CatalogDataSourceCompactionConfig; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedOverlord; -import org.apache.druid.testing.embedded.indexing.Resources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.apache.druid.timeline.DataSegment; import org.joda.time.Period; @@ -51,7 +50,8 @@ public class CatalogCompactionTest extends EmbeddedClusterTestBase { - private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord() + .addProperty("druid.catalog.client.maxSyncRetries", "0"); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() .addProperty("druid.manager.segments.useIncrementalCache", "always"); private final EmbeddedBroker broker = new EmbeddedBroker() @@ -64,8 +64,8 @@ protected EmbeddedDruidCluster createCluster() .useLatchableEmitter() .addExtension(CatalogClientModule.class) .addExtension(CatalogCoordinatorModule.class) - .addServer(coordinator) .addServer(overlord) + .addServer(coordinator) .addServer(broker) .addServer(new EmbeddedIndexer()) .addServer(new EmbeddedHistorical()); @@ -81,7 +81,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() .segmentsMetadataStorage() .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) ); - Assertions.assertEquals(10, segments.size()); + Assertions.assertEquals(3, segments.size()); segments.forEach( segment -> Assertions.assertTrue(Granularities.DAY.isAligned(segment.getInterval())) ); @@ -136,21 +136,26 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() private void runIngestionAtDayGranularity() { final String taskId = IdUtils.getRandomId(); - final Object task = createIndexTaskForInlineData( - taskId, - StringUtils.replace(Resources.CSV_DATA_10_DAYS, "\n", "\\n") - ); + final Object task = createIndexTaskForInlineData(taskId); cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); cluster.callApi().waitForTaskToSucceed(taskId, overlord); } - private Object createIndexTaskForInlineData(String taskId, String inlineDataCsv) + private Object createIndexTaskForInlineData(String taskId) { - return EmbeddedClusterApis.createTaskFromPayload( - taskId, - StringUtils.format(Resources.INDEX_TASK_PAYLOAD_WITH_INLINE_DATA, inlineDataCsv, dataSource) - ); + final String inlineDataCsv = + "2025-06-01T00:00:00.000Z,shirt,105" + + "\n2025-06-02T00:00:00.000Z,trousers,210" + + "\n2025-06-03T00:00:00.000Z,jeans,150"; + return TaskBuilder.ofTypeIndex() + .dataSource(dataSource) + .isoTimestampColumn("time") + .csvInputFormatWithColumns("time", "item", "value") + .inlineInputSourceWithData(inlineDataCsv) + .segmentGranularity("DAY") + .dimensions() + .withId(taskId); } private void enableCompactionSupervisor() From 44a6547ed098072b4191ca02fa183a9c01d8dd46 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Jul 2025 11:20:18 +0530 Subject: [PATCH 34/38] Add MoreResources, move utility methods to EmbeddedClusterApis --- .../embedded/compact/AutoCompactionTest.java | 275 ++++++++---------- .../embedded/compact/CompactionTaskTest.java | 52 ++-- .../embedded/compact/CompactionTestBase.java | 57 +--- .../embedded/indexing/IndexTaskTest.java | 2 +- .../indexing/KafkaClusterMetricsTest.java | 2 +- .../embedded/indexing/MoreResources.java | 70 +++++ .../testing/embedded/EmbeddedClusterApis.java | 118 ++++++++ .../testing/embedded/indexing/Resources.java | 43 --- 8 files changed, 335 insertions(+), 284 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java rename {embedded-tests => services}/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java (70%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 77cf7356a2ee..cda666e39a11 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Ordering; import org.apache.datasketches.hll.TgtHllType; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.TaskPayloadResponse; @@ -47,6 +46,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -75,10 +75,12 @@ import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.indexing.Resources; import org.apache.druid.timeline.DataSegment; import org.hamcrest.Matcher; @@ -109,7 +111,7 @@ public class AutoCompactionTest extends CompactionTestBase { private static final Logger LOG = new Logger(AutoCompactionTest.class); - private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; + private static final Supplier INDEX_TASK = MoreResources.Task.BASIC_INDEX; private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); @@ -233,8 +235,8 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 2 segments across 1 days... verifySegmentsCount(2); verifyScanResult("added", "...||31"); @@ -271,7 +273,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again @@ -289,8 +291,8 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 2 segments across 1 days... verifySegmentsCount(2); @@ -335,7 +337,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again @@ -353,8 +355,8 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 2 segments across 1 days... verifySegmentsCount(2); verifyScanResult("added", "31||31"); @@ -380,7 +382,7 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again @@ -403,8 +405,8 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput updateCompactionTaskSlot(0.1, 2); } try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 2 segments across 1 days... verifySegmentsCount(2); verifyScanResult("added", "31||31"); @@ -428,7 +430,7 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again @@ -446,8 +448,8 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() // added = null, count = 2, sum_added = 62 loadData(INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 2 segments across 1 days... verifySegmentsCount(2); verifyScanResult("ingested_events", "2||2"); @@ -473,7 +475,7 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() verifyScanResult("COUNT(*)", "1"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); // Verify rollup segments does not get compacted again @@ -489,8 +491,6 @@ public void testAutoCompactionPreservesCreateBitmapIndexInDimensionSchema(Compac { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total) verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -526,8 +526,6 @@ public void testAutoCompactionRollsUpMultiValueDimensionsWithoutUnnest(Compactio { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total) verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -563,8 +561,8 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -574,7 +572,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, AutoCompactionSnapshot.ScheduleStatus.RUNNING, @@ -592,7 +590,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, AutoCompactionSnapshot.ScheduleStatus.RUNNING, @@ -614,8 +612,8 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -632,7 +630,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng forceTriggerAutoCompaction(10); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(10, 1); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); if (engine == CompactionEngine.NATIVE) { // HashedPartitionsSpec not supported by MSQ. @@ -645,7 +643,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng forceTriggerAutoCompaction(6); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(hashedPartitionsSpec, 6); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); } LOG.info("Auto compaction test with range partitioning"); @@ -670,7 +668,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(expectedRangePartitionsSpec, 2); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); } } @@ -680,8 +678,8 @@ public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine eng { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -695,7 +693,7 @@ public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine eng verifySegmentsCompacted(0, null); // Auto compaction stats should be deleted as compacation config was deleted Assertions.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); } } @@ -706,8 +704,8 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception updateCompactionTaskSlot(0, 0); loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -717,7 +715,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(0, null); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); Assertions.assertNull(compactionResource.getCompactionStatus(fullDatasourceName)); // Update compaction slots to be 1 updateCompactionTaskSlot(1, 1); @@ -725,7 +723,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, AutoCompactionSnapshot.ScheduleStatus.RUNNING, @@ -747,7 +745,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(intervalsBeforeCompaction); + verifySegmentIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, AutoCompactionSnapshot.ScheduleStatus.RUNNING, @@ -790,8 +788,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -809,16 +807,12 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( engine ); - List expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + List expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); LOG.info("Auto compaction test with MONTH segment granularity, dropExisting is true"); @@ -839,19 +833,17 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( // new MONTH segments for data and tombstones for days with no data // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones - final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = + final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsAfterYEARCompactionButBeforeMONTHCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( + intervalsAfterYEARCompactionButBeforeMONTHCompaction, + newGranularity + ); forceTriggerAutoCompaction(12); verifyQuery(INDEX_QUERIES_RESOURCE); verifyTombstones(10); verifySegmentsCompacted(12, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); LOG.info("Auto compaction test with SEMESTER segment granularity, dropExisting is true, over tombstones"); // only reason is semester and not quarter or month is to minimize time in the test but to @@ -877,10 +869,11 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( verifySegmentsCompacted(2, 1000); expectedIntervalAfterCompaction = - Arrays.asList("2013-01-01T00:00:00.000Z/2013-07-01T00:00:00.000Z", - "2013-07-01T00:00:00.000Z/2014-01-01T00:00:00.000Z" + Arrays.asList( + Intervals.of("2013-01-01T00:00:00.000Z/2013-07-01T00:00:00.000Z"), + Intervals.of("2013-07-01T00:00:00.000Z/2014-01-01T00:00:00.000Z") ); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); // verify that autocompaction completed before List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); @@ -918,8 +911,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -937,16 +930,12 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT engine ); - List expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + List expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); LOG.info("Auto compaction test with MONTH segment granularity, dropExisting is true"); @@ -967,19 +956,17 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT // new MONTH segments for data and tombstones for days with no data // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones - final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = + final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsAfterYEARCompactionButBeforeMONTHCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( + intervalsAfterYEARCompactionButBeforeMONTHCompaction, + newGranularity + ); forceTriggerAutoCompaction(12); verifyQuery(INDEX_QUERIES_RESOURCE); verifyTombstones(10); verifySegmentsCompacted(12, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); // Now compact again over tombstones but with dropExisting set to false: LOG.info("Auto compaction test with SEMESTER segment granularity, dropExisting is false, over tombstones"); @@ -1002,10 +989,10 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT expectedIntervalAfterCompaction = Arrays.asList( - "2013-01-01T00:00:00.000Z/2013-07-01T00:00:00.000Z", - "2013-07-01T00:00:00.000Z/2014-01-01T00:00:00.000Z" + Intervals.of("2013-01-01T00:00:00.000Z/2013-07-01T00:00:00.000Z"), + Intervals.of("2013-07-01T00:00:00.000Z/2014-01-01T00:00:00.000Z") ); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); // verify that autocompaction completed before List compactTasksBefore = getCompleteTasksForDataSource(fullDatasourceName); @@ -1020,8 +1007,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1038,16 +1025,12 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse LOG.info("Auto compaction test with YEAR segment granularity"); - List expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + List expectedIntervalAfterCompaction + = EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); newGranularity = Granularities.DAY; // Set dropExisting to false @@ -1067,15 +1050,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse // The version for the YEAR segment is still the latest for 2013-01-01 to 2013-08-31 and 2013-09-02 to 2014-01-01. // Hence, all three segments are available and the expected intervals are combined from the DAY and YEAR segment granularities // (which are 2013-08-31 to 2013-09-01, 2013-09-01 to 2013-09-02 and 2013-01-01 to 2014-01-01) - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + expectedIntervalAfterCompaction.addAll( + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity) + ); forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(3, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -1085,8 +1066,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(Compacti { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1102,19 +1083,15 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(Compacti LOG.info("Auto compaction test with YEAR segment granularity"); - List expectedIntervalAfterCompaction = new ArrayList<>(); - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + List expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); // Since the new segmentGranularity is YEAR, it will have mixed versions inside the same time chunk // There will be an old version (for the first day interval) from the initial ingestion and // a newer version (for the second day interval) from the first compaction forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, 1000); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -1124,8 +1101,6 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1157,8 +1132,6 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1191,8 +1164,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1201,18 +1174,14 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula // Set dropExisting to true submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); - List expectedIntervalAfterCompaction = new ArrayList<>(); + List expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); // We will still have one visible segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) // and four overshadowed segments - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); loadData(INDEX_TASK); verifySegmentsCount(5); @@ -1222,31 +1191,29 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula // two segments with interval of 2013-08-31/2013-09-01 (newly ingested with DAY) // and two segments with interval of 2013-09-01/2013-09-02 (newly ingested with DAY) expectedIntervalAfterCompaction.addAll(intervalsBeforeCompaction); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); newGranularity = Granularities.MONTH; - final List intervalsAfterYEARButBeforeMONTHCompaction = + final List intervalsAfterYEARButBeforeMONTHCompaction = getSegmentIntervals(fullDatasourceName); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); // verify: - expectedIntervalAfterCompaction = new ArrayList<>(); + expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( + intervalsAfterYEARButBeforeMONTHCompaction, + Granularities.MONTH + ); // The previous segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) will be // completely overshadowed by a combination of tombstones and segments with data. // We will only have one segment with interval of 2013-08-01/2013-09-01 (compacted with MONTH) // and one segment with interval of 2013-09-01/2013-10-01 (compacted with MONTH) // plus ten tombstones for the remaining months, thus expecting 12 intervals... - for (String interval : intervalsAfterYEARButBeforeMONTHCompaction) { - for (Interval newinterval : Granularities.MONTH.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } forceTriggerAutoCompaction(12); verifyQuery(INDEX_QUERIES_RESOURCE); verifyTombstones(10); verifySegmentsCompacted(12, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -1255,8 +1222,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + // 4 segments across 2 days (4 total)... verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -1271,17 +1238,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula CompactionEngine.NATIVE ); - List expectedIntervalAfterCompaction = new ArrayList<>(); + List expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, newGranularity); // We wil have one segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); loadData(INDEX_TASK); verifySegmentsCount(5); @@ -1291,7 +1254,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula // two segments with interval of 2013-08-31/2013-09-01 (newly ingested with DAY) // and two segments with interval of 2013-09-01/2013-09-02 (newly ingested with DAY) expectedIntervalAfterCompaction.addAll(intervalsBeforeCompaction); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); newGranularity = Granularities.MONTH; // Set dropExisting to false @@ -1304,26 +1267,20 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula ); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity - expectedIntervalAfterCompaction = new ArrayList<>(); + expectedIntervalAfterCompaction = + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, Granularities.YEAR); // Since dropExisting is set to false... // We wil have one segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) from before the compaction - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : Granularities.YEAR.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } // one segments with interval of 2013-09-01/2013-10-01 (compacted with MONTH) // and one segments with interval of 2013-10-01/2013-11-01 (compacted with MONTH) - for (String interval : intervalsBeforeCompaction) { - for (Interval newinterval : Granularities.MONTH.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - expectedIntervalAfterCompaction.add(newinterval.toString()); - } - } + expectedIntervalAfterCompaction.addAll( + EmbeddedClusterApis.createAlignedIntervals(intervalsBeforeCompaction, Granularities.MONTH) + ); forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(3, MAX_ROWS_PER_SEGMENT_COMPACTED); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -1484,8 +1441,6 @@ public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) th // "namespace", "continent", "country", "region", "city" loadData(INDEX_TASK_WITH_DIMENSION_SPEC); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1527,8 +1482,8 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(Ordering.natural().reversed()); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + intervalsBeforeCompaction.sort(Comparators.intervalsByStartThenEnd().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1571,8 +1526,8 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(Ordering.natural().reversed()); + final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + intervalsBeforeCompaction.sort(Comparators.intervalsByStartThenEnd().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1867,7 +1822,7 @@ private void deleteCompactionConfig() throws Exception * the datasource after compaction. */ private void forceTriggerAutoCompaction( - List intervals, + List intervals, boolean useSupervisors, int numExpectedSegmentsAfterCompaction ) throws Exception @@ -1876,7 +1831,7 @@ private void forceTriggerAutoCompaction( // Enable compaction for the requested intervals final FixedIntervalOrderPolicy policy = new FixedIntervalOrderPolicy( intervals.stream().map( - interval -> new FixedIntervalOrderPolicy.Candidate(fullDatasourceName, Intervals.of(interval)) + interval -> new FixedIntervalOrderPolicy.Candidate(fullDatasourceName, interval) ).collect(Collectors.toList()) ); updateClusterConfig( @@ -1884,7 +1839,7 @@ private void forceTriggerAutoCompaction( ); // Wait for scheduler to pick up the compaction job - // TODO: make this latch-based + // Instead of sleep, we can latch on a relevant metric later Thread.sleep(30_000); waitForCompactionToFinish(numExpectedSegmentsAfterCompaction); @@ -1924,7 +1879,7 @@ private void verifySegmentsCompacted(int expectedCompactedSegmentCount, Integer private void verifyTombstones(int expectedCompactedTombstoneCount) { - Set segments = getFullSegmentsMetadata(dataSource); + Set segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); int actualTombstoneCount = 0; for (DataSegment segment : segments) { if (segment.isTombstone()) { @@ -1936,7 +1891,7 @@ private void verifyTombstones(int expectedCompactedTombstoneCount) private void verifySegmentsCompacted(PartitionsSpec partitionsSpec, int expectedCompactedSegmentCount) { - Set segments = getFullSegmentsMetadata(dataSource); + Set segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); List foundCompactedSegments = new ArrayList<>(); for (DataSegment segment : segments) { if (segment.getLastCompactionState() != null) { @@ -1954,7 +1909,7 @@ private void verifySegmentsCompacted(PartitionsSpec partitionsSpec, int expected private void verifySegmentsCompactedDimensionSchema(List dimensionSchemas) { - Set segments = getFullSegmentsMetadata(dataSource); + Set segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); List foundCompactedSegments = new ArrayList<>(); for (DataSegment segment : segments) { if (segment.getLastCompactionState() != null) { @@ -2035,9 +1990,7 @@ private List getCompleteTasksForDataSource(String dataSource) private TaskPayloadResponse getTaskPayload(String taskId) { - return cluster.callApi().onLeaderOverlord( - o -> o.taskPayload(taskId) - ); + return cluster.callApi().onLeaderOverlord(o -> o.taskPayload(taskId)); } private Set getTaskIdsForState(String state, String dataSource) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java index 05cf12afeb9d..73e0e967b9cb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java @@ -41,13 +41,14 @@ import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.indexing.Resources; import org.joda.time.Interval; -import org.joda.time.chrono.ISOChronology; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -62,7 +63,7 @@ public class CompactionTaskTest extends CompactionTestBase { - private static final Supplier INDEX_TASK = Resources.Task.BASIC_INDEX; + private static final Supplier INDEX_TASK = MoreResources.Task.BASIC_INDEX; private static final List> INDEX_QUERIES_RESOURCE = List.of( Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z"), @@ -103,7 +104,7 @@ public class CompactionTaskTest extends CompactionTestBase .ioConfig(new CompactionIntervalSpec(Intervals.of("2013-08-31/2013-09-02"), null), true); private static final Supplier INDEX_TASK_WITH_TIMESTAMP = - () -> Resources.Task.BASIC_INDEX.get().dimensions( + () -> MoreResources.Task.BASIC_INDEX.get().dimensions( "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", "namespace", "continent", "country", "region", "city", "timestamp" @@ -160,8 +161,7 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); runQueries(INDEX_QUERIES_RESOURCE); @@ -174,7 +174,7 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti checkNumberOfSegments(2); runQueries(INDEX_QUERIES_HOUR_RESOURCE); verifySegmentsHaveQueryGranularity("HOUR", 2); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); // QueryGranularity was HOUR, now we will change it to MINUTE (QueryGranularity changed to finer) compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), null, Granularities.MINUTE); @@ -186,7 +186,7 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti checkNumberOfSegments(2); runQueries(INDEX_QUERIES_HOUR_RESOURCE); verifySegmentsHaveQueryGranularity("MINUTE", 2); - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -197,8 +197,7 @@ public void testParallelHashedCompaction() throws Exception runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); runQueries(INDEX_QUERIES_RESOURCE); @@ -209,8 +208,7 @@ public void testParallelHashedCompaction() throws Exception runQueries(INDEX_QUERIES_RESOURCE); verifySegmentsHaveQueryGranularity("SECOND", 2); - - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); Map reports = cluster.callApi().onLeaderOverlord(o -> o.taskReportAsMap(taskId)); Assertions.assertTrue(reports != null && !reports.isEmpty()); @@ -241,8 +239,7 @@ public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularity runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); runQueries(INDEX_QUERIES_RESOURCE); @@ -253,14 +250,11 @@ public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularity runQueries(INDEX_QUERIES_YEAR_RESOURCE); verifySegmentsHaveQueryGranularity("YEAR", 1); - List newIntervals = new ArrayList<>(); - for (String interval : expectedIntervalAfterCompaction) { - for (Interval newinterval : Granularities.YEAR.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - newIntervals.add(newinterval.toString()); - } - } - expectedIntervalAfterCompaction = newIntervals; - checkCompactionIntervals(expectedIntervalAfterCompaction); + expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( + expectedIntervalAfterCompaction, + Granularities.YEAR + ); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } @@ -280,8 +274,7 @@ private void loadDataAndCompact( runTask(indexTask, fullDatasourceName); // 4 segments across 2 days checkNumberOfSegments(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); + List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); runQueries(INDEX_QUERIES_RESOURCE); @@ -294,15 +287,12 @@ private void loadDataAndCompact( verifySegmentsHaveQueryGranularity("SECOND", 2); if (newSegmentGranularity != null) { - List newIntervals = new ArrayList<>(); - for (String interval : expectedIntervalAfterCompaction) { - for (Interval newinterval : newSegmentGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { - newIntervals.add(newinterval.toString()); - } - } - expectedIntervalAfterCompaction = newIntervals; + expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( + expectedIntervalAfterCompaction, + newSegmentGranularity + ); } - checkCompactionIntervals(expectedIntervalAfterCompaction); + verifySegmentIntervals(expectedIntervalAfterCompaction); } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java index 6d500d4d3823..13c101b83373 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java @@ -20,10 +20,7 @@ package org.apache.druid.testing.embedded.compact; import org.apache.druid.indexing.common.task.TaskBuilder; -import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.EmbeddedHistorical; @@ -31,16 +28,12 @@ import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; -import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.jupiter.api.Assertions; import java.io.Closeable; -import java.util.Comparator; import java.util.List; import java.util.Set; -import java.util.TreeSet; -import java.util.stream.Collectors; public abstract class CompactionTestBase extends EmbeddedClusterTestBase { @@ -66,9 +59,7 @@ protected EmbeddedDruidCluster createCluster() */ protected Closeable unloader(String dataSource) { - return () -> { - overlord.bindings().segmentsMetadataStorage().markAllSegmentsAsUnused(dataSource); - }; + return cluster.callApi().createUnloader(dataSource); } /** @@ -78,17 +69,15 @@ protected Closeable unloader(String dataSource) */ protected String runTask(TaskBuilder taskBuilder, String dataSource) { - final String taskId = EmbeddedClusterApis.newTaskId(dataSource); - cluster.callApi().onLeaderOverlord( - o -> o.runTask(taskId, taskBuilder.dataSource(dataSource).withId(taskId)) + return cluster.callApi().runTask( + (ds, taskId) -> taskBuilder.dataSource(ds).withId(taskId), + dataSource, + overlord, + coordinator ); - cluster.callApi().waitForTaskToSucceed(taskId, overlord); - cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator); - - return taskId; } - protected void checkCompactionIntervals(List expectedIntervals) + protected void verifySegmentIntervals(List expectedIntervals) { Assertions.assertEquals( Set.copyOf(expectedIntervals), @@ -96,39 +85,13 @@ protected void checkCompactionIntervals(List expectedIntervals) ); } - protected Set getFullSegmentsMetadata(String dataSource) - { - return overlord - .bindings() - .segmentsMetadataStorage() - .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE); - } - - protected List getSegmentIntervals(String dataSource) + protected List getSegmentIntervals(String dataSource) { - final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); - final Set sortedIntervals = new TreeSet<>(comparator); - - final Set allUsedSegments = getFullSegmentsMetadata(dataSource); - for (DataSegment segment : allUsedSegments) { - sortedIntervals.add(segment.getInterval()); - } - - return sortedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + return cluster.callApi().getSortedSegmentIntervals(dataSource, overlord); } protected void verifySegmentsCount(int numExpectedSegments) { - int segmentCount = getFullSegmentsMetadata(dataSource).size(); - Assertions.assertEquals(numExpectedSegments, segmentCount, "Segment count mismatch"); - Assertions.assertEquals( - String.valueOf(segmentCount), - cluster.runSql( - "SELECT COUNT(*) FROM sys.segments" - + " WHERE datasource='%s' AND is_overshadowed = 0 AND is_available = 1", - dataSource - ), - "Segment count mismatch in sys table" - ); + cluster.callApi().verifyNumVisibleSegmentsIs(numExpectedSegments, dataSource, overlord); } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java index 28fb7bdb5015..e5578e83e5a2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java @@ -48,7 +48,7 @@ import java.util.stream.IntStream; /** - * Simulation tests for batch {@link IndexTask} using inline datasources. + * Embedded tests for batch {@link IndexTask} using inline datasources. */ public class IndexTaskTest extends EmbeddedClusterTestBase { diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index b5e88cdaa21f..8717fcf223f4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -64,7 +64,7 @@ import java.util.stream.Collectors; /** - * Simulation test to emit cluster metrics using a {@link KafkaEmitter} and then + * Embedded test to emit cluster metrics using a {@link KafkaEmitter} and then * ingest them back into the cluster with a {@code KafkaSupervisor}. */ @SuppressWarnings("resource") diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java new file mode 100644 index 000000000000..539ec5d0680b --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java @@ -0,0 +1,70 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; + +import java.util.function.Supplier; + +/** + * Additional utility methods used in embedded tests that are not added to + * {@code Resources} to avoid cyclical dependencies. + */ +public class MoreResources +{ + /** + * Task payload builders. + */ + public static class Task + { + public static final Supplier BASIC_INDEX = + () -> TaskBuilder + .ofTypeIndex() + .jsonInputFormat() + .localInputSourceWithFiles( + Resources.DataFile.TINY_WIKI_1_JSON, + Resources.DataFile.TINY_WIKI_2_JSON, + Resources.DataFile.TINY_WIKI_3_JSON + ) + .timestampColumn("timestamp") + .dimensions( + "page", + "language", "tags", "user", "unpatrolled", "newPage", "robot", + "anonymous", "namespace", "continent", "country", "region", "city" + ) + .metricAggregates( + new CountAggregatorFactory("ingested_events"), + new DoubleSumAggregatorFactory("added", "added"), + new DoubleSumAggregatorFactory("deleted", "deleted"), + new DoubleSumAggregatorFactory("delta", "delta"), + new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), + new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), + new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null) + ) + .dynamicPartitionWithMaxRows(3) + .granularitySpec("DAY", "SECOND", true) + .appendToExisting(false); + } +} diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index c458479d794f..422873a1b175 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -32,16 +32,26 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; import org.junit.jupiter.api.Assertions; +import java.io.Closeable; +import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.function.Function; /** @@ -103,6 +113,26 @@ public String runSql(String sql, Object... args) } } + /** + * Creates a Task using the given builder and runs it. + * + * @return ID of the task. + */ + public String runTask( + TaskBuilder taskBuilder, + String dataSource, + EmbeddedOverlord overlord, + EmbeddedCoordinator coordinator + ) + { + final String taskId = EmbeddedClusterApis.newTaskId(dataSource); + onLeaderOverlord(o -> o.runTask(taskId, taskBuilder.build(dataSource, taskId))); + waitForTaskToSucceed(taskId, overlord); + waitForAllSegmentsToBeAvailable(dataSource, coordinator); + + return taskId; + } + /** * Waits for the given task to finish successfully. If the given * {@link EmbeddedOverlord} is not the leader, this method can only return by @@ -127,6 +157,56 @@ public void waitForTaskToFinish(String taskId, EmbeddedOverlord overlord) ); } + /** + * Retrieves all used segments from the metadata store (or cache if applicable). + */ + public Set getVisibleUsedSegments(String dataSource, EmbeddedOverlord overlord) + { + return overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE); + } + + /** + * Returns intervals of all visible used segments sorted using the + * {@link Comparators#intervalsByStartThenEnd()}. + */ + public List getSortedSegmentIntervals(String dataSource, EmbeddedOverlord overlord) + { + final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); + final TreeSet sortedIntervals = new TreeSet<>(comparator); + + final Set allUsedSegments = getVisibleUsedSegments(dataSource, overlord); + for (DataSegment segment : allUsedSegments) { + sortedIntervals.add(segment.getInterval()); + } + + return new ArrayList<>(sortedIntervals); + } + + /** + * Verifies that the number of visible used segments is the same as expected. + */ + public void verifyNumVisibleSegmentsIs(int numExpectedSegments, String dataSource, EmbeddedOverlord overlord) + { + int segmentCount = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).size(); + Assertions.assertEquals( + numExpectedSegments, + segmentCount, + "Segment count mismatch" + ); + Assertions.assertEquals( + String.valueOf(segmentCount), + cluster.runSql( + "SELECT COUNT(*) FROM sys.segments WHERE datasource='%s'" + + " AND is_overshadowed = 0 AND is_available = 1", + dataSource + ), + "Segment count mismatch in sys.segments table" + ); + } + /** * Waits for all used segments (including overshadowed) of the given datasource * to be loaded on historicals. @@ -145,6 +225,15 @@ public void waitForAllSegmentsToBeAvailable(String dataSource, EmbeddedCoordinat ); } + /** + * Returns a {@link Closeable} that deletes all the data for the given datasource + * on {@link Closeable#close()}. + */ + public Closeable createUnloader(String dataSource) + { + return () -> onLeaderOverlord(o -> o.markSegmentsAsUnused(dataSource)); + } + /** * Fetches the status of the given task from the cluster and verifies that it * matches the expected status. @@ -184,6 +273,8 @@ public SupervisorStatus getSupervisorStatus(String supervisorId) throw new ISE("Could not find supervisor[%s]", supervisorId); } + // STATIC UTILITY METHODS + /** * Creates a random datasource name prefixed with {@link TestDataSource#WIKI}. */ @@ -215,8 +306,35 @@ public static Map deserializeJsonToMap(String payload) } } + /** + * Creates a list of intervals that align with the given target granularity + * and overlap the original list of given intervals. If the original list is + * sorted, the returned list would be sorted too. + */ + public static List createAlignedIntervals( + List original, + Granularity targetGranularity + ) + { + final List alignedIntervals = new ArrayList<>(); + for (Interval interval : original) { + for (Interval alignedInterval : + targetGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { + alignedIntervals.add(alignedInterval); + } + } + + return alignedIntervals; + } + private static T getResult(ListenableFuture future) { return FutureUtils.getUnchecked(future, true); } + + @FunctionalInterface + public interface TaskBuilder + { + Object build(String dataSource, String taskId); + } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java b/services/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java similarity index 70% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java rename to services/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java index 48ddd6534145..7af3d3a7c7a2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/indexing/Resources.java @@ -19,15 +19,6 @@ package org.apache.druid.testing.embedded.indexing; -import org.apache.druid.indexing.common.task.TaskBuilder; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; - -import java.util.function.Supplier; - /** * Constants and utility methods used in embedded cluster tests. */ @@ -103,38 +94,4 @@ public static class Query + " GROUP BY 1" + " HAVING added_times_ten > 9000"; } - - /** - * Task payload builders. - */ - public static class Task - { - public static final Supplier BASIC_INDEX = - () -> TaskBuilder - .ofTypeIndex() - .jsonInputFormat() - .localInputSourceWithFiles( - DataFile.TINY_WIKI_1_JSON, - DataFile.TINY_WIKI_2_JSON, - DataFile.TINY_WIKI_3_JSON - ) - .timestampColumn("timestamp") - .dimensions( - "page", - "language", "tags", "user", "unpatrolled", "newPage", "robot", - "anonymous", "namespace", "continent", "country", "region", "city" - ) - .metricAggregates( - new CountAggregatorFactory("ingested_events"), - new DoubleSumAggregatorFactory("added", "added"), - new DoubleSumAggregatorFactory("deleted", "deleted"), - new DoubleSumAggregatorFactory("delta", "delta"), - new SketchMergeAggregatorFactory("thetaSketch", "user", null, null, null, null), - new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", null, null, null, null, true), - new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", null) - ) - .dynamicPartitionWithMaxRows(3) - .granularitySpec("DAY", "SECOND", true) - .appendToExisting(false); - } } From a95cedd2cebd790ce8cb6cadf193115c603a1c95 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Jul 2025 11:47:53 +0530 Subject: [PATCH 35/38] Move more methods --- .../embedded/compact/AutoCompactionTest.java | 26 +++------ .../embedded/compact/CompactionTaskTest.java | 56 ++++++------------- .../embedded/compact/CompactionTestBase.java | 6 ++ .../testing/embedded/EmbeddedClusterApis.java | 34 +++++++++++ 4 files changed, 67 insertions(+), 55 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index cda666e39a11..15d5f09a0f38 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -118,7 +118,6 @@ public class AutoCompactionTest extends CompactionTestBase private static final Supplier INDEX_TASK_WITH_DIMENSION_SPEC = () -> INDEX_TASK.get().granularitySpec("DAY", "DAY", true); - private static final String SELECT_APPROX_COUNT_DISTINCT = Resources.Query.SELECT_APPROX_COUNT_DISTINCT; private static final List> INDEX_QUERIES_RESOURCE = List.of( Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z"), Pair.of(Resources.Query.SELECT_APPROX_COUNT_DISTINCT, "5,5"), @@ -243,7 +242,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyScanResult("ingested_events", "2||..."); verifyScanResult("sum_added", "62||..."); verifyScanResult("COUNT(*)", "2"); - verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "2,2"); + verifyDistinctCount("2,2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -270,7 +269,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyScanResult("ingested_events", "3"); verifyScanResult("sum_added", "93.0"); verifyScanResult("COUNT(*)", "1"); - verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); + verifyDistinctCount("3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); verifySegmentIntervals(intervalsBeforeCompaction); @@ -300,7 +299,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyScanResult("added", "...||31"); verifyScanResult("ingested_events", "2||..."); verifyScanResult("sum_added", "62||..."); - verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "2,2"); + verifyDistinctCount("2,2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -334,7 +333,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifyScanResult("ingested_events", "3"); verifyScanResult("sum_added", "93"); verifyScanResult("COUNT(*)", "1"); - verifyQuery(SELECT_APPROX_COUNT_DISTINCT, "3,3"); + verifyDistinctCount("3,3"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); verifySegmentIntervals(intervalsBeforeCompaction); @@ -1634,19 +1633,12 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception runTask(taskBuilder, fullDatasourceName); } - private void verifyQuery(List> queries) + private void verifyDistinctCount(String result) { - queries.forEach( - query -> verifyQuery(query.lhs, query.rhs) - ); - } - - private void verifyQuery(String query, String result) - { - Assertions.assertEquals( - result, - cluster.runSql(query, dataSource), - StringUtils.format("Query[%s] failed", query) + cluster.callApi().verifySqlQuery( + Resources.Query.SELECT_APPROX_COUNT_DISTINCT, + dataSource, + result ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java index 73e0e967b9cb..1c451fc7434c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java @@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -160,19 +159,19 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti try (final Closeable ignored = unloader(fullDatasourceName)) { runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days - checkNumberOfSegments(4); + verifySegmentsCount(4); List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); - runQueries(INDEX_QUERIES_RESOURCE); + verifyQuery(INDEX_QUERIES_RESOURCE); // QueryGranularity was SECOND, now we will change it to HOUR (QueryGranularity changed to coarser) compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), null, Granularities.HOUR); // The original 4 segments should be compacted into 2 new segments since data only has 2 days and the compaction // segmentGranularity is DAY - checkNumberOfSegments(2); - runQueries(INDEX_QUERIES_HOUR_RESOURCE); + verifySegmentsCount(2); + verifyQuery(INDEX_QUERIES_HOUR_RESOURCE); verifySegmentsHaveQueryGranularity("HOUR", 2); verifySegmentIntervals(expectedIntervalAfterCompaction); @@ -183,8 +182,8 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti // is the same. Since QueryGranularity is changed to finer qranularity, the data will remains the same. (data // will just be bucketed to a finer qranularity but roll up will not be different // i.e. 2020-10-29T05:00 will just be bucketed to 2020-10-29T05:00:00) - checkNumberOfSegments(2); - runQueries(INDEX_QUERIES_HOUR_RESOURCE); + verifySegmentsCount(2); + verifyQuery(INDEX_QUERIES_HOUR_RESOURCE); verifySegmentsHaveQueryGranularity("MINUTE", 2); verifySegmentIntervals(expectedIntervalAfterCompaction); } @@ -196,16 +195,16 @@ public void testParallelHashedCompaction() throws Exception try (final Closeable ignored = unloader(fullDatasourceName)) { runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days - checkNumberOfSegments(4); + verifySegmentsCount(4); List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); - runQueries(INDEX_QUERIES_RESOURCE); + verifyQuery(INDEX_QUERIES_RESOURCE); String taskId = compactData(PARALLEL_COMPACTION_TASK.get(), null, null); // The original 4 segments should be compacted into 2 new segments - checkNumberOfSegments(2); - runQueries(INDEX_QUERIES_RESOURCE); + verifySegmentsCount(2); + verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsHaveQueryGranularity("SECOND", 2); verifySegmentIntervals(expectedIntervalAfterCompaction); @@ -238,16 +237,16 @@ public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularity try (final Closeable ignored = unloader(fullDatasourceName)) { runTask(INDEX_TASK.get(), fullDatasourceName); // 4 segments across 2 days - checkNumberOfSegments(4); + verifySegmentsCount(4); List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); - runQueries(INDEX_QUERIES_RESOURCE); + verifyQuery(INDEX_QUERIES_RESOURCE); compactData(COMPACTION_TASK_ALLOW_NON_ALIGNED.get(), Granularities.YEAR, Granularities.YEAR); // The original 4 segments should be compacted into 1 new segment - checkNumberOfSegments(1); - runQueries(INDEX_QUERIES_YEAR_RESOURCE); + verifySegmentsCount(1); + verifyQuery(INDEX_QUERIES_YEAR_RESOURCE); verifySegmentsHaveQueryGranularity("YEAR", 1); expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( @@ -273,17 +272,17 @@ private void loadDataAndCompact( try (final Closeable ignored = unloader(fullDatasourceName)) { runTask(indexTask, fullDatasourceName); // 4 segments across 2 days - checkNumberOfSegments(4); + verifySegmentsCount(4); List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); verifySegmentsHaveQueryGranularity("SECOND", 4); - runQueries(INDEX_QUERIES_RESOURCE); + verifyQuery(INDEX_QUERIES_RESOURCE); compactData(compactionResource, newSegmentGranularity, null); // The original 4 segments should be compacted into 2 new segments - checkNumberOfSegments(2); - runQueries(INDEX_QUERIES_RESOURCE); + verifySegmentsCount(2); + verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsHaveQueryGranularity("SECOND", 2); if (newSegmentGranularity != null) { @@ -347,23 +346,4 @@ private void verifySegmentsHaveQueryGranularity(String expectedQueryGranularity, Assertions.assertEquals(expectedResults, trimmedResult); } - - private void checkNumberOfSegments(int numExpectedSegments) - { - verifySegmentsCount(numExpectedSegments); - } - - private void runQueries(List> queries) - { - if (queries == null) { - return; - } - for (Pair query : queries) { - Assertions.assertEquals( - query.rhs, - cluster.runSql(query.lhs, dataSource), - StringUtils.format("Query[%s] failed", query.lhs) - ); - } - } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java index 13c101b83373..561885decc2d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java @@ -20,6 +20,7 @@ package org.apache.druid.testing.embedded.compact; import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.embedded.EmbeddedBroker; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; @@ -94,4 +95,9 @@ protected void verifySegmentsCount(int numExpectedSegments) { cluster.callApi().verifyNumVisibleSegmentsIs(numExpectedSegments, dataSource, overlord); } + + protected void verifyQuery(List> queries) + { + cluster.callApi().verifySqlQueries(queries, dataSource); + } } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 422873a1b175..aaa2dc87e380 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; @@ -113,6 +114,39 @@ public String runSql(String sql, Object... args) } } + /** + * Runs the given SQL queries and verifies the results. + * + * @param queryResultPairs List of SQL query and CSV result pairs. Each query + * must contain a {@code %s} placeholder for the datasource. + * @param dataSource Datasource for which the queries should be run + */ + public void verifySqlQueries(List> queryResultPairs, String dataSource) + { + if (queryResultPairs == null) { + return; + } + queryResultPairs.forEach( + pair -> verifySqlQuery(pair.lhs, dataSource, pair.rhs) + ); + } + + /** + * Runs the given SQL query for a datasource and verifies the result. + * + * @param query Must contain a {@code %s} placeholder for the datasource. + * @param dataSource Datasource for which the query should be run. + * @param expectedResultCsv Expected result as a CSV String. + */ + public void verifySqlQuery(String query, String dataSource, String expectedResultCsv) + { + Assertions.assertEquals( + expectedResultCsv, + cluster.runSql(query, dataSource), + StringUtils.format("Query[%s] failed", query) + ); + } + /** * Creates a Task using the given builder and runs it. * From 5c22b78661cb8b66f57de74f53c0cacbad47023b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Jul 2025 11:52:13 +0530 Subject: [PATCH 36/38] Clean up --- .../embedded/compact/CompactionTestBase.java | 7 ++++++- .../testing/embedded/EmbeddedClusterApis.java | 18 ------------------ 2 files changed, 6 insertions(+), 19 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java index 561885decc2d..9e524eabe92d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java @@ -98,6 +98,11 @@ protected void verifySegmentsCount(int numExpectedSegments) protected void verifyQuery(List> queries) { - cluster.callApi().verifySqlQueries(queries, dataSource); + if (queries == null) { + return; + } + queries.forEach( + pair -> cluster.callApi().verifySqlQuery(pair.lhs, dataSource, pair.rhs) + ); } } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index aaa2dc87e380..f8bb3d66cb8d 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -31,7 +31,6 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; @@ -114,23 +113,6 @@ public String runSql(String sql, Object... args) } } - /** - * Runs the given SQL queries and verifies the results. - * - * @param queryResultPairs List of SQL query and CSV result pairs. Each query - * must contain a {@code %s} placeholder for the datasource. - * @param dataSource Datasource for which the queries should be run - */ - public void verifySqlQueries(List> queryResultPairs, String dataSource) - { - if (queryResultPairs == null) { - return; - } - queryResultPairs.forEach( - pair -> verifySqlQuery(pair.lhs, dataSource, pair.rhs) - ); - } - /** * Runs the given SQL query for a datasource and verifies the result. * From ee9886a688977e057098f11b13376a1b07c55626 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Jul 2025 12:08:40 +0530 Subject: [PATCH 37/38] More clean up --- .../embedded/compact/AutoCompactionTest.java | 42 +++++++++---------- .../compact/CompactionResourceTestClient.java | 26 ++---------- .../compact/CompactionSparseColumnTest.java | 8 ++-- .../embedded/compact/CompactionTaskTest.java | 18 ++++---- .../embedded/compact/CompactionTestBase.java | 13 ++---- 5 files changed, 42 insertions(+), 65 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 15d5f09a0f38..b2149b7bfde3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -234,7 +234,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 2 segments across 1 days... verifySegmentsCount(2); @@ -290,7 +290,7 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 2 segments across 1 days... verifySegmentsCount(2); @@ -354,7 +354,7 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 2 segments across 1 days... verifySegmentsCount(2); @@ -404,7 +404,7 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput updateCompactionTaskSlot(0.1, 2); } try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 2 segments across 1 days... verifySegmentsCount(2); @@ -447,7 +447,7 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() // added = null, count = 2, sum_added = 62 loadData(INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 2 segments across 1 days... verifySegmentsCount(2); @@ -560,7 +560,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -611,7 +611,7 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine eng { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -677,7 +677,7 @@ public void testAutoCompactionDutyCanDeleteCompactionConfig(CompactionEngine eng { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -703,7 +703,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception updateCompactionTaskSlot(0, 0); loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -787,7 +787,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -833,7 +833,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = - getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(); expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( intervalsAfterYEARCompactionButBeforeMONTHCompaction, newGranularity @@ -910,7 +910,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -956,7 +956,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT // Hence, we will only have 2013-08 to 2013-09 months with data // plus 12 tombstones final List intervalsAfterYEARCompactionButBeforeMONTHCompaction = - getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(); expectedIntervalAfterCompaction = EmbeddedClusterApis.createAlignedIntervals( intervalsAfterYEARCompactionButBeforeMONTHCompaction, newGranularity @@ -1006,7 +1006,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1065,7 +1065,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(Compacti { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1163,7 +1163,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1194,7 +1194,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula newGranularity = Granularities.MONTH; final List intervalsAfterYEARButBeforeMONTHCompaction = - getSegmentIntervals(fullDatasourceName); + getSegmentIntervals(); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); @@ -1221,7 +1221,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1481,7 +1481,7 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); intervalsBeforeCompaction.sort(Comparators.intervalsByStartThenEnd().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1525,7 +1525,7 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { - final List intervalsBeforeCompaction = getSegmentIntervals(fullDatasourceName); + final List intervalsBeforeCompaction = getSegmentIntervals(); intervalsBeforeCompaction.sort(Comparators.intervalsByStartThenEnd().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1630,7 +1630,7 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception taskBuilder.granularitySpec(granularitySpec); } - runTask(taskBuilder, fullDatasourceName); + runTask(taskBuilder); } private void verifyDistinctCount(String result) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java index cdbb395721ca..aafcf14a2bb9 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionResourceTestClient.java @@ -38,7 +38,6 @@ import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -48,6 +47,10 @@ import java.util.List; import java.util.Map; +/** + * The methods in this class should eventually be updated to use + * {@code CoordinatorClient} or {@code OverlordClient} as applicable. + */ public class CompactionResourceTestClient { private static final Logger log = new Logger(CompactionResourceTestClient.class); @@ -139,27 +142,6 @@ public void deleteDataSourceCompactionConfig(final String dataSource) throws Exc } } - /** - * For all purposes, use the new APIs {@link #getClusterConfig()} or - * {@link #getAllCompactionConfigs()}. - */ - @Deprecated - public DruidCompactionConfig getCoordinatorCompactionConfig() throws Exception - { - String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); - StatusResponseHolder response = httpClient().go( - new Request(HttpMethod.GET, new URL(url)), responseHandler - ).get(); - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while getting compaction config status[%s] content[%s]", - response.getStatus(), - response.getContent() - ); - } - return jsonMapper.readValue(response.getContent(), new TypeReference<>() {}); - } - public List getAllCompactionConfigs() throws Exception { String url = StringUtils.format("%s/compaction/config/datasources", getOverlordURL()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java index 364bcac077ba..f819095b6496 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSparseColumnTest.java @@ -92,7 +92,7 @@ public void testCompactionPerfectRollUpWithoutDimensionSpec() throws Exception // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get(), dataSource); + runTask(COMPACTION_TASK.get()); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -119,7 +119,7 @@ public void testCompactionPerfectRollUpWithLexicographicDimensionSpec() throws E // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dimensions("dimA", "dimB", "dimC"), dataSource); + runTask(COMPACTION_TASK.get().dimensions("dimA", "dimB", "dimC")); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -145,7 +145,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw // Load and verify initial data loadAndVerifyDataWithSparseColumn(); // Compaction with perfect roll up. Rolls with "X", "H" (for the first and second columns respectively) should be roll up - runTask(COMPACTION_TASK.get().dimensions("dimC", "dimB", "dimA"), dataSource); + runTask(COMPACTION_TASK.get().dimensions("dimC", "dimB", "dimA")); // Verify compacted data. // Compacted data only have one segments. First segment have the following rows: @@ -166,7 +166,7 @@ public void testCompactionPerfectRollUpWithNonLexicographicDimensionSpec() throw private void loadAndVerifyDataWithSparseColumn() { - runTask(INDEX_TASK.get(), dataSource); + runTask(INDEX_TASK.get()); List>>> expectedResultBeforeCompaction = new ArrayList<>(); // First segments have the following rows: List> segment1Rows = ImmutableList.of( diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java index 1c451fc7434c..a9ad05ca2b53 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java @@ -157,10 +157,10 @@ public void testCompactionWithSegmentGranularityInGranularitySpec() throws Excep public void testCompactionWithQueryGranularityInGranularitySpec() throws Exception { try (final Closeable ignored = unloader(fullDatasourceName)) { - runTask(INDEX_TASK.get(), fullDatasourceName); + runTask(INDEX_TASK.get()); // 4 segments across 2 days verifySegmentsCount(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + List expectedIntervalAfterCompaction = getSegmentIntervals(); verifySegmentsHaveQueryGranularity("SECOND", 4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -193,10 +193,10 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti public void testParallelHashedCompaction() throws Exception { try (final Closeable ignored = unloader(fullDatasourceName)) { - runTask(INDEX_TASK.get(), fullDatasourceName); + runTask(INDEX_TASK.get()); // 4 segments across 2 days verifySegmentsCount(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + List expectedIntervalAfterCompaction = getSegmentIntervals(); verifySegmentsHaveQueryGranularity("SECOND", 4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -235,10 +235,10 @@ public void testParallelHashedCompaction() throws Exception public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularitySpec() throws Exception { try (final Closeable ignored = unloader(fullDatasourceName)) { - runTask(INDEX_TASK.get(), fullDatasourceName); + runTask(INDEX_TASK.get()); // 4 segments across 2 days verifySegmentsCount(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + List expectedIntervalAfterCompaction = getSegmentIntervals(); verifySegmentsHaveQueryGranularity("SECOND", 4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -270,10 +270,10 @@ private void loadDataAndCompact( ) throws Exception { try (final Closeable ignored = unloader(fullDatasourceName)) { - runTask(indexTask, fullDatasourceName); + runTask(indexTask); // 4 segments across 2 days verifySegmentsCount(4); - List expectedIntervalAfterCompaction = getSegmentIntervals(fullDatasourceName); + List expectedIntervalAfterCompaction = getSegmentIntervals(); verifySegmentsHaveQueryGranularity("SECOND", 4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -310,7 +310,7 @@ private String compactData( if (newSegmentGranularity != null) { template.segmentGranularity(newSegmentGranularity); } - return runTask(template, fullDatasourceName); + return runTask(template); } private void verifySegmentsHaveQueryGranularity(String expectedQueryGranularity, int segmentCount) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java index 9e524eabe92d..683abe00199c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTestBase.java @@ -64,11 +64,9 @@ protected Closeable unloader(String dataSource) } /** - * Creates a Task using the given builder and runs it. - * - * @return ID of the task. + * Creates and runs a task for the current {@link #dataSource}. */ - protected String runTask(TaskBuilder taskBuilder, String dataSource) + protected String runTask(TaskBuilder taskBuilder) { return cluster.callApi().runTask( (ds, taskId) -> taskBuilder.dataSource(ds).withId(taskId), @@ -82,11 +80,11 @@ protected void verifySegmentIntervals(List expectedIntervals) { Assertions.assertEquals( Set.copyOf(expectedIntervals), - Set.copyOf(getSegmentIntervals(dataSource)) + Set.copyOf(getSegmentIntervals()) ); } - protected List getSegmentIntervals(String dataSource) + protected List getSegmentIntervals() { return cluster.callApi().getSortedSegmentIntervals(dataSource, overlord); } @@ -98,9 +96,6 @@ protected void verifySegmentsCount(int numExpectedSegments) protected void verifyQuery(List> queries) { - if (queries == null) { - return; - } queries.forEach( pair -> cluster.callApi().verifySqlQuery(pair.lhs, dataSource, pair.rhs) ); From 8832555896796ef92d77f2ecadc7976ded73fcbd Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Jul 2025 13:23:32 +0530 Subject: [PATCH 38/38] Remove usages of || --- .../embedded/compact/AutoCompactionTest.java | 52 +++++++++---------- 1 file changed, 25 insertions(+), 27 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index b2149b7bfde3..c867305e7221 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -238,9 +238,9 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis // 2 segments across 1 days... verifySegmentsCount(2); - verifyScanResult("added", "...||31"); - verifyScanResult("ingested_events", "2||..."); - verifyScanResult("sum_added", "62||..."); + verifyScanResult("added", "...\n31"); + verifyScanResult("ingested_events", "2\n..."); + verifyScanResult("sum_added", "62\n..."); verifyScanResult("COUNT(*)", "2"); verifyDistinctCount("2,2"); @@ -296,9 +296,9 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis verifySegmentsCount(2); verifyScanResult("COUNT(*)", "2"); - verifyScanResult("added", "...||31"); - verifyScanResult("ingested_events", "2||..."); - verifyScanResult("sum_added", "62||..."); + verifyScanResult("added", "...\n31"); + verifyScanResult("ingested_events", "2\n..."); + verifyScanResult("sum_added", "62\n..."); verifyDistinctCount("2,2"); submitCompactionConfig( @@ -358,7 +358,7 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws // 2 segments across 1 days... verifySegmentsCount(2); - verifyScanResult("added", "31||31"); + verifyScanResult("added", "31\n31"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( @@ -408,7 +408,7 @@ public void testAutoCompactionWithMetricColumnSameAsInputColShouldOverwriteInput // 2 segments across 1 days... verifySegmentsCount(2); - verifyScanResult("added", "31||31"); + verifyScanResult("added", "31\n31"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( @@ -451,8 +451,8 @@ public void testAutoCompactionOnlyRowsWithMetricShouldPreserveExistingMetrics() // 2 segments across 1 days... verifySegmentsCount(2); - verifyScanResult("ingested_events", "2||2"); - verifyScanResult("sum_added", "62||62"); + verifyScanResult("ingested_events", "2\n2"); + verifyScanResult("sum_added", "62\n62"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( @@ -1291,7 +1291,7 @@ public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegm GranularitySpec specs = new UniformGranularitySpec(Granularities.MONTH, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -1309,7 +1309,7 @@ public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegm // does not have data on every week on the month forceTriggerAutoCompaction(3); // Make sure that no data is lost after compaction - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); List tasks = getCompleteTasksForDataSource(fullDatasourceName); @@ -1338,7 +1338,7 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe GranularitySpec specs = new UniformGranularitySpec(Granularities.WEEK, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -1353,7 +1353,7 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe // we expect the compaction task's interval to align with the MONTH segmentGranularity (2013-08-01 to 2013-10-01) forceTriggerAutoCompaction(2); // Make sure that no data is lost after compaction - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); List tasks = getCompleteTasksForDataSource(fullDatasourceName); @@ -1380,7 +1380,7 @@ public void testAutoCompactionDutyWithRollup() throws Exception GranularitySpec specs = new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -1410,7 +1410,7 @@ public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) GranularitySpec specs = new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, true, List.of(new Interval("2013-08-31/2013-09-02", chrono))); loadData(INDEX_TASK_WITH_GRANULARITY_SPEC, specs); try (final Closeable ignored = unloader(fullDatasourceName)) { - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( MAX_ROWS_PER_SEGMENT_COMPACTED, @@ -1444,7 +1444,7 @@ public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) th verifySegmentsCount(4); // Result is not rollup - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); // Compact and change dimension to only "language" @@ -1488,7 +1488,7 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce // Result is not rollup // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); // Compact and filter with selector on dim "page" and value "Striker Eureka" @@ -1531,7 +1531,7 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws verifySegmentsCount(4); // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); // Compact and add longSum and doubleSum metrics @@ -1549,8 +1549,8 @@ public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws // Result should be the same with the addition of new metrics, "double_sum_added" and "long_sum_added". // These new metrics should have the same value as the input field "added" - verifyScanResult("double_sum_added", "57.0||459.0"); - verifyScanResult("long_sum_added", "57||459"); + verifyScanResult("double_sum_added", "57.0\n459.0"); + verifyScanResult("long_sum_added", "57\n459"); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1578,7 +1578,7 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception // Result is not rollup // For dim "page", result has values "Gypsy Danger" and "Striker Eureka" - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); submitCompactionConfig( @@ -1593,12 +1593,12 @@ public void testAutoCompactionDutyWithOverlappingInterval() throws Exception ); // Compact the MONTH segment forceTriggerAutoCompaction(2); - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); // Compact the WEEK segment forceTriggerAutoCompaction(2); - verifyScanResult("added", "57.0||459.0"); + verifyScanResult("added", "57.0\n459.0"); verifyScanResult("COUNT(*)", "2"); // Verify all task succeed @@ -1646,8 +1646,7 @@ private void verifyDistinctCount(String result) * Verifies the result of a SELECT query * * @param field Field to select - * @param result CSV result with special strings {@code ||} to represent - * new-lines and {@code ...} to represent an empty string. + * @param result CSV result with special string {@code ...} to represent an empty string. */ private void verifyScanResult(String field, String result) { @@ -1658,7 +1657,6 @@ private void verifyScanResult(String field, String result) // replace empty placeholder with empty string result = StringUtils.replace(result, "...", "\"\""); - result = StringUtils.replace(result, "||", "\n"); Assertions.assertEquals( result,