A Replacing task must read segments created before it acquired its lock (#15085)

* Replacing tasks must read segments created before they acquired their locks
This commit is contained in:
AmatyaAvadhanula 2023-10-19 11:13:07 +05:30 committed by GitHub
parent fa311dd0b6
commit a8febd457c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 478 additions and 125 deletions

View File

@ -40,6 +40,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
@ -364,7 +365,7 @@ public class MaterializedViewSupervisor implements Supervisor
// Pair<interval -> max(created_date), interval -> list<DataSegment>>
Pair<Map<Interval, String>, Map<Interval, List<DataSegment>>> baseSegmentsSnapshot =
getMaxCreateDateAndBaseSegments(
metadataStorageCoordinator.retrieveUsedSegmentsAndCreatedDates(spec.getBaseDataSource())
metadataStorageCoordinator.retrieveUsedSegmentsAndCreatedDates(spec.getBaseDataSource(), Intervals.ETERNITY)
);
// baseSegments are used to create HadoopIndexTask
Map<Interval, List<DataSegment>> baseSegments = baseSegmentsSnapshot.rhs;

View File

@ -0,0 +1,209 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.batch.parallel.AbstractBatchSubtask;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.metadata.ReplaceTaskLock;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.SegmentTimeline;
import org.joda.time.Interval;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
/**
* This action exists in addition to retrieveUsedSegmentsAction because that action suffers
* from a race condition described by the following sequence of events:
*
* -Segments S1, S2, S3 exist
* -Compact acquires a replace lock
* -A concurrent appending job publishes a segment S4 which needs to be upgraded to the replace lock's version
* -Compact task processes S1-S4 to create new segments
* -Compact task publishes new segments and carries S4 forward to the new version
*
* This can lead to the data in S4 being duplicated
*
* This TaskAction returns a collection of segments which have data within the specified interval and are marked as
* used, and have been created before a REPLACE lock, if any, was acquired.
* This ensures that a consistent set of segments is returned each time this action is called
*/
public class RetrieveSegmentsToReplaceAction implements TaskAction<Collection<DataSegment>>
{
private static final Logger log = new Logger(RetrieveSegmentsToReplaceAction.class);
@JsonIgnore
private final String dataSource;
@JsonIgnore
private final Interval interval;
@JsonCreator
public RetrieveSegmentsToReplaceAction(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
@Override
public TypeReference<Collection<DataSegment>> getReturnTypeReference()
{
return new TypeReference<Collection<DataSegment>>() {};
}
@Override
public Collection<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{
// The DruidInputSource can be used to read from one datasource and write to another.
// In such a case, the race condition described in the class-level docs cannot occur,
// and the action can simply fetch all visible segments for the datasource and interval
if (!task.getDataSource().equals(dataSource)) {
return retrieveAllVisibleSegments(toolbox);
}
final String supervisorId;
if (task instanceof AbstractBatchSubtask) {
supervisorId = ((AbstractBatchSubtask) task).getSupervisorTaskId();
} else {
supervisorId = task.getId();
}
final Set<ReplaceTaskLock> replaceLocksForTask = toolbox
.getTaskLockbox()
.getAllReplaceLocksForDatasource(task.getDataSource())
.stream()
.filter(lock -> supervisorId.equals(lock.getSupervisorTaskId()))
.collect(Collectors.toSet());
// If there are no replace locks for the task, simply fetch all visible segments for the interval
if (replaceLocksForTask.isEmpty()) {
return retrieveAllVisibleSegments(toolbox);
}
Map<Interval, Map<String, Set<DataSegment>>> intervalToCreatedToSegments = new HashMap<>();
for (Pair<DataSegment, String> segmentAndCreatedDate :
toolbox.getIndexerMetadataStorageCoordinator().retrieveUsedSegmentsAndCreatedDates(dataSource, interval)) {
final DataSegment segment = segmentAndCreatedDate.lhs;
final String created = segmentAndCreatedDate.rhs;
intervalToCreatedToSegments.computeIfAbsent(segment.getInterval(), s -> new HashMap<>())
.computeIfAbsent(created, c -> new HashSet<>())
.add(segment);
}
Set<DataSegment> allSegmentsToBeReplaced = new HashSet<>();
for (final Map.Entry<Interval, Map<String, Set<DataSegment>>> entry : intervalToCreatedToSegments.entrySet()) {
final Interval segmentInterval = entry.getKey();
String lockVersion = null;
for (ReplaceTaskLock replaceLock : replaceLocksForTask) {
if (replaceLock.getInterval().contains(segmentInterval)) {
lockVersion = replaceLock.getVersion();
}
}
final Map<String, Set<DataSegment>> createdToSegmentsMap = entry.getValue();
for (Map.Entry<String, Set<DataSegment>> createdAndSegments : createdToSegmentsMap.entrySet()) {
if (lockVersion == null || lockVersion.compareTo(createdAndSegments.getKey()) > 0) {
allSegmentsToBeReplaced.addAll(createdAndSegments.getValue());
} else {
for (DataSegment segment : createdAndSegments.getValue()) {
log.info("Ignoring segment[%s] as it has created_date[%s] greater than the REPLACE lock version[%s]",
segment.getId(), createdAndSegments.getKey(), lockVersion);
}
}
}
}
return SegmentTimeline.forSegments(allSegmentsToBeReplaced)
.findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE);
}
private Collection<DataSegment> retrieveAllVisibleSegments(TaskActionToolbox toolbox)
{
return toolbox.getIndexerMetadataStorageCoordinator()
.retrieveUsedSegmentsForInterval(dataSource, interval, Segments.ONLY_VISIBLE);
}
@Override
public boolean isAudited()
{
return false;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RetrieveSegmentsToReplaceAction that = (RetrieveSegmentsToReplaceAction) o;
if (!dataSource.equals(that.dataSource)) {
return false;
}
return interval.equals(that.interval);
}
@Override
public int hashCode()
{
return Objects.hash(dataSource, interval);
}
@Override
public String toString()
{
return getClass().getSimpleName() + "{" +
"dataSource='" + dataSource + '\'' +
", interval=" + interval +
'}';
}
}

View File

@ -38,6 +38,7 @@ import java.util.concurrent.Future;
@JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class),
@JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class),
@JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class),
@JsonSubTypes.Type(name = "retrieveSegmentsToReplace", value = RetrieveSegmentsToReplaceAction.class),
// Type name doesn't correspond to the name of the class for backward compatibility.
@JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class),
// Type name doesn't correspond to the name of the class for backward compatibility.

