diff --git a/core/src/main/java/org/apache/druid/indexer/IngestionState.java b/core/src/main/java/org/apache/druid/indexer/IngestionState.java index 26d46166674e..ddbd0a995776 100644 --- a/core/src/main/java/org/apache/druid/indexer/IngestionState.java +++ b/core/src/main/java/org/apache/druid/indexer/IngestionState.java @@ -24,5 +24,6 @@ public enum IngestionState NOT_STARTED, DETERMINE_PARTITIONS, BUILD_SEGMENTS, + SEGMENT_AVAILABILITY_WAIT, COMPLETED } diff --git a/docs/ingestion/compaction.md b/docs/ingestion/compaction.md index 239523d401be..f6846dc558a5 100644 --- a/docs/ingestion/compaction.md +++ b/docs/ingestion/compaction.md @@ -111,7 +111,7 @@ To perform a manual compaction, you submit a compaction task. Compaction tasks m |`dimensionsSpec`|Custom dimensions spec. The compaction task uses the specified dimensions spec if it exists instead of generating one.|No| |`metricsSpec`|Custom metrics spec. The compaction task uses the specified metrics spec rather than generating one.|No| |`segmentGranularity`|When set, the compaction task changes the segment granularity for the given interval. Deprecated. Use `granularitySpec`. |No.| -|`tuningConfig`|[Parallel indexing task tuningConfig](native-batch.md#tuningconfig)|No| +|`tuningConfig`|[Parallel indexing task tuningConfig](native-batch.md#tuningconfig). Note that your tuning config cannot contain a non-zero value for `awaitSegmentAvailabilityTimeoutMillis` because it is not supported by compaction tasks at this time.|No| |`context`|[Task context](./tasks.md#context)|No| |`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` and `queryGranularity` for the compacted segments. See [Compaction granularitySpec](#compaction-granularity-spec).|No| diff --git a/docs/ingestion/hadoop.md b/docs/ingestion/hadoop.md index 37f491c84787..4fd4bccc91c7 100644 --- a/docs/ingestion/hadoop.md +++ b/docs/ingestion/hadoop.md @@ -336,6 +336,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |logParseExceptions|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no(default = false)| |maxParseExceptions|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overrides `ignoreInvalidRows` if `maxParseExceptions` is defined.|no(default = unlimited)| |useYarnRMJobStatusFallback|Boolean|If the Hadoop jobs created by the indexing task are unable to retrieve their completion status from the JobHistory server, and this parameter is true, the indexing task will try to fetch the application status from `http:///ws/v1/cluster/apps/`, where `` is the value of `yarn.resourcemanager.webapp.address` in your Hadoop configuration. This flag is intended as a fallback for cases where an indexing task's jobs succeed, but the JobHistory server is unavailable, causing the indexing task to fail because it cannot determine the job statuses.|no (default = true)| +|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait will occur. If `> 0`, the task will wait for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task will exit as successful, but the segments were not confirmed to have become available for query.|no (default = 0)| ### `jobProperties` diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 0cb1f59fea53..5e7b337e13f5 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -197,6 +197,7 @@ A sample task is shown below: |id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| |spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes| |context|Context containing various task configuration parameters. See below for more details.|no| +|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait will occur. If `> 0`, the task will wait for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task will exit as successful, but the segments were not confirmed to have become available for query. Note for compaction tasks: you should not set this to a non-zero value because it is not supported by the compaction task type at this time.|no (default = 0)| ### `dataSchema` @@ -250,6 +251,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |taskStatusCheckPeriodMs|Polling period in milliseconds to check running task statuses.|1000|no| |chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no| |chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no| +|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait will occur. If `> 0`, the task will wait for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task will exit as successful, but the segments were not confirmed to have become available for query.|no (default = 0)| ### Split Hint Spec diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index db63a7316d1f..be21a6558fff 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -199,7 +199,8 @@ public HadoopIndexTask createTask(Interval interval, String version, List allowedHadoopPrefix, final @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, final @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - final @JsonProperty("useYarnRMJobStatusFallback") @Nullable Boolean useYarnRMJobStatusFallback + final @JsonProperty("useYarnRMJobStatusFallback") @Nullable Boolean useYarnRMJobStatusFallback, + final @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis ) { this.workingPath = workingPath; @@ -176,6 +179,12 @@ public HadoopTuningConfig( this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; this.useYarnRMJobStatusFallback = useYarnRMJobStatusFallback == null ? true : useYarnRMJobStatusFallback; + + if (awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis < 0) { + this.awaitSegmentAvailabilityTimeoutMillis = DEFAULT_AWAIT_SEGMENT_AVAILABILITY_TIMEOUT_MILLIS; + } else { + this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; + } } @Nullable @@ -334,6 +343,12 @@ public boolean isUseYarnRMJobStatusFallback() return useYarnRMJobStatusFallback; } + @JsonProperty + public long getAwaitSegmentAvailabilityTimeoutMillis() + { + return awaitSegmentAvailabilityTimeoutMillis; + } + public HadoopTuningConfig withWorkingPath(String path) { return new HadoopTuningConfig( @@ -361,7 +376,8 @@ public HadoopTuningConfig withWorkingPath(String path) allowedHadoopPrefix, logParseExceptions, maxParseExceptions, - useYarnRMJobStatusFallback + useYarnRMJobStatusFallback, + awaitSegmentAvailabilityTimeoutMillis ); } @@ -392,7 +408,8 @@ public HadoopTuningConfig withVersion(String ver) allowedHadoopPrefix, logParseExceptions, maxParseExceptions, - useYarnRMJobStatusFallback + useYarnRMJobStatusFallback, + awaitSegmentAvailabilityTimeoutMillis ); } @@ -423,7 +440,8 @@ public HadoopTuningConfig withShardSpecs(Map> specs allowedHadoopPrefix, logParseExceptions, maxParseExceptions, - useYarnRMJobStatusFallback + useYarnRMJobStatusFallback, + awaitSegmentAvailabilityTimeoutMillis ); } } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 644ae240262d..404d5ed67b8f 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -481,6 +481,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( null, null, null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index a127d519cc2b..1bb672cb36c4 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -235,6 +235,7 @@ public DetermineHashedPartitionsJobTest( null, null, null, + null, null ) ); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index 047c1049c0ff..b73b47cacc1b 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -344,6 +344,7 @@ public DeterminePartitionsJobTest( null, null, null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index 9eabd4187a32..86e35d08ab9d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -257,6 +257,7 @@ HadoopIngestionSpec build() null, null, null, + null, null ); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java index 277f590cecc7..230c6c0a949d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java @@ -63,6 +63,7 @@ public void testSerde() throws Exception null, null, null, + null, null ); @@ -86,6 +87,7 @@ public void testSerde() throws Exception Assert.assertEquals(0, actual.getNumBackgroundPersistThreads()); Assert.assertEquals(true, actual.isForceExtendableShardSpecs()); Assert.assertEquals(true, actual.isUseExplicitVersion()); + Assert.assertEquals(0, actual.getAwaitSegmentAvailabilityTimeoutMillis()); } public static T jsonReadWriteRead(String s, Class klass) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index c0254c4fbe43..97967bdb3b7e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -547,6 +547,7 @@ public void setUp() throws Exception null, null, null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 3188762bc0b0..9115eddfc086 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -187,6 +187,7 @@ public void setup() throws Exception null, null, null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java index 68505be91cf1..8fc3e6c28084 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java @@ -79,6 +79,7 @@ public class GranularityPathSpecTest null, null, null, + null, null ); diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTuningConfigModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTuningConfigModule.java index 4799f2ad2805..56e502259a6b 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTuningConfigModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTuningConfigModule.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.initialization.DruidModule; @@ -39,7 +40,8 @@ public List getJacksonModules() new SimpleModule(IndexingServiceTuningConfigModule.class.getSimpleName()) .registerSubtypes( new NamedType(IndexTuningConfig.class, "index"), - new NamedType(ParallelIndexTuningConfig.class, "index_parallel") + new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), + new NamedType(CompactionTask.CompactionTuningConfig.class, "compaction") ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java index 3b7b780561ef..bb149e54b50f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java @@ -41,17 +41,22 @@ public class IngestionStatsAndErrorsTaskReportData @Nullable private String errorMsg; + @JsonProperty + private boolean segmentAvailabilityConfirmed; + public IngestionStatsAndErrorsTaskReportData( @JsonProperty("ingestionState") IngestionState ingestionState, @JsonProperty("unparseableEvents") Map unparseableEvents, @JsonProperty("rowStats") Map rowStats, - @JsonProperty("errorMsg") @Nullable String errorMsg + @JsonProperty("errorMsg") @Nullable String errorMsg, + @JsonProperty("segmentAvailabilityConfirmed") boolean segmentAvailabilityConfirmed ) { this.ingestionState = ingestionState; this.unparseableEvents = unparseableEvents; this.rowStats = rowStats; this.errorMsg = errorMsg; + this.segmentAvailabilityConfirmed = segmentAvailabilityConfirmed; } @JsonProperty @@ -79,6 +84,12 @@ public String getErrorMsg() return errorMsg; } + @JsonProperty + public boolean isSegmentAvailabilityConfirmed() + { + return segmentAvailabilityConfirmed; + } + public static IngestionStatsAndErrorsTaskReportData getPayloadFromTaskReports( Map taskReports ) @@ -100,13 +111,20 @@ public boolean equals(Object o) return getIngestionState() == that.getIngestionState() && Objects.equals(getUnparseableEvents(), that.getUnparseableEvents()) && Objects.equals(getRowStats(), that.getRowStats()) && - Objects.equals(getErrorMsg(), that.getErrorMsg()); + Objects.equals(getErrorMsg(), that.getErrorMsg()) && + Objects.equals(isSegmentAvailabilityConfirmed(), that.isSegmentAvailabilityConfirmed()); } @Override public int hashCode() { - return Objects.hash(getIngestionState(), getUnparseableEvents(), getRowStats(), getErrorMsg()); + return Objects.hash( + getIngestionState(), + getUnparseableEvents(), + getRowStats(), + getErrorMsg(), + isSegmentAvailabilityConfirmed() + ); } @Override @@ -117,6 +135,7 @@ public String toString() ", unparseableEvents=" + unparseableEvents + ", rowStats=" + rowStats + ", errorMsg='" + errorMsg + '\'' + + ", segmentAvailabilityConfirmed=" + segmentAvailabilityConfirmed + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index cbdfc68975da..bc1e00b3d04e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -49,6 +49,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentArchiver; @@ -59,7 +60,6 @@ import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 2ef1f88ac0a7..5112fa9f4eb5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -47,6 +47,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentArchiver; @@ -55,7 +56,6 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index a51a3c248dbe..53734dd90d36 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -45,10 +45,13 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.granularity.IntervalsByGranularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; @@ -72,6 +75,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -87,6 +93,8 @@ public abstract class AbstractBatchIndexTask extends AbstractTask { private static final Logger log = new Logger(AbstractBatchIndexTask.class); + protected boolean segmentAvailabilityConfirmationCompleted = false; + @GuardedBy("this") private final TaskResourceCleaner resourceCloserOnAbnormalExit = new TaskResourceCleaner(); @@ -580,6 +588,63 @@ protected static List findInputSegments( } } + /** + * Wait for segments to become available on the cluster. If waitTimeout is reached, giveup on waiting. This is a + * QoS method that can be used to make Batch Ingest tasks wait to finish until their ingested data is available on + * the cluster. Doing so gives an end user assurance that a Successful task status means their data is available + * for querying. + * + * @param toolbox {@link TaskToolbox} object with for assisting with task work. + * @param segmentsToWaitFor {@link List} of segments to wait for availability. + * @param waitTimeout Millis to wait before giving up + * @return True if all segments became available, otherwise False. + */ + protected boolean waitForSegmentAvailability( + TaskToolbox toolbox, + List segmentsToWaitFor, + long waitTimeout + ) + { + if (segmentsToWaitFor.isEmpty()) { + log.info("Asked to wait for segments to be available, but I wasn't provided with any segments."); + return true; + } else if (waitTimeout < 0) { + log.warn("Asked to wait for availability for < 0 seconds?! Requested waitTimeout: [%s]", waitTimeout); + return false; + } + log.info("Waiting for [%d] segments to be loaded by the cluster...", segmentsToWaitFor.size()); + + try ( + SegmentHandoffNotifier notifier = toolbox.getSegmentHandoffNotifierFactory() + .createSegmentHandoffNotifier(segmentsToWaitFor.get(0).getDataSource()) + ) { + + ExecutorService exec = Execs.directExecutor(); + CountDownLatch doneSignal = new CountDownLatch(segmentsToWaitFor.size()); + + notifier.start(); + for (DataSegment s : segmentsToWaitFor) { + notifier.registerSegmentHandoffCallback( + new SegmentDescriptor(s.getInterval(), s.getVersion(), s.getShardSpec().getPartitionNum()), + exec, + () -> { + log.debug( + "Confirmed availability for [%s]. Removing from list of segments to wait for", + s.getId() + ); + doneSignal.countDown(); + } + ); + } + return doneSignal.await(waitTimeout, TimeUnit.MILLISECONDS); + } + catch (InterruptedException e) { + log.warn("Interrupted while waiting for segment availablity; Unable to confirm availability!"); + Thread.currentThread().interrupt(); + return false; + } + } + private static class LockGranularityDetermineResult { private final LockGranularity lockGranularity; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 3e5bd4f09ff8..550f81886f1f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -581,6 +581,14 @@ && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory). && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); } + /** + * Return a map of reports for the task. + * + * A successfull task should always have a null errorMsg. A falied task should always have a non-null + * errorMsg. + * + * @return Map of reports for the task. + */ private Map getTaskCompletionReports() { return TaskReport.buildTaskReports( @@ -590,7 +598,8 @@ private Map getTaskCompletionReports() ingestionState, getTaskCompletionUnparseableEvents(), getTaskCompletionRowStats(), - errorMsg + errorMsg, + errorMsg == null ) ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index ed4bf7eb2b3a..18f5f71d54b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -37,6 +37,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -77,21 +78,25 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; +import org.joda.time.Duration; import org.joda.time.Interval; import javax.annotation.Nonnull; @@ -146,7 +151,7 @@ public class CompactionTask extends AbstractBatchIndexTask @Nullable private final ClientCompactionTaskGranularitySpec granularitySpec; @Nullable - private final ParallelIndexTuningConfig tuningConfig; + private final CompactionTuningConfig tuningConfig; @JsonIgnore private final SegmentProvider segmentProvider; @JsonIgnore @@ -234,17 +239,51 @@ public CompactionTask( } @VisibleForTesting - static ParallelIndexTuningConfig getTuningConfig(TuningConfig tuningConfig) + static CompactionTuningConfig getTuningConfig(TuningConfig tuningConfig) { - if (tuningConfig instanceof ParallelIndexTuningConfig) { - return (ParallelIndexTuningConfig) tuningConfig; + if (tuningConfig instanceof CompactionTuningConfig) { + return (CompactionTuningConfig) tuningConfig; + } else if (tuningConfig instanceof ParallelIndexTuningConfig) { + final ParallelIndexTuningConfig parallelIndexTuningConfig = (ParallelIndexTuningConfig) tuningConfig; + return new CompactionTuningConfig( + null, + parallelIndexTuningConfig.getMaxRowsPerSegment(), + parallelIndexTuningConfig.getAppendableIndexSpec(), + parallelIndexTuningConfig.getMaxRowsInMemory(), + parallelIndexTuningConfig.getMaxBytesInMemory(), + parallelIndexTuningConfig.isSkipBytesInMemoryOverheadCheck(), + parallelIndexTuningConfig.getMaxTotalRows(), + parallelIndexTuningConfig.getNumShards(), + parallelIndexTuningConfig.getSplitHintSpec(), + parallelIndexTuningConfig.getPartitionsSpec(), + parallelIndexTuningConfig.getIndexSpec(), + parallelIndexTuningConfig.getIndexSpecForIntermediatePersists(), + parallelIndexTuningConfig.getMaxPendingPersists(), + parallelIndexTuningConfig.isForceGuaranteedRollup(), + parallelIndexTuningConfig.isReportParseExceptions(), + parallelIndexTuningConfig.getPushTimeout(), + parallelIndexTuningConfig.getSegmentWriteOutMediumFactory(), + null, + parallelIndexTuningConfig.getMaxNumConcurrentSubTasks(), + parallelIndexTuningConfig.getMaxRetry(), + parallelIndexTuningConfig.getTaskStatusCheckPeriodMs(), + parallelIndexTuningConfig.getChatHandlerTimeout(), + parallelIndexTuningConfig.getChatHandlerNumRetries(), + parallelIndexTuningConfig.getMaxNumSegmentsToMerge(), + parallelIndexTuningConfig.getTotalNumMergeTasks(), + parallelIndexTuningConfig.isLogParseExceptions(), + parallelIndexTuningConfig.getMaxParseExceptions(), + parallelIndexTuningConfig.getMaxSavedParseExceptions(), + parallelIndexTuningConfig.getMaxColumnsToMerge(), + parallelIndexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis() + ); } else if (tuningConfig instanceof IndexTuningConfig) { final IndexTuningConfig indexTuningConfig = (IndexTuningConfig) tuningConfig; - return new ParallelIndexTuningConfig( + return new CompactionTuningConfig( null, indexTuningConfig.getMaxRowsPerSegment(), indexTuningConfig.getAppendableIndexSpec(), - indexTuningConfig.getMaxRowsPerSegment(), + indexTuningConfig.getMaxRowsInMemory(), indexTuningConfig.getMaxBytesInMemory(), indexTuningConfig.isSkipBytesInMemoryOverheadCheck(), indexTuningConfig.getMaxTotalRows(), @@ -269,12 +308,14 @@ static ParallelIndexTuningConfig getTuningConfig(TuningConfig tuningConfig) indexTuningConfig.isLogParseExceptions(), indexTuningConfig.getMaxParseExceptions(), indexTuningConfig.getMaxSavedParseExceptions(), - indexTuningConfig.getMaxColumnsToMerge() + indexTuningConfig.getMaxColumnsToMerge(), + indexTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis() ); } else { throw new ISE( - "Unknown tuningConfig type: [%s], Must be either [%s] or [%s]", + "Unknown tuningConfig type: [%s], Must be in [%s, %s, %s]", tuningConfig.getClass().getName(), + CompactionTuningConfig.class.getName(), ParallelIndexTuningConfig.class.getName(), IndexTuningConfig.class.getName() ); @@ -514,7 +555,7 @@ static List createIngestionSchema( toolbox.getIndexIO() ); - final ParallelIndexTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); + final CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { // original granularity @@ -950,18 +991,18 @@ void checkSegments(LockGranularity lockGranularityInUse, List lates static class PartitionConfigurationManager { @Nullable - private final ParallelIndexTuningConfig tuningConfig; + private final CompactionTuningConfig tuningConfig; - PartitionConfigurationManager(@Nullable ParallelIndexTuningConfig tuningConfig) + PartitionConfigurationManager(@Nullable CompactionTuningConfig tuningConfig) { this.tuningConfig = tuningConfig; } @Nullable - ParallelIndexTuningConfig computeTuningConfig() + CompactionTuningConfig computeTuningConfig() { - ParallelIndexTuningConfig newTuningConfig = tuningConfig == null - ? ParallelIndexTuningConfig.defaultConfig() + CompactionTuningConfig newTuningConfig = tuningConfig == null + ? CompactionTuningConfig.defaultConfig() : tuningConfig; PartitionsSpec partitionsSpec = newTuningConfig.getGivenOrDefaultPartitionsSpec(); if (partitionsSpec instanceof DynamicPartitionsSpec) { @@ -1082,4 +1123,163 @@ public CompactionTask build() ); } } + + /** + * Compcation Task Tuning Config. + * + * An extension of ParallelIndexTuningConfig. As of now, all this TuningConfig + * does is fail if the TuningConfig contains + * `awaitSegmentAvailabilityTimeoutMillis` that is != 0 since it is not + * supported for Compcation Tasks. + */ + public static class CompactionTuningConfig extends ParallelIndexTuningConfig + { + public static final String TYPE = "compaction"; + + public static CompactionTuningConfig defaultConfig() + { + return new CompactionTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 0L + ); + } + + @JsonCreator + public CompactionTuningConfig( + @JsonProperty("targetPartitionSize") @Deprecated @Nullable Integer targetPartitionSize, + @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, + @JsonProperty("maxTotalRows") @Deprecated @Nullable Long maxTotalRows, + @JsonProperty("numShards") @Deprecated @Nullable Integer numShards, + @JsonProperty("splitHintSpec") @Nullable SplitHintSpec splitHintSpec, + @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, + @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("pushTimeout") @Nullable Long pushTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("maxNumSubTasks") @Deprecated @Nullable Integer maxNumSubTasks, + @JsonProperty("maxNumConcurrentSubTasks") @Nullable Integer maxNumConcurrentSubTasks, + @JsonProperty("maxRetry") @Nullable Integer maxRetry, + @JsonProperty("taskStatusCheckPeriodMs") @Nullable Long taskStatusCheckPeriodMs, + @JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout, + @JsonProperty("chatHandlerNumRetries") @Nullable Integer chatHandlerNumRetries, + @JsonProperty("maxNumSegmentsToMerge") @Nullable Integer maxNumSegmentsToMerge, + @JsonProperty("totalNumMergeTasks") @Nullable Integer totalNumMergeTasks, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, + @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis + ) + { + super( + targetPartitionSize, + maxRowsPerSegment, + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxTotalRows, + numShards, + splitHintSpec, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory, + maxNumSubTasks, + maxNumConcurrentSubTasks, + maxRetry, + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis + ); + + Preconditions.checkArgument( + awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis == 0, + "awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task" + ); + } + + @Override + public CompactionTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) + { + return new CompactionTuningConfig( + null, + null, + getAppendableIndexSpec(), + getMaxRowsInMemory(), + getMaxBytesInMemory(), + isSkipBytesInMemoryOverheadCheck(), + null, + null, + getSplitHintSpec(), + partitionsSpec, + getIndexSpec(), + getIndexSpecForIntermediatePersists(), + getMaxPendingPersists(), + isForceGuaranteedRollup(), + isReportParseExceptions(), + getPushTimeout(), + getSegmentWriteOutMediumFactory(), + null, + getMaxNumConcurrentSubTasks(), + getMaxRetry(), + getTaskStatusCheckPeriodMs(), + getChatHandlerTimeout(), + getChatHandlerNumRetries(), + getMaxNumSegmentsToMerge(), + getTotalNumMergeTasks(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions(), + getMaxColumnsToMerge(), + getAwaitSegmentAvailabilityTimeoutMillis() + ); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index c11ce31c59a9..b66ae471136c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -79,6 +79,7 @@ import java.io.File; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -439,8 +440,21 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); if (buildSegmentsStatus.getDataSegments() != null) { - ingestionState = IngestionState.COMPLETED; toolbox.publishSegments(buildSegmentsStatus.getDataSegments()); + + // Try to wait for segments to be loaded by the cluster if the tuning config specifies a non-zero value + // for awaitSegmentAvailabilityTimeoutMillis + if (spec.getTuningConfig().getAwaitSegmentAvailabilityTimeoutMillis() > 0) { + ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; + ArrayList segmentsToWaitFor = new ArrayList<>(buildSegmentsStatus.getDataSegments()); + segmentAvailabilityConfirmationCompleted = waitForSegmentAvailability( + toolbox, + segmentsToWaitFor, + spec.getTuningConfig().getAwaitSegmentAvailabilityTimeoutMillis() + ); + } + + ingestionState = IngestionState.COMPLETED; toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { @@ -533,7 +547,8 @@ private Map getTaskCompletionReports() ingestionState, null, getTaskCompletionRowStats(), - errorMsg + errorMsg, + segmentAvailabilityConfirmationCompleted ) ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index bb1ee7714633..75ae4d23fdf2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -525,7 +525,8 @@ private Map getTaskCompletionReports() ingestionState, getTaskCompletionUnparseableEvents(), getTaskCompletionRowStats(), - errorMsg + errorMsg, + segmentAvailabilityConfirmationCompleted ) ) ); @@ -918,6 +919,18 @@ private TaskStatus generateAndPublishSegments( awaitPublish(driver.publishAll(inputSegments, segmentsFoundForDrop, publisher, annotateFunction), pushTimeout); appenderator.close(); + // Try to wait for segments to be loaded by the cluster if the tuning config specifies a non-zero value + // for awaitSegmentAvailabilityTimeoutMillis + if (tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis() > 0 && published != null) { + ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; + ArrayList segmentsToWaitFor = new ArrayList<>(published.getSegments()); + segmentAvailabilityConfirmationCompleted = waitForSegmentAvailability( + toolbox, + segmentsToWaitFor, + tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis() + ); + } + ingestionState = IngestionState.COMPLETED; if (published == null) { log.error("Failed to publish segments, aborting!"); @@ -1169,6 +1182,7 @@ public static class IndexTuningConfig implements AppenderatorConfig private final boolean logParseExceptions; private final int maxParseExceptions; private final int maxSavedParseExceptions; + private final long awaitSegmentAvailabilityTimeoutMillis; @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @@ -1238,7 +1252,8 @@ public IndexTuningConfig( @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, + @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis ) { this( @@ -1265,7 +1280,8 @@ public IndexTuningConfig( logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - maxColumnsToMerge + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis ); Preconditions.checkArgument( @@ -1276,7 +1292,7 @@ public IndexTuningConfig( private IndexTuningConfig() { - this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); + this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( @@ -1296,7 +1312,8 @@ private IndexTuningConfig( @Nullable Boolean logParseExceptions, @Nullable Integer maxParseExceptions, @Nullable Integer maxSavedParseExceptions, - @Nullable Integer maxColumnsToMerge + @Nullable Integer maxColumnsToMerge, + @Nullable Long awaitSegmentAvailabilityTimeoutMillis ) { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; @@ -1337,6 +1354,11 @@ private IndexTuningConfig( this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; + if (awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis < 0) { + this.awaitSegmentAvailabilityTimeoutMillis = DEFAULT_AWAIT_SEGMENT_AVAILABILITY_TIMEOUT_MILLIS; + } else { + this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; + } } @Override @@ -1359,7 +1381,8 @@ public IndexTuningConfig withBasePersistDirectory(File dir) logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - maxColumnsToMerge + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis ); } @@ -1382,7 +1405,8 @@ public IndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - maxColumnsToMerge + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis ); } @@ -1574,6 +1598,12 @@ public Period getIntermediatePersistPeriod() return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } + @JsonProperty + public long getAwaitSegmentAvailabilityTimeoutMillis() + { + return awaitSegmentAvailabilityTimeoutMillis; + } + @Override public boolean equals(Object o) { @@ -1600,7 +1630,8 @@ public boolean equals(Object o) Objects.equals(indexSpec, that.indexSpec) && Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && + Objects.equals(awaitSegmentAvailabilityTimeoutMillis, that.awaitSegmentAvailabilityTimeoutMillis); } @Override @@ -1623,7 +1654,8 @@ public int hashCode() logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + awaitSegmentAvailabilityTimeoutMillis ); } @@ -1647,6 +1679,7 @@ public String toString() ", maxParseExceptions=" + maxParseExceptions + ", maxSavedParseExceptions=" + maxSavedParseExceptions + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + ", awaitSegmentAvailabilityTimeoutMillis=" + awaitSegmentAvailabilityTimeoutMillis + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 03927ad08bd1..60984e8a9dbb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -36,14 +36,18 @@ import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; +import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; 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.indexing.common.Counters; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; @@ -166,6 +170,8 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen @MonotonicNonNull private volatile TaskToolbox toolbox; + private long awaitSegmentAvailabilityTimeoutMillis; + @JsonCreator public ParallelIndexSupervisorTask( @JsonProperty("id") String id, @@ -200,6 +206,8 @@ public ParallelIndexSupervisorTask( if (missingIntervalsInOverwriteMode) { addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } + + awaitSegmentAvailabilityTimeoutMillis = ingestionSchema.getTuningConfig().getAwaitSegmentAvailabilityTimeoutMillis(); } private static void checkPartitionsSpecForForceGuaranteedRollup(PartitionsSpec partitionsSpec) @@ -493,6 +501,24 @@ private boolean isParallelMode() return isParallelMode(baseInputSource, ingestionSchema.getTuningConfig()); } + /** + * Attempt to wait for indexed segments to become available on the cluster. + * @param reportsMap Map containing information with published segments that we are going to wait for. + */ + private void waitForSegmentAvailability(Map reportsMap) + { + ArrayList segmentsToWaitFor = new ArrayList<>(); + reportsMap.values() + .forEach(report -> { + segmentsToWaitFor.addAll(report.getNewSegments()); + }); + segmentAvailabilityConfirmationCompleted = waitForSegmentAvailability( + toolbox, + segmentsToWaitFor, + awaitSegmentAvailabilityTimeoutMillis + ); + } + /** * Run the single phase parallel indexing for best-effort rollup. In this mode, each sub task created by * the supervisor task reads data and generates segments individually. @@ -508,8 +534,16 @@ private TaskStatus runSinglePhaseParallel(TaskToolbox toolbox) throws Exception if (state.isSuccess()) { //noinspection ConstantConditions publishSegments(toolbox, runner.getReports()); + if (awaitSegmentAvailabilityTimeoutMillis > 0) { + waitForSegmentAvailability(runner.getReports()); + } } - return TaskStatus.fromCode(getId(), state); + TaskStatus taskStatus = TaskStatus.fromCode(getId(), state); + toolbox.getTaskReportFileWriter().write( + getId(), + getTaskCompletionReports(taskStatus, segmentAvailabilityConfirmationCompleted) + ); + return taskStatus; } /** @@ -646,9 +680,17 @@ private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throw if (state.isSuccess()) { //noinspection ConstantConditions publishSegments(toolbox, mergeRunner.getReports()); + if (awaitSegmentAvailabilityTimeoutMillis > 0) { + waitForSegmentAvailability(mergeRunner.getReports()); + } } - return TaskStatus.fromCode(getId(), state); + TaskStatus taskStatus = TaskStatus.fromCode(getId(), state); + toolbox.getTaskReportFileWriter().write( + getId(), + getTaskCompletionReports(taskStatus, segmentAvailabilityConfirmationCompleted) + ); + return taskStatus; } private TaskStatus runRangePartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception @@ -708,9 +750,17 @@ private TaskStatus runRangePartitionMultiPhaseParallel(TaskToolbox toolbox) thro TaskState mergeState = runNextPhase(mergeRunner); if (mergeState.isSuccess()) { publishSegments(toolbox, mergeRunner.getReports()); + if (awaitSegmentAvailabilityTimeoutMillis > 0) { + waitForSegmentAvailability(mergeRunner.getReports()); + } } - return TaskStatus.fromCode(getId(), mergeState); + TaskStatus taskStatus = TaskStatus.fromCode(getId(), mergeState); + toolbox.getTaskReportFileWriter().write( + getId(), + getTaskCompletionReports(taskStatus, segmentAvailabilityConfirmationCompleted) + ); + return taskStatus; } private static Map mergeCardinalityReports(Collection reports) @@ -976,6 +1026,30 @@ private TaskStatus runSequential(TaskToolbox toolbox) throws Exception } } + /** + * Generate an IngestionStatsAndErrorsTaskReport for the task. + * + * @param taskStatus {@link TaskStatus} + * @param segmentAvailabilityConfirmed Whether or not the segments were confirmed to be available for query when + * when the task completed. + * @return + */ + private Map getTaskCompletionReports(TaskStatus taskStatus, boolean segmentAvailabilityConfirmed) + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + IngestionState.COMPLETED, + new HashMap<>(), + new HashMap<>(), + taskStatus.getErrorMsg(), + segmentAvailabilityConfirmed + ) + ) + ); + } + private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig) { return new IndexTuningConfig( @@ -1001,7 +1075,8 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), tuningConfig.getMaxSavedParseExceptions(), - tuningConfig.getMaxColumnsToMerge() + tuningConfig.getMaxColumnsToMerge(), + tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 065504eb7e49..f474c1efca9b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -101,6 +101,7 @@ public static ParallelIndexTuningConfig defaultConfig() null, null, null, + null, null ); } @@ -135,7 +136,8 @@ public ParallelIndexTuningConfig( @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, + @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis ) { super( @@ -161,7 +163,8 @@ public ParallelIndexTuningConfig( logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - maxColumnsToMerge + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis ); if (maxNumSubTasks != null && maxNumConcurrentSubTasks != null) { @@ -284,7 +287,8 @@ public ParallelIndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpe isLogParseExceptions(), getMaxParseExceptions(), getMaxSavedParseExceptions(), - getMaxColumnsToMerge() + getMaxColumnsToMerge(), + getAwaitSegmentAvailabilityTimeoutMillis() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ea8a5ba2ed7f..6f0728d2c0be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1050,6 +1050,18 @@ private synchronized void persistSequences() throws IOException log.info("Saved sequence metadata to disk: %s", sequences); } + /** + * Return a map of reports for the task. + * + * A successfull task should always have a null errorMsg. Segments availability is inherently confirmed + * if the task was succesful. + * + * A falied task should always have a non-null errorMsg. Segment availability is never confirmed if the task + * was not successful. + * + * @param errorMsg Nullable error message for the task. null if task succeeded. + * @return Map of reports for the task. + */ private Map getTaskCompletionReports(@Nullable String errorMsg) { return TaskReport.buildTaskReports( @@ -1059,7 +1071,8 @@ private Map getTaskCompletionReports(@Nullable String errorM ingestionState, getTaskCompletionUnparseableEvents(), getTaskCompletionRowStats(), - errorMsg + errorMsg, + errorMsg == null ) ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index ef75df5a52d5..a5e889d2726b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -37,6 +37,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentArchiver; import org.apache.druid.segment.loading.DataSegmentKiller; @@ -45,7 +46,6 @@ import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index abc504b308d3..46fa4fac6a1c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -109,14 +109,14 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index a73e44b7dc36..6e93f3ad4707 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -255,6 +255,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException null, null, null, + null, null ) ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 4693f3ad485c..95826fa23e1c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -113,6 +113,7 @@ public CompactionTaskParallelRunTest(LockGranularity lockGranularity) public void setup() throws IOException { getObjectMapper().registerSubtypes(ParallelIndexTuningConfig.class, DruidInputSource.class); + getObjectMapper().registerSubtypes(CompactionTask.CompactionTuningConfig.class, DruidInputSource.class); inputDir = temporaryFolder.newFolder(); final File tmpFile = File.createTempFile("druid", "index", inputDir); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 5eadc388db0f..384d26f46b97 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -315,6 +315,7 @@ public void testRunWithHashPartitioning() throws Exception null, null, null, + null, null ) ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 4f386abe23d9..f9f3a66cab3a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -170,7 +170,7 @@ public class CompactionTaskTest Intervals.of("2017-06-30/2017-07-01") ); private static final Map MIXED_TYPE_COLUMN_MAP = new HashMap<>(); - private static final ParallelIndexTuningConfig TUNING_CONFIG = createTuningConfig(); + private static final CompactionTask.CompactionTuningConfig TUNING_CONFIG = createTuningConfig(); private static final TestUtils TEST_UTILS = new TestUtils(); private static final Map SEGMENT_MAP = new HashMap<>(); @@ -307,9 +307,9 @@ private static List findDimensions(int startIndex, Interval segmentInter return dimensions; } - private static ParallelIndexTuningConfig createTuningConfig() + private static CompactionTask.CompactionTuningConfig createTuningConfig() { - return new ParallelIndexTuningConfig( + return new CompactionTask.CompactionTuningConfig( null, null, // null to compute maxRowsPerSegment automatically null, @@ -343,6 +343,7 @@ private static ParallelIndexTuningConfig createTuningConfig() null, null, null, + null, null ); } @@ -575,6 +576,7 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws null, null, null, + null, null ), null, @@ -606,6 +608,165 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws assertEquals(expectedFromJson, fromJson); } + @Test + public void testGetTuningConfigWithIndexTuningConfig() + { + IndexTuningConfig indexTuningConfig = new IndexTuningConfig( + null, + null, // null to compute maxRowsPerSegment automatically + null, + 500000, + 1000000L, + null, + null, + null, + null, + null, + null, + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.LZ4, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + null, + null, + true, + false, + null, + null, + null, + null, + null, + null, + null, + null + ); + + CompactionTask.CompactionTuningConfig compactionTuningConfig = new CompactionTask.CompactionTuningConfig( + null, + null, + null, + 500000, + 1000000L, + null, + null, + null, + null, + null, + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.LZ4, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + null, + null, + true, + false, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(indexTuningConfig)); + + } + + @Test + public void testGetTuningConfigWithParallelIndexTuningConfig() + { + ParallelIndexTuningConfig parallelIndexTuningConfig = new ParallelIndexTuningConfig( + null, + null, // null to compute maxRowsPerSegment automatically + null, + 500000, + 1000000L, + null, + null, + null, + null, + null, + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.LZ4, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + null, + null, + true, + false, + 5000L, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + CompactionTask.CompactionTuningConfig compactionTuningConfig = new CompactionTask.CompactionTuningConfig( + null, + null, // null to compute maxRowsPerSegment automatically + null, + 500000, + 1000000L, + null, + null, + null, + null, + null, + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.LZ4, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + null, + null, + true, + false, + 5000L, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(parallelIndexTuningConfig)); + } + @Test public void testSerdeWithUnknownTuningConfigThrowingError() throws IOException { @@ -711,7 +872,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept @Test public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOException, SegmentLoadingException { - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( 100000, null, null, @@ -745,6 +906,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null, null, + null, null ); final List ingestionSpecs = CompactionTask.createIngestionSchema( @@ -782,7 +944,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio @Test public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, SegmentLoadingException { - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( null, null, null, @@ -816,6 +978,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, Segm null, null, null, + null, null ); final List ingestionSpecs = CompactionTask.createIngestionSchema( @@ -853,7 +1016,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, Segm @Test public void testCreateIngestionSchemaWithNumShards() throws IOException, SegmentLoadingException { - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( null, null, null, @@ -887,6 +1050,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException, Segment null, null, null, + null, null ); final List ingestionSpecs = CompactionTask.createIngestionSchema( @@ -1413,7 +1577,7 @@ private void assertIngestionSchema( expectedDimensionsSpecs, expectedMetricsSpec, expectedSegmentIntervals, - new ParallelIndexTuningConfig( + new CompactionTask.CompactionTuningConfig( null, null, null, @@ -1447,6 +1611,7 @@ private void assertIngestionSchema( null, null, null, + null, null ), expectedSegmentGranularity, @@ -1459,7 +1624,7 @@ private void assertIngestionSchema( List expectedDimensionsSpecs, List expectedMetricsSpec, List expectedSegmentIntervals, - ParallelIndexTuningConfig expectedTuningConfig, + CompactionTask.CompactionTuningConfig expectedTuningConfig, Granularity expectedSegmentGranularity, Granularity expectedQueryGranularity ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java new file mode 100644 index 000000000000..2d496db89fe4 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java @@ -0,0 +1,202 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; + +public class CompactionTuningConfigTest +{ + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setup() + { + mapper.registerSubtypes(new NamedType(CompactionTask.CompactionTuningConfig.class, "compcation")); + } + + @Test + public void testSerdeDefault() throws IOException + { + final CompactionTask.CompactionTuningConfig tuningConfig = CompactionTask.CompactionTuningConfig.defaultConfig(); + final byte[] json = mapper.writeValueAsBytes(tuningConfig); + final ParallelIndexTuningConfig fromJson = (CompactionTask.CompactionTuningConfig) mapper.readValue(json, TuningConfig.class); + Assert.assertEquals(fromJson, tuningConfig); + } + + @Test + public void testSerdeWithNonZeroAwaitSegmentAvailabilityTimeoutMillis() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task"); + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( + null, + null, + null, + 10, + 1000L, + null, + null, + null, + null, + new DynamicPartitionsSpec(100, 100L), + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + new IndexSpec(), + 1, + false, + true, + 10000L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + null, + 250, + 100, + 20L, + new Duration(3600), + 128, + null, + null, + false, + null, + null, + null, + 5L + ); + } + + @Test + public void testSerdeWithZeroAwaitSegmentAvailabilityTimeoutMillis() + { + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( + null, + null, + null, + 10, + 1000L, + null, + null, + null, + null, + new DynamicPartitionsSpec(100, 100L), + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + new IndexSpec(), + 1, + false, + true, + 10000L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + null, + 250, + 100, + 20L, + new Duration(3600), + 128, + null, + null, + false, + null, + null, + null, + 0L + ); + Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); + } + + @Test + public void testSerdeWithNullAwaitSegmentAvailabilityTimeoutMillis() + { + final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( + null, + null, + null, + 10, + 1000L, + null, + null, + null, + null, + new DynamicPartitionsSpec(100, 100L), + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + new IndexSpec(), + 1, + false, + true, + 10000L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + null, + 250, + 100, + 20L, + new Duration(3600), + 128, + null, + null, + false, + null, + null, + null, + null + ); + Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); + } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(CompactionTask.CompactionTuningConfig.class) + .usingGetClass() + .verify(); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java index 52b6c72796cf..5d3a164ce66d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java @@ -84,7 +84,8 @@ public void testSerdeTuningConfigWithDynamicPartitionsSpec() throws IOException true, 10, 100, - 1234 + 1234, + 0L ); assertSerdeTuningConfig(tuningConfig); } @@ -120,7 +121,8 @@ public void testSerdeTuningConfigWithHashedPartitionsSpec() throws IOException true, 10, 100, - null + null, + -1L ); assertSerdeTuningConfig(tuningConfig); } @@ -156,7 +158,8 @@ public void testSerdeTuningConfigWithDeprecatedDynamicPartitionsSpec() throws IO true, 10, 100, - null + null, + 1L ); assertSerdeTuningConfig(tuningConfig); } @@ -192,7 +195,8 @@ public void testSerdeTuningConfigWithDeprecatedHashedPartitionsSpec() throws IOE true, 10, 100, - 1234 + 1234, + null ); assertSerdeTuningConfig(tuningConfig); } @@ -230,6 +234,7 @@ public void testForceGuaranteedRollupWithDynamicPartitionsSpec() true, 10, 100, + null, null ); } @@ -267,6 +272,7 @@ public void testBestEffortRollupWithHashedPartitionsSpec() true, 10, 100, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 3364c422440a..55c1f150c316 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -49,6 +49,7 @@ import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; @@ -71,6 +72,8 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -84,14 +87,18 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; +import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; @@ -973,6 +980,185 @@ public void testBestEffortRollup() throws Exception } } + @Test + public void testWaitForSegmentAvailabilityNoSegments() throws IOException + { + final File tmpDir = temporaryFolder.newFolder(); + + TaskToolbox mockToolbox = EasyMock.createMock(TaskToolbox.class); + List segmentsToWaitFor = new ArrayList<>(); + IndexTask indexTask = new IndexTask( + null, + null, + createDefaultIngestionSpec( + jsonMapper, + tmpDir, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + null, + createTuningConfigWithMaxRowsPerSegment(2, true), + false, + false + ), + null + ); + + EasyMock.replay(mockToolbox); + Assert.assertTrue(indexTask.waitForSegmentAvailability(mockToolbox, segmentsToWaitFor, 1000)); + EasyMock.verify(mockToolbox); + } + + @Test + public void testWaitForSegmentAvailabilityInvalidWaitTimeout() throws IOException + { + final File tmpDir = temporaryFolder.newFolder(); + + TaskToolbox mockToolbox = EasyMock.createMock(TaskToolbox.class); + List segmentsToWaitFor = new ArrayList<>(); + segmentsToWaitFor.add(EasyMock.createMock(DataSegment.class)); + IndexTask indexTask = new IndexTask( + null, + null, + createDefaultIngestionSpec( + jsonMapper, + tmpDir, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + null, + createTuningConfigWithMaxRowsPerSegment(2, true), + false, + false + ), + null + ); + + EasyMock.replay(mockToolbox); + Assert.assertFalse(indexTask.waitForSegmentAvailability(mockToolbox, segmentsToWaitFor, -1)); + EasyMock.verify(mockToolbox); + } + + @Test + public void testWaitForSegmentAvailabilityMultipleSegmentsTimeout() throws IOException + { + final File tmpDir = temporaryFolder.newFolder(); + + TaskToolbox mockToolbox = EasyMock.createMock(TaskToolbox.class); + SegmentHandoffNotifierFactory mockFactory = EasyMock.createMock(SegmentHandoffNotifierFactory.class); + SegmentHandoffNotifier mockNotifier = EasyMock.createMock(SegmentHandoffNotifier.class); + + DataSegment mockDataSegment1 = EasyMock.createMock(DataSegment.class); + DataSegment mockDataSegment2 = EasyMock.createMock(DataSegment.class); + List segmentsToWaitFor = new ArrayList<>(); + segmentsToWaitFor.add(mockDataSegment1); + segmentsToWaitFor.add(mockDataSegment2); + + IndexTask indexTask = new IndexTask( + null, + null, + createDefaultIngestionSpec( + jsonMapper, + tmpDir, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + null, + createTuningConfigWithMaxRowsPerSegment(2, true), + false, + false + ), + null + ); + + EasyMock.expect(mockDataSegment1.getInterval()).andReturn(Intervals.of("1970-01-01/2100-01-01")).once(); + EasyMock.expect(mockDataSegment1.getVersion()).andReturn("dummyString").once(); + EasyMock.expect(mockDataSegment1.getShardSpec()).andReturn(EasyMock.createMock(ShardSpec.class)).once(); + EasyMock.expect(mockDataSegment2.getInterval()).andReturn(Intervals.of("1970-01-01/2100-01-01")).once(); + EasyMock.expect(mockDataSegment2.getVersion()).andReturn("dummyString").once(); + EasyMock.expect(mockDataSegment2.getShardSpec()).andReturn(EasyMock.createMock(ShardSpec.class)).once(); + + EasyMock.expect(mockToolbox.getSegmentHandoffNotifierFactory()).andReturn(mockFactory).once(); + EasyMock.expect(mockDataSegment1.getDataSource()).andReturn("MockDataSource").once(); + EasyMock.expect(mockFactory.createSegmentHandoffNotifier("MockDataSource")).andReturn(mockNotifier).once(); + mockNotifier.start(); + EasyMock.expectLastCall().once(); + mockNotifier.registerSegmentHandoffCallback(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().andReturn(true).times(2); + mockNotifier.close(); + EasyMock.expectLastCall().once(); + + + EasyMock.replay(mockToolbox); + EasyMock.replay(mockDataSegment1, mockDataSegment2); + EasyMock.replay(mockFactory, mockNotifier); + + Assert.assertFalse(indexTask.waitForSegmentAvailability(mockToolbox, segmentsToWaitFor, 1000)); + EasyMock.verify(mockToolbox); + EasyMock.verify(mockDataSegment1, mockDataSegment2); + EasyMock.verify(mockFactory, mockNotifier); + } + + @Test + public void testWaitForSegmentAvailabilityMultipleSegmentsSuccess() throws IOException + { + final File tmpDir = temporaryFolder.newFolder(); + + TaskToolbox mockToolbox = EasyMock.createMock(TaskToolbox.class); + + DataSegment mockDataSegment1 = EasyMock.createMock(DataSegment.class); + DataSegment mockDataSegment2 = EasyMock.createMock(DataSegment.class); + List segmentsToWaitFor = new ArrayList<>(); + segmentsToWaitFor.add(mockDataSegment1); + segmentsToWaitFor.add(mockDataSegment2); + + IndexTask indexTask = new IndexTask( + null, + null, + createDefaultIngestionSpec( + jsonMapper, + tmpDir, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + null, + createTuningConfigWithMaxRowsPerSegment(2, true), + false, + false + ), + null + ); + + EasyMock.expect(mockDataSegment1.getInterval()).andReturn(Intervals.of("1970-01-01/1971-01-01")).once(); + EasyMock.expect(mockDataSegment1.getVersion()).andReturn("dummyString").once(); + EasyMock.expect(mockDataSegment1.getShardSpec()).andReturn(EasyMock.createMock(ShardSpec.class)).once(); + EasyMock.expect(mockDataSegment1.getId()).andReturn(SegmentId.dummy("MockDataSource")).once(); + EasyMock.expect(mockDataSegment2.getInterval()).andReturn(Intervals.of("1971-01-01/1972-01-01")).once(); + EasyMock.expect(mockDataSegment2.getVersion()).andReturn("dummyString").once(); + EasyMock.expect(mockDataSegment2.getShardSpec()).andReturn(EasyMock.createMock(ShardSpec.class)).once(); + EasyMock.expect(mockDataSegment2.getId()).andReturn(SegmentId.dummy("MockDataSource")).once(); + + EasyMock.expect(mockToolbox.getSegmentHandoffNotifierFactory()) + .andReturn(new NoopSegmentHandoffNotifierFactory()) + .once(); + EasyMock.expect(mockDataSegment1.getDataSource()).andReturn("MockDataSource").once(); + + EasyMock.replay(mockToolbox); + EasyMock.replay(mockDataSegment1, mockDataSegment2); + + Assert.assertTrue(indexTask.waitForSegmentAvailability(mockToolbox, segmentsToWaitFor, 30000)); + EasyMock.verify(mockToolbox); + EasyMock.verify(mockDataSegment1, mockDataSegment2); + } + private static void populateRollupTestData(File tmpFile) throws IOException { try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { @@ -1160,6 +1346,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception true, 7, 7, + null, null ); @@ -1293,6 +1480,7 @@ public void testMultipleParseExceptionsFailure() throws Exception true, 2, 5, + null, null ); @@ -1418,6 +1606,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc true, 2, 5, + null, null ); @@ -2231,6 +2420,7 @@ static IndexTuningConfig createTuningConfig( null, null, 1, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 5deecafe1f59..571566223a0e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -97,6 +97,8 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.RealtimeTuningConfig; @@ -104,8 +106,6 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java index d133b3ac8da2..65071b2ac194 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java @@ -55,7 +55,8 @@ public void testSerde() throws Exception ImmutableMap.of( "number", 1234 ), - "an error message" + "an error message", + true ) ); String report1serialized = jsonMapper.writeValueAsString(report1); @@ -64,7 +65,7 @@ public void testSerde() throws Exception IngestionStatsAndErrorsTaskReport.class ); Assert.assertEquals(report1, report2); - + Assert.assertEquals(report1.hashCode(), report2.hashCode()); Map reportMap1 = TaskReport.buildTaskReports(report1); String reportMapSerialized = jsonMapper.writeValueAsString(reportMap1); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index 4bbd34b3c41b..2d026d6a61dd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -269,7 +269,8 @@ public void testIndexTaskSerde() throws Exception null, null, null, - null + null, + 1L ) ), null @@ -307,6 +308,7 @@ public void testIndexTaskSerde() throws Exception Assert.assertEquals(taskTuningConfig.getNumShards(), task2TuningConfig.getNumShards()); Assert.assertEquals(taskTuningConfig.getMaxRowsPerSegment(), task2TuningConfig.getMaxRowsPerSegment()); Assert.assertEquals(taskTuningConfig.isReportParseExceptions(), task2TuningConfig.isReportParseExceptions()); + Assert.assertEquals(taskTuningConfig.getAwaitSegmentAvailabilityTimeoutMillis(), task2TuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); } @Test @@ -352,6 +354,7 @@ public void testIndexTaskwithResourceSerde() throws Exception null, null, null, + null, null ) ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 86d7ad145838..ef7c668b2df1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -169,6 +169,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase null, null, null, + null, null ); @@ -253,6 +254,7 @@ protected ParallelIndexTuningConfig newTuningConfig( null, null, null, + null, null ); } @@ -542,6 +544,7 @@ public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) ); objectMapper.registerSubtypes( new NamedType(ParallelIndexSupervisorTask.class, ParallelIndexSupervisorTask.TYPE), + new NamedType(CompactionTask.CompactionTuningConfig.class, CompactionTask.CompactionTuningConfig.TYPE), new NamedType(SinglePhaseSubTask.class, SinglePhaseSubTask.TYPE), new NamedType(PartialHashSegmentGenerateTask.class, PartialHashSegmentGenerateTask.TYPE), new NamedType(PartialRangeSegmentGenerateTask.class, PartialRangeSegmentGenerateTask.TYPE), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 843ca4cca3cb..489035c21676 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -192,6 +192,7 @@ private ParallelIndexSupervisorTask newTask( null, null, null, + null, null ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 224b034a0faa..abac3efecaea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -440,6 +440,7 @@ private TestSupervisorTask newTask( null, null, null, + null, null ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 949654871d0b..542d39cb4a7a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -274,6 +274,7 @@ ParallelIndexIngestionSpec build() null, null, null, + null, null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index b29b6cf8e06d..a28fb1efe1d0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -227,6 +227,7 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA false, null, null, + null, null ); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 6a15eeb1e79a..107470834e40 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -185,6 +185,7 @@ ParallelIndexTuningConfig build() logParseExceptions, maxParseExceptions, 25, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index 8434e2f69818..dae67edaf029 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -101,6 +101,7 @@ public void testSerdeWithMaxRowsPerSegment() false, null, null, + null, null ); final byte[] json = mapper.writeValueAsBytes(tuningConfig); @@ -146,6 +147,7 @@ public void testSerdeWithMaxNumConcurrentSubTasks() throws IOException false, null, null, + null, null ); final byte[] json = mapper.writeValueAsBytes(tuningConfig); @@ -191,6 +193,7 @@ public void testSerdeWithMaxNumSubTasks() throws IOException false, null, null, + null, null ); final byte[] json = mapper.writeValueAsBytes(tuningConfig); @@ -238,6 +241,7 @@ public void testSerdeWithMaxNumSubTasksAndMaxNumConcurrentSubTasks() false, null, null, + null, null ); } @@ -282,6 +286,7 @@ public void testConstructorWithHashedPartitionsSpecAndNonForceGuaranteedRollupFa false, null, null, + null, null ); } @@ -326,6 +331,7 @@ public void testConstructorWithSingleDimensionPartitionsSpecAndNonForceGuarantee false, null, null, + null, null ); } @@ -370,6 +376,7 @@ public void testConstructorWithDynamicPartitionsSpecAndForceGuaranteedRollupFail false, null, null, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index f08579607564..df9502ce1c71 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -364,6 +364,7 @@ public void testWith1MaxNumConcurrentSubTasks() null, null, null, + null, null ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 01a1d2424e0d..34ae0ab0e0e7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -71,13 +71,13 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 400f7da3dbbe..1213f9eaa2e9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -48,11 +48,11 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 8f220870c474..e3ba195fce1a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -113,6 +113,8 @@ import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.RealtimeTuningConfig; @@ -128,8 +130,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; @@ -766,6 +766,7 @@ public void testIndexTask() throws Exception null, null, null, + null, null ) ), @@ -849,6 +850,7 @@ public void testIndexTaskFailure() throws Exception null, null, null, + null, null ) ), @@ -1277,6 +1279,7 @@ public void testResumeTasks() throws Exception null, null, null, + null, null ) ), @@ -1387,6 +1390,7 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception null, null, null, + null, null ) ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index dbc223394c56..1bb33f0b457c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -42,9 +42,9 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index fdf7aa44c2f2..2a6e79316328 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -49,9 +49,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.IndexerZkConfig; import org.apache.druid.server.initialization.ServerConfig; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java index cf8f1601e320..6e40ee2346a3 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java @@ -223,7 +223,7 @@ public RowIngestionMetersTotals getTaskStats(String taskId) } } - private Map getTaskReport(String taskId) + public Map getTaskReport(String taskId) { try { StatusResponseHolder response = makeRequest( diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index cad6e2a73f79..990d573f23ec 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -495,6 +495,11 @@ private void loadData(String indexTask) throws Exception { String taskSpec = getResourceAsString(indexTask); taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName); + taskSpec = StringUtils.replace( + taskSpec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); final String taskID = indexer.submitTask(taskSpec); LOG.info("TaskID for loading index task %s", taskID); indexer.waitUntilTaskCompletes(taskID); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractAzureInputHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractAzureInputHadoopIndexTest.java index 2a6894ef91d0..a9c1cc1286ce 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractAzureInputHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractAzureInputHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; @@ -31,7 +32,7 @@ public abstract class AbstractAzureInputHadoopIndexTest extends AbstractITBatchI private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_azure_input_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - void doTest() throws Exception + void doTest(Pair segmentAvailabilityConfirmationPair) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -76,7 +77,8 @@ void doTest() throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractGcsInputHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractGcsInputHadoopIndexTest.java index 244a2f5438d0..0f7f1d16ff3d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractGcsInputHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractGcsInputHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; @@ -31,7 +32,7 @@ public abstract class AbstractGcsInputHadoopIndexTest extends AbstractITBatchInd private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_gcs_input_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - void doTest() throws Exception + void doTest(Pair segmentAvailabilityConfirmationPair) throws Exception { final String indexDatasource = "wikipedia_hadoop_index_test_" + UUID.randomUUID(); try ( @@ -72,7 +73,8 @@ void doTest() throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractS3InputHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractS3InputHadoopIndexTest.java index 86dcdfaacff7..13f0dd9e869d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractS3InputHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/AbstractS3InputHadoopIndexTest.java @@ -21,6 +21,7 @@ import com.google.inject.Inject; import org.apache.druid.common.aws.AWSCredentialsConfig; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; @@ -36,7 +37,7 @@ public abstract class AbstractS3InputHadoopIndexTest extends AbstractITBatchInde private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_s3_input_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - void doTest() throws Exception + void doTest(Pair segmentAvailabilityConfirmationPair) throws Exception { final String indexDatasource = "wikipedia_hadoop_index_test_" + UUID.randomUUID(); try ( @@ -89,7 +90,8 @@ void doTest() throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToAzureHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToAzureHadoopIndexTest.java index 47a5d9f8a858..16321337890d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToAzureHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToAzureHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -45,6 +46,6 @@ public class ITAzureInputToAzureHadoopIndexTest extends AbstractAzureInputHadoop { public void testGcsIndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToHdfsHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToHdfsHadoopIndexTest.java index 915c14c408f1..45cbf06c46dc 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToHdfsHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITAzureInputToHdfsHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -45,6 +46,6 @@ public class ITAzureInputToHdfsHadoopIndexTest extends AbstractAzureInputHadoopI { public void testGcsIndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToGcsHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToGcsHadoopIndexTest.java index 5b0e4ec95579..ca986b1dd089 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToGcsHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToGcsHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -46,6 +47,6 @@ public class ITGcsInputToGcsHadoopIndexTest extends AbstractGcsInputHadoopIndexT { public void testGcsIndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToHdfsHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToHdfsHadoopIndexTest.java index 314d5d7164d6..c39792b5bf8c 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToHdfsHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToHdfsHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -46,6 +47,6 @@ public class ITGcsInputToHdfsHadoopIndexTest extends AbstractGcsInputHadoopIndex { public void testGcsIndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java index 898c7d58845d..a61fae4e0e73 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java @@ -20,12 +20,17 @@ package org.apache.druid.tests.hadoop; import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; 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.java.util.common.logger.Logger; +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.apache.druid.tests.indexer.AbstractITBatchIndexTest; import org.apache.druid.timeline.partition.HashPartitionFunction; @@ -68,6 +73,14 @@ public class ITHadoopIndexTest extends AbstractITBatchIndexTest private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json"; private static final String REINDEX_DATASOURCE = "wikipedia_hadoop_reindex_test"; + 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() { @@ -114,7 +127,8 @@ public void testLegacyITHadoopIndexTest() throws Exception BATCH_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } @@ -142,6 +156,11 @@ public void testIndexData(DimensionBasedPartitionsSpec partitionsSpec) throws Ex "%%PARTITIONS_SPEC%%", jsonMapper.writeValueAsString(partitionsSpec) ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString(0) + ); return spec; } @@ -157,14 +176,131 @@ public void testIndexData(DimensionBasedPartitionsSpec partitionsSpec) throws Ex INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( indexDatasource, reindexDatasource, REINDEX_TASK, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test Hadoop Batch Ingestion with 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 job. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailability() throws Exception + { + String indexDatasource = INDEX_DATASOURCE + "_" + UUID.randomUUID(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + final Function specPathsTransform = spec -> { + try { + String path = "/batch_index/json"; + spec = StringUtils.replace( + spec, + "%%INPUT_PATHS%%", + path + ); + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString( + new HashedPartitionsSpec(3, null, null) + ) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString(600000) + ); + + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + specPathsTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test Hadoop Batch Indexing with non-zero value for awaitSegmentAvailabilityTimeoutMillis. The coordinator + * is paused when the task runs. This should result in a successful task with a flag in the task report indicating + * that we did not confirm segment availability. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + String indexDatasource = INDEX_DATASOURCE + "_" + UUID.randomUUID(); + + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + final Function specPathsTransform = spec -> { + try { + String path = "/batch_index/json"; + spec = StringUtils.replace( + spec, + "%%INPUT_PATHS%%", + path + ); + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString( + new HashedPartitionsSpec(3, null, null) + ) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString(1) + ); + + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + specPathsTransform, + INDEX_QUERIES_RESOURCE, + false, + false, + false, + new Pair<>(true, false) + ); + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); + ITRetryUtil.retryUntilTrue( + () -> coordinatorClient.areSegmentsLoaded(indexDatasource + config.getExtraDatasourceNameSuffix()), "Segment Load For: " + indexDatasource + config.getExtraDatasourceNameSuffix() ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToHdfsHadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToHdfsHadoopIndexTest.java index 01aa8e006d1b..d59d3d869187 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToHdfsHadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToHdfsHadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -44,6 +45,6 @@ public class ITS3InputToHdfsHadoopIndexTest extends AbstractS3InputHadoopIndexTe @Test() public void testS3IndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToS3HadoopIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToS3HadoopIndexTest.java index 27cedcdc75dd..2e1667924f4e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToS3HadoopIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToS3HadoopIndexTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.hadoop; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -44,6 +45,6 @@ public class ITS3InputToS3HadoopIndexTest extends AbstractS3InputHadoopIndexTest @Test() public void testS3IndexData() throws Exception { - doTest(); + doTest(new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractAzureInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractAzureInputSourceParallelIndexTest.java index 8ca08a0ef1e5..9ce161f6ccd9 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractAzureInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractAzureInputSourceParallelIndexTest.java @@ -68,7 +68,10 @@ public static Object[][] resources() }; } - void doTest(Pair azureInputSource) throws Exception + void doTest( + Pair azureInputSource, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -125,7 +128,8 @@ void doTest(Pair azureInputSource) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractGcsInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractGcsInputSourceParallelIndexTest.java index 805619a2d182..f720a5d5943e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractGcsInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractGcsInputSourceParallelIndexTest.java @@ -68,7 +68,10 @@ public static Object[][] resources() }; } - void doTest(Pair gcsInputSource) throws Exception + void doTest( + Pair gcsInputSource, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -125,7 +128,8 @@ void doTest(Pair gcsInputSource) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractHdfsInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractHdfsInputSourceParallelIndexTest.java index 742238accb05..eaedde10bbfe 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractHdfsInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractHdfsInputSourceParallelIndexTest.java @@ -58,7 +58,11 @@ public static Object[][] resources() }; } - void doTest(Pair hdfsInputSource, InputFormatDetails inputFormatDetails) throws Exception + void doTest( + Pair hdfsInputSource, + InputFormatDetails inputFormatDetails, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -115,7 +119,8 @@ void doTest(Pair hdfsInputSource, InputFormatDetails inputFormatDe INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index 65fbbe7dc941..acab50df3d35 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -23,6 +23,9 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.druid.indexer.partitions.SecondaryPartitionType; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionCardinalityTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; @@ -31,6 +34,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.java.util.common.ISE; 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.logger.Logger; import org.apache.druid.testing.IntegrationTestingConfig; @@ -103,10 +107,20 @@ protected void doIndexTest( String queryFilePath, boolean waitForNewVersion, boolean runTestQueries, - boolean waitForSegmentsToLoad + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair ) throws IOException { - doIndexTest(dataSource, indexTaskFilePath, Function.identity(), queryFilePath, waitForNewVersion, runTestQueries, waitForSegmentsToLoad); + doIndexTest( + dataSource, + indexTaskFilePath, + Function.identity(), + queryFilePath, + waitForNewVersion, + runTestQueries, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); } protected void doIndexTest( @@ -116,7 +130,8 @@ protected void doIndexTest( String queryFilePath, boolean waitForNewVersion, boolean runTestQueries, - boolean waitForSegmentsToLoad + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair ) throws IOException { final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); @@ -128,7 +143,13 @@ protected void doIndexTest( ) ); - submitTaskAndWait(taskSpec, fullDatasourceName, waitForNewVersion, waitForSegmentsToLoad); + submitTaskAndWait( + taskSpec, + fullDatasourceName, + waitForNewVersion, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); if (runTestQueries) { doTestQuery(dataSource, queryFilePath); } @@ -164,10 +185,18 @@ protected void doReindexTest( String baseDataSource, String reindexDataSource, String reindexTaskFilePath, - String queryFilePath + String queryFilePath, + Pair segmentAvailabilityConfirmationPair ) throws IOException { - doReindexTest(baseDataSource, reindexDataSource, Function.identity(), reindexTaskFilePath, queryFilePath); + doReindexTest( + baseDataSource, + reindexDataSource, + Function.identity(), + reindexTaskFilePath, + queryFilePath, + segmentAvailabilityConfirmationPair + ); } void doReindexTest( @@ -175,7 +204,8 @@ void doReindexTest( String reindexDataSource, Function taskSpecTransform, String reindexTaskFilePath, - String queryFilePath + String queryFilePath, + Pair segmentAvailabilityConfirmationPair ) throws IOException { final String fullBaseDatasourceName = baseDataSource + config.getExtraDatasourceNameSuffix(); @@ -195,7 +225,13 @@ void doReindexTest( taskSpec = taskSpecTransform.apply(taskSpec); - submitTaskAndWait(taskSpec, fullReindexDatasourceName, false, true); + submitTaskAndWait( + taskSpec, + fullReindexDatasourceName, + false, + true, + segmentAvailabilityConfirmationPair + ); try { String queryResponseTemplate; try { @@ -231,15 +267,32 @@ void doIndexTestSqlTest( String indexTaskFilePath, String queryFilePath ) throws IOException + { + doIndexTestSqlTest( + dataSource, + indexTaskFilePath, + queryFilePath, + Function.identity() + ); + } + void doIndexTestSqlTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath, + Function taskSpecTransform + ) throws IOException { final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); - final String taskSpec = StringUtils.replace( - getResourceAsString(indexTaskFilePath), - "%%DATASOURCE%%", - fullDatasourceName + final String taskSpec = taskSpecTransform.apply( + StringUtils.replace( + getResourceAsString(indexTaskFilePath), + "%%DATASOURCE%%", + fullDatasourceName + ) ); - submitTaskAndWait(taskSpec, fullDatasourceName, false, true); + Pair dummyPair = new Pair<>(false, false); + submitTaskAndWait(taskSpec, fullDatasourceName, false, true, dummyPair); try { sqlQueryHelper.testQueriesFromFile(queryFilePath); } @@ -253,7 +306,8 @@ private void submitTaskAndWait( String taskSpec, String dataSourceName, boolean waitForNewVersion, - boolean waitForSegmentsToLoad + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair ) { final List oldVersions = waitForNewVersion ? coordinator.getAvailableSegments(dataSourceName) : null; @@ -280,6 +334,18 @@ private void submitTaskAndWait( ); } + if (segmentAvailabilityConfirmationPair.lhs != null && segmentAvailabilityConfirmationPair.lhs) { + TaskReport reportRaw = indexer.getTaskReport(taskID).get("ingestionStatsAndErrors"); + IngestionStatsAndErrorsTaskReport report = (IngestionStatsAndErrorsTaskReport) reportRaw; + IngestionStatsAndErrorsTaskReportData reportData = (IngestionStatsAndErrorsTaskReportData) report.getPayload(); + if (segmentAvailabilityConfirmationPair.rhs != null) { + Assert.assertEquals( + Boolean.valueOf(reportData.isSegmentAvailabilityConfirmed()), + segmentAvailabilityConfirmationPair.rhs + ); + } + } + // IT*ParallelIndexTest do a second round of ingestion to replace segements in an existing // data source. For that second round we need to make sure the coordinator actually learned // about the new segments befor waiting for it to report that all segments are loaded; otherwise diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index e97f52472a58..6c5067b70c39 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -105,6 +105,11 @@ protected void loadData(String indexTask, final String fullDatasourceName) throw { String taskSpec = getResourceAsString(indexTask); taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName); + taskSpec = StringUtils.replace( + taskSpec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); final String taskID = indexer.submitTask(taskSpec); LOG.info("TaskID for loading index task %s", taskID); indexer.waitUntilTaskCompletes(taskID); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractLocalInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractLocalInputSourceParallelIndexTest.java index 8203d03e68c9..79dc7ab49534 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractLocalInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractLocalInputSourceParallelIndexTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nonnull; @@ -34,12 +35,19 @@ public abstract class AbstractLocalInputSourceParallelIndexTest extends Abstract private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - public void doIndexTest(InputFormatDetails inputFormatDetails) throws Exception + public void doIndexTest( + InputFormatDetails inputFormatDetails, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { - doIndexTest(inputFormatDetails, ImmutableMap.of()); + doIndexTest(inputFormatDetails, ImmutableMap.of(), segmentAvailabilityConfirmationPair); } - public void doIndexTest(InputFormatDetails inputFormatDetails, @Nonnull Map extraInputFormatMap) throws Exception + public void doIndexTest( + InputFormatDetails inputFormatDetails, + @Nonnull Map extraInputFormatMap, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); Map inputFormatMap = new ImmutableMap.Builder().putAll(extraInputFormatMap) @@ -99,7 +107,8 @@ public void doIndexTest(InputFormatDetails inputFormatDetails, @Nonnull Map inputSource) throws Exception + void doTest( + Pair inputSource, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -125,7 +128,8 @@ void doTest(Pair inputSource) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractS3InputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractS3InputSourceParallelIndexTest.java index 3962c8d82b99..1d378d5dee67 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractS3InputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractS3InputSourceParallelIndexTest.java @@ -68,7 +68,10 @@ public static Object[][] resources() }; } - void doTest(Pair s3InputSource) throws Exception + void doTest( + Pair s3InputSource, + Pair segmentAvailabilityConfirmationPair + ) throws Exception { final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); try ( @@ -125,7 +128,8 @@ void doTest(Pair s3InputSource) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java index f523ddc16f56..5e9071e45507 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java @@ -25,6 +25,7 @@ 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.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; @@ -96,11 +97,11 @@ public void doIndexTest(List partitionsSpecList, List e final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); ) { // Submit initial ingestion task - submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(0), false); + submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(0), false, new Pair<>(false, false)); verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(0)); doTestQuery(indexDatasource, INDEX_QUERIES_INITIAL_INGESTION_RESOURCE); // Submit append ingestion task - submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(1), true); + submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(1), true, new Pair<>(false, false)); verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(1)); doTestQuery(indexDatasource, INDEX_QUERIES_POST_APPEND_PRE_COMPACT_RESOURCE); // Submit compaction task @@ -115,7 +116,8 @@ public void doIndexTest(List partitionsSpecList, List e private void submitIngestionTaskAndVerify( String indexDatasource, PartitionsSpec partitionsSpec, - boolean appendToExisting + boolean appendToExisting, + Pair segmentAvailabilityConfirmationPair ) throws Exception { InputFormatDetails inputFormatDetails = InputFormatDetails.JSON; @@ -180,7 +182,8 @@ private void submitIngestionTaskAndVerify( null, false, false, - true + true, + segmentAvailabilityConfirmationPair ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToAzureParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToAzureParallelIndexTest.java index e8594ccf515e..cbb8bba877ae 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToAzureParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToAzureParallelIndexTest.java @@ -44,6 +44,6 @@ public class ITAzureToAzureParallelIndexTest extends AbstractAzureInputSourcePar @Test(dataProvider = "resources") public void testAzureIndexData(Pair azureInputSource) throws Exception { - doTest(azureInputSource); + doTest(azureInputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToHdfsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToHdfsParallelIndexTest.java index cd1d76a0e32f..ca7ee72e02f6 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToHdfsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAzureToHdfsParallelIndexTest.java @@ -46,6 +46,6 @@ public class ITAzureToHdfsParallelIndexTest extends AbstractAzureInputSourcePara @Test(dataProvider = "resources") public void testAzureIndexData(Pair azureInputSource) throws Exception { - doTest(azureInputSource); + doTest(azureInputSource, new Pair<>(false, false)); } } 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 index 373bef35f5f4..a463e389d79e 100644 --- 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 @@ -20,10 +20,15 @@ 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; @@ -49,6 +54,14 @@ public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTes 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() { @@ -75,6 +88,11 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception "%%FORCE_GUARANTEED_ROLLUP%%", Boolean.toString(false) ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); return StringUtils.replace( spec, "%%PARTITIONS_SPEC%%", @@ -93,7 +111,8 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); // Index again, this time only choosing the second data file, and without explicit intervals chosen. @@ -105,7 +124,8 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception REINDEX_QUERIES_RESOURCE, true, true, - true + true, + new Pair<>(false, false) ); doReindexTest( @@ -113,7 +133,8 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_INGEST_SEGMENT_DATASOURCE, rollupTransform, INDEX_INGEST_SEGMENT_TASK, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); // with DruidInputSource instead of IngestSegmentFirehose @@ -122,7 +143,118 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_DRUID_INPUT_SOURCE_DATASOURCE, rollupTransform, INDEX_DRUID_INPUT_SOURCE_TASK, - REINDEX_QUERIES_RESOURCE + 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/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java index 3a0583144bf5..c1350f550684 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java @@ -19,6 +19,8 @@ package org.apache.druid.tests.indexer; +import com.fasterxml.jackson.core.JsonProcessingException; +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; @@ -59,13 +61,28 @@ public void testIndexData() throws Exception throw new RuntimeException(e); } }; + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + doIndexTest( INDEX_DATASOURCE, INDEX_TASK, + transform, INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doIndexTest( COMBINING_INDEX_DATASOURCE, @@ -74,7 +91,8 @@ public void testIndexData() throws Exception COMBINING_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java index dacd9851230f..ce49b457b0aa 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +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; @@ -108,7 +109,8 @@ public void testIndexData() throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doIndexTest( COMBINING_INDEX_DATASOURCE, @@ -117,7 +119,8 @@ public void testIndexData() throws Exception COMBINING_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } 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 index 798332270bd8..957c8a5522c4 100644 --- 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 @@ -190,6 +190,13 @@ private void loadDataAndCompact( 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); compactData(compactionResource, newSegmentGranularity, null); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToGcsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToGcsParallelIndexTest.java index 75949d7bc095..927cb3ee2812 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToGcsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToGcsParallelIndexTest.java @@ -45,6 +45,6 @@ public class ITGcsToGcsParallelIndexTest extends AbstractGcsInputSourceParallelI @Test(dataProvider = "resources") public void testGcsIndexData(Pair gcsInputSource) throws Exception { - doTest(gcsInputSource); + doTest(gcsInputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToHdfsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToHdfsParallelIndexTest.java index ec2e09803264..2c9a42e1a6a2 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToHdfsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITGcsToHdfsParallelIndexTest.java @@ -47,6 +47,6 @@ public class ITGcsToHdfsParallelIndexTest extends AbstractGcsInputSourceParallel @Test(dataProvider = "resources") public void testGcsIndexData(Pair gcsInputSource) throws Exception { - doTest(gcsInputSource); + doTest(gcsInputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToAzureParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToAzureParallelIndexTest.java index 0abd874d3e41..e0cee7f8a55a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToAzureParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToAzureParallelIndexTest.java @@ -42,6 +42,6 @@ public class ITHdfsToAzureParallelIndexTest extends AbstractHdfsInputSourceParal @Test(dataProvider = "resources") public void testHdfsIndexData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.JSON); + doTest(hdfsInputSource, InputFormatDetails.JSON, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToGcsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToGcsParallelIndexTest.java index c30e1a545094..5e302a14f770 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToGcsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToGcsParallelIndexTest.java @@ -43,6 +43,6 @@ public class ITHdfsToGcsParallelIndexTest extends AbstractHdfsInputSourceParalle @Test(dataProvider = "resources") public void testHdfsIndexData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.JSON); + doTest(hdfsInputSource, InputFormatDetails.JSON, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToHdfsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToHdfsParallelIndexTest.java index 71f227c8dbed..42073544d148 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToHdfsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToHdfsParallelIndexTest.java @@ -41,18 +41,18 @@ public class ITHdfsToHdfsParallelIndexTest extends AbstractHdfsInputSourceParall @Test(dataProvider = "resources") public void testHdfsIndexJsonData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.JSON); + doTest(hdfsInputSource, InputFormatDetails.JSON, new Pair<>(false, false)); } @Test(dataProvider = "resources") public void testHdfsIndexOrcData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.ORC); + doTest(hdfsInputSource, InputFormatDetails.ORC, new Pair<>(false, false)); } @Test(dataProvider = "resources") public void testHdfsIndexParquetData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.PARQUET); + doTest(hdfsInputSource, InputFormatDetails.PARQUET, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToS3ParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToS3ParallelIndexTest.java index 564c0c19f447..5e674fda106d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToS3ParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHdfsToS3ParallelIndexTest.java @@ -42,6 +42,6 @@ public class ITHdfsToS3ParallelIndexTest extends AbstractHdfsInputSourceParallel @Test(dataProvider = "resources") public void testHdfsIndexData(Pair hdfsInputSource) throws Exception { - doTest(hdfsInputSource, InputFormatDetails.JSON); + doTest(hdfsInputSource, InputFormatDetails.JSON, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java index 69d1eeaaa94f..3d63ea2172ae 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.indexer; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -46,7 +47,8 @@ public void doTest() throws IOException INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index 9563306a0a70..ef24182f95cb 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -19,12 +19,20 @@ package org.apache.druid.tests.indexer; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.inject.Inject; +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.annotations.Guice; import org.testng.annotations.Test; import java.io.Closeable; +import java.util.function.Function; @Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE}) @Guice(moduleFactory = DruidTestModuleFactory.class) @@ -57,6 +65,14 @@ public class ITIndexerTest extends AbstractITBatchIndexTest private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_TASK = "/indexer/wikipedia_index_with_merge_column_limit_task.json"; private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE = "wikipedia_index_with_merge_column_limit_test"; + 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 { @@ -67,25 +83,43 @@ public void testIndexData() throws Exception final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + doIndexTest( INDEX_DATASOURCE, INDEX_TASK, + transform, INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( INDEX_DATASOURCE, reindexDatasource, REINDEX_TASK, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); doReindexTest( INDEX_DATASOURCE, reindexDatasourceWithDruidInputSource, REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); } } @@ -106,19 +140,22 @@ public void testReIndexDataWithTimestamp() throws Exception INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( INDEX_WITH_TIMESTAMP_DATASOURCE, reindexDatasource, REINDEX_TASK, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); doReindexTest( INDEX_WITH_TIMESTAMP_DATASOURCE, reindexDatasourceWithDruidInputSource, REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); } } @@ -139,19 +176,102 @@ public void testMERGEIndexData() throws Exception MERGE_INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( MERGE_INDEX_DATASOURCE, reindexDatasource, MERGE_REINDEX_TASK, - MERGE_REINDEX_QUERIES_RESOURCE + MERGE_REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); doReindexTest( MERGE_INDEX_DATASOURCE, reindexDatasourceWithDruidInputSource, MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, - MERGE_INDEX_QUERIES_RESOURCE + MERGE_INDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test that task reports indicate the ingested segments were loaded before the configured timeout expired. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailability() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("600000") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test that the task still succeeds if the segments do not become available before the configured wait timeout + * expires. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("1") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + 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" ); } } @@ -169,7 +289,8 @@ public void testIndexWithMergeColumnLimitData() throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java index a0c1014105a9..34fa4d908188 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -58,42 +59,42 @@ public void testAvroInputFormatIndexDataIngestionSpecWithSchema() throws Excepti "type", "record", "name", "wikipedia", "fields", fieldList); - doIndexTest(InputFormatDetails.AVRO, ImmutableMap.of("schema", schema)); + doIndexTest(InputFormatDetails.AVRO, ImmutableMap.of("schema", schema), new Pair<>(false, false)); } @Test public void testAvroInputFormatIndexDataIngestionSpecWithoutSchema() throws Exception { - doIndexTest(InputFormatDetails.AVRO); + doIndexTest(InputFormatDetails.AVRO, new Pair<>(false, false)); } @Test public void testJsonInputFormatIndexDataIngestionSpecWithSchema() throws Exception { - doIndexTest(InputFormatDetails.JSON); + doIndexTest(InputFormatDetails.JSON, new Pair<>(false, false)); } @Test public void testTsvInputFormatIndexDataIngestionSpecWithSchema() throws Exception { - doIndexTest(InputFormatDetails.TSV, ImmutableMap.of("findColumnsFromHeader", true)); + doIndexTest(InputFormatDetails.TSV, ImmutableMap.of("findColumnsFromHeader", true), new Pair<>(false, false)); } @Test public void testParquetInputFormatIndexDataIngestionSpecWithSchema() throws Exception { - doIndexTest(InputFormatDetails.PARQUET); + doIndexTest(InputFormatDetails.PARQUET, new Pair<>(false, false)); } @Test public void testOrcInputFormatIndexDataIngestionSpecWithSchema() throws Exception { - doIndexTest(InputFormatDetails.ORC); + doIndexTest(InputFormatDetails.ORC, new Pair<>(false, false)); } @Test public void testCsvInputFormatIndexDataIngestionSpecWithSchema() throws Exception { - doIndexTest(InputFormatDetails.CSV, ImmutableMap.of("findColumnsFromHeader", true)); + doIndexTest(InputFormatDetails.CSV, ImmutableMap.of("findColumnsFromHeader", true), new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOssToOssParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOssToOssParallelIndexTest.java index ea989598b4bd..003b41452a57 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOssToOssParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOssToOssParallelIndexTest.java @@ -44,6 +44,6 @@ public class ITOssToOssParallelIndexTest extends AbstractOssInputSourceParallelI @Test(dataProvider = "resources") public void testAliyunOssIndexData(Pair ossInputSource) throws Exception { - doTest(ossInputSource); + doTest(ossInputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java index f47a9663c79b..1ff7b18b6767 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +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; @@ -150,7 +151,8 @@ private void submitIngestionTaskAndVerify( null, false, false, - true + true, + new Pair<>(false, false) ); } } 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 index fef886dda2f7..1bb1a79bd49c 100644 --- 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 @@ -23,6 +23,7 @@ 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; @@ -77,6 +78,11 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception "%%FORCE_GUARANTEED_ROLLUP%%", Boolean.toString(true) ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); return StringUtils.replace( spec, "%%PARTITIONS_SPEC%%", @@ -95,7 +101,8 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( @@ -103,7 +110,9 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_INGEST_SEGMENT_DATASOURCE, rollupTransform, INDEX_INGEST_SEGMENT_TASK, - INDEX_QUERIES_RESOURCE + INDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); // with DruidInputSource instead of IngestSegmentFirehose @@ -112,7 +121,8 @@ public void testIndexData(PartitionsSpec partitionsSpec) throws Exception INDEX_DRUID_INPUT_SOURCE_DATASOURCE, rollupTransform, INDEX_DRUID_INPUT_SOURCE_TASK, - INDEX_QUERIES_RESOURCE + INDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3OverrideCredentialsIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3OverrideCredentialsIndexTest.java index ec949f17fed3..79912218ce9e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3OverrideCredentialsIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3OverrideCredentialsIndexTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; +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; @@ -129,7 +130,8 @@ public void testS3WithValidOverrideCredentialsIndexDataShouldSucceed() throws Ex INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToHdfsParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToHdfsParallelIndexTest.java index b57f7bc975e9..a5869447156e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToHdfsParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToHdfsParallelIndexTest.java @@ -46,6 +46,6 @@ public class ITS3ToHdfsParallelIndexTest extends AbstractS3InputSourceParallelIn @Test(dataProvider = "resources") public void testS3IndexData(Pair s3InputSource) throws Exception { - doTest(s3InputSource); + doTest(s3InputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToS3ParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToS3ParallelIndexTest.java index e8f8b361fd64..0a85d38fe16f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToS3ParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITS3ToS3ParallelIndexTest.java @@ -44,6 +44,6 @@ public class ITS3ToS3ParallelIndexTest extends AbstractS3InputSourceParallelInde @Test(dataProvider = "resources") public void testS3IndexData(Pair s3InputSource) throws Exception { - doTest(s3InputSource); + doTest(s3InputSource, new Pair<>(false, false)); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSqlInputSourceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSqlInputSourceTest.java index 92e9db809916..30c0aedd7688 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSqlInputSourceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSqlInputSourceTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +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; @@ -95,7 +96,8 @@ public void testIndexData(List sqlQueries) throws Exception INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java index df742e4698a5..7582ae46bf14 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java @@ -19,6 +19,8 @@ package org.apache.druid.tests.indexer; +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; @@ -26,6 +28,7 @@ import org.testng.annotations.Test; import java.io.Closeable; +import java.util.function.Function; @Test(groups = TestNGGroup.BATCH_INDEX) @Guice(moduleFactory = DruidTestModuleFactory.class) @@ -43,10 +46,25 @@ public void testIndexData() throws Exception try ( final Closeable ignored = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()) ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + doIndexTestSqlTest( INDEX_DATASOURCE, INDEX_TASK, - SYSTEM_QUERIES_RESOURCE + SYSTEM_QUERIES_RESOURCE, + transform ); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java index 269c74d4cd19..a75f9a933c5a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java @@ -19,7 +19,10 @@ package org.apache.druid.tests.indexer; +import com.fasterxml.jackson.core.JsonProcessingException; import com.google.inject.Inject; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; @@ -30,6 +33,7 @@ import java.io.Closeable; import java.util.concurrent.TimeUnit; +import java.util.function.Function; @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITTestCoordinatorPausedTest extends AbstractITBatchIndexTest @@ -53,13 +57,29 @@ public void testCoordinatorPause() throws Exception final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()) ) { coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + doIndexTest( INDEX_DATASOURCE, INDEX_TASK, + transform, INDEX_QUERIES_RESOURCE, false, false, - false + false, + new Pair<>(false, false) ); TimeUnit.MINUTES.sleep(3); if (coordinatorClient.areSegmentsLoaded(INDEX_DATASOURCE)) { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java index 4a5116402873..e98623dcd899 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java @@ -19,6 +19,7 @@ package org.apache.druid.tests.indexer; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; @@ -56,13 +57,15 @@ public void testIndexAndReIndexWithTransformSpec() throws IOException INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( INDEX_DATASOURCE, reindexDatasourceWithDruidInputSource, REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); } } @@ -83,13 +86,15 @@ public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOExcep INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); doReindexTest( INDEX_DATASOURCE, reindexDatasource, REINDEX_TASK, - REINDEX_QUERIES_RESOURCE + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) ); } } @@ -108,7 +113,8 @@ public void testIndexWithFirehoseAndTransforms() throws IOException INDEX_QUERIES_RESOURCE, false, true, - true + true, + new Pair<>(false, false) ); } } diff --git a/integration-tests/src/test/resources/hadoop/wikipedia_hadoop_index_task.json b/integration-tests/src/test/resources/hadoop/wikipedia_hadoop_index_task.json index d30214b93141..76600f57cf87 100644 --- a/integration-tests/src/test/resources/hadoop/wikipedia_hadoop_index_task.json +++ b/integration-tests/src/test/resources/hadoop/wikipedia_hadoop_index_task.json @@ -80,6 +80,7 @@ "tuningConfig": { "type": "hadoop", "partitionsSpec": %%PARTITIONS_SPEC%%, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%%, "jobProperties": { "fs.permissions.umask-mode": "022", "fs.default.name" : "hdfs://druid-it-hadoop:9000", diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json index 1364567fbff2..1e6952f56f61 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json @@ -79,7 +79,8 @@ }, "tuningConfig": { "type": "index", - "maxRowsPerSegment": 3 + "maxRowsPerSegment": 3, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% } } } \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json index 4781d39b0249..e83b1109da02 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -70,7 +70,8 @@ "type": "maxSize", "maxSplitSize": 1 }, - "partitionsSpec": %%PARTITIONS_SPEC%% + "partitionsSpec": %%PARTITIONS_SPEC%%, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% } } } \ No newline at end of file diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifier.java similarity index 94% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java rename to server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifier.java index 2e97258a52da..fa884022a3a3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifier.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.coordinator.CoordinatorClient; @@ -60,7 +60,7 @@ public CoordinatorBasedSegmentHandoffNotifier( @Override public boolean registerSegmentHandoffCallback(SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable) { - log.info("Adding SegmentHandoffCallback for dataSource[%s] Segment[%s]", dataSource, descriptor); + log.debug("Adding SegmentHandoffCallback for dataSource[%s] Segment[%s]", dataSource, descriptor); Pair prev = handOffCallbacks.putIfAbsent( descriptor, new Pair<>(exec, handOffRunnable) @@ -106,7 +106,7 @@ void checkForSegmentHandoffs() handOffComplete = isHandOffComplete(loadedSegments, descriptor); } if (handOffComplete) { - log.info("Segment Handoff complete for dataSource[%s] Segment[%s]", dataSource, descriptor); + log.debug("Segment Handoff complete for dataSource[%s] Segment[%s]", dataSource, descriptor); entry.getValue().lhs.execute(entry.getValue().rhs); itr.remove(); } @@ -122,7 +122,7 @@ void checkForSegmentHandoffs() } } if (!handOffCallbacks.isEmpty()) { - log.info("Still waiting for Handoff for Segments : [%s]", handOffCallbacks.keySet()); + log.info("Still waiting for Handoff for [%d] Segments", handOffCallbacks.size()); } } catch (Throwable t) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierConfig.java b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierConfig.java similarity index 95% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierConfig.java rename to server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierConfig.java index 285a4929d0d3..2ee769b38570 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierConfig.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Duration; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierFactory.java b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierFactory.java similarity index 96% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierFactory.java rename to server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierFactory.java index d92bea767526..aecb8e8f35be 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierFactory.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import com.google.inject.Inject; import org.apache.druid.client.coordinator.CoordinatorClient; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifier.java b/server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifier.java similarity index 97% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifier.java rename to server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifier.java index 9c77b1f07fcc..022e205727d3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifier.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifier.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import org.apache.druid.query.SegmentDescriptor; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifierFactory.java b/server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifierFactory.java similarity index 94% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifierFactory.java rename to server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifierFactory.java index 96aeb5fcd3a9..12cfde8d9084 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SegmentHandoffNotifierFactory.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/SegmentHandoffNotifierFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; public interface SegmentHandoffNotifierFactory diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java index 156063ec4499..266204520ce5 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java @@ -40,6 +40,7 @@ public interface TuningConfig int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; boolean DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK = false; + long DEFAULT_AWAIT_SEGMENT_AVAILABILITY_TIMEOUT_MILLIS = 0L; /** * The incremental index implementation to use diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java index 2aa5d43a2eaa..bea2890fcf0f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java @@ -43,6 +43,7 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; @@ -52,7 +53,6 @@ import org.apache.druid.segment.realtime.plumber.Committers; import org.apache.druid.segment.realtime.plumber.Plumber; import org.apache.druid.segment.realtime.plumber.RejectionPolicy; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.VersioningPolicy; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java index 99adff0a1ba4..8d188111f64e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java @@ -22,13 +22,13 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.SegmentPublisher; import org.apache.druid.segment.realtime.plumber.Plumber; import org.apache.druid.segment.realtime.plumber.PlumberSchool; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.coordination.DataSegmentAnnouncer; public class AppenderatorPlumberSchool implements PlumberSchool diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index f510df37ac6a..53212e29ac9e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -36,11 +36,11 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java index 536c1aef7188..9a51143d64b2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java @@ -21,6 +21,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import java.util.concurrent.Executor; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 578f3f27b323..0c97d856a0e8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java index f4d5c43f17af..3f808f9817b7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -31,6 +31,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.join.JoinableFactory; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java rename to server/src/test/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierTest.java index f5534bd46a3b..31933aa4a824 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/org/apache/druid/segment/handoff/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import com.google.common.collect.Sets; import org.apache.druid.client.ImmutableSegmentLoadInfo; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index 7cba86e1fa0f..78210d3bc191 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -20,12 +20,12 @@ package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.realtime.SegmentPublisher; import org.apache.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; import org.apache.druid.segment.realtime.plumber.NoopRejectionPolicyFactory; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.easymock.EasyMock; import org.junit.Assert; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index a3614afea30f..fa1f7a9472f9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -38,10 +38,10 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index deb80b3cc169..f2944224a1e7 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -46,6 +46,8 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 44cc2005dc15..b6b4336162bf 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -94,6 +94,9 @@ import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; +import org.apache.druid.segment.handoff.CoordinatorBasedSegmentHandoffNotifierConfig; +import org.apache.druid.segment.handoff.CoordinatorBasedSegmentHandoffNotifierFactory; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.loading.DataSegmentArchiver; import org.apache.druid.segment.loading.DataSegmentKiller; @@ -106,9 +109,6 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig; -import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.SegmentManager; diff --git a/website/.spelling b/website/.spelling index 76a4a83bba13..a81b0799a873 100644 --- a/website/.spelling +++ b/website/.spelling @@ -979,6 +979,7 @@ InputSplit JobHistory a.example.com assumeGrouped +awaitSegmentAvailabilityTimeoutMillis cleanupOnFailure combineText connectURI @@ -1070,6 +1071,7 @@ timeChunk totalNumMergeTasks StaticS3Firehose prefetchTriggerBytes +awaitSegmentAvailabilityTimeoutMillis - ../docs/ingestion/schema-design.md product_category product_id