Skip to content

Commit

Permalink
[Backport] Check for handoff of upgraded segments (#16162) (#16344)
Browse files Browse the repository at this point in the history
Changes:
1) Check for handoff of upgraded realtime segments.
2) Drop sink only when all associated realtime segments have been abandoned.
3) Delete pending segments upon commit to prevent unnecessary upgrades and
partition space exhaustion when a concurrent replace happens. This also prevents
potential data duplication.
4) Register pending segment upgrade only on those tasks to which the segment is associated.

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
  • Loading branch information
AmatyaAvadhanula and kfaraz committed Apr 30, 2024
1 parent ff18055 commit b18d836
Show file tree
Hide file tree
Showing 33 changed files with 581 additions and 392 deletions.
Expand Up @@ -298,12 +298,6 @@ public LagStats computeLagStats()
throw new UnsupportedOperationException("Compute Lag Stats not supported in MaterializedViewSupervisor");
}

@Override
public Set<String> getActiveRealtimeSequencePrefixes()
{
throw new UnsupportedOperationException();
}

@Override
public int getActiveTaskGroupsCount()
{
Expand Down
Expand Up @@ -207,8 +207,6 @@ public void testMaterializedViewSupervisorSpecCreated()

Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.getActiveTaskGroupsCount());

Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.getActiveRealtimeSequencePrefixes());

Callable<Integer> noop = new Callable<Integer>() {
@Override
public Integer call()
Expand Down
Expand Up @@ -32,7 +32,6 @@
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.AbstractTask;
import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask;
import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.msq.exec.MSQTasks;
import org.apache.druid.msq.exec.Worker;
Expand All @@ -46,7 +45,7 @@
import java.util.Set;

@JsonTypeName(MSQWorkerTask.TYPE)
public class MSQWorkerTask extends AbstractTask implements PendingSegmentAllocatingTask
public class MSQWorkerTask extends AbstractTask
{
public static final String TYPE = "query_worker";

Expand Down Expand Up @@ -126,13 +125,6 @@ public Set<ResourceAction> getInputSourceResources()
return ImmutableSet.of();
}

@Override
public String getTaskAllocatorId()
{
return getControllerTaskId();
}


@Override
public boolean isReady(final TaskActionClient taskActionClient)
{
Expand Down
Expand Up @@ -108,12 +108,4 @@ public void testGetInputSourceResources()
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertTrue(msqWorkerTask.getInputSourceResources().isEmpty());
}

@Test
public void testGetTaskAllocatorId()
{
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertEquals(controllerTaskId, msqWorkerTask.getTaskAllocatorId());
}

}
Expand Up @@ -34,13 +34,10 @@
import org.apache.druid.metadata.ReplaceTaskLock;
import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;