View File

@ -19,6 +19,7 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.TaskResource;
@ -29,16 +30,20 @@ import java.util.Map;
public abstract class AbstractBatchSubtask extends AbstractBatchIndexTask
{
private final String supervisorTaskId;
protected AbstractBatchSubtask(
String id,
@Nullable String groupId,
@Nullable TaskResource taskResource,
String dataSource,
@Nullable Map<String, Object> context,
@Nonnull IngestionMode ingestionMode
@Nonnull IngestionMode ingestionMode,
@Nonnull String supervisorTaskId
)
{
super(id, groupId, taskResource, dataSource, context, -1, ingestionMode);
this.supervisorTaskId = supervisorTaskId;
}
/**
@ -46,4 +51,13 @@ public abstract class AbstractBatchSubtask extends AbstractBatchIndexTask
* This ID is used to identify duplicate work of retry tasks for the same spec.
*/
public abstract String getSubtaskSpecId();
/**
* @return Task ID of the {@code ParallelIndexSupervisorTask} which launched this sub-task.
*/
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
}

View File

@ -69,7 +69,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final String subtaskSpecId;
private final ObjectMapper jsonMapper;
@ -95,7 +94,8 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
taskResource,
ingestionSchema.getDataSchema(),
ingestionSchema.getTuningConfig(),
context
context,
supervisorTaskId
);
Preconditions.checkArgument(
@ -107,7 +107,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
this.subtaskSpecId = subtaskSpecId;
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.jsonMapper = jsonMapper;
}
@ -123,12 +122,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
return ingestionSchema;
}
@JsonProperty
private String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
@Override
public String getSubtaskSpecId()
@ -163,7 +156,7 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
{
if (!getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) {
return tryTimeChunkLock(
new SurrogateTaskActionClient(supervisorTaskId, taskActionClient),
new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient),
getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals()
);
} else {
@ -274,7 +267,7 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
{
final ParallelIndexSupervisorTaskClient taskClient =
toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
getSupervisorTaskId(),
ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
);

View File

@ -82,7 +82,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final String subtaskSpecId;
// For testing
@ -136,7 +135,8 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
taskResource,
ingestionSchema.getDataSchema(),
ingestionSchema.getTuningConfig(),
context
context,
supervisorTaskId
);
Preconditions.checkArgument(
@ -148,7 +148,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
this.subtaskSpecId = subtaskSpecId;
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.dedupInputRowFilterSupplier = dedupRowDimValueFilterSupplier;
}
@ -164,12 +163,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
return ingestionSchema;
}
@JsonProperty
private String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
@Override
public String getSubtaskSpecId()
@ -204,7 +197,7 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
{
if (!getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) {
return tryTimeChunkLock(
new SurrogateTaskActionClient(supervisorTaskId, taskActionClient),
new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient),
getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals()
);
} else {
@ -326,7 +319,7 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
private void sendReport(TaskToolbox toolbox, DimensionDistributionReport report)
{
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
getSupervisorTaskId(),
ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
);

View File

@ -63,7 +63,6 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final String subtaskSpecId;
@Nullable
private final Map<Interval, Integer> intervalToNumShardsOverride;
@ -96,7 +95,6 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
this.subtaskSpecId = subtaskSpecId;
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.intervalToNumShardsOverride = intervalToNumShardsOverride;
}
@ -112,12 +110,6 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
@Override
public String getSubtaskSpecId()
@ -158,7 +150,7 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
return tryTimeChunkLock(
new SurrogateTaskActionClient(supervisorTaskId, taskActionClient),
new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient),
getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals()
);
}
@ -175,7 +167,7 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
getDataSource(),
getSubtaskSpecId(),
granularitySpec,
new SupervisorTaskAccess(supervisorTaskId, taskClient),
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
createHashPartitionAnalysisFromPartitionsSpec(
granularitySpec,
partitionsSpec,

View File

@ -64,7 +64,6 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
private static final String PROP_SPEC = "spec";
private static final boolean SKIP_NULL = true;
private final String supervisorTaskId;
private final String subtaskSpecId;
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
@ -98,7 +97,6 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
this.subtaskSpecId = subtaskSpecId;
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.intervalToPartitions = intervalToPartitions;
}
@ -131,12 +129,6 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
@Override
public String getSubtaskSpecId()
@ -176,7 +168,7 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
public boolean isReady(TaskActionClient taskActionClient) throws IOException
{
return tryTimeChunkLock(
new SurrogateTaskActionClient(supervisorTaskId, taskActionClient),
new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient),
getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals()
);
}
@ -194,7 +186,7 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
getDataSource(),
getSubtaskSpecId(),
ingestionSchema.getDataSchema().getGranularitySpec(),
new SupervisorTaskAccess(supervisorTaskId, taskClient),
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
partitionAnalysis
);
}

View File

@ -94,7 +94,8 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
taskResource,
ingestionSchema.getDataSchema(),
ingestionSchema.getTuningConfig(),
context
context,
supervisorTaskId
);
Preconditions.checkArgument(

View File

@ -78,7 +78,6 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
private final PartialSegmentMergeIOConfig ioConfig;
private final int numAttempts;
private final String supervisorTaskId;
private final String subtaskSpecId;
PartialSegmentMergeTask(
@ -101,7 +100,8 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
taskResource,
dataSchema,
tuningConfig,
context
context,
supervisorTaskId
);
Preconditions.checkArgument(
@ -111,7 +111,6 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
this.subtaskSpecId = subtaskSpecId;
this.ioConfig = ioConfig;
this.numAttempts = numAttempts;
this.supervisorTaskId = supervisorTaskId;
}
@JsonProperty
@ -120,12 +119,6 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
return numAttempts;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
@Override
public String getSubtaskSpecId()
@ -151,7 +144,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
}
final List<TaskLock> locks = toolbox.getTaskActionClient().submit(
new SurrogateAction<>(supervisorTaskId, new LockListAction())
new SurrogateAction<>(getSupervisorTaskId(), new LockListAction())
);
final Map<Interval, String> intervalToVersion = Maps.newHashMapWithExpectedSize(locks.size());
locks.forEach(lock -> {
@ -179,7 +172,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
LOG.info("Fetch took [%s] seconds", fetchTime);
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
getSupervisorTaskId(),
getTuningConfig().getChatHandlerTimeout(),
getTuningConfig().getChatHandlerNumRetries()
);
@ -225,7 +218,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
);
FileUtils.mkdirp(partitionDir);
for (PartitionLocation location : entryPerBucketId.getValue()) {
final File unzippedDir = toolbox.getShuffleClient().fetchSegmentFile(partitionDir, supervisorTaskId, location);
final File unzippedDir = toolbox.getShuffleClient().fetchSegmentFile(partitionDir, getSupervisorTaskId(), location);
intervalToUnzippedFiles.computeIfAbsent(interval, k -> new Int2ObjectOpenHashMap<>())
.computeIfAbsent(bucketId, k -> new ArrayList<>())
.add(unzippedDir);

View File

@ -49,10 +49,11 @@ abstract class PerfectRollupWorkerTask extends AbstractBatchSubtask
@Nullable TaskResource taskResource,
DataSchema dataSchema,
ParallelIndexTuningConfig tuningConfig,
@Nullable Map<String, Object> context
@Nullable Map<String, Object> context,
String supervisorTaskId
)
{
super(id, groupId, taskResource, dataSchema.getDataSource(), context, IngestionMode.NONE);
super(id, groupId, taskResource, dataSchema.getDataSource(), context, IngestionMode.NONE, supervisorTaskId);
Preconditions.checkArgument(
tuningConfig.isForceGuaranteedRollup(),

View File

@ -118,7 +118,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final String subtaskSpecId;
/**
@ -169,7 +168,8 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
taskResource,
ingestionSchema.getDataSchema().getDataSource(),
context,
AbstractTask.computeBatchIngestionMode(ingestionSchema.getIOConfig())
AbstractTask.computeBatchIngestionMode(ingestionSchema.getIOConfig()),
supervisorTaskId
);
if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) {
@ -179,7 +179,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
this.subtaskSpecId = subtaskSpecId;
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.missingIntervalsInOverwriteMode = ingestionSchema.getIOConfig().isAppendToExisting() != true
&& ingestionSchema.getDataSchema()
.getGranularitySpec()
@ -217,7 +216,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
public boolean isReady(TaskActionClient taskActionClient) throws IOException
{
return determineLockGranularityAndTryLock(
new SurrogateTaskActionClient(supervisorTaskId, taskActionClient),
new SurrogateTaskActionClient(getSupervisorTaskId(), taskActionClient),
ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals()
);
}
@ -234,12 +233,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@Override
@JsonProperty
public String getSubtaskSpecId()
@ -272,7 +265,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(toolbox);
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
getSupervisorTaskId(),
ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
);

View File

@ -48,10 +48,9 @@ import org.apache.druid.data.input.impl.SplittableInputSource;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
import org.apache.druid.indexing.common.actions.RetrieveSegmentsToReplaceAction;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.firehose.WindowedSegmentId;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.Comparators;
@ -552,14 +551,7 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
} else {
try {
usedSegments = toolbox.getTaskActionClient()
.submit(
new RetrieveUsedSegmentsAction(
dataSource,
null,
Collections.singletonList(interval),
Segments.ONLY_VISIBLE
)
);
.submit(new RetrieveSegmentsToReplaceAction(dataSource, interval));
}
catch (IOException e) {
LOG.error(e, "Error retrieving the used segments for interval[%s].", interval);

View File

@ -111,7 +111,8 @@ public class PerfectRollupWorkerTaskTest
null,
createDataSchema(granularitySpecInputIntervals),
createTuningConfig(forceGuaranteedRollup, partitionsSpec),
null
null,
"supervisor-id"
);
}
@ -149,10 +150,11 @@ public class PerfectRollupWorkerTaskTest
@Nullable TaskResource taskResource,
DataSchema dataSchema,
ParallelIndexTuningConfig tuningConfig,
@Nullable Map<String, Object> context
@Nullable Map<String, Object> context,
String supervisorId
)
{
super(id, groupId, taskResource, dataSchema, tuningConfig, context);
super(id, groupId, taskResource, dataSchema, tuningConfig, context, supervisorId);
}
@Override

View File

@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskStorageDirTracker;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.actions.RetrieveSegmentsToReplaceAction;
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
@ -845,6 +846,54 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase
verifyIntervalHasVisibleSegments(YEAR_23, segmentV20, segmentV21, segmentV22, segmentV23);
}
@Test
public void testSegmentsToReplace()
{
final SegmentIdWithShardSpec pendingSegmentV01
= appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH);
Assert.assertEquals(SEGMENT_V0, pendingSegmentV01.getVersion());
Assert.assertEquals(JAN_23, pendingSegmentV01.getInterval());
final DataSegment segment1 = asSegment(pendingSegmentV01);
appendTask.commitAppendSegments(segment1);
final SegmentIdWithShardSpec pendingSegmentV02
= appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH);
Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV02.asSegmentId());
Assert.assertEquals(SEGMENT_V0, pendingSegmentV02.getVersion());
Assert.assertEquals(JAN_23, pendingSegmentV02.getInterval());
verifyInputSegments(replaceTask, JAN_23, segment1);
replaceTask.acquireReplaceLockOn(JAN_23);
final DataSegment segment2 = asSegment(pendingSegmentV02);
appendTask.commitAppendSegments(segment2);
// Despite segment2 existing, it is not chosen to be replaced because it was created after the tasklock was acquired
verifyInputSegments(replaceTask, JAN_23, segment1);
replaceTask.releaseLock(JAN_23);
final SegmentIdWithShardSpec pendingSegmentV03
= appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH);
Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV03.asSegmentId());
Assert.assertNotEquals(pendingSegmentV02.asSegmentId(), pendingSegmentV03.asSegmentId());
Assert.assertEquals(SEGMENT_V0, pendingSegmentV03.getVersion());
Assert.assertEquals(JAN_23, pendingSegmentV03.getInterval());
final DataSegment segment3 = asSegment(pendingSegmentV03);
appendTask.commitAppendSegments(segment3);
appendTask.releaseLock(JAN_23);
replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23);
// The new lock was acquired before segment3 was created but it doesn't contain the month's interval
// So, all three segments are chosen
verifyInputSegments(replaceTask, JAN_23, segment1, segment2, segment3);
replaceTask.releaseLock(FIRST_OF_JAN_23);
// All the segments are chosen when there's no lock
verifyInputSegments(replaceTask, JAN_23, segment1, segment2, segment3);
}
@Nullable
private DataSegment findSegmentWith(String version, Map<String, Object> loadSpec, Set<DataSegment> segments)
{
@ -901,6 +950,23 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase
}
}
private void verifyInputSegments(Task task, Interval interval, DataSegment... expectedSegments)
{
try {
final TaskActionClient taskActionClient = taskActionClientFactory.create(task);
Collection<DataSegment> allUsedSegments = taskActionClient.submit(
new RetrieveSegmentsToReplaceAction(
WIKI,
interval
)
);
Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments));
}
catch (IOException e) {
throw new ISE(e, "Error while fetching segments to replace in interval[%s]", interval);
}
}
private TaskToolboxFactory createToolboxFactory(
TaskConfig taskConfig,
TaskActionClientFactory taskActionClientFactory

View File

@ -89,7 +89,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
}
@Override
public List<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource)
public List<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval)
{
return ImmutableList.of();
}

View File

@ -82,8 +82,9 @@ public interface IndexerMetadataStorageCoordinator
Collection<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility);
/**
*
* Retrieve all published segments which are marked as used and the created_date of these segments belonging to the
* given data source from the metadata store.
* given data source and interval from the metadata store.
*
* Unlike other similar methods in this interface, this method doesn't accept a {@link Segments} "visibility"
* parameter. The returned collection may include overshadowed segments and their created_dates, as if {@link
@ -91,10 +92,11 @@ public interface IndexerMetadataStorageCoordinator
* if needed.
*
* @param dataSource The data source to query
* @param interval The interval to query
*
* @return The DataSegments and the related created_date of segments
*/
Collection<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource);
Collection<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval);
/**
* Retrieve all published segments which may include any data in the given intervals and are marked as used from the

View File

@ -174,15 +174,34 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
}
@Override
public List<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource)
public List<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource, Interval interval)
{
String rawQueryString = "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true";
final String queryString = StringUtils.format(rawQueryString, dbTables.getSegmentsTable());
StringBuilder queryBuilder = new StringBuilder(
"SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true"
);
final List<Interval> intervals = new ArrayList<>();
// Do not need an interval condition if the interval is ETERNITY
if (!Intervals.isEternity(interval)) {
intervals.add(interval);
}
SqlSegmentsMetadataQuery.appendConditionForIntervalsAndMatchMode(
queryBuilder,
intervals,
SqlSegmentsMetadataQuery.IntervalMode.OVERLAPS,
connector
);
final String queryString = StringUtils.format(queryBuilder.toString(), dbTables.getSegmentsTable());
return connector.retryWithHandle(
handle -> {
Query<Map<String, Object>> query = handle
.createQuery(queryString)
.bind("dataSource", dataSource);
SqlSegmentsMetadataQuery.bindQueryIntervals(query, intervals);
return query
.map((int index, ResultSet r, StatementContext ctx) ->
new Pair<>(

View File

@ -261,6 +261,82 @@ public class SqlSegmentsMetadataQuery
return null;
}
/**
* Append the condition for the interval and match mode to the given string builder with a partial query
* @param sb - StringBuilder containing the paritial query with SELECT clause and WHERE condition for used, datasource
* @param intervals - intervals to fetch the segments for
* @param matchMode - Interval match mode - overlaps or contains
* @param connector - SQL connector
*/
public static void appendConditionForIntervalsAndMatchMode(
final StringBuilder sb,
final Collection<Interval> intervals,
final IntervalMode matchMode,
final SQLMetadataConnector connector
)
{
if (intervals.isEmpty()) {
return;
}
sb.append(" AND (");
for (int i = 0; i < intervals.size(); i++) {
sb.append(
matchMode.makeSqlCondition(
connector.getQuoteString(),
StringUtils.format(":start%d", i),
StringUtils.format(":end%d", i)
)
);
// Add a special check for a segment which have one end at eternity and the other at some finite value. Since
// we are using string comparison, a segment with this start or end will not be returned otherwise.
if (matchMode.equals(IntervalMode.OVERLAPS)) {
sb.append(StringUtils.format(
" OR (start = '%s' AND \"end\" != '%s' AND \"end\" > :start%d)",
Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i
));
sb.append(StringUtils.format(
" OR (start != '%s' AND \"end\" = '%s' AND start < :end%d)",
Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i
));
}
if (i != intervals.size() - 1) {
sb.append(" OR ");
}
}
// Add a special check for a single segment with eternity. Since we are using string comparison, a segment with
// this start and end will not be returned otherwise.
// Known Issue: https://github.com/apache/druid/issues/12860
if (matchMode.equals(IntervalMode.OVERLAPS)) {
sb.append(StringUtils.format(
" OR (start = '%s' AND \"end\" = '%s')", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd()
));
}
sb.append(")");
}
/**
* Given a Query object bind the input intervals to it
* @param query Query to fetch segments
* @param intervals Intervals to fetch segments for
*/
public static void bindQueryIntervals(final Query<Map<String, Object>> query, final Collection<Interval> intervals)
{
if (intervals.isEmpty()) {
return;
}
final Iterator<Interval> iterator = intervals.iterator();
for (int i = 0; iterator.hasNext(); i++) {
Interval interval = iterator.next();
query.bind(StringUtils.format("start%d", i), interval.getStart().toString())
.bind(StringUtils.format("end%d", i), interval.getEnd().toString());
}
}
private CloseableIterator<DataSegment> retrieveSegments(
final String dataSource,
final Collection<Interval> intervals,
@ -275,36 +351,8 @@ public class SqlSegmentsMetadataQuery
final StringBuilder sb = new StringBuilder();
sb.append("SELECT payload FROM %s WHERE used = :used AND dataSource = :dataSource");
if (compareAsString && !intervals.isEmpty()) {
sb.append(" AND (");
for (int i = 0; i < intervals.size(); i++) {
sb.append(
matchMode.makeSqlCondition(
connector.getQuoteString(),
StringUtils.format(":start%d", i),
StringUtils.format(":end%d", i)
)
);
// Add a special check for a segment which have one end at eternity and the other at some finite value. Since
// we are using string comparison, a segment with this start or end will not be returned otherwise.
if (matchMode.equals(IntervalMode.OVERLAPS)) {
sb.append(StringUtils.format(" OR (start = '%s' AND \"end\" != '%s' AND \"end\" > :start%d)", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i));
sb.append(StringUtils.format(" OR (start != '%s' AND \"end\" = '%s' AND start < :end%d)", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd(), i));
}
if (i != intervals.size() - 1) {
sb.append(" OR ");
}
}
// Add a special check for a single segment with eternity. Since we are using string comparison, a segment with
// this start and end will not be returned otherwise.
// Known Issue: https://github.com/apache/druid/issues/12860
if (matchMode.equals(IntervalMode.OVERLAPS)) {
sb.append(StringUtils.format(" OR (start = '%s' AND \"end\" = '%s')", Intervals.ETERNITY.getStart(), Intervals.ETERNITY.getEnd()));
}
sb.append(")");
if (compareAsString) {
appendConditionForIntervalsAndMatchMode(sb, intervals, matchMode, connector);
}
final Query<Map<String, Object>> sql = handle
@ -317,12 +365,7 @@ public class SqlSegmentsMetadataQuery
}
if (compareAsString) {
final Iterator<Interval> iterator = intervals.iterator();
for (int i = 0; iterator.hasNext(); i++) {
Interval interval = iterator.next();
sql.bind(StringUtils.format("start%d", i), interval.getStart().toString())
.bind(StringUtils.format("end%d", i), interval.getEnd().toString());
}
bindQueryIntervals(sql, intervals);
}
final ResultIterator<DataSegment> resultIterator =

View File

@ -2554,6 +2554,52 @@ public class IndexerSQLMetadataStorageCoordinatorTest
);
}
@Test
public void testRetrieveUsedSegmentsAndCreatedDates()
{
insertUsedSegments(ImmutableSet.of(defaultSegment));
List<Pair<DataSegment, String>> resultForIntervalOnTheLeft =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2000/2001"));
Assert.assertTrue(resultForIntervalOnTheLeft.isEmpty());
List<Pair<DataSegment, String>> resultForIntervalOnTheRight =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("3000/3001"));
Assert.assertTrue(resultForIntervalOnTheRight.isEmpty());
List<Pair<DataSegment, String>> resultForExactInterval =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), defaultSegment.getInterval());
Assert.assertEquals(1, resultForExactInterval.size());
Assert.assertEquals(defaultSegment, resultForExactInterval.get(0).lhs);
List<Pair<DataSegment, String>> resultForIntervalWithLeftOverlap =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2000/2015-01-02"));
Assert.assertEquals(resultForExactInterval, resultForIntervalWithLeftOverlap);
List<Pair<DataSegment, String>> resultForIntervalWithRightOverlap =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.of("2015-01-01/3000"));
Assert.assertEquals(resultForExactInterval, resultForIntervalWithRightOverlap);
List<Pair<DataSegment, String>> resultForEternity =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Intervals.ETERNITY);
Assert.assertEquals(resultForExactInterval, resultForEternity);
}
@Test
public void testRetrieveUsedSegmentsAndCreatedDatesFetchesEternityForAnyInterval()
{
insertUsedSegments(ImmutableSet.of(eternitySegment, firstHalfEternityRangeSegment, secondHalfEternityRangeSegment));
List<Pair<DataSegment, String>> resultForRandomInterval =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), defaultSegment.getInterval());
Assert.assertEquals(3, resultForRandomInterval.size());
List<Pair<DataSegment, String>> resultForEternity =
coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), eternitySegment.getInterval());
Assert.assertEquals(3, resultForEternity.size());
}
private static class DS
{
static final String WIKI = "wiki";