import javax.annotation.Nullable;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -155,7 +152,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox)
// failure to upgrade pending segments does not affect success of the commit
if (publishResult.isSuccess() && toolbox.getSupervisorManager() != null) {
try {
registerUpgradedPendingSegmentsOnSupervisor(task, toolbox);
registerUpgradedPendingSegmentsOnSupervisor(task, toolbox, publishResult.getUpgradedPendingSegments());
}
catch (Exception e) {
log.error(e, "Error while upgrading pending segments for task[%s]", task.getId());
Expand All @@ -168,7 +165,11 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox)
/**
* Registers upgraded pending segments on the active supervisor, if any
*/
private void registerUpgradedPendingSegmentsOnSupervisor(Task task, TaskActionToolbox toolbox)
private void registerUpgradedPendingSegmentsOnSupervisor(
Task task,
TaskActionToolbox toolbox,
List<PendingSegmentRecord> upgradedPendingSegments
)
{
final SupervisorManager supervisorManager = toolbox.getSupervisorManager();
final Optional<String> activeSupervisorIdWithAppendLock =
Expand All @@ -178,42 +179,10 @@ private void registerUpgradedPendingSegmentsOnSupervisor(Task task, TaskActionTo
return;
}

final Set<ReplaceTaskLock> replaceLocksForTask = toolbox
.getTaskLockbox()
.getAllReplaceLocksForDatasource(task.getDataSource())
.stream()
.filter(lock -> task.getId().equals(lock.getSupervisorTaskId()))
.collect(Collectors.toSet());


Set<PendingSegmentRecord> pendingSegments = new HashSet<>();
for (ReplaceTaskLock replaceLock : replaceLocksForTask) {
pendingSegments.addAll(
toolbox.getIndexerMetadataStorageCoordinator()
.getPendingSegments(task.getDataSource(), replaceLock.getInterval())
);
}
Map<String, SegmentIdWithShardSpec> idToPendingSegment = new HashMap<>();
pendingSegments.forEach(pendingSegment -> idToPendingSegment.put(
pendingSegment.getId().asSegmentId().toString(),
pendingSegment.getId()
));
Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> segmentToParent = new HashMap<>();
pendingSegments.forEach(pendingSegment -> {
if (pendingSegment.getUpgradedFromSegmentId() != null
&& !pendingSegment.getUpgradedFromSegmentId().equals(pendingSegment.getId().asSegmentId().toString())) {
segmentToParent.put(
pendingSegment.getId(),
idToPendingSegment.get(pendingSegment.getUpgradedFromSegmentId())
);
}
});

segmentToParent.forEach(
(newId, oldId) -> supervisorManager.registerNewVersionOfPendingSegmentOnSupervisor(
upgradedPendingSegments.forEach(
upgradedPendingSegment -> supervisorManager.registerUpgradedPendingSegmentOnSupervisor(
activeSupervisorIdWithAppendLock.get(),
oldId,
newId
upgradedPendingSegment
)
);
}
Expand Down
Expand Up @@ -43,7 +43,6 @@
import org.apache.druid.indexing.common.task.BatchAppenderators;
import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask;
import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch;
import org.apache.druid.indexing.common.task.SegmentAllocators;
import org.apache.druid.indexing.common.task.TaskResource;
Expand Down Expand Up @@ -109,7 +108,7 @@
* generates and pushes segments, and reports them to the {@link SinglePhaseParallelIndexTaskRunner} instead of
* publishing on its own.
*/
public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHandler, PendingSegmentAllocatingTask
public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHandler
{
public static final String TYPE = "single_phase_sub_task";
public static final String OLD_TYPE_NAME = "index_sub";
Expand Down Expand Up @@ -240,12 +239,6 @@ public String getSubtaskSpecId()
return subtaskSpecId;
}

@Override
public String getTaskAllocatorId()
{
return getGroupId();
}

@Override
public TaskStatus runTask(final TaskToolbox toolbox) throws Exception
{
Expand Down
Expand Up @@ -1242,7 +1242,7 @@ public void remove(final Task task)
idsInSameGroup.remove(task.getId());
if (idsInSameGroup.isEmpty()) {
final int pendingSegmentsDeleted
= metadataStorageCoordinator.deletePendingSegmentsForTaskGroup(taskAllocatorId);
= metadataStorageCoordinator.deletePendingSegmentsForTaskAllocatorId(taskAllocatorId);
log.info(
"Deleted [%d] entries from pendingSegments table for pending segments group [%s] with APPEND locks.",
pendingSegmentsDeleted, taskAllocatorId
Expand Down
Expand Up @@ -33,9 +33,9 @@
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.MetadataSupervisorManager;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;

import javax.annotation.Nullable;

Expand Down Expand Up @@ -308,16 +308,19 @@ public boolean checkPointDataSourceMetadata(
* allows the supervisor to include the pending segment in queries fired against
* that segment version.
*/
public boolean registerNewVersionOfPendingSegmentOnSupervisor(
public boolean registerUpgradedPendingSegmentOnSupervisor(
String supervisorId,
SegmentIdWithShardSpec basePendingSegment,
SegmentIdWithShardSpec newSegmentVersion
PendingSegmentRecord upgradedPendingSegment
)
{
try {
Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null");
Preconditions.checkNotNull(basePendingSegment, "rootPendingSegment cannot be null");
Preconditions.checkNotNull(newSegmentVersion, "newSegmentVersion cannot be null");
Preconditions.checkNotNull(upgradedPendingSegment, "upgraded pending segment cannot be null");
Preconditions.checkNotNull(upgradedPendingSegment.getTaskAllocatorId(), "taskAllocatorId cannot be null");
Preconditions.checkNotNull(
upgradedPendingSegment.getUpgradedFromSegmentId(),
"upgradedFromSegmentId cannot be null"
);

Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(supervisorId);
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
Expand All @@ -326,12 +329,12 @@ public boolean registerNewVersionOfPendingSegmentOnSupervisor(
}

SeekableStreamSupervisor<?, ?, ?> seekableStreamSupervisor = (SeekableStreamSupervisor<?, ?, ?>) supervisor.lhs;
seekableStreamSupervisor.registerNewVersionOfPendingSegment(basePendingSegment, newSegmentVersion);
seekableStreamSupervisor.registerNewVersionOfPendingSegment(upgradedPendingSegment);
return true;
}
catch (Exception e) {
log.error(e, "PendingSegmentRecord[%s] mapping update request to version[%s] on Supervisor[%s] failed",
basePendingSegment.asSegmentId(), newSegmentVersion.getVersion(), supervisorId);
log.error(e, "Failed to upgrade pending segment[%s] to new pending segment[%s] on Supervisor[%s].",
upgradedPendingSegment.getUpgradedFromSegmentId(), upgradedPendingSegment.getId().getVersion(), supervisorId);
}
return false;
}
Expand Down

This file was deleted.

Expand Up @@ -21,8 +21,8 @@

import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.joda.time.DateTime;

import java.util.List;
Expand Down Expand Up @@ -158,15 +158,14 @@ ListenableFuture<Boolean> setEndOffsetsAsync(
* Update the task state to redirect queries for later versions to the root pending segment.
* The task also announces that it is serving the segments belonging to the subsequent versions.
* The update is processed only if the task is serving the original pending segment.
* @param taskId - task id
* @param basePendingSegment - the pending segment that was originally allocated
* @param newVersionOfSegment - the ids belonging to the versions to which the root segment needs to be updated
*
* @param taskId - task id
* @param pendingSegmentRecord - the ids belonging to the versions to which the root segment needs to be updated
* @return true if the update succeeds
*/
ListenableFuture<Boolean> registerNewVersionOfPendingSegmentAsync(
String taskId,
SegmentIdWithShardSpec basePendingSegment,
SegmentIdWithShardSpec newVersionOfSegment
PendingSegmentRecord pendingSegmentRecord
);

Class<PartitionIdType> getPartitionType();
Expand Down
Expand Up @@ -43,6 +43,7 @@
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler;
import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.rpc.HttpResponseException;
import org.apache.druid.rpc.IgnoreHttpResponseHandler;
import org.apache.druid.rpc.RequestBuilder;
Expand All @@ -57,7 +58,6 @@
import org.apache.druid.rpc.StandardRetryPolicy;
import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.DateTime;
Expand Down Expand Up @@ -197,13 +197,12 @@ public ListenableFuture<Map<PartitionIdType, SequenceOffsetType>> getEndOffsetsA
@Override
public ListenableFuture<Boolean> registerNewVersionOfPendingSegmentAsync(
String taskId,
SegmentIdWithShardSpec basePendingSegment,
SegmentIdWithShardSpec newVersionOfSegment
PendingSegmentRecord pendingSegmentRecord
)
{
final RequestBuilder requestBuilder
= new RequestBuilder(HttpMethod.POST, "/pendingSegmentVersion")
.jsonContent(jsonMapper, new PendingSegmentVersions(basePendingSegment, newVersionOfSegment));
.jsonContent(jsonMapper, pendingSegmentRecord);

return makeRequest(taskId, requestBuilder)
.handler(IgnoreHttpResponseHandler.INSTANCE)
Expand Down
Expand Up @@ -78,6 +78,7 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.segment.incremental.ParseExceptionHandler;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.incremental.RowIngestionMeters;
Expand Down Expand Up @@ -1575,18 +1576,15 @@ public Response setEndOffsetsHTTP(
@Path("/pendingSegmentVersion")
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
public Response registerNewVersionOfPendingSegment(
PendingSegmentVersions pendingSegmentVersions,
public Response registerUpgradedPendingSegment(
PendingSegmentRecord upgradedPendingSegment,
// this field is only for internal purposes, shouldn't be usually set by users
@Context final HttpServletRequest req
)
{
authorizationCheck(req, Action.WRITE);
try {
((StreamAppenderator) appenderator).registerNewVersionOfPendingSegment(
pendingSegmentVersions.getBaseSegment(),
pendingSegmentVersions.getNewVersion()
);
((StreamAppenderator) appenderator).registerUpgradedPendingSegment(upgradedPendingSegment);
return Response.ok().build();
}
catch (DruidException e) {
Expand All @@ -1598,8 +1596,8 @@ public Response registerNewVersionOfPendingSegment(
catch (Exception e) {
log.error(
e,
"Could not register new version[%s] of pending segment[%s]",
pendingSegmentVersions.getNewVersion(), pendingSegmentVersions.getBaseSegment()
"Could not register pending segment[%s] upgraded from[%s]",
upgradedPendingSegment.getId().asSegmentId(), upgradedPendingSegment.getUpgradedFromSegmentId()
);
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
}
Expand Down

0 comments on commit b18d836

Please sign in to comment.