From 4327aad749b0aadeec4fea8be7bfde6c6b5d126e Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 26 Sep 2023 16:44:04 +0530 Subject: [PATCH 01/31] Facilitate supervisor to update pending segment mapping in tasks --- .../MaterializedViewSupervisor.java | 7 +++++ .../supervisor/SupervisorManager.java | 20 ++++++++++++++ .../SeekableStreamIndexTaskClient.java | 8 ++++++ ...eekableStreamIndexTaskClientAsyncImpl.java | 21 +++++++++++++++ .../SeekableStreamIndexTaskRunner.java | 26 +++++++++++++++++++ .../supervisor/SeekableStreamSupervisor.java | 26 +++++++++++++++++++ ...TestIndexerMetadataStorageCoordinator.java | 6 +++++ .../IndexerMetadataStorageCoordinator.java | 2 ++ .../supervisor/NoopSupervisorSpec.java | 6 +++++ .../overlord/supervisor/Supervisor.java | 3 +++ .../IndexerSQLMetadataStorageCoordinator.java | 6 +++++ .../appenderator/SinkQuerySegmentWalker.java | 20 +++++++++++++- .../appenderator/StreamAppenderator.java | 11 ++++++++ 13 files changed, 161 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index ac2738534da9..b4ba7ac1c2ef 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -49,6 +49,7 @@ import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SqlSegmentsMetadataManager; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; import org.joda.time.Interval; @@ -289,6 +290,12 @@ public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) // do nothing } + @Override + public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) + { + // do nothing + } + @Override public LagStats computeLagStats() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 2cd926bae907..6fdf1f7a1400 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.segment.incremental.ParseExceptionReport; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import javax.annotation.Nullable; @@ -247,6 +248,25 @@ public boolean checkPointDataSourceMetadata( return false; } + public boolean updatePendingSegmentMapping(String supervisorId, SegmentIdWithShardSpec rootPendingSegment) + { + try { + Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null"); + Preconditions.checkNotNull(rootPendingSegment, "rootPendingSegment cannot be null"); + + Pair supervisor = supervisors.get(supervisorId); + + Preconditions.checkNotNull(supervisor, "supervisor could not be found"); + + supervisor.lhs.updatePendingSegmentMapping(rootPendingSegment); + return true; + } + catch (Exception e) { + log.error(e, "Pending segment mapping update request failed"); + } + return false; + } + /** * Stops a supervisor with a given id and then removes it from the list. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 18631626d0f6..806bcd6c692c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -22,10 +22,12 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.segment.incremental.ParseExceptionReport; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; public interface SeekableStreamIndexTaskClient @@ -153,6 +155,12 @@ ListenableFuture setEndOffsetsAsync( */ ListenableFuture getStatusAsync(String id); + ListenableFuture updatePendingSegmentMapping( + String id, + SegmentIdWithShardSpec rootPendingSegment, + Set versionsOfPendingSegment + ); + Class getPartitionType(); Class getSequenceType(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 9d6d49e00bf2..8b7b9c616775 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -37,6 +37,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -57,6 +58,7 @@ 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; @@ -68,6 +70,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -193,6 +196,24 @@ public ListenableFuture> getEndOffsetsA .go(); } + @Override + public ListenableFuture updatePendingSegmentMapping( + String id, + SegmentIdWithShardSpec rootPendingSegment, + Set versionsOfPendingSegment + ) + { + final RequestBuilder requestBuilder = new RequestBuilder( + HttpMethod.POST, + "updatePendingSegmentMapping" + ).jsonContent(jsonMapper, Pair.of(rootPendingSegment, versionsOfPendingSegment)); + + return makeRequest(id, requestBuilder) + .handler(IgnoreHttpResponseHandler.INSTANCE) + .onSuccess(r -> true) + .go(); + } + @Override public ListenableFuture setEndOffsetsAsync( final String id, 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 e44dfe9a451e..f540c0ba5d4c 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 @@ -72,6 +72,7 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -82,7 +83,9 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.server.security.Access; @@ -1538,6 +1541,20 @@ public Response setEndOffsetsHTTP( return setEndOffsets(sequences, finish); } + @POST + @Path("updatePendingSegmentMapping") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response updatePendingSegmentMapping( + Pair> rootPendingSegmentToVersionMapping, + // this field is only for internal purposes, shouldn't be usually set by users + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.WRITE); + return updatePendingSegmentMapping(rootPendingSegmentToVersionMapping.lhs, rootPendingSegmentToVersionMapping.rhs); + } + public Map doGetRowStats() { Map returnMap = new HashMap<>(); @@ -1742,6 +1759,15 @@ public Response setEndOffsets( return Response.ok(sequenceNumbers).build(); } + private Response updatePendingSegmentMapping( + SegmentIdWithShardSpec rootPendingSegment, + Set versionsOfPendingSegment + ) + { + ((StreamAppenderator) appenderator).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + return Response.ok().build(); + } + private void resetNextCheckpointTime() { nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 2f6cb008b842..db5f73e3b89d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -97,6 +97,7 @@ import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import javax.annotation.Nonnull; @@ -1092,6 +1093,31 @@ public void resetOffsets(@Nonnull DataSourceMetadata resetDataSourceMetadata) addNotice(new ResetOffsetsNotice(resetDataSourceMetadata)); } + @Override + public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) + { + for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { + for (String taskId : taskGroup.taskIds()) { + taskClient.updatePendingSegmentMapping( + taskId, + rootPendingSegment, + indexerMetadataStorageCoordinator.getAllVersionsOfPendingSegment(rootPendingSegment) + ); + } + } + for (List taskGroupList : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroupList) { + for (String taskId : taskGroup.taskIds()) { + taskClient.updatePendingSegmentMapping( + taskId, + rootPendingSegment, + indexerMetadataStorageCoordinator.getAllVersionsOfPendingSegment(rootPendingSegment) + ); + } + } + } + } + public ReentrantLock getRecordSupplierLock() { return recordSupplierLock; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 34d2e44552a7..6fd749201b7b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -253,6 +253,12 @@ public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) return null; } + @Override + public Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment) + { + return Collections.emptySet(); + } + public Set getPublished() { return ImmutableSet.copyOf(published); diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 3cbabea78fae..7f4c6c193909 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -406,4 +406,6 @@ SegmentPublishResult commitMetadataOnly( */ DataSegment retrieveSegmentForId(String id, boolean includeUnused); + Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment); + } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java index e733ef6c233d..a7439bc44914 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.security.ResourceAction; import javax.annotation.Nonnull; @@ -185,6 +186,11 @@ public int getActiveTaskGroupsCount() { return -1; } + + @Override + public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) + { + } }; } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index bcfc5ebe8196..22f7dd8b5740 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.segment.incremental.ParseExceptionReport; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import javax.annotation.Nullable; import java.util.List; @@ -87,6 +88,8 @@ default Boolean isHealthy() */ void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata); + void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment); + /** * Computes maxLag, totalLag and avgLag */ diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 7eaac692f7ce..bdd4efa12b4d 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2338,6 +2338,12 @@ public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) ); } + @Override + public Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment) + { + return Collections.emptySet(); + } + private static class PendingSegmentsRecord { private final String sequenceName; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 46315dbc0de2..2abc2e24e776 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -69,7 +69,10 @@ import org.joda.time.Interval; import java.io.Closeable; +import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; @@ -92,6 +95,7 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker private final Cache cache; private final CacheConfig cacheConfig; private final CachePopulatorStats cachePopulatorStats; + private final Map newIdToRootPendingSegment = new ConcurrentHashMap<>(); public SinkQuerySegmentWalker( String dataSource, @@ -182,7 +186,8 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable> perSegmentRunners = Iterables.transform( specs, - descriptor -> { + newDescriptor -> { + final SegmentDescriptor descriptor = newIdToRootPendingSegment.getOrDefault(newDescriptor, newDescriptor); final PartitionChunk chunk = sinkTimeline.findChunk( descriptor.getInterval(), descriptor.getVersion(), @@ -297,6 +302,19 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ); } + public void updatePendingSegmentMapping( + SegmentIdWithShardSpec rootPendingSegment, + Set versionsOfPendingSegment + ) + { + for (SegmentIdWithShardSpec versionOfPendingSegment : versionsOfPendingSegment) { + newIdToRootPendingSegment.put( + versionOfPendingSegment.asSegmentId().toDescriptor(), + rootPendingSegment.asSegmentId().toDescriptor() + ); + } + } + @VisibleForTesting String getDataSource() { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index e95852bfddb1..be50de901ec3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1026,6 +1026,17 @@ public void closeNow() } } + public void updatePendingSegmentMapping( + SegmentIdWithShardSpec rootPendingSegment, + Set versionsOfPendingSegment + ) + { + if (!sinks.containsKey(rootPendingSegment) || droppingSinks.contains(rootPendingSegment)) { + return; + } + ((SinkQuerySegmentWalker) texasRanger).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + } + private void lockBasePersistDirectory() { if (basePersistDirLock == null) { From f93fbd3e42934dbcbe4bf11d3c3634611ae49ab0 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 5 Oct 2023 19:09:50 +0530 Subject: [PATCH 02/31] Add CommitRealtimeSegmentsAndMetadataAction --- ...mmitRealtimeSegmentsAndMetadataAction.java | 168 ++++++++++++++++++ .../SegmentTransactionalAppendAction.java | 16 +- .../SegmentTransactionalInsertAction.java | 44 +---- .../SegmentTransactionalReplaceAction.java | 20 +-- .../indexing/common/actions/TaskAction.java | 1 + .../AppenderatorDriverRealtimeIndexTask.java | 2 +- .../indexing/common/task/IndexTaskUtils.java | 49 +++-- .../SeekableStreamIndexTaskRunner.java | 18 +- .../seekablestream/SequenceMetadata.java | 42 +++-- .../task/concurrent/ActionsTestTask.java | 28 +++ .../ConcurrentReplaceAndAppendTest.java | 90 +++++++++- .../seekablestream/SequenceMetadataTest.java | 6 +- .../emitter/service/SegmentMetadataEvent.java | 13 ++ 13 files changed, 373 insertions(+), 124 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java new file mode 100644 index 000000000000..67ad47080723 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Preconditions; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.timeline.DataSegment; + +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Task action to commit realtime segments and metadata when using APPEND task locks. + *

+ * This action performs the following operations in a single transaction: + *

    + *
  • Commit the append segments
  • + *
  • Upgrade the append segments to all visible REPLACE versions
  • + *
  • Commit start and end {@link DataSourceMetadata}.
  • + *
+ * This action differs from {@link SegmentTransactionalInsertAction} as it is used + * only with APPEND locks and also upgrades segments as needed. + */ +public class CommitRealtimeSegmentsAndMetadataAction implements TaskAction +{ + /** + * Set of segments to be inserted into metadata storage + */ + private final Set segments; + + private final DataSourceMetadata startMetadata; + private final DataSourceMetadata endMetadata; + + public static CommitRealtimeSegmentsAndMetadataAction create( + Set segments, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) + { + return new CommitRealtimeSegmentsAndMetadataAction(segments, startMetadata, endMetadata); + } + + @JsonCreator + private CommitRealtimeSegmentsAndMetadataAction( + @JsonProperty("segments") Set segments, + @JsonProperty("startMetadata") DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") DataSourceMetadata endMetadata + ) + { + Preconditions.checkArgument( + segments != null && !segments.isEmpty(), + "Segments to commit should not be empty" + ); + this.segments = segments; + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + @JsonProperty + public DataSourceMetadata getStartMetadata() + { + return startMetadata; + } + + @JsonProperty + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + /** + * Performs some sanity checks and publishes the given segments. + */ + @Override + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) + { + final SegmentPublishResult publishResult; + + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + + try { + publishResult = toolbox.getTaskLockbox().doInCriticalSection( + task, + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), + CriticalAction.builder() + .onValidLocks( + // TODO: this might need to call a new method which does the following in the same transaction + // - commit append segments + // - upgrade append segments to replace versions + // - commit metadata + () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( + segments, + startMetadata, + endMetadata + ) + ) + .onInvalidLocks( + () -> SegmentPublishResult.fail( + "Invalid task locks. Maybe they are revoked by a higher priority task." + + " Please check the overlord log for details." + ) + ) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); + return publishResult; + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "CommitRealtimeSegmentsAndMetadataAction{" + + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + ", startMetadata=" + startMetadata + + ", endMetadata=" + endMetadata + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 171c4f6640f3..36b69270f991 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -26,7 +26,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; @@ -107,20 +106,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } - // Emit metrics - final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); - IndexTaskUtils.setTaskDimensions(metricBuilder, task); - - if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); - for (DataSegment segment : retVal.getSegments()) { - IndexTaskUtils.setSegmentDimensions(metricBuilder, segment); - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); - } - } else { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); - } - + IndexTaskUtils.emitSegmentPublishMetrics(retVal, task, toolbox); return retVal; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 9b23db71d464..5a9ca0cacdfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -33,13 +33,8 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.emitter.service.SegmentMetadataEvent; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -222,47 +217,10 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } - // Emit metrics - final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); - IndexTaskUtils.setTaskDimensions(metricBuilder, task); - - if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); - } else { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); - } - - // getSegments() should return an empty set if announceHistoricalSegments() failed - for (DataSegment segment : retVal.getSegments()) { - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - metricBuilder.setDimension( - DruidMetrics.PARTITIONING_TYPE, - segment.getShardSpec() == null ? null : segment.getShardSpec().getType() - ); - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); - // Emit the segment related metadata using the configured emitters. - // There is a possibility that some segments' metadata event might get missed if the - // server crashes after commiting segment but before emitting the event. - this.emitSegmentMetadata(segment, toolbox); - } - + IndexTaskUtils.emitSegmentPublishMetrics(retVal, task, toolbox); return retVal; } - private void emitSegmentMetadata(DataSegment segment, TaskActionToolbox toolbox) - { - SegmentMetadataEvent event = new SegmentMetadataEvent( - segment.getDataSource(), - DateTime.now(DateTimeZone.UTC), - segment.getInterval().getStart(), - segment.getInterval().getEnd(), - segment.getVersion(), - segment.getLastCompactionState() != null - ); - - toolbox.getEmitter().emit(event); - } - private void checkWithSegmentLock() { final Map> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 5a1228e1dd1d..119a59ea648a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -27,9 +27,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.ReplaceTaskLock; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; @@ -111,23 +109,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } - // Emit metrics - final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); - IndexTaskUtils.setTaskDimensions(metricBuilder, task); - - if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); - - for (DataSegment segment : retVal.getSegments()) { - final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); - metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); - } - } else { - toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); - } - + IndexTaskUtils.emitSegmentPublishMetrics(retVal, task, toolbox); return retVal; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 171d53b9cdd6..0948c036ea73 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -38,6 +38,7 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), + @JsonSubTypes.Type(name = "commitRealtimeSegments", value = CommitRealtimeSegmentsAndMetadataAction.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. 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 dfa1f85fde72..8293c51a768f 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 @@ -696,7 +696,7 @@ private void publishSegments( ); pendingHandoffs.add(Futures.transformAsync( publishFuture, - (AsyncFunction) driver::registerHandoff, + driver::registerHandoff, MoreExecutors.directExecutor() )); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java index 20f7584c8eb2..79a3e8993a8c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java @@ -20,8 +20,10 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.emitter.service.SegmentMetadataEvent; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.ParseExceptionReport; @@ -35,7 +37,6 @@ import org.apache.druid.server.security.ResourceType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CircularBuffer; -import org.joda.time.DateTime; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; @@ -45,29 +46,6 @@ public class IndexTaskUtils { - @Nullable - public static List getMessagesFromSavedParseExceptions( - CircularBuffer savedParseExceptions, - boolean includeTimeOfException - ) - { - if (savedParseExceptions == null) { - return null; - } - - List events = new ArrayList<>(); - for (int i = 0; i < savedParseExceptions.size(); i++) { - if (includeTimeOfException) { - DateTime timeOfException = DateTimes.utc(savedParseExceptions.getLatest(i).getTimeOfExceptionMillis()); - events.add(timeOfException + ", " + savedParseExceptions.getLatest(i).getMessage()); - } else { - events.add(savedParseExceptions.getLatest(i).getMessage()); - } - } - - return events; - } - @Nullable public static List getReportListFromSavedParseExceptions( CircularBuffer savedParseExceptionReports @@ -152,4 +130,25 @@ public static void setSegmentDimensions( metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); } + + public static void emitSegmentPublishMetrics( + SegmentPublishResult publishResult, + Task task, + TaskActionToolbox toolbox + ) + { + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + + if (publishResult.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); + for (DataSegment segment : publishResult.getSegments()) { + IndexTaskUtils.setSegmentDimensions(metricBuilder, segment); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); + toolbox.getEmitter().emit(SegmentMetadataEvent.create(segment, DateTimes.nowUtc())); + } + } else { + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); + } + } } 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 e44dfe9a451e..c2b548cd779a 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 @@ -60,6 +60,7 @@ import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -319,7 +320,8 @@ public void initializeSequences() throws IOException previous.getValue(), current.getValue(), true, - exclusiveStartPartitions + exclusiveStartPartitions, + getTaskLockType() ) ); previous = current; @@ -334,7 +336,8 @@ public void initializeSequences() throws IOException previous.getValue(), endOffsets, false, - exclusiveStartPartitions + exclusiveStartPartitions, + getTaskLockType() ) ); } else { @@ -345,7 +348,8 @@ public void initializeSequences() throws IOException ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(), endOffsets, false, - ioConfig.getStartSequenceNumbers().getExclusivePartitions() + ioConfig.getStartSequenceNumbers().getExclusivePartitions(), + getTaskLockType() ) ); } @@ -925,6 +929,11 @@ public void onFailure(Throwable t) return TaskStatus.success(task.getId()); } + private TaskLockType getTaskLockType() + { + return TaskLocks.determineLockTypeForAppend(task.getContext()); + } + private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException { // Check if any publishFuture failed. @@ -1709,7 +1718,8 @@ public Response setEndOffsets( sequenceNumbers, endOffsets, false, - exclusiveStartPartitions + exclusiveStartPartitions, + getTaskLockType() ); log.info( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index 161a36de2fd0..aa009c674900 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -25,8 +25,13 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.Committer; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.CommitRealtimeSegmentsAndMetadataAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.TaskAction; +import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -54,6 +59,7 @@ public class SequenceMetadata private final String sequenceName; private final Set exclusiveStartPartitions; private final Set assignments; + private final TaskLockType taskLockType; private final boolean sentinel; /** * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because @@ -73,7 +79,8 @@ public SequenceMetadata( @JsonProperty("startOffsets") Map startOffsets, @JsonProperty("endOffsets") Map endOffsets, @JsonProperty("checkpointed") boolean checkpointed, - @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions + @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions, + @JsonProperty("taskLockType") TaskLockType taskLockType ) { Preconditions.checkNotNull(sequenceName); @@ -86,6 +93,7 @@ public SequenceMetadata( this.assignments = new HashSet<>(startOffsets.keySet()); this.checkpointed = checkpointed; this.sentinel = false; + this.taskLockType = taskLockType; this.exclusiveStartPartitions = exclusiveStartPartitions == null ? Collections.emptySet() : exclusiveStartPartitions; @@ -139,6 +147,12 @@ public Map getEndOffsets() } } + @JsonProperty + public TaskLockType getTaskLockType() + { + return taskLockType; + } + @JsonProperty public boolean isSentinel() { @@ -363,7 +377,7 @@ public SegmentPublishResult publishAnnotatedSegments( ); } - final SegmentTransactionalInsertAction action; + final TaskAction action; if (segmentsToPush.isEmpty()) { // If a task ingested no data but made progress reading through its assigned partitions, @@ -395,19 +409,21 @@ public SegmentPublishResult publishAnnotatedSegments( ); } } else if (useTransaction) { - action = SegmentTransactionalInsertAction.appendAction( - segmentsToPush, - runner.createDataSourceMetadata( - new SeekableStreamStartSequenceNumbers<>( - finalPartitions.getStream(), - getStartOffsets(), - exclusiveStartPartitions - ) - ), - runner.createDataSourceMetadata(finalPartitions) + final DataSourceMetadata startMetadata = runner.createDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>( + finalPartitions.getStream(), + getStartOffsets(), + exclusiveStartPartitions + ) ); + final DataSourceMetadata endMetadata = runner.createDataSourceMetadata(finalPartitions); + action = taskLockType == TaskLockType.APPEND + ? CommitRealtimeSegmentsAndMetadataAction.create(segmentsToPush, startMetadata, endMetadata) + : SegmentTransactionalInsertAction.appendAction(segmentsToPush, startMetadata, endMetadata); } else { - action = SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null); + action = taskLockType == TaskLockType.APPEND + ? SegmentTransactionalAppendAction.create(segmentsToPush) + : SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null); } return toolbox.getTaskActionClient().submit(action); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index b78efcbc3469..6a1a92641d3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -23,6 +23,7 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; @@ -60,6 +61,11 @@ public TaskLock acquireReplaceLockOn(Interval interval) return runAction(new TimeChunkLockTryAcquireAction(TaskLockType.REPLACE, interval)); } + public Void releaseLock(Interval interval) + { + return runAction(new LockReleaseAction(interval)); + } + public TaskLock acquireAppendLockOn(Interval interval) { return runAction(new TimeChunkLockTryAcquireAction(TaskLockType.APPEND, interval)); @@ -97,6 +103,28 @@ public SegmentIdWithShardSpec allocateSegmentForTimestamp(DateTime timestamp, Gr ); } + public SegmentIdWithShardSpec allocateSegmentForTimestamp( + DateTime timestamp, + Granularity preferredSegmentGranularity, + String sequenceName + ) + { + return runAction( + new SegmentAllocateAction( + getDataSource(), + timestamp, + Granularities.SECOND, + preferredSegmentGranularity, + getId() + "__" + sequenceName, + null, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + TaskLockType.APPEND + ) + ); + } + private T runAction(TaskAction action) { return execute(() -> client.submit(action)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 293503b1c723..678d17aefbf4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -66,11 +66,14 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; @@ -626,10 +629,10 @@ public void testLockReplaceAllocateLockReplaceLockReplaceAppend() // Allocate an append segment for v1 final ActionsTestTask appendTask1 = createAndStartTask(); - appendTask1.acquireAppendLockOn(YEAR_23); final SegmentIdWithShardSpec pendingSegmentV11 = appendTask1.allocateSegmentForTimestamp(YEAR_23.getStart(), Granularities.YEAR); - Assert.assertEquals(segmentV10.getVersion(), pendingSegmentV11.getVersion()); + Assert.assertEquals(v1, pendingSegmentV11.getVersion()); + Assert.assertEquals(YEAR_23, pendingSegmentV11.getInterval()); // Commit replace segment for v2 final ActionsTestTask replaceTask2 = createAndStartTask(); @@ -771,6 +774,89 @@ public void testMultipleGranularities() verifyIntervalHasVisibleSegments(YEAR_23, segmentV10, segmentV11, segmentV13); } + @Test + public void testSegmentIsAllocatedAtLatestVersion() + { + final SegmentIdWithShardSpec pendingSegmentV01 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(SEGMENT_V0, pendingSegmentV01.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV01.getInterval()); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + verifyIntervalHasUsedSegments(JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10); + + final SegmentIdWithShardSpec pendingSegmentV12 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV12.asSegmentId()); + Assert.assertEquals(v1, pendingSegmentV12.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV12.getInterval()); + + replaceTask.releaseLock(JAN_23); + final ActionsTestTask replaceTask2 = createAndStartTask(); + final String v2 = replaceTask2.acquireReplaceLockOn(JAN_23).getVersion(); + final DataSegment segmentV20 = createSegment(JAN_23, v2); + replaceTask2.commitReplaceSegments(segmentV20); + verifyIntervalHasUsedSegments(JAN_23, segmentV10, segmentV20); + verifyIntervalHasVisibleSegments(JAN_23, segmentV20); + + final SegmentIdWithShardSpec pendingSegmentV23 + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertNotEquals(pendingSegmentV01.asSegmentId(), pendingSegmentV23.asSegmentId()); + Assert.assertEquals(v2, pendingSegmentV23.getVersion()); + Assert.assertEquals(JAN_23, pendingSegmentV23.getInterval()); + + // Commit the append segments + final DataSegment segmentV01 = asSegment(pendingSegmentV01); + final DataSegment segmentV12 = asSegment(pendingSegmentV12); + final DataSegment segmentV23 = asSegment(pendingSegmentV23); + + Set appendedSegments + = appendTask.commitAppendSegments(segmentV01, segmentV12, segmentV23).getSegments(); + Assert.assertEquals(3 + 3, appendedSegments.size()); + + // Verify that the original append segments have been committed + Assert.assertTrue(appendedSegments.remove(segmentV01)); + Assert.assertTrue(appendedSegments.remove(segmentV12)); + Assert.assertTrue(appendedSegments.remove(segmentV23)); + + // Verify that segmentV01 has been upgraded to both v1 and v2 + final DataSegment segmentV11 = findSegmentWith(v1, segmentV01.getLoadSpec(), appendedSegments); + Assert.assertNotNull(segmentV11); + final DataSegment segmentV21 = findSegmentWith(v2, segmentV01.getLoadSpec(), appendedSegments); + Assert.assertNotNull(segmentV21); + + // Verify that segmentV12 has been upgraded to v2 + final DataSegment segmentV22 = findSegmentWith(v2, segmentV12.getLoadSpec(), appendedSegments); + Assert.assertNotNull(segmentV22); + + // Verify that segmentV23 is not downgraded to v1 + final DataSegment segmentV13 = findSegmentWith(v1, segmentV23.getLoadSpec(), appendedSegments); + Assert.assertNull(segmentV13); + + verifyIntervalHasUsedSegments( + YEAR_23, + segmentV01, + segmentV10, segmentV11, segmentV12, + segmentV20, segmentV21, segmentV22, segmentV23 + ); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV20, segmentV21, segmentV22, segmentV23); + } + + @Nullable + private DataSegment findSegmentWith(String version, Map loadSpec, Set segments) { + for (DataSegment segment : segments) { + if (version.equals(segment.getVersion()) + && Objects.equals(segment.getLoadSpec(), loadSpec)) { + return segment; + } + } + + return null; + } + private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) { final SegmentId id = pendingSegment.asSegmentId(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java index aae07194bb94..af7410bd1d79 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java @@ -76,7 +76,8 @@ public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNull ImmutableMap.of(), ImmutableMap.of(), true, - ImmutableSet.of() + ImmutableSet.of(), + null ); TransactionalSegmentPublisher transactionalSegmentPublisher = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, true); @@ -109,7 +110,8 @@ public void testPublishAnnotatedSegmentsSucceedIfDropSegmentsAndOverwriteSegment ImmutableMap.of(), ImmutableMap.of(), true, - ImmutableSet.of() + ImmutableSet.of(), + null ); TransactionalSegmentPublisher transactionalSegmentPublisher = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, false); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java index bc3769b62361..7e249f72d0a6 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonValue; import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; /** @@ -62,6 +63,18 @@ public class SegmentMetadataEvent implements Event */ private final boolean isCompacted; + public static SegmentMetadataEvent create(DataSegment segment, DateTime eventTime) + { + return new SegmentMetadataEvent( + segment.getDataSource(), + eventTime, + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getVersion(), + segment.getLastCompactionState() != null + ); + } + public SegmentMetadataEvent( String dataSource, DateTime createdTime, From 1182c74e113ffc0265c850cf7bd0ed0170609770 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 6 Oct 2023 10:12:38 +0530 Subject: [PATCH 03/31] Add IndexerMetadataStorageCoordinator.commitAppendSegmentsAndMetadata --- ...mmitRealtimeSegmentsAndMetadataAction.java | 31 +++-- .../SegmentTransactionalAppendAction.java | 3 - .../AppenderatorDriverRealtimeIndexTask.java | 1 - .../ConcurrentReplaceAndAppendTest.java | 3 +- ...TestIndexerMetadataStorageCoordinator.java | 11 ++ .../IndexerMetadataStorageCoordinator.java | 15 +++ .../IndexerSQLMetadataStorageCoordinator.java | 109 ++++++++++++++---- 7 files changed, 134 insertions(+), 39 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java index 67ad47080723..7a73db0d4a4a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java @@ -23,14 +23,20 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Preconditions; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; +import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -107,28 +113,35 @@ public TypeReference getReturnTypeReference() }; } - /** - * Performs some sanity checks and publishes the given segments. - */ @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { - final SegmentPublishResult publishResult; + // Verify that all the locks are of expected type + final List locks = toolbox.getTaskLockbox().findLocksForTask(task); + for (TaskLock lock : locks) { + if (lock.getType() != TaskLockType.APPEND) { + throw InvalidInput.exception( + "Cannot use action[%s] for task[%s] as it is holding a lock of type[%s] instead of [APPEND].", + "CommitRealtimeSegmentsAndMetadata", task.getId(), lock.getType() + ); + } + } TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + final String datasource = task.getDataSource(); + final Map segmentToReplaceLock + = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + final SegmentPublishResult publishResult; try { publishResult = toolbox.getTaskLockbox().doInCriticalSection( task, segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - // TODO: this might need to call a new method which does the following in the same transaction - // - commit append segments - // - upgrade append segments to replace versions - // - commit metadata - () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegmentsAndMetadata( segments, + segmentToReplaceLock, startMetadata, endMetadata ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 36b69270f991..994454a9a4b2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -69,9 +69,6 @@ public TypeReference getReturnTypeReference() }; } - /** - * Performs some sanity checks and publishes the given segments. - */ @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { 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 8293c51a768f..3a599dd485be 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 @@ -27,7 +27,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.AsyncFunction; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 678d17aefbf4..22f21fb79b62 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -846,7 +846,8 @@ public void testSegmentIsAllocatedAtLatestVersion() } @Nullable - private DataSegment findSegmentWith(String version, Map loadSpec, Set segments) { + private DataSegment findSegmentWith(String version, Map loadSpec, Set segments) + { for (DataSegment segment : segments) { if (version.equals(segment.getVersion()) && Objects.equals(segment.getLoadSpec(), loadSpec)) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 34d2e44552a7..624279e8141b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -175,6 +175,17 @@ public SegmentPublishResult commitAppendSegments( return SegmentPublishResult.ok(commitSegments(appendSegments)); } + @Override + public SegmentPublishResult commitAppendSegmentsAndMetadata( + Set appendSegments, + Map appendSegmentToReplaceLock, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) + { + return SegmentPublishResult.ok(commitSegments(appendSegments)); + } + @Override public SegmentPublishResult commitSegmentsAndMetadata( Set segments, diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 3cbabea78fae..de1aadd320d3 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -300,6 +300,21 @@ SegmentPublishResult commitAppendSegments( Map appendSegmentToReplaceLock ); + /** + * Commits segments created by an APPEND task. This method also handles segment + * upgrade scenarios that may result from concurrent append and replace. Also + * commits start and end {@link DataSourceMetadata}. + * + * @see #commitAppendSegments + * @see #commitSegmentsAndMetadata + */ + SegmentPublishResult commitAppendSegmentsAndMetadata( + Set appendSegments, + Map appendSegmentToReplaceLock, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ); + /** * Commits segments created by a REPLACE task. This method also handles the * segment upgrade scenarios that may result from concurrent append and replace. diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 7eaac692f7ce..74e4011ada05 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -438,33 +438,28 @@ public SegmentPublishResult commitAppendSegments( final Map appendSegmentToReplaceLock ) { - verifySegmentsToCommit(appendSegments); - - final String dataSource = appendSegments.iterator().next().getDataSource(); - final Set upgradedSegments = connector.retryTransaction( - (handle, transactionStatus) - -> getSegmentsToUpgradeOnAppend(handle, dataSource, appendSegments), - 0, - SQLMetadataConnector.DEFAULT_MAX_TRIES + return commitAppendSegmentsAndMetadataInTransaction( + appendSegments, + appendSegmentToReplaceLock, + null, + null ); + } - // Create entries for all required versions of the append segments - final Set allSegmentsToInsert = new HashSet<>(appendSegments); - allSegmentsToInsert.addAll(upgradedSegments); - - try { - return connector.retryTransaction( - (handle, transactionStatus) -> { - insertIntoUpgradeSegmentsTable(handle, appendSegmentToReplaceLock); - return SegmentPublishResult.ok(insertSegments(handle, allSegmentsToInsert)); - }, - 3, - getSqlMetadataMaxRetry() - ); - } - catch (CallbackFailedException e) { - return SegmentPublishResult.fail(e.getMessage()); - } + @Override + public SegmentPublishResult commitAppendSegmentsAndMetadata( + Set appendSegments, + Map appendSegmentToReplaceLock, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) + { + return commitAppendSegmentsAndMetadataInTransaction( + appendSegments, + appendSegmentToReplaceLock, + startMetadata, + endMetadata + ); } @Override @@ -971,6 +966,70 @@ private static class CheckExistingSegmentIdResult } } + private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( + Set appendSegments, + Map appendSegmentToReplaceLock, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) + { + verifySegmentsToCommit(appendSegments); + if ((startMetadata == null && endMetadata != null) + || (startMetadata != null && endMetadata == null)) { + throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); + } + + final String dataSource = appendSegments.iterator().next().getDataSource(); + final Set upgradedSegments = connector.retryTransaction( + (handle, transactionStatus) + -> getSegmentsToUpgradeOnAppend(handle, dataSource, appendSegments), + 0, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + + // Create entries for all required versions of the append segments + final Set allSegmentsToInsert = new HashSet<>(appendSegments); + allSegmentsToInsert.addAll(upgradedSegments); + + final AtomicBoolean metadataNotUpdated = new AtomicBoolean(false); + try { + return connector.retryTransaction( + (handle, transactionStatus) -> { + metadataNotUpdated.set(false); + + if (startMetadata != null) { + final DataStoreMetadataUpdateResult metadataUpdateResult + = updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); + + if (metadataUpdateResult.isFailed()) { + transactionStatus.setRollbackOnly(); + metadataNotUpdated.set(true); + + if (metadataUpdateResult.canRetry()) { + throw new RetryTransactionException(metadataUpdateResult.getErrorMsg()); + } else { + throw new RuntimeException(metadataUpdateResult.getErrorMsg()); + } + } + } + + insertIntoUpgradeSegmentsTable(handle, appendSegmentToReplaceLock); + return SegmentPublishResult.ok(insertSegments(handle, allSegmentsToInsert)); + }, + 3, + getSqlMetadataMaxRetry() + ); + } + catch (CallbackFailedException e) { + if (metadataNotUpdated.get()) { + // Return failed result if metadata was definitely not updated + return SegmentPublishResult.fail(e.getMessage()); + } else { + throw e; + } + } + } + private void insertPendingSegmentsIntoMetastore( Handle handle, Map createdSegments, From 9ddfd5e8462c17043412dd307147c7541b433b45 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 7 Oct 2023 09:03:29 +0530 Subject: [PATCH 04/31] Remove extra task action --- ...mmitRealtimeSegmentsAndMetadataAction.java | 181 ------------------ .../SegmentTransactionalAppendAction.java | 81 +++++++- .../indexing/common/actions/TaskAction.java | 1 - .../common/task/AbstractBatchIndexTask.java | 2 +- .../seekablestream/SequenceMetadata.java | 5 +- .../task/concurrent/ActionsTestTask.java | 2 +- .../seekablestream/SequenceMetadataTest.java | 24 +-- .../service/SegmentMetadataEventTest.java | 31 +++ 8 files changed, 119 insertions(+), 208 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java deleted file mode 100644 index 7a73db0d4a4a..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitRealtimeSegmentsAndMetadataAction.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.common.actions; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Preconditions; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.task.IndexTaskUtils; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.metadata.ReplaceTaskLock; -import org.apache.druid.segment.SegmentUtils; -import org.apache.druid.timeline.DataSegment; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * Task action to commit realtime segments and metadata when using APPEND task locks. - *

- * This action performs the following operations in a single transaction: - *

    - *
  • Commit the append segments
  • - *
  • Upgrade the append segments to all visible REPLACE versions
  • - *
  • Commit start and end {@link DataSourceMetadata}.
  • - *
- * This action differs from {@link SegmentTransactionalInsertAction} as it is used - * only with APPEND locks and also upgrades segments as needed. - */ -public class CommitRealtimeSegmentsAndMetadataAction implements TaskAction -{ - /** - * Set of segments to be inserted into metadata storage - */ - private final Set segments; - - private final DataSourceMetadata startMetadata; - private final DataSourceMetadata endMetadata; - - public static CommitRealtimeSegmentsAndMetadataAction create( - Set segments, - DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata - ) - { - return new CommitRealtimeSegmentsAndMetadataAction(segments, startMetadata, endMetadata); - } - - @JsonCreator - private CommitRealtimeSegmentsAndMetadataAction( - @JsonProperty("segments") Set segments, - @JsonProperty("startMetadata") DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") DataSourceMetadata endMetadata - ) - { - Preconditions.checkArgument( - segments != null && !segments.isEmpty(), - "Segments to commit should not be empty" - ); - this.segments = segments; - this.startMetadata = startMetadata; - this.endMetadata = endMetadata; - } - - @JsonProperty - public Set getSegments() - { - return segments; - } - - @JsonProperty - public DataSourceMetadata getStartMetadata() - { - return startMetadata; - } - - @JsonProperty - public DataSourceMetadata getEndMetadata() - { - return endMetadata; - } - - @Override - public TypeReference getReturnTypeReference() - { - return new TypeReference() - { - }; - } - - @Override - public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) - { - // Verify that all the locks are of expected type - final List locks = toolbox.getTaskLockbox().findLocksForTask(task); - for (TaskLock lock : locks) { - if (lock.getType() != TaskLockType.APPEND) { - throw InvalidInput.exception( - "Cannot use action[%s] for task[%s] as it is holding a lock of type[%s] instead of [APPEND].", - "CommitRealtimeSegmentsAndMetadata", task.getId(), lock.getType() - ); - } - } - - TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final String datasource = task.getDataSource(); - final Map segmentToReplaceLock - = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); - - final SegmentPublishResult publishResult; - try { - publishResult = toolbox.getTaskLockbox().doInCriticalSection( - task, - segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), - CriticalAction.builder() - .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegmentsAndMetadata( - segments, - segmentToReplaceLock, - startMetadata, - endMetadata - ) - ) - .onInvalidLocks( - () -> SegmentPublishResult.fail( - "Invalid task locks. Maybe they are revoked by a higher priority task." - + " Please check the overlord log for details." - ) - ) - .build() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - - IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); - return publishResult; - } - - @Override - public boolean isAudited() - { - return true; - } - - @Override - public String toString() - { - return "CommitRealtimeSegmentsAndMetadataAction{" + - ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + - ", startMetadata=" + startMetadata + - ", endMetadata=" + endMetadata + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 994454a9a4b2..125a008cdc90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -22,14 +22,20 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; +import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -41,18 +47,40 @@ public class SegmentTransactionalAppendAction implements TaskAction { private final Set segments; + @Nullable + private final DataSourceMetadata startMetadata; + @Nullable + private final DataSourceMetadata endMetadata; - public static SegmentTransactionalAppendAction create(Set segments) + public static SegmentTransactionalAppendAction forSegments(Set segments) { - return new SegmentTransactionalAppendAction(segments); + return new SegmentTransactionalAppendAction(segments, null, null); + } + + public static SegmentTransactionalAppendAction forSegmentsAndMetadata( + Set segments, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) + { + return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata); } @JsonCreator private SegmentTransactionalAppendAction( - @JsonProperty("segments") Set segments + @JsonProperty("segments") Set segments, + @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata ) { this.segments = segments; + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; + + if ((startMetadata == null && endMetadata != null) + || (startMetadata != null && endMetadata == null)) { + throw InvalidInput.exception("startMetadata and endMetadata must either be both null or both non-null."); + } } @JsonProperty @@ -61,6 +89,20 @@ public Set getSegments() return segments; } + @JsonProperty + @Nullable + public DataSourceMetadata getStartMetadata() + { + return startMetadata; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + @Override public TypeReference getReturnTypeReference() { @@ -72,24 +114,45 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { + // Verify that all the locks are of expected type + final List locks = toolbox.getTaskLockbox().findLocksForTask(task); + for (TaskLock lock : locks) { + if (lock.getType() != TaskLockType.APPEND) { + throw InvalidInput.exception( + "Cannot use action[%s] for task[%s] as it is holding a lock of type[%s] instead of [APPEND].", + "CommitRealtimeSegmentsAndMetadata", task.getId(), lock.getType() + ); + } + } + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); final String datasource = task.getDataSource(); final Map segmentToReplaceLock = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + final CriticalAction.Action publishAction; + if (startMetadata == null) { + publishAction = () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( + segments, + segmentToReplaceLock + ); + } else { + publishAction = () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegmentsAndMetadata( + segments, + segmentToReplaceLock, + startMetadata, + endMetadata + ); + } + final SegmentPublishResult retVal; try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() - .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( - segments, - segmentToReplaceLock - ) - ) + .onValidLocks(publishAction) .onInvalidLocks( () -> SegmentPublishResult.fail( "Invalid task locks. Maybe they are revoked by a higher priority task." diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 0948c036ea73..171d53b9cdd6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -38,7 +38,6 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), - @JsonSubTypes.Type(name = "commitRealtimeSegments", value = CommitRealtimeSegmentsAndMetadataAction.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. 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 ea61f37c7e90..276d82eedc84 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 @@ -415,7 +415,7 @@ protected TaskAction buildPublishAction( case REPLACE: return SegmentTransactionalReplaceAction.create(segmentsToPublish); case APPEND: - return SegmentTransactionalAppendAction.create(segmentsToPublish); + return SegmentTransactionalAppendAction.forSegments(segmentsToPublish); default: return SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index aa009c674900..47ccbaa00b35 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.Committer; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.CommitRealtimeSegmentsAndMetadataAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -418,11 +417,11 @@ public SegmentPublishResult publishAnnotatedSegments( ); final DataSourceMetadata endMetadata = runner.createDataSourceMetadata(finalPartitions); action = taskLockType == TaskLockType.APPEND - ? CommitRealtimeSegmentsAndMetadataAction.create(segmentsToPush, startMetadata, endMetadata) + ? SegmentTransactionalAppendAction.forSegmentsAndMetadata(segmentsToPush, startMetadata, endMetadata) : SegmentTransactionalInsertAction.appendAction(segmentsToPush, startMetadata, endMetadata); } else { action = taskLockType == TaskLockType.APPEND - ? SegmentTransactionalAppendAction.create(segmentsToPush) + ? SegmentTransactionalAppendAction.forSegments(segmentsToPush) : SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index 6a1a92641d3f..69a8b6cc1030 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -81,7 +81,7 @@ public SegmentPublishResult commitReplaceSegments(DataSegment... segments) public SegmentPublishResult commitAppendSegments(DataSegment... segments) { return runAction( - SegmentTransactionalAppendAction.create(Sets.newHashSet(segments)) + SegmentTransactionalAppendAction.forSegments(Sets.newHashSet(segments)) ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java index af7410bd1d79..fbe63ffe2689 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java @@ -29,9 +29,8 @@ import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; -import org.junit.Rule; +import org.junit.Assert; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.ArgumentMatchers; import org.mockito.Mock; @@ -43,9 +42,6 @@ @RunWith(MockitoJUnitRunner.class) public class SequenceMetadataTest { - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Mock private SeekableStreamIndexTaskRunner mockSeekableStreamIndexTaskRunner; @@ -59,7 +55,7 @@ public class SequenceMetadataTest private TaskToolbox mockTaskToolbox; @Test - public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNullAndNotEmpty() throws Exception + public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNullAndNotEmpty() { DataSegment dataSegment = DataSegment.builder() .dataSource("foo") @@ -79,14 +75,18 @@ public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNull ImmutableSet.of(), null ); - TransactionalSegmentPublisher transactionalSegmentPublisher = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, true); + TransactionalSegmentPublisher transactionalSegmentPublisher + = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, true); - expectedException.expect(ISE.class); - expectedException.expectMessage( - "Stream ingestion task unexpectedly attempted to overwrite segments: " + SegmentUtils.commaSeparatedIdentifiers(notNullNotEmptySegment) + ISE exception = Assert.assertThrows( + ISE.class, + () -> transactionalSegmentPublisher.publishAnnotatedSegments(notNullNotEmptySegment, ImmutableSet.of(), null) + ); + Assert.assertEquals( + "Stream ingestion task unexpectedly attempted to overwrite segments: " + + SegmentUtils.commaSeparatedIdentifiers(notNullNotEmptySegment), + exception.getMessage() ); - - transactionalSegmentPublisher.publishAnnotatedSegments(notNullNotEmptySegment, ImmutableSet.of(), null); } @Test diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEventTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEventTest.java index 83a4fcba7dc5..a926b004c0af 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEventTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEventTest.java @@ -21,6 +21,11 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -51,4 +56,30 @@ public void testBasicEvent() event.toMap() ); } + + @Test + public void testCreate() + { + final DataSegment segment = DataSegment.builder() + .dataSource("wiki") + .interval(Intervals.of("2023/2024")) + .shardSpec(new NumberedShardSpec(1, 1)) + .version("v1") + .size(100) + .build(); + final DateTime eventTime = DateTimes.nowUtc(); + SegmentMetadataEvent event = SegmentMetadataEvent.create(segment, eventTime); + Assert.assertEquals( + EventMap.builder() + .put(SegmentMetadataEvent.FEED, "segment_metadata") + .put(SegmentMetadataEvent.DATASOURCE, segment.getDataSource()) + .put(SegmentMetadataEvent.CREATED_TIME, eventTime) + .put(SegmentMetadataEvent.START_TIME, segment.getInterval().getStart()) + .put(SegmentMetadataEvent.END_TIME, segment.getInterval().getEnd()) + .put(SegmentMetadataEvent.VERSION, segment.getVersion()) + .put(SegmentMetadataEvent.IS_COMPACTED, false) + .build(), + event.toMap() + ); + } } From 527bb792d50bb642dfd1e76680122b22fdba188b Mon Sep 17 00:00:00 2001 From: Amatya Date: Sun, 8 Oct 2023 16:02:54 +0530 Subject: [PATCH 05/31] Handle upgraded segment announcing and unannouncing --- .../SeekableStreamIndexTaskClient.java | 9 ++++ ...eekableStreamIndexTaskClientAsyncImpl.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 11 ++-- .../appenderator/StreamAppenderator.java | 53 +++++++++++++++++-- 4 files changed, 68 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 806bcd6c692c..094b0234c98c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -155,6 +155,15 @@ ListenableFuture setEndOffsetsAsync( */ ListenableFuture getStatusAsync(String id); + /** + * 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 id - task id + * @param rootPendingSegment - the pending segment that was originally allocated + * @param versionsOfPendingSegment - the ids belonging to the versions to which the root segment needs to be updated + * @return true if the update succeeds + */ ListenableFuture updatePendingSegmentMapping( String id, SegmentIdWithShardSpec rootPendingSegment, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 8b7b9c616775..ab71406df716 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -205,7 +205,7 @@ public ListenableFuture updatePendingSegmentMapping( { final RequestBuilder requestBuilder = new RequestBuilder( HttpMethod.POST, - "updatePendingSegmentMapping" + "pendingSegmentMapping" ).jsonContent(jsonMapper, Pair.of(rootPendingSegment, versionsOfPendingSegment)); return makeRequest(id, requestBuilder) 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 460a1096adde..21721c12836f 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 @@ -1545,7 +1545,7 @@ public Response setEndOffsetsHTTP( } @POST - @Path("updatePendingSegmentMapping") + @Path("pendingSegmentMapping") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response updatePendingSegmentMapping( @@ -1767,8 +1767,13 @@ private Response updatePendingSegmentMapping( Set versionsOfPendingSegment ) { - ((StreamAppenderator) appenderator).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); - return Response.ok().build(); + try { + ((StreamAppenderator) appenderator).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + return Response.ok().build(); + } + catch (IOException e) { + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } } private void resetNextCheckpointTime() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index be50de901ec3..043372777fab 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -72,6 +72,7 @@ import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.joda.time.Interval; @@ -86,6 +87,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -153,6 +155,8 @@ public class StreamAppenderator implements Appenderator private final AtomicBoolean closed = new AtomicBoolean(false); + private final ConcurrentMap> rootPendingSegmentToNewerVersions; + private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; // use intermediate executor so that deadlock conditions can be prevented @@ -216,6 +220,7 @@ public class StreamAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); this.useMaxMemoryEstimates = useMaxMemoryEstimates; + rootPendingSegmentToNewerVersions = new ConcurrentHashMap<>(); } @Override @@ -998,7 +1003,7 @@ public void closeNow() log.debug("Shutting down immediately..."); for (Map.Entry entry : sinks.entrySet()) { try { - segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); + unannounceRootSegmentAndUpgradedVersions(entry.getValue()); } catch (Exception e) { log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) @@ -1026,15 +1031,57 @@ public void closeNow() } } + private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOException + { + final DataSegment rootSegment = sink.getSegment(); + segmentAnnouncer.unannounceSegment(rootSegment); + for (SegmentIdWithShardSpec newId : rootPendingSegmentToNewerVersions.get(rootSegment.getId())) { + final DataSegment newSegment = new DataSegment( + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + rootSegment.getLoadSpec(), + rootSegment.getDimensions(), + rootSegment.getMetrics(), + newId.getShardSpec(), + rootSegment.getBinaryVersion(), + rootSegment.getSize() + ); + segmentAnnouncer.unannounceSegment(newSegment); + } + rootPendingSegmentToNewerVersions.remove(rootSegment.getId()); + } + public void updatePendingSegmentMapping( SegmentIdWithShardSpec rootPendingSegment, Set versionsOfPendingSegment - ) + ) throws IOException { if (!sinks.containsKey(rootPendingSegment) || droppingSinks.contains(rootPendingSegment)) { return; } + + // Update query mapping with SinkQuerySegmentWalker ((SinkQuerySegmentWalker) texasRanger).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + + // Announce segments + rootPendingSegmentToNewerVersions.putIfAbsent(rootPendingSegment.asSegmentId(), new HashSet<>()); + final DataSegment rootSegment = sinks.get(rootPendingSegment).getSegment(); + for (SegmentIdWithShardSpec idWithShardSpec : versionsOfPendingSegment) { + final DataSegment newSegment = new DataSegment( + idWithShardSpec.getDataSource(), + idWithShardSpec.getInterval(), + idWithShardSpec.getVersion(), + rootSegment.getLoadSpec(), + rootSegment.getDimensions(), + rootSegment.getMetrics(), + idWithShardSpec.getShardSpec(), + rootSegment.getBinaryVersion(), + rootSegment.getSize() + ); + segmentAnnouncer.announceSegment(newSegment); + rootPendingSegmentToNewerVersions.get(rootPendingSegment.asSegmentId()).add(idWithShardSpec); + } } private void lockBasePersistDirectory() @@ -1338,7 +1385,7 @@ public Void apply(@Nullable Object input) // Unannounce the segment. try { - segmentAnnouncer.unannounceSegment(sink.getSegment()); + unannounceRootSegmentAndUpgradedVersions(sink); } catch (Exception e) { log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) From e19b2cad4a73923e209dca1826675cb74cc983fa Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 9 Oct 2023 21:07:20 +0530 Subject: [PATCH 06/31] Upgrade pending segments --- .../SegmentTransactionalReplaceAction.java | 35 ++- .../indexing/common/actions/TaskLocks.java | 2 + .../druid/indexing/overlord/TaskLockbox.java | 3 +- ...TestIndexerMetadataStorageCoordinator.java | 18 +- .../IndexerMetadataStorageCoordinator.java | 12 + .../IndexerSQLMetadataStorageCoordinator.java | 220 +++++++++++++++--- .../druid/metadata/SQLMetadataConnector.java | 4 + 7 files changed, 257 insertions(+), 37 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 119a59ea648a..238b6a0e7743 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -27,10 +27,13 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -40,6 +43,8 @@ */ public class SegmentTransactionalReplaceAction implements TaskAction { + private static final Logger log = new Logger(SegmentTransactionalReplaceAction.class); + /** * Set of segments to be inserted into metadata storage */ @@ -86,9 +91,9 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) final Set replaceLocksForTask = toolbox.getTaskLockbox().findReplaceLocksForTask(task); - final SegmentPublishResult retVal; + final SegmentPublishResult publishResult; try { - retVal = toolbox.getTaskLockbox().doInCriticalSection( + publishResult = toolbox.getTaskLockbox().doInCriticalSection( task, segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() @@ -109,8 +114,30 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } - IndexTaskUtils.emitSegmentPublishMetrics(retVal, task, toolbox); - return retVal; + IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); + + if (publishResult.isSuccess()) { + // If upgrade of pending segments fails, the segments will still get upgraded + // when the corresponding APPEND task commits the segments. + // Thus, the upgrade of pending segments should not be done in the same + // transaction as the commit of replace segments and failure to upgrade + // pending segments should not affect success of replace commit. + try { + List upgradedPendingSegments = + toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegments(segments); + log.info( + "Upgraded [%d] pending segments for REPLACE task[%s]: [%s]", + upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments + ); + + // These upgraded pending segments should be forwarded to the SupervisorManager + } + catch (Exception e) { + log.error(e, "Error while upgrading pending segments for task[%s]", task.getId()); + } + } + + return publishResult; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index bb835997801f..d60649359256 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -125,6 +125,8 @@ public static boolean isLockCoversSegments( && timeChunkLock.getDataSource().equals(segment.getDataSource()) && (timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0 || TaskLockType.APPEND.equals(timeChunkLock.getType())); + // APPEND locks always have the version DateTimes.EPOCH (1970-01-01) + // and cover the segments irrespective of the segment version } else { final SegmentLock segmentLock = (SegmentLock) lock; return segmentLock.getInterval().contains(segment.getInterval()) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 761c0b591605..b7aab8505346 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -29,6 +29,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; +import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; @@ -941,7 +942,7 @@ private Set getNonRevokedReplaceLocks(List posse // Replace locks are always held by the supervisor task if (posse.taskIds.size() > 1) { - throw new ISE( + throw DruidException.defensive( "Replace lock[%s] for datasource[%s] is held by multiple tasks[%s]", lock, datasource, posse.taskIds ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 624279e8141b..dfaa1eb86e43 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -113,7 +113,11 @@ public List retrieveUnusedSegmentsForInterval(String dataSource, In } @Override - public List retrieveUnusedSegmentsForInterval(String dataSource, Interval interval, @Nullable Integer limit) + public List retrieveUnusedSegmentsForInterval( + String dataSource, + Interval interval, + @Nullable Integer limit + ) { synchronized (unusedSegments) { Stream resultStream = unusedSegments.stream(); @@ -233,6 +237,18 @@ public SegmentIdWithShardSpec allocatePendingSegment( ); } + @Override + public Set upgradePendingSegments(Set replaceSegments) + { + return Collections.emptySet(); + } + + @Override + public Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec segmentIdWithShardSpec) + { + return Collections.emptySet(); + } + @Override public int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval) { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index de1aadd320d3..540616ab7e07 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -334,6 +334,18 @@ SegmentPublishResult commitReplaceSegments( Set locksHeldByReplaceTask ); + /** + * Creates new versions for the pending segments that overlap with the given + * replace segments being committed. + * + * @param replaceSegments Segments being committed by a REPLACE task + * @return List of pending segments chosen for upgrade. The returned list does + * not contain the new versions of the pending segments. + */ + Set upgradePendingSegments(Set replaceSegments); + + Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec segmentIdWithShardSpec); + /** * Retrieves data source's metadata from the metadata store. Returns null if there is no metadata. */ diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 74e4011ada05..93d6bf932b5b 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -105,6 +105,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class); private static final int MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE = 100; + private static final String UPGRADED_PENDING_SEGMENT_PREFIX = "upgraded_to_replace_version_"; + private final ObjectMapper jsonMapper; private final MetadataStorageTablesConfig dbTables; private final SQLMetadataConnector connector; @@ -237,44 +239,45 @@ public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interv /** * Fetches all the pending segments, whose interval overlaps with the given - * search interval from the metadata store. + * search interval from the metadata store. Returns a Map from the + * pending segment ID to the sequence name. */ - private Set getPendingSegmentsForIntervalWithHandle( + private Map getPendingSegmentsForIntervalWithHandle( final Handle handle, final String dataSource, final Interval interval ) throws IOException { - final Set identifiers = new HashSet<>(); - - final ResultIterator dbSegments = + final ResultIterator dbSegments = handle.createQuery( StringUtils.format( // This query might fail if the year has a different number of digits // See https://github.com/apache/druid/pull/11582 for a similar issue // Using long for these timestamps instead of varchar would give correct time comparisons - "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", + "SELECT sequence_name, payload FROM %1$s" + + " WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", dbTables.getPendingSegmentsTable(), connector.getQuoteString() ) ) .bind("dataSource", dataSource) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) - .map(ByteArrayMapper.FIRST) + .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r)) .iterator(); + final Map pendingSegmentToSequenceName = new HashMap<>(); while (dbSegments.hasNext()) { - final byte[] payload = dbSegments.next(); - final SegmentIdWithShardSpec identifier = jsonMapper.readValue(payload, SegmentIdWithShardSpec.class); + PendingSegmentsRecord record = dbSegments.next(); + final SegmentIdWithShardSpec identifier = jsonMapper.readValue(record.payload, SegmentIdWithShardSpec.class); if (interval.overlaps(identifier.getInterval())) { - identifiers.add(identifier); + pendingSegmentToSequenceName.put(identifier, record.sequenceName); } } dbSegments.close(); - return identifiers; + return pendingSegmentToSequenceName; } private SegmentTimeline getTimelineForIntervalsWithHandle( @@ -417,7 +420,7 @@ public SegmentPublishResult commitReplaceSegments( (handle, transactionStatus) -> { final Set segmentsToInsert = new HashSet<>(replaceSegments); segmentsToInsert.addAll( - getSegmentsToUpgradeOnReplace(handle, replaceSegments, locksHeldByReplaceTask) + createUpgradedVersionsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask) ); return SegmentPublishResult.ok( insertSegments(handle, segmentsToInsert) @@ -596,6 +599,157 @@ public SegmentIdWithShardSpec allocatePendingSegment( ); } + @Override + public Set upgradePendingSegments(Set replaceSegments) + { + if (replaceSegments.isEmpty()) { + return Collections.emptySet(); + } + + // Any replace interval has exactly one version of segments + final Map replaceIntervalToMaxId = new HashMap<>(); + for (DataSegment segment : replaceSegments) { + DataSegment committedMaxId = replaceIntervalToMaxId.get(segment.getInterval()); + if (committedMaxId == null + || committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) { + replaceIntervalToMaxId.put(segment.getInterval(), segment); + } + } + + final String datasource = replaceSegments.iterator().next().getDataSource(); + return connector.retryWithHandle( + handle -> upgradePendingSegments(handle, datasource, replaceIntervalToMaxId) + ); + } + + @Override + public Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec pendingSegment) + { + return connector.retryWithHandle( + handle -> findAllVersionsOfPendingSegment(handle, pendingSegment) + ); + } + + private Set findAllVersionsOfPendingSegment( + Handle handle, + SegmentIdWithShardSpec pendingSegment + ) throws IOException + { + final Interval interval = pendingSegment.getInterval(); + final Query> query = handle + .createQuery( + StringUtils.format( + "SELECT payload " + + "FROM %s WHERE " + + "dataSource = :dataSource AND " + + "start = :start AND " + + "%2$send%2$s = :end AND " + + "sequence_prev_id = :sequence_prev_id", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + ) + ) + .bind("dataSource", pendingSegment.getDataSource()) + .bind("sequence_prev_id", pendingSegment.asSegmentId().toString()) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + + final ResultIterator dbSegments = query + .map(ByteArrayMapper.FIRST) + .iterator(); + + final Set allVersions = new HashSet<>(); + while (dbSegments.hasNext()) { + final byte[] payload = dbSegments.next(); + final SegmentIdWithShardSpec segmentId = + jsonMapper.readValue(payload, SegmentIdWithShardSpec.class); + allVersions.add(segmentId); + } + + return allVersions; + } + + /** + * Finds pending segments contained in each replace interval and upgrades them + * to the replace version. + */ + private Set upgradePendingSegments( + Handle handle, + String datasource, + Map replaceIntervalToMaxId + ) throws IOException + { + final Map newPendingSegmentVersions = new HashMap<>(); + + for (Map.Entry entry : replaceIntervalToMaxId.entrySet()) { + final Interval replaceInterval = entry.getKey(); + final DataSegment maxSegmentId = entry.getValue(); + final String replaceVersion = maxSegmentId.getVersion(); + + final int numCorePartitions = maxSegmentId.getShardSpec().getNumCorePartitions(); + int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum(); + + final Map overlappingPendingSegments + = getPendingSegmentsForIntervalWithHandle(handle, datasource, replaceInterval); + + for (Map.Entry overlappingPendingSegment + : overlappingPendingSegments.entrySet()) { + final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getKey(); + final String pendingSegmentSequence = overlappingPendingSegment.getValue(); + if (shouldUpgradePendingSegment(pendingSegmentId, pendingSegmentSequence, replaceInterval, replaceVersion)) { + // There cannot be any duplicates because this version not been committed before + newPendingSegmentVersions.put( + new SegmentCreateRequest( + UPGRADED_PENDING_SEGMENT_PREFIX + replaceVersion, + pendingSegmentId.toString(), + replaceVersion, + NumberedPartialShardSpec.instance() + ), + new SegmentIdWithShardSpec( + datasource, + replaceInterval, + replaceVersion, + new NumberedShardSpec(++currentPartitionNumber, numCorePartitions) + ) + ); + } + } + } + + // Do not skip lineage check so that the sequence_name_prev_id_sha1 + // includes hash of both sequence_name and prev_segment_id + int numInsertedPendingSegments = insertPendingSegmentsIntoMetastore( + handle, + newPendingSegmentVersions, + datasource, + false + ); + log.info( + "Inserted total [%d] new versions for [%d] pending segments.", + numInsertedPendingSegments, newPendingSegmentVersions.size() + ); + + return new HashSet<>(newPendingSegmentVersions.values()); + } + + private boolean shouldUpgradePendingSegment( + SegmentIdWithShardSpec pendingSegmentId, + String pendingSegmentSequenceName, + Interval replaceInterval, + String replaceVersion + ) + { + if (pendingSegmentId.getVersion().compareTo(replaceVersion) >= 0) { + return false; + } else if (!replaceInterval.contains(pendingSegmentId.getInterval())) { + return false; + } else { + // Do not upgrade already upgraded pending segment + return pendingSegmentSequenceName == null + || !pendingSegmentSequenceName.startsWith(UPGRADED_PENDING_SEGMENT_PREFIX); + } + } + @Nullable private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( final Handle handle, @@ -716,7 +870,6 @@ private Map allocatePendingSegment handle, createdSegments, dataSource, - interval, skipSegmentLineageCheck ); @@ -982,7 +1135,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( final String dataSource = appendSegments.iterator().next().getDataSource(); final Set upgradedSegments = connector.retryTransaction( (handle, transactionStatus) - -> getSegmentsToUpgradeOnAppend(handle, dataSource, appendSegments), + -> createUpgradedVersionsOfAppendSegments(handle, dataSource, appendSegments), 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); @@ -1030,11 +1183,10 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } } - private void insertPendingSegmentsIntoMetastore( + private int insertPendingSegmentsIntoMetastore( Handle handle, Map createdSegments, String dataSource, - Interval interval, boolean skipSegmentLineageCheck ) throws JsonProcessingException { @@ -1055,6 +1207,8 @@ private void insertPendingSegmentsIntoMetastore( for (Map.Entry entry : segmentIdToRequest.entrySet()) { final SegmentCreateRequest request = entry.getValue(); final SegmentIdWithShardSpec segmentId = entry.getKey(); + final Interval interval = segmentId.getInterval(); + insertBatch.add() .bind("id", segmentId.toString()) .bind("dataSource", dataSource) @@ -1069,7 +1223,8 @@ private void insertPendingSegmentsIntoMetastore( ) .bind("payload", jsonMapper.writeValueAsBytes(segmentId)); } - insertBatch.execute(); + int[] updated = insertBatch.execute(); + return Arrays.stream(updated).sum(); } private void insertPendingSegmentIntoMetastore( @@ -1105,7 +1260,7 @@ private void insertPendingSegmentIntoMetastore( } /** - * Allocates and returns any extra versions that need to be committed for the + * Creates and returns any extra versions that need to be committed for the * given append segments. *

* This is typically needed when a REPLACE task started and finished after @@ -1113,7 +1268,7 @@ private void insertPendingSegmentIntoMetastore( * there would be some used segments in the DB with versions higher than these * append segments. */ - private Set getSegmentsToUpgradeOnAppend( + private Set createUpgradedVersionsOfAppendSegments( Handle handle, String dataSource, Set segmentsToAppend @@ -1138,17 +1293,17 @@ private Set getSegmentsToUpgradeOnAppend( Segments.INCLUDING_OVERSHADOWED ); - final Map> committedVersionToIntervals = new HashMap<>(); - final Map> committedIntervalToSegments = new HashMap<>(); + final Map> overlappingVersionToIntervals = new HashMap<>(); + final Map> overlappingIntervalToSegments = new HashMap<>(); for (DataSegment segment : overlappingSegments) { - committedVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + overlappingVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) .add(segment.getInterval()); - committedIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>()) + overlappingIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>()) .add(segment); } final Set upgradedSegments = new HashSet<>(); - for (Map.Entry> entry : committedVersionToIntervals.entrySet()) { + for (Map.Entry> entry : overlappingVersionToIntervals.entrySet()) { final String upgradeVersion = entry.getKey(); Map> segmentsToUpgrade = getSegmentsWithVersionLowerThan( upgradeVersion, @@ -1156,12 +1311,12 @@ private Set getSegmentsToUpgradeOnAppend( appendVersionToSegments ); for (Map.Entry> upgradeEntry : segmentsToUpgrade.entrySet()) { - Set segmentsUpgradedToVersion = upgradeSegmentsToVersion( + Set segmentsUpgradedToVersion = createUpgradedVersionOfSegments( handle, upgradeVersion, upgradeEntry.getKey(), upgradeEntry.getValue(), - committedIntervalToSegments + overlappingIntervalToSegments ); log.info("Upgraded [%d] segments to version[%s].", segmentsUpgradedToVersion.size(), upgradeVersion); upgradedSegments.addAll(segmentsUpgradedToVersion); @@ -1212,7 +1367,7 @@ private Map> getSegmentsWithVersionLowerThan( * Computes new Segment IDs for the {@code segmentsToUpgrade} being upgraded * to the given {@code upgradeVersion}. */ - private Set upgradeSegmentsToVersion( + private Set createUpgradedVersionOfSegments( Handle handle, String upgradeVersion, Interval interval, @@ -1237,7 +1392,7 @@ private Set upgradeSegmentsToVersion( // Get pending segments for the new version, if any final String dataSource = segmentsToUpgrade.iterator().next().getDataSource(); final Set pendingSegments - = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); + = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet(); // Determine new IDs for each append segment by taking into account both // committed and pending segments for this version @@ -1337,7 +1492,7 @@ private Map createNewSegments( // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. final Set pendingSegments = - getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); + getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet(); final Map createdSegments = new HashMap<>(); final Map sequenceHashToSegment = new HashMap<>(); @@ -1547,7 +1702,7 @@ private SegmentIdWithShardSpec createNewSegment( handle, dataSource, interval - ); + ).keySet(); if (committedMaxId != null) { pendings.add(committedMaxId); } @@ -1725,7 +1880,10 @@ private Set announceHistoricalSegmentBatch( return toInsertSegments; } - private Set getSegmentsToUpgradeOnReplace( + /** + * Creates new versions of segments appended while a REPLACE task was in progress. + */ + private Set createUpgradedVersionsOfAppendSegmentsAfterReplace( final Handle handle, final Set replaceSegments, final Set locksHeldByReplaceTask diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 6feaf9e07a38..b518a210a9cf 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -285,6 +285,10 @@ tableName, getPayloadType(), getQuoteString(), getCollation() StringUtils.format( "CREATE INDEX idx_%1$s_datasource_sequence ON %1$s(dataSource, sequence_name)", tableName + ), + StringUtils.format( + "CREATE INDEX idx_%1$s_datasource_sequence_prev_id ON %1$s(dataSource, sequence_prev_id)", + tableName ) ) ); From 260d7bdfa85a33c0cb91a63c4eb875dda88a17e9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 9 Oct 2023 21:08:20 +0530 Subject: [PATCH 07/31] Use correct jdbi query object --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 93d6bf932b5b..2aca6032ff98 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -636,7 +636,7 @@ private Set findAllVersionsOfPendingSegment( ) throws IOException { final Interval interval = pendingSegment.getInterval(); - final Query> query = handle + final Query> query = handle .createQuery( StringUtils.format( "SELECT payload " From b7a7e6f6cde6ded39c411d2381fc91e6ec2d6c1e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 9 Oct 2023 21:09:02 +0530 Subject: [PATCH 08/31] Remove extra index on pending_segments table --- .../java/org/apache/druid/metadata/SQLMetadataConnector.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index b518a210a9cf..6feaf9e07a38 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -285,10 +285,6 @@ tableName, getPayloadType(), getQuoteString(), getCollation() StringUtils.format( "CREATE INDEX idx_%1$s_datasource_sequence ON %1$s(dataSource, sequence_name)", tableName - ), - StringUtils.format( - "CREATE INDEX idx_%1$s_datasource_sequence_prev_id ON %1$s(dataSource, sequence_prev_id)", - tableName ) ) ); From 46a1b1e521db55d680de8a26c5cb8e1b1c0ac62b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 9 Oct 2023 21:11:34 +0530 Subject: [PATCH 09/31] Fix SegmentTransactionalReplaceAction --- .../common/actions/SegmentTransactionalReplaceAction.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 238b6a0e7743..6001a5dc093f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -33,7 +33,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; -import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -123,7 +122,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) // transaction as the commit of replace segments and failure to upgrade // pending segments should not affect success of replace commit. try { - List upgradedPendingSegments = + Set upgradedPendingSegments = toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegments(segments); log.info( "Upgraded [%d] pending segments for REPLACE task[%s]: [%s]", From 3df81289892b75bbbe523bf2adeef73014b0b4a7 Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 01:51:37 +0530 Subject: [PATCH 10/31] Use the correct lock type --- .../indexing/seekablestream/SeekableStreamIndexTask.java | 3 ++- .../seekablestream/SeekableStreamIndexTaskRunner.java | 3 ++- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 4 ++++ .../segment/realtime/appenderator/StreamAppenderator.java | 3 +++ 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 3aca46fbfaea..d74ee5c0be26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -34,6 +34,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.TaskResource; @@ -106,7 +107,7 @@ public SeekableStreamIndexTask( this.lockGranularityToUse = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK) ? LockGranularity.TIME_CHUNK : LockGranularity.SEGMENT; - this.lockTypeToUse = getContextValue(Tasks.USE_SHARED_LOCK, false) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + this.lockTypeToUse = TaskLocks.determineLockTypeForAppend(getContext()); } protected static String getFormattedGroupId(String dataSource, String type) 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 21721c12836f..09a1d1c931de 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 @@ -64,6 +64,7 @@ import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -447,7 +448,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } else { final TaskLock lock = toolbox.getTaskActionClient().submit( new TimeChunkLockAcquireAction( - TaskLockType.EXCLUSIVE, + TaskLocks.determineLockTypeForAppend(task.getContext()), segmentId.getInterval(), 1000L ) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index bdd4efa12b4d..53e4bfafd225 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1278,7 +1278,11 @@ private Map createNewSegments( // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. final Set pendingSegments = +<<<<<<< Updated upstream getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); +======= + new HashSet<>(getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet()); +>>>>>>> Stashed changes final Map createdSegments = new HashMap<>(); final Map sequenceHashToSegment = new HashMap<>(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 043372777fab..1ec5a02569e2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1035,6 +1035,9 @@ private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOExcept { final DataSegment rootSegment = sink.getSegment(); segmentAnnouncer.unannounceSegment(rootSegment); + if (!rootPendingSegmentToNewerVersions.containsKey(rootSegment.getId())) { + return; + } for (SegmentIdWithShardSpec newId : rootPendingSegmentToNewerVersions.get(rootSegment.getId())) { final DataSegment newSegment = new DataSegment( newId.getDataSource(), From c131323fc807055a9222dc561892cb8ee1b542ee Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 01:58:44 +0530 Subject: [PATCH 11/31] Remove unused import --- .../indexing/seekablestream/SeekableStreamIndexTaskRunner.java | 1 - 1 file changed, 1 deletion(-) 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 31fba5b3da60..12e67df8bf3c 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 @@ -65,7 +65,6 @@ import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; From a4f605d0d224eb05bd7bd220860eaa74bcee4f97 Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 02:57:04 +0530 Subject: [PATCH 12/31] Use mutable set --- .../metadata/IndexerSQLMetadataStorageCoordinator.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index abc08759f8df..70a545568e42 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1698,11 +1698,9 @@ private SegmentIdWithShardSpec createNewSegment( // across all shard specs (published + pending). // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. - final Set pendings = getPendingSegmentsForIntervalWithHandle( - handle, - dataSource, - interval - ).keySet(); + final Set pendings = new HashSet<>( + getPendingSegmentsForIntervalWithHandle( handle, dataSource, interval).keySet() + ); if (committedMaxId != null) { pendings.add(committedMaxId); } From 04ba92c0b2727f9203746575df5f7790ede5d22a Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 11:01:01 +0530 Subject: [PATCH 13/31] Fix tests and other misc changes --- .../SegmentTransactionalAppendAction.java | 2 +- .../SegmentTransactionalReplaceAction.java | 9 +++++++- .../supervisor/SupervisorManager.java | 20 ++++++++++++++++++ .../IndexerSQLMetadataStorageCoordinator.java | 21 ++++++++++--------- 4 files changed, 40 insertions(+), 12 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 125a008cdc90..67b701718cae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -120,7 +120,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) if (lock.getType() != TaskLockType.APPEND) { throw InvalidInput.exception( "Cannot use action[%s] for task[%s] as it is holding a lock of type[%s] instead of [APPEND].", - "CommitRealtimeSegmentsAndMetadata", task.getId(), lock.getType() + "SegmentTransactionalAppendAction", task.getId(), lock.getType() ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 6001a5dc093f..0a21b3f4a9ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; @@ -115,7 +116,13 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); - if (publishResult.isSuccess()) { + final Set activeSupervisorIds = new HashSet<>(); + if (toolbox.getSupervisorManager() != null) { + activeSupervisorIds.addAll( + toolbox.getSupervisorManager().getSeekableStreamSupervisorIdsForDatasource(task.getDataSource()) + ); + } + if (publishResult.isSuccess() && !activeSupervisorIds.isEmpty()) { // If upgrade of pending segments fails, the segments will still get upgraded // when the corresponding APPEND task commits the segments. // Thus, the upgrade of pending segments should not be done in the same diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 6fdf1f7a1400..abe281702435 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -34,6 +35,7 @@ import javax.annotation.Nullable; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -70,6 +72,24 @@ public Set getSupervisorIds() return supervisors.keySet(); } + public Set getSeekableStreamSupervisorIdsForDatasource(String datasource) + { + final Set retVal = new HashSet<>(); + for (Map.Entry> entry : supervisors.entrySet()) { + final String supervisorId = entry.getKey(); + final Supervisor supervisor = entry.getValue().lhs; + final SupervisorSpec supervisorSpec = entry.getValue().rhs; + if (!(supervisor instanceof SeekableStreamSupervisor)) { + continue; + } + if (supervisorSpec.isSuspended() || !supervisorSpec.getDataSources().contains(datasource)) { + continue; + } + retVal.add(supervisorId); + } + return retVal; + } + public Optional getSupervisorSpec(String id) { Pair supervisor = supervisors.get(id); diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 70a545568e42..6b3f0d45ccfa 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1392,7 +1392,7 @@ private Set createUpgradedVersionOfSegments( // Get pending segments for the new version, if any final String dataSource = segmentsToUpgrade.iterator().next().getDataSource(); final Set pendingSegments - = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet(); + = new HashSet<>(getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet()); // Determine new IDs for each append segment by taking into account both // committed and pending segments for this version @@ -1542,23 +1542,24 @@ private SegmentIdWithShardSpec createNewSegment( { final PartialShardSpec partialShardSpec = request.getPartialShardSpec(); final String existingVersion = request.getVersion(); + final Set mutablePendingSegments = new HashSet<>(pendingSegments); // Include the committedMaxId while computing the overallMaxId if (committedMaxId != null) { - pendingSegments.add(committedMaxId); + mutablePendingSegments.add(committedMaxId); } // If there is an existing chunk, find the max id with the same version as the existing chunk. // There may still be a pending segment with a higher version (but no corresponding used segments) // which may generate a clash with an existing segment once the new id is generated final SegmentIdWithShardSpec overallMaxId = - pendingSegments.stream() - .filter(id -> id.getShardSpec().sharePartitionSpace(partialShardSpec)) - .filter(id -> versionOfExistingChunk == null - || id.getVersion().equals(versionOfExistingChunk)) - .max(Comparator.comparing(SegmentIdWithShardSpec::getVersion) - .thenComparing(id -> id.getShardSpec().getPartitionNum())) - .orElse(null); + mutablePendingSegments.stream() + .filter(id -> id.getShardSpec().sharePartitionSpace(partialShardSpec)) + .filter(id -> versionOfExistingChunk == null + || id.getVersion().equals(versionOfExistingChunk)) + .max(Comparator.comparing(SegmentIdWithShardSpec::getVersion) + .thenComparing(id -> id.getShardSpec().getPartitionNum())) + .orElse(null); // Determine the version of the new segment final String newSegmentVersion; @@ -1699,7 +1700,7 @@ private SegmentIdWithShardSpec createNewSegment( // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. final Set pendings = new HashSet<>( - getPendingSegmentsForIntervalWithHandle( handle, dataSource, interval).keySet() + getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet() ); if (committedMaxId != null) { pendings.add(committedMaxId); From 727491282aeaee85f35416636efc4620b7f6c3df Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 15:58:42 +0530 Subject: [PATCH 14/31] Add realtime segment update and announcement to replace action --- .../actions/SegmentTransactionalReplaceAction.java | 6 ++++++ .../SeekableStreamIndexTaskClientAsyncImpl.java | 11 ++++++++--- .../SeekableStreamIndexTaskRunner.java | 14 +++++++------- .../supervisor/SeekableStreamSupervisor.java | 4 ++-- .../IndexerMetadataStorageCoordinator.java | 3 --- .../IndexerSQLMetadataStorageCoordinator.java | 6 ------ .../appenderator/SinkQuerySegmentWalker.java | 8 +++++--- .../realtime/appenderator/StreamAppenderator.java | 2 +- 8 files changed, 29 insertions(+), 25 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 0a21b3f4a9ac..ccdc27b17a91 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -136,6 +136,12 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments ); + for (String supervisorId : activeSupervisorIds) { + for (SegmentIdWithShardSpec pendingSegment : upgradedPendingSegments) { + toolbox.getSupervisorManager().updatePendingSegmentMapping(supervisorId, pendingSegment); + } + } + // These upgraded pending segments should be forwarded to the SupervisorManager } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index ab71406df716..0543ed310876 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -37,7 +37,6 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -203,10 +202,16 @@ public ListenableFuture updatePendingSegmentMapping( Set versionsOfPendingSegment ) { + if (versionsOfPendingSegment.isEmpty()) { + return Futures.immediateFuture(true); + } + final List allVersionsOfPendingSegment = new ArrayList<>(); + allVersionsOfPendingSegment.add(rootPendingSegment); + allVersionsOfPendingSegment.addAll(versionsOfPendingSegment); final RequestBuilder requestBuilder = new RequestBuilder( HttpMethod.POST, - "pendingSegmentMapping" - ).jsonContent(jsonMapper, Pair.of(rootPendingSegment, versionsOfPendingSegment)); + "/pendingSegmentMapping" + ).jsonContent(jsonMapper, allVersionsOfPendingSegment); return makeRequest(id, requestBuilder) .handler(IgnoreHttpResponseHandler.INSTANCE) 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 12e67df8bf3c..1d14b1553106 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 @@ -1558,13 +1558,13 @@ public Response setEndOffsetsHTTP( @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response updatePendingSegmentMapping( - Pair> rootPendingSegmentToVersionMapping, + List allVersionsOfPendingSegment, // this field is only for internal purposes, shouldn't be usually set by users @Context final HttpServletRequest req ) { authorizationCheck(req, Action.WRITE); - return updatePendingSegmentMapping(rootPendingSegmentToVersionMapping.lhs, rootPendingSegmentToVersionMapping.rhs); + return updatePendingSegmentMapping(allVersionsOfPendingSegment); } public Map doGetRowStats() @@ -1772,13 +1772,13 @@ public Response setEndOffsets( return Response.ok(sequenceNumbers).build(); } - private Response updatePendingSegmentMapping( - SegmentIdWithShardSpec rootPendingSegment, - Set versionsOfPendingSegment - ) + private Response updatePendingSegmentMapping(List allVersionsOfPendingSegment) { try { - ((StreamAppenderator) appenderator).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + ((StreamAppenderator) appenderator).updatePendingSegmentMapping( + allVersionsOfPendingSegment.get(0), + allVersionsOfPendingSegment.subList(1, allVersionsOfPendingSegment.size()) + ); return Response.ok().build(); } catch (IOException e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index db5f73e3b89d..b83be30f86b7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1101,7 +1101,7 @@ public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegmen taskClient.updatePendingSegmentMapping( taskId, rootPendingSegment, - indexerMetadataStorageCoordinator.getAllVersionsOfPendingSegment(rootPendingSegment) + indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) ); } } @@ -1111,7 +1111,7 @@ public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegmen taskClient.updatePendingSegmentMapping( taskId, rootPendingSegment, - indexerMetadataStorageCoordinator.getAllVersionsOfPendingSegment(rootPendingSegment) + indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) ); } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index de443bc878fc..c40167ccc739 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -432,7 +432,4 @@ SegmentPublishResult commitMetadataOnly( * @return DataSegment used segment corresponding to given id */ DataSegment retrieveSegmentForId(String id, boolean includeUnused); - - Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment); - } diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 6b3f0d45ccfa..f126522970db 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2554,12 +2554,6 @@ public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) ); } - @Override - public Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment) - { - return Collections.emptySet(); - } - private static class PendingSegmentsRecord { private final String sequenceName; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 2abc2e24e776..fb9774b3b10f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -69,10 +69,11 @@ import org.joda.time.Interval; import java.io.Closeable; -import java.util.Map; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; @@ -95,7 +96,8 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker private final Cache cache; private final CacheConfig cacheConfig; private final CachePopulatorStats cachePopulatorStats; - private final Map newIdToRootPendingSegment = new ConcurrentHashMap<>(); + private final ConcurrentMap newIdToRootPendingSegment + = new ConcurrentHashMap<>(); public SinkQuerySegmentWalker( String dataSource, @@ -304,7 +306,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final public void updatePendingSegmentMapping( SegmentIdWithShardSpec rootPendingSegment, - Set versionsOfPendingSegment + List versionsOfPendingSegment ) { for (SegmentIdWithShardSpec versionOfPendingSegment : versionsOfPendingSegment) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 1ec5a02569e2..cb7af2612769 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1057,7 +1057,7 @@ private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOExcept public void updatePendingSegmentMapping( SegmentIdWithShardSpec rootPendingSegment, - Set versionsOfPendingSegment + List versionsOfPendingSegment ) throws IOException { if (!sinks.containsKey(rootPendingSegment) || droppingSinks.contains(rootPendingSegment)) { From 409167b3a76be06279caca5cbdfaf757b64b2d81 Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 10 Oct 2023 16:44:46 +0530 Subject: [PATCH 15/31] Fix compilation --- .../seekablestream/SeekableStreamIndexTaskRunner.java | 1 - .../test/TestIndexerMetadataStorageCoordinator.java | 6 ------ .../realtime/appenderator/SinkQuerySegmentWalker.java | 1 - 3 files changed, 8 deletions(-) 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 1d14b1553106..fb4ff1a3876f 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 @@ -73,7 +73,6 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.incremental.ParseExceptionHandler; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 8b94265cae87..dfaa1eb86e43 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -280,12 +280,6 @@ public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) return null; } - @Override - public Set getAllVersionsOfPendingSegment(SegmentIdWithShardSpec rootPendingSegment) - { - return Collections.emptySet(); - } - public Set getPublished() { return ImmutableSet.copyOf(published); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index fb9774b3b10f..00260ea1d254 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -71,7 +71,6 @@ import java.io.Closeable; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; From 37d7197d6230505ece2cdfc38df71554b816ee72 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 11 Oct 2023 11:22:35 +0530 Subject: [PATCH 16/31] Return set of segments to be upgraded rather than the upgraded ones --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index f126522970db..8b358d606836 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -680,6 +680,7 @@ private Set upgradePendingSegments( ) throws IOException { final Map newPendingSegmentVersions = new HashMap<>(); + final Set upgradedPendingSegments = new HashSet<>(); for (Map.Entry entry : replaceIntervalToMaxId.entrySet()) { final Interval replaceInterval = entry.getKey(); @@ -697,6 +698,7 @@ private Set upgradePendingSegments( final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getKey(); final String pendingSegmentSequence = overlappingPendingSegment.getValue(); if (shouldUpgradePendingSegment(pendingSegmentId, pendingSegmentSequence, replaceInterval, replaceVersion)) { + upgradedPendingSegments.add(pendingSegmentId); // There cannot be any duplicates because this version not been committed before newPendingSegmentVersions.put( new SegmentCreateRequest( @@ -729,7 +731,7 @@ private Set upgradePendingSegments( numInsertedPendingSegments, newPendingSegmentVersions.size() ); - return new HashSet<>(newPendingSegmentVersions.values()); + return upgradedPendingSegments; } private boolean shouldUpgradePendingSegment( From 85fecb257186d9259f53aea5b0e0c39e5d25b38d Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 11 Oct 2023 14:00:53 +0530 Subject: [PATCH 17/31] Do not wait indefinitely for handoff of overshadowed segments --- .../druid/server/http/DataSourcesResource.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 1e146736da13..879cfc6eb11c 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -61,6 +61,7 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -885,14 +886,21 @@ public Response isHandOffComplete( return Response.ok(true).build(); } - TimelineLookup timeline = serverInventoryView.getTimeline( + VersionedIntervalTimeline timeline = serverInventoryView.getTimeline( new TableDataSource(dataSourceName) ); if (timeline == null) { - log.debug("No timeline found for datasource[%s]", dataSourceName); + log.error("No timeline found for datasource[%s]", dataSourceName); return Response.ok(false).build(); } + // If the segment being handed off has a lower version than the current chunk's, do not wait. + // This can happen when a concurrent replace occurs and there are multiple versions of segments being appended + if (!timeline.lookup(Intervals.of(interval)).isEmpty() + && timeline.lookup(Intervals.of(interval)).get(0).getVersion().compareTo(version) > 0) { + return Response.ok(true).build(); + } + Iterable servedSegmentsInInterval = prepareServedSegmentsInInterval(timeline, theInterval); if (isSegmentLoaded(servedSegmentsInInterval, descriptor)) { From 760ac0d4cdd347059963226e4e2a343d5ad577a8 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 11 Oct 2023 14:25:09 +0530 Subject: [PATCH 18/31] Fix fetching --- .../metadata/IndexerSQLMetadataStorageCoordinator.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 8b358d606836..3ff79f520080 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -635,24 +635,18 @@ private Set findAllVersionsOfPendingSegment( SegmentIdWithShardSpec pendingSegment ) throws IOException { - final Interval interval = pendingSegment.getInterval(); final Query> query = handle .createQuery( StringUtils.format( "SELECT payload " + "FROM %s WHERE " + "dataSource = :dataSource AND " - + "start = :start AND " - + "%2$send%2$s = :end AND " + "sequence_prev_id = :sequence_prev_id", - dbTables.getPendingSegmentsTable(), - connector.getQuoteString() + dbTables.getPendingSegmentsTable() ) ) .bind("dataSource", pendingSegment.getDataSource()) - .bind("sequence_prev_id", pendingSegment.asSegmentId().toString()) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()); + .bind("sequence_prev_id", pendingSegment.asSegmentId().toString()); final ResultIterator dbSegments = query .map(ByteArrayMapper.FIRST) From 0735d9c2dfcdf762c8e678a9059e33c0d6dae147 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 11 Oct 2023 17:00:12 +0800 Subject: [PATCH 19/31] Add some comments --- .../SegmentTransactionalReplaceAction.java | 2 - .../IndexerMetadataStorageCoordinator.java | 25 ++++- .../IndexerSQLMetadataStorageCoordinator.java | 100 ++++++++++-------- 3 files changed, 77 insertions(+), 50 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index ccdc27b17a91..6517f3945699 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -141,8 +141,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) toolbox.getSupervisorManager().updatePendingSegmentMapping(supervisorId, pendingSegment); } } - - // These upgraded pending segments should be forwarded to the SupervisorManager } catch (Exception e) { log.error(e, "Error while upgrading pending segments for task[%s]", task.getId()); diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index c40167ccc739..23df95f493d1 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -335,15 +335,32 @@ SegmentPublishResult commitReplaceSegments( ); /** - * Creates new versions for the pending segments that overlap with the given - * replace segments being committed. + * Creates and inserts new IDs for the pending segments hat overlap with the given + * replace segments being committed. The newly created pending segment IDs: + *

    + *
  • Have the same interval and version as that of an overlapping segment + * committed by the REPLACE task.
  • + *
  • Cannot be committed but are only used to serve realtime queries against + * those versions.
  • + *
* * @param replaceSegments Segments being committed by a REPLACE task - * @return List of pending segments chosen for upgrade. The returned list does - * not contain the new versions of the pending segments. + * @return List of pending segments for which new IDs have been created. + * The returned list does not contain the new IDs themselves. */ Set upgradePendingSegments(Set replaceSegments); + /** + * Finds all versions of a pending segment allocated to an appending task. + * + * @param segmentIdWithShardSpec Pending segment that was originally allocated + * to an appending task. Allocated segments take + * the highest version that exists at the time of + * allocation. + * @return All versions of the given pending segment in the metadata store. + * Different versions of the same pending segment may have different shard + * specs and intervals, but they refer to the same data. + */ Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec segmentIdWithShardSpec); /** diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 8b358d606836..f357904fc17c 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -105,7 +105,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class); private static final int MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE = 100; - private static final String UPGRADED_PENDING_SEGMENT_PREFIX = "upgraded_to_replace_version_"; + private static final String UPGRADED_PENDING_SEGMENT_PREFIX = "upgraded_to_version__"; private final ObjectMapper jsonMapper; private final MetadataStorageTablesConfig dbTables; @@ -420,7 +420,7 @@ public SegmentPublishResult commitReplaceSegments( (handle, transactionStatus) -> { final Set segmentsToInsert = new HashSet<>(replaceSegments); segmentsToInsert.addAll( - createUpgradedVersionsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask) + createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask) ); return SegmentPublishResult.ok( insertSegments(handle, segmentsToInsert) @@ -670,8 +670,16 @@ private Set findAllVersionsOfPendingSegment( } /** - * Finds pending segments contained in each replace interval and upgrades them - * to the replace version. + * Creates and inserts new IDs for the pending segments contained in each replace + * interval. The newly created pending segment IDs + *
    + *
  • Have the same interval and version as that of an overlapping segment + * committed by the REPLACE task.
  • + *
  • Cannot be committed but are only used to serve realtime queries against + * those versions.
  • + *
+ * + * @return Set of pending segments for which new IDs have been created. */ private Set upgradePendingSegments( Handle handle, @@ -699,7 +707,9 @@ private Set upgradePendingSegments( final String pendingSegmentSequence = overlappingPendingSegment.getValue(); if (shouldUpgradePendingSegment(pendingSegmentId, pendingSegmentSequence, replaceInterval, replaceVersion)) { upgradedPendingSegments.add(pendingSegmentId); - // There cannot be any duplicates because this version not been committed before + // Ensure unique sequence_name_prev_id_sha1 by setting + // sequence_prev_id -> pendingSegmentId + // sequence_name -> prefix + replaceVersion newPendingSegmentVersions.put( new SegmentCreateRequest( UPGRADED_PENDING_SEGMENT_PREFIX + replaceVersion, @@ -1135,16 +1145,16 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } final String dataSource = appendSegments.iterator().next().getDataSource(); - final Set upgradedSegments = connector.retryTransaction( + final Set segmentIdsForNewVersions = connector.retryTransaction( (handle, transactionStatus) - -> createUpgradedVersionsOfAppendSegments(handle, dataSource, appendSegments), + -> createNewIdsForAppendSegments(handle, dataSource, appendSegments), 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); // Create entries for all required versions of the append segments final Set allSegmentsToInsert = new HashSet<>(appendSegments); - allSegmentsToInsert.addAll(upgradedSegments); + allSegmentsToInsert.addAll(segmentIdsForNewVersions); final AtomicBoolean metadataNotUpdated = new AtomicBoolean(false); try { @@ -1262,15 +1272,12 @@ private void insertPendingSegmentIntoMetastore( } /** - * Creates and returns any extra versions that need to be committed for the - * given append segments. - *

- * This is typically needed when a REPLACE task started and finished after - * these append segments had already been allocated. As such, - * there would be some used segments in the DB with versions higher than these - * append segments. + * Creates new IDs for the given append segments if a REPLACE task started and + * finished after these append segments had already been allocated. The newly + * created IDs belong to the same interval and version as the segments committed + * by the REPLACE task. */ - private Set createUpgradedVersionsOfAppendSegments( + private Set createNewIdsForAppendSegments( Handle handle, String dataSource, Set segmentsToAppend @@ -1313,12 +1320,18 @@ private Set createUpgradedVersionsOfAppendSegments( appendVersionToSegments ); for (Map.Entry> upgradeEntry : segmentsToUpgrade.entrySet()) { - Set segmentsUpgradedToVersion = createUpgradedVersionOfSegments( + final Interval upgradeInterval = upgradeEntry.getKey(); + final Set segmentsAlreadyOnVersion + = overlappingIntervalToSegments.getOrDefault(upgradeInterval, Collections.emptySet()) + .stream() + .filter(s -> s.getVersion().equals(upgradeVersion)) + .collect(Collectors.toSet()); + Set segmentsUpgradedToVersion = createNewIdsForAppendSegmentsWithVersion( handle, upgradeVersion, - upgradeEntry.getKey(), + upgradeInterval, upgradeEntry.getValue(), - overlappingIntervalToSegments + segmentsAlreadyOnVersion ); log.info("Upgraded [%d] segments to version[%s].", segmentsUpgradedToVersion.size(), upgradeVersion); upgradedSegments.addAll(segmentsUpgradedToVersion); @@ -1366,23 +1379,20 @@ private Map> getSegmentsWithVersionLowerThan( } /** - * Computes new Segment IDs for the {@code segmentsToUpgrade} being upgraded - * to the given {@code upgradeVersion}. + * Computes new segment IDs that belong to the upgradeInterval and upgradeVersion. + * + * @param committedSegments Segments that already exist in the upgradeInterval + * at upgradeVersion. */ - private Set createUpgradedVersionOfSegments( + private Set createNewIdsForAppendSegmentsWithVersion( Handle handle, String upgradeVersion, - Interval interval, + Interval upgradeInterval, Set segmentsToUpgrade, - Map> committedSegmentsByInterval + Set committedSegments ) throws IOException { - final Set committedSegments - = committedSegmentsByInterval.getOrDefault(interval, Collections.emptySet()) - .stream() - .filter(s -> s.getVersion().equals(upgradeVersion)) - .collect(Collectors.toSet()); - + // Find the committed segments with the higest partition number SegmentIdWithShardSpec committedMaxId = null; for (DataSegment committedSegment : committedSegments) { if (committedMaxId == null @@ -1391,14 +1401,14 @@ private Set createUpgradedVersionOfSegments( } } - // Get pending segments for the new version, if any + // Get pending segments for the new version to determine the next partition number to allocate final String dataSource = segmentsToUpgrade.iterator().next().getDataSource(); - final Set pendingSegments - = new HashSet<>(getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).keySet()); + final Set pendingSegmentIds + = getPendingSegmentsForIntervalWithHandle(handle, dataSource, upgradeInterval).keySet(); + final Set allAllocatedIds = new HashSet<>(pendingSegmentIds); - // Determine new IDs for each append segment by taking into account both - // committed and pending segments for this version - final Set upgradedSegments = new HashSet<>(); + // Create new IDs for each append segment + final Set newSegmentIds = new HashSet<>(); for (DataSegment segment : segmentsToUpgrade) { SegmentCreateRequest request = new SegmentCreateRequest( segment.getId() + "__" + upgradeVersion, @@ -1406,19 +1416,21 @@ private Set createUpgradedVersionOfSegments( upgradeVersion, NumberedPartialShardSpec.instance() ); - // allocate new segment id + + // Create new segment ID based on committed segments, allocated pending segments + // and new IDs created so far in this method final SegmentIdWithShardSpec newId = createNewSegment( request, dataSource, - interval, + upgradeInterval, upgradeVersion, committedMaxId, - pendingSegments + allAllocatedIds ); - // Add to set of pending segments so that shard specs are computed taking the new id into account - pendingSegments.add(newId); - upgradedSegments.add( + // Update the set so that subsequent segment IDs use a higher partition number + allAllocatedIds.add(newId); + newSegmentIds.add( DataSegment.builder(segment) .interval(newId.getInterval()) .version(newId.getVersion()) @@ -1427,7 +1439,7 @@ private Set createUpgradedVersionOfSegments( ); } - return upgradedSegments; + return newSegmentIds; } private Map createNewSegments( @@ -1884,7 +1896,7 @@ private Set announceHistoricalSegmentBatch( /** * Creates new versions of segments appended while a REPLACE task was in progress. */ - private Set createUpgradedVersionsOfAppendSegmentsAfterReplace( + private Set createNewIdsOfAppendSegmentsAfterReplace( final Handle handle, final Set replaceSegments, final Set locksHeldByReplaceTask From bb5b25ffec6bc0d5324a9be93d433610739dfad3 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 11 Oct 2023 18:07:30 +0530 Subject: [PATCH 20/31] Address review feedback --- .../SegmentTransactionalReplaceAction.java | 18 +++++------ .../supervisor/SupervisorManager.java | 8 ++--- .../SeekableStreamIndexTaskClient.java | 2 +- ...eekableStreamIndexTaskClientAsyncImpl.java | 2 +- .../seekablestream/SequenceMetadata.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 4 +-- .../appenderator/StreamAppenderator.java | 32 +++++++++++-------- 7 files changed, 34 insertions(+), 34 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 6517f3945699..45748bf22783 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -33,7 +33,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; -import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; @@ -116,13 +115,14 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); - final Set activeSupervisorIds = new HashSet<>(); + final String activeSupervisorId; if (toolbox.getSupervisorManager() != null) { - activeSupervisorIds.addAll( - toolbox.getSupervisorManager().getSeekableStreamSupervisorIdsForDatasource(task.getDataSource()) - ); + activeSupervisorId = toolbox.getSupervisorManager() + .getSeekableStreamSupervisorIdForDatasource(task.getDataSource()); + } else { + activeSupervisorId = null; } - if (publishResult.isSuccess() && !activeSupervisorIds.isEmpty()) { + if (publishResult.isSuccess() && activeSupervisorId != null) { // If upgrade of pending segments fails, the segments will still get upgraded // when the corresponding APPEND task commits the segments. // Thus, the upgrade of pending segments should not be done in the same @@ -136,10 +136,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments ); - for (String supervisorId : activeSupervisorIds) { - for (SegmentIdWithShardSpec pendingSegment : upgradedPendingSegments) { - toolbox.getSupervisorManager().updatePendingSegmentMapping(supervisorId, pendingSegment); - } + for (SegmentIdWithShardSpec pendingSegment : upgradedPendingSegments) { + toolbox.getSupervisorManager().updatePendingSegmentMapping(activeSupervisorId, pendingSegment); } } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index abe281702435..efb3e4e756d4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -35,7 +35,6 @@ import javax.annotation.Nullable; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -72,9 +71,8 @@ public Set getSupervisorIds() return supervisors.keySet(); } - public Set getSeekableStreamSupervisorIdsForDatasource(String datasource) + public String getSeekableStreamSupervisorIdForDatasource(String datasource) { - final Set retVal = new HashSet<>(); for (Map.Entry> entry : supervisors.entrySet()) { final String supervisorId = entry.getKey(); final Supervisor supervisor = entry.getValue().lhs; @@ -85,9 +83,9 @@ public Set getSeekableStreamSupervisorIdsForDatasource(String datasource if (supervisorSpec.isSuspended() || !supervisorSpec.getDataSources().contains(datasource)) { continue; } - retVal.add(supervisorId); + return supervisorId; } - return retVal; + return null; } public Optional getSupervisorSpec(String id) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 094b0234c98c..0c749d45cf7a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -164,7 +164,7 @@ ListenableFuture setEndOffsetsAsync( * @param versionsOfPendingSegment - the ids belonging to the versions to which the root segment needs to be updated * @return true if the update succeeds */ - ListenableFuture updatePendingSegmentMapping( + ListenableFuture updatePendingSegmentMappingAsync( String id, SegmentIdWithShardSpec rootPendingSegment, Set versionsOfPendingSegment diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 0543ed310876..66d2bed6fdeb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -196,7 +196,7 @@ public ListenableFuture> getEndOffsetsA } @Override - public ListenableFuture updatePendingSegmentMapping( + public ListenableFuture updatePendingSegmentMappingAsync( String id, SegmentIdWithShardSpec rootPendingSegment, Set versionsOfPendingSegment diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index 47ccbaa00b35..b5a65e99462c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -79,7 +79,7 @@ public SequenceMetadata( @JsonProperty("endOffsets") Map endOffsets, @JsonProperty("checkpointed") boolean checkpointed, @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions, - @JsonProperty("taskLockType") TaskLockType taskLockType + @JsonProperty("taskLockType") @Nullable TaskLockType taskLockType ) { Preconditions.checkNotNull(sequenceName); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index b83be30f86b7..6450e2c9e626 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1098,7 +1098,7 @@ public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegmen { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (String taskId : taskGroup.taskIds()) { - taskClient.updatePendingSegmentMapping( + taskClient.updatePendingSegmentMappingAsync( taskId, rootPendingSegment, indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) @@ -1108,7 +1108,7 @@ public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegmen for (List taskGroupList : pendingCompletionTaskGroups.values()) { for (TaskGroup taskGroup : taskGroupList) { for (String taskId : taskGroup.taskIds()) { - taskClient.updatePendingSegmentMapping( + taskClient.updatePendingSegmentMappingAsync( taskId, rootPendingSegment, indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index cb7af2612769..2209cf082deb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1038,21 +1038,25 @@ private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOExcept if (!rootPendingSegmentToNewerVersions.containsKey(rootSegment.getId())) { return; } - for (SegmentIdWithShardSpec newId : rootPendingSegmentToNewerVersions.get(rootSegment.getId())) { - final DataSegment newSegment = new DataSegment( - newId.getDataSource(), - newId.getInterval(), - newId.getVersion(), - rootSegment.getLoadSpec(), - rootSegment.getDimensions(), - rootSegment.getMetrics(), - newId.getShardSpec(), - rootSegment.getBinaryVersion(), - rootSegment.getSize() - ); - segmentAnnouncer.unannounceSegment(newSegment); + try { + for (SegmentIdWithShardSpec newId : rootPendingSegmentToNewerVersions.get(rootSegment.getId())) { + final DataSegment newSegment = new DataSegment( + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + rootSegment.getLoadSpec(), + rootSegment.getDimensions(), + rootSegment.getMetrics(), + newId.getShardSpec(), + rootSegment.getBinaryVersion(), + rootSegment.getSize() + ); + segmentAnnouncer.unannounceSegment(newSegment); + } + } + finally { + rootPendingSegmentToNewerVersions.remove(rootSegment.getId()); } - rootPendingSegmentToNewerVersions.remove(rootSegment.getId()); } public void updatePendingSegmentMapping( From 73707a8730d2c9ee2173c4e3cb0d57562023c250 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 12 Oct 2023 01:06:28 +0800 Subject: [PATCH 21/31] Simplify updateSegmentMapping method, remove findAllVersionsOfPendingSegment --- .../MaterializedViewSupervisor.java | 7 -- .../SegmentTransactionalReplaceAction.java | 55 +++++++++------ .../supervisor/SupervisorManager.java | 28 +++++--- .../supervisor/SeekableStreamSupervisor.java | 10 +-- ...TestIndexerMetadataStorageCoordinator.java | 10 +-- .../IndexerMetadataStorageCoordinator.java | 18 +---- .../supervisor/NoopSupervisorSpec.java | 6 -- .../overlord/supervisor/Supervisor.java | 3 - .../IndexerSQLMetadataStorageCoordinator.java | 70 +++++-------------- 9 files changed, 75 insertions(+), 132 deletions(-) diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index b4ba7ac1c2ef..ac2738534da9 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -49,7 +49,6 @@ import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SqlSegmentsMetadataManager; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; import org.joda.time.Interval; @@ -290,12 +289,6 @@ public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) // do nothing } - @Override - public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) - { - // do nothing - } - @Override public LagStats computeLagStats() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 45748bf22783..9d1e5bd2862d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -22,17 +22,20 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -115,30 +118,12 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) IndexTaskUtils.emitSegmentPublishMetrics(publishResult, task, toolbox); - final String activeSupervisorId; - if (toolbox.getSupervisorManager() != null) { - activeSupervisorId = toolbox.getSupervisorManager() - .getSeekableStreamSupervisorIdForDatasource(task.getDataSource()); - } else { - activeSupervisorId = null; - } - if (publishResult.isSuccess() && activeSupervisorId != null) { - // If upgrade of pending segments fails, the segments will still get upgraded - // when the corresponding APPEND task commits the segments. - // Thus, the upgrade of pending segments should not be done in the same - // transaction as the commit of replace segments and failure to upgrade - // pending segments should not affect success of replace commit. + // Upgrade any overlapping pending segments + // Do not perform upgrade in the same transaction as replace commit so that + // failure to upgrade pending segments does not affect success of the commit + if (publishResult.isSuccess() && toolbox.getSupervisorManager() != null) { try { - Set upgradedPendingSegments = - toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegments(segments); - log.info( - "Upgraded [%d] pending segments for REPLACE task[%s]: [%s]", - upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments - ); - - for (SegmentIdWithShardSpec pendingSegment : upgradedPendingSegments) { - toolbox.getSupervisorManager().updatePendingSegmentMapping(activeSupervisorId, pendingSegment); - } + tryUpgradeOverlappingPendingSegments(task, toolbox); } catch (Exception e) { log.error(e, "Error while upgrading pending segments for task[%s]", task.getId()); @@ -148,6 +133,30 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) return publishResult; } + /** + * Tries to upgrade any pending segments that overlap with the committed segments. + */ + private void tryUpgradeOverlappingPendingSegments(Task task, TaskActionToolbox toolbox) + { + final SupervisorManager supervisorManager = toolbox.getSupervisorManager(); + final Optional activeSupervisorId = supervisorManager.getActiveSupervisorIdForDatasource(task.getDataSource()); + if (!activeSupervisorId.isPresent()) { + return; + } + + Map upgradedPendingSegments = + toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegments(segments); + log.info( + "Upgraded [%d] pending segments for REPLACE task[%s]: [%s]", + upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments + ); + + upgradedPendingSegments.forEach( + (oldId, newId) -> toolbox.getSupervisorManager() + .updatePendingSegmentMapping(activeSupervisorId.get(), oldId, newId) + ); + } + @Override public boolean isAudited() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index efb3e4e756d4..1e0cc3ece8d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -71,21 +71,20 @@ public Set getSupervisorIds() return supervisors.keySet(); } - public String getSeekableStreamSupervisorIdForDatasource(String datasource) + public Optional getActiveSupervisorIdForDatasource(String datasource) { for (Map.Entry> entry : supervisors.entrySet()) { final String supervisorId = entry.getKey(); final Supervisor supervisor = entry.getValue().lhs; final SupervisorSpec supervisorSpec = entry.getValue().rhs; - if (!(supervisor instanceof SeekableStreamSupervisor)) { - continue; + if (supervisor instanceof SeekableStreamSupervisor + && !supervisorSpec.isSuspended() + && supervisorSpec.getDataSources().contains(datasource)) { + return Optional.of(supervisorId); } - if (supervisorSpec.isSuspended() || !supervisorSpec.getDataSources().contains(datasource)) { - continue; - } - return supervisorId; } - return null; + + return Optional.absent(); } public Optional getSupervisorSpec(String id) @@ -266,17 +265,24 @@ public boolean checkPointDataSourceMetadata( return false; } - public boolean updatePendingSegmentMapping(String supervisorId, SegmentIdWithShardSpec rootPendingSegment) + public boolean updatePendingSegmentMapping( + String supervisorId, + SegmentIdWithShardSpec rootPendingSegment, + SegmentIdWithShardSpec upgradedPendingSegment + ) { try { Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null"); Preconditions.checkNotNull(rootPendingSegment, "rootPendingSegment cannot be null"); Pair supervisor = supervisors.get(supervisorId); - Preconditions.checkNotNull(supervisor, "supervisor could not be found"); + if (!(supervisor.lhs instanceof SeekableStreamSupervisor)) { + return false; + } - supervisor.lhs.updatePendingSegmentMapping(rootPendingSegment); + SeekableStreamSupervisor seekableStreamSupervisor = (SeekableStreamSupervisor) supervisor.lhs; + seekableStreamSupervisor.updatePendingSegmentMapping(rootPendingSegment, upgradedPendingSegment); return true; } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 6450e2c9e626..4e1850f2f3ef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1093,15 +1093,17 @@ public void resetOffsets(@Nonnull DataSourceMetadata resetDataSourceMetadata) addNotice(new ResetOffsetsNotice(resetDataSourceMetadata)); } - @Override - public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) + public void updatePendingSegmentMapping( + SegmentIdWithShardSpec rootPendingSegment, + SegmentIdWithShardSpec upgradedPendingSegment + ) { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (String taskId : taskGroup.taskIds()) { taskClient.updatePendingSegmentMappingAsync( taskId, rootPendingSegment, - indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) + Collections.singleton(upgradedPendingSegment) ); } } @@ -1111,7 +1113,7 @@ public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegmen taskClient.updatePendingSegmentMappingAsync( taskId, rootPendingSegment, - indexerMetadataStorageCoordinator.findAllVersionsOfPendingSegment(rootPendingSegment) + Collections.singleton(upgradedPendingSegment) ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index dfaa1eb86e43..ab3bcc0ae606 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -238,15 +238,9 @@ public SegmentIdWithShardSpec allocatePendingSegment( } @Override - public Set upgradePendingSegments(Set replaceSegments) + public Map upgradePendingSegments(Set replaceSegments) { - return Collections.emptySet(); - } - - @Override - public Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec segmentIdWithShardSpec) - { - return Collections.emptySet(); + return Collections.emptyMap(); } @Override diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 23df95f493d1..fc6bd3c308cb 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -345,23 +345,9 @@ SegmentPublishResult commitReplaceSegments( * * * @param replaceSegments Segments being committed by a REPLACE task - * @return List of pending segments for which new IDs have been created. - * The returned list does not contain the new IDs themselves. + * @return Map from originally allocated pending segment to its new upgraded ID. */ - Set upgradePendingSegments(Set replaceSegments); - - /** - * Finds all versions of a pending segment allocated to an appending task. - * - * @param segmentIdWithShardSpec Pending segment that was originally allocated - * to an appending task. Allocated segments take - * the highest version that exists at the time of - * allocation. - * @return All versions of the given pending segment in the metadata store. - * Different versions of the same pending segment may have different shard - * specs and intervals, but they refer to the same data. - */ - Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec segmentIdWithShardSpec); + Map upgradePendingSegments(Set replaceSegments); /** * Retrieves data source's metadata from the metadata store. Returns null if there is no metadata. diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java index a7439bc44914..e733ef6c233d 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.security.ResourceAction; import javax.annotation.Nonnull; @@ -186,11 +185,6 @@ public int getActiveTaskGroupsCount() { return -1; } - - @Override - public void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment) - { - } }; } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index 22f7dd8b5740..bcfc5ebe8196 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -25,7 +25,6 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.segment.incremental.ParseExceptionReport; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import javax.annotation.Nullable; import java.util.List; @@ -88,8 +87,6 @@ default Boolean isHealthy() */ void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata); - void updatePendingSegmentMapping(SegmentIdWithShardSpec rootPendingSegment); - /** * Computes maxLag, totalLag and avgLag */ diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 28df0afcca99..39efaf1fb3e5 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -600,10 +600,12 @@ public SegmentIdWithShardSpec allocatePendingSegment( } @Override - public Set upgradePendingSegments(Set replaceSegments) + public Map upgradePendingSegments( + Set replaceSegments + ) { if (replaceSegments.isEmpty()) { - return Collections.emptySet(); + return Collections.emptyMap(); } // Any replace interval has exactly one version of segments @@ -622,47 +624,6 @@ public Set upgradePendingSegments(Set repla ); } - @Override - public Set findAllVersionsOfPendingSegment(SegmentIdWithShardSpec pendingSegment) - { - return connector.retryWithHandle( - handle -> findAllVersionsOfPendingSegment(handle, pendingSegment) - ); - } - - private Set findAllVersionsOfPendingSegment( - Handle handle, - SegmentIdWithShardSpec pendingSegment - ) throws IOException - { - final Query> query = handle - .createQuery( - StringUtils.format( - "SELECT payload " - + "FROM %s WHERE " - + "dataSource = :dataSource AND " - + "sequence_prev_id = :sequence_prev_id", - dbTables.getPendingSegmentsTable() - ) - ) - .bind("dataSource", pendingSegment.getDataSource()) - .bind("sequence_prev_id", pendingSegment.asSegmentId().toString()); - - final ResultIterator dbSegments = query - .map(ByteArrayMapper.FIRST) - .iterator(); - - final Set allVersions = new HashSet<>(); - while (dbSegments.hasNext()) { - final byte[] payload = dbSegments.next(); - final SegmentIdWithShardSpec segmentId = - jsonMapper.readValue(payload, SegmentIdWithShardSpec.class); - allVersions.add(segmentId); - } - - return allVersions; - } - /** * Creates and inserts new IDs for the pending segments contained in each replace * interval. The newly created pending segment IDs @@ -673,16 +634,16 @@ private Set findAllVersionsOfPendingSegment( * those versions. * * - * @return Set of pending segments for which new IDs have been created. + * @return Map from original pending segment to the new upgraded ID. */ - private Set upgradePendingSegments( + private Map upgradePendingSegments( Handle handle, String datasource, Map replaceIntervalToMaxId ) throws IOException { final Map newPendingSegmentVersions = new HashMap<>(); - final Set upgradedPendingSegments = new HashSet<>(); + final Map pendingSegmentToNewId = new HashMap<>(); for (Map.Entry entry : replaceIntervalToMaxId.entrySet()) { final Interval replaceInterval = entry.getKey(); @@ -700,10 +661,15 @@ private Set upgradePendingSegments( final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getKey(); final String pendingSegmentSequence = overlappingPendingSegment.getValue(); if (shouldUpgradePendingSegment(pendingSegmentId, pendingSegmentSequence, replaceInterval, replaceVersion)) { - upgradedPendingSegments.add(pendingSegmentId); // Ensure unique sequence_name_prev_id_sha1 by setting // sequence_prev_id -> pendingSegmentId // sequence_name -> prefix + replaceVersion + SegmentIdWithShardSpec newId = new SegmentIdWithShardSpec( + datasource, + replaceInterval, + replaceVersion, + new NumberedShardSpec(++currentPartitionNumber, numCorePartitions) + ); newPendingSegmentVersions.put( new SegmentCreateRequest( UPGRADED_PENDING_SEGMENT_PREFIX + replaceVersion, @@ -711,13 +677,9 @@ private Set upgradePendingSegments( replaceVersion, NumberedPartialShardSpec.instance() ), - new SegmentIdWithShardSpec( - datasource, - replaceInterval, - replaceVersion, - new NumberedShardSpec(++currentPartitionNumber, numCorePartitions) - ) + newId ); + pendingSegmentToNewId.put(pendingSegmentId, newId); } } } @@ -735,7 +697,7 @@ private Set upgradePendingSegments( numInsertedPendingSegments, newPendingSegmentVersions.size() ); - return upgradedPendingSegments; + return pendingSegmentToNewId; } private boolean shouldUpgradePendingSegment( From aeaaadd9f2d8f312c3390ebf6985bdc93fe92c59 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 12 Oct 2023 13:26:12 +0800 Subject: [PATCH 22/31] Update pending segment mapping API --- .../SegmentTransactionalReplaceAction.java | 2 +- .../common/task/AbstractBatchIndexTask.java | 10 ++-- .../druid/indexing/common/task/IndexTask.java | 8 ++- .../parallel/ParallelIndexSupervisorTask.java | 7 ++- .../supervisor/SupervisorManager.java | 16 ++++-- .../PendingSegmentVersions.java | 56 +++++++++++++++++++ .../SeekableStreamIndexTaskClient.java | 15 +++-- ...eekableStreamIndexTaskClientAsyncImpl.java | 24 +++----- .../SeekableStreamIndexTaskRunner.java | 22 +++++--- .../supervisor/SeekableStreamSupervisor.java | 18 ++---- .../appenderator/SinkQuerySegmentWalker.java | 21 +++---- .../appenderator/StreamAppenderator.java | 43 +++++++------- .../server/http/DataSourcesResource.java | 19 ++++--- 13 files changed, 157 insertions(+), 104 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PendingSegmentVersions.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 9d1e5bd2862d..a9baf10ece40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -153,7 +153,7 @@ private void tryUpgradeOverlappingPendingSegments(Task task, TaskActionToolbox t upgradedPendingSegments.forEach( (oldId, newId) -> toolbox.getSupervisorManager() - .updatePendingSegmentMapping(activeSupervisorId.get(), oldId, newId) + .registerNewVersionOfPendingSegmentOnSupervisor(activeSupervisorId.get(), oldId, newId) ); } 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 276d82eedc84..fe19b35391e9 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 @@ -401,16 +401,16 @@ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranular /** * Builds a TaskAction to publish segments based on the type of locks that this - * task acquires (determined by context property {@link Tasks#TASK_LOCK_TYPE}). + * task acquires. + * + * @see #determineLockType */ protected TaskAction buildPublishAction( Set segmentsToBeOverwritten, - Set segmentsToPublish + Set segmentsToPublish, + TaskLockType lockType ) { - TaskLockType lockType = TaskLockType.valueOf( - getContextValue(Tasks.TASK_LOCK_TYPE, Tasks.DEFAULT_TASK_LOCK_TYPE.name()) - ); switch (lockType) { case REPLACE: return SegmentTransactionalReplaceAction.create(segmentsToPublish); 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 a2ca4f869ea7..d880f3eb86a4 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 @@ -50,6 +50,7 @@ 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.TaskLockType; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; @@ -910,10 +911,11 @@ private TaskStatus generateAndPublishSegments( throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType()); } - + final TaskLockType taskLockType = getTaskLockHelper().getLockTypeToUse(); final TransactionalSegmentPublisher publisher = - (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit(buildPublishAction(segmentsToBeOverwritten, segmentsToPublish)); + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( + buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, taskLockType) + ); String effectiveId = getContextValue(CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, null); if (effectiveId == null) { 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 d3e218623cda..e99ef35d9423 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 @@ -42,6 +42,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +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.TaskActionClient; @@ -1167,9 +1168,11 @@ private void publishSegments( } } + final TaskLockType taskLockType = getTaskLockHelper().getLockTypeToUse(); final TransactionalSegmentPublisher publisher = - (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit(buildPublishAction(segmentsToBeOverwritten, segmentsToPublish)); + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( + buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, taskLockType) + ); final boolean published = newSegments.isEmpty() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 1e0cc3ece8d6..177123c900b6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -265,15 +265,21 @@ public boolean checkPointDataSourceMetadata( return false; } - public boolean updatePendingSegmentMapping( + /** + * Registers a new version of the given pending segment on a supervisor. This + * allows the supervisor to include the pending segment in queries fired against + * that segment version. + */ + public boolean registerNewVersionOfPendingSegmentOnSupervisor( String supervisorId, - SegmentIdWithShardSpec rootPendingSegment, - SegmentIdWithShardSpec upgradedPendingSegment + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newSegmentVersion ) { try { Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null"); - Preconditions.checkNotNull(rootPendingSegment, "rootPendingSegment cannot be null"); + Preconditions.checkNotNull(basePendingSegment, "rootPendingSegment cannot be null"); + Preconditions.checkNotNull(newSegmentVersion, "newSegmentVersion cannot be null"); Pair supervisor = supervisors.get(supervisorId); Preconditions.checkNotNull(supervisor, "supervisor could not be found"); @@ -282,7 +288,7 @@ public boolean updatePendingSegmentMapping( } SeekableStreamSupervisor seekableStreamSupervisor = (SeekableStreamSupervisor) supervisor.lhs; - seekableStreamSupervisor.updatePendingSegmentMapping(rootPendingSegment, upgradedPendingSegment); + seekableStreamSupervisor.registerNewVersionOfPendingSegment(basePendingSegment, newSegmentVersion); return true; } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PendingSegmentVersions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PendingSegmentVersions.java new file mode 100644 index 000000000000..146b0afc4b9d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PendingSegmentVersions.java @@ -0,0 +1,56 @@ +/* + * 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.seekablestream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; + +/** + * Contains a new version of an existing base pending segment. Used by realtime + * tasks to serve queries against multiple versions of the same pending segment. + */ +public class PendingSegmentVersions +{ + private final SegmentIdWithShardSpec baseSegment; + private final SegmentIdWithShardSpec newVersion; + + @JsonCreator + public PendingSegmentVersions( + @JsonProperty("baseSegment") SegmentIdWithShardSpec baseSegment, + @JsonProperty("newVersion") SegmentIdWithShardSpec newVersion + ) + { + this.baseSegment = baseSegment; + this.newVersion = newVersion; + } + + @JsonProperty + public SegmentIdWithShardSpec getBaseSegment() + { + return baseSegment; + } + + @JsonProperty + public SegmentIdWithShardSpec getNewVersion() + { + return newVersion; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 0c749d45cf7a..5e5924249608 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeMap; public interface SeekableStreamIndexTaskClient @@ -159,15 +158,15 @@ ListenableFuture 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 id - task id - * @param rootPendingSegment - the pending segment that was originally allocated - * @param versionsOfPendingSegment - the ids belonging to the versions to which the root segment needs to be updated + * @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 * @return true if the update succeeds */ - ListenableFuture updatePendingSegmentMappingAsync( - String id, - SegmentIdWithShardSpec rootPendingSegment, - Set versionsOfPendingSegment + ListenableFuture registerNewVersionOfPendingSegmentAsync( + String taskId, + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newVersionOfSegment ); Class getPartitionType(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 66d2bed6fdeb..40d475909e68 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -69,7 +69,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -196,24 +195,17 @@ public ListenableFuture> getEndOffsetsA } @Override - public ListenableFuture updatePendingSegmentMappingAsync( - String id, - SegmentIdWithShardSpec rootPendingSegment, - Set versionsOfPendingSegment + public ListenableFuture registerNewVersionOfPendingSegmentAsync( + String taskId, + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newVersionOfSegment ) { - if (versionsOfPendingSegment.isEmpty()) { - return Futures.immediateFuture(true); - } - final List allVersionsOfPendingSegment = new ArrayList<>(); - allVersionsOfPendingSegment.add(rootPendingSegment); - allVersionsOfPendingSegment.addAll(versionsOfPendingSegment); - final RequestBuilder requestBuilder = new RequestBuilder( - HttpMethod.POST, - "/pendingSegmentMapping" - ).jsonContent(jsonMapper, allVersionsOfPendingSegment); + final RequestBuilder requestBuilder + = new RequestBuilder(HttpMethod.POST, "/pendingSegmentVersion") + .jsonContent(jsonMapper, new PendingSegmentVersions(basePendingSegment, newVersionOfSegment)); - return makeRequest(id, requestBuilder) + return makeRequest(taskId, requestBuilder) .handler(IgnoreHttpResponseHandler.INSTANCE) .onSuccess(r -> true) .go(); 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 fb4ff1a3876f..537a04f3c84a 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 @@ -1553,17 +1553,20 @@ public Response setEndOffsetsHTTP( } @POST - @Path("pendingSegmentMapping") + @Path("/pendingSegmentVersion") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response updatePendingSegmentMapping( - List allVersionsOfPendingSegment, + public Response registerNewVersionOfPendingSegment( + PendingSegmentVersions pendingSegmentVersions, // this field is only for internal purposes, shouldn't be usually set by users @Context final HttpServletRequest req ) { authorizationCheck(req, Action.WRITE); - return updatePendingSegmentMapping(allVersionsOfPendingSegment); + return registerNewVersionOfPendingSegment( + pendingSegmentVersions.getBaseSegment(), + pendingSegmentVersions.getNewVersion() + ); } public Map doGetRowStats() @@ -1771,12 +1774,15 @@ public Response setEndOffsets( return Response.ok(sequenceNumbers).build(); } - private Response updatePendingSegmentMapping(List allVersionsOfPendingSegment) + private Response registerNewVersionOfPendingSegment( + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newSegmentVersion + ) { try { - ((StreamAppenderator) appenderator).updatePendingSegmentMapping( - allVersionsOfPendingSegment.get(0), - allVersionsOfPendingSegment.subList(1, allVersionsOfPendingSegment.size()) + ((StreamAppenderator) appenderator).registerNewVersionOfPendingSegment( + basePendingSegment, + newSegmentVersion ); return Response.ok().build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 4e1850f2f3ef..1d05169e3fb1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1093,28 +1093,20 @@ public void resetOffsets(@Nonnull DataSourceMetadata resetDataSourceMetadata) addNotice(new ResetOffsetsNotice(resetDataSourceMetadata)); } - public void updatePendingSegmentMapping( - SegmentIdWithShardSpec rootPendingSegment, - SegmentIdWithShardSpec upgradedPendingSegment + public void registerNewVersionOfPendingSegment( + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newSegmentVersion ) { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (String taskId : taskGroup.taskIds()) { - taskClient.updatePendingSegmentMappingAsync( - taskId, - rootPendingSegment, - Collections.singleton(upgradedPendingSegment) - ); + taskClient.registerNewVersionOfPendingSegmentAsync(taskId, basePendingSegment, newSegmentVersion); } } for (List taskGroupList : pendingCompletionTaskGroups.values()) { for (TaskGroup taskGroup : taskGroupList) { for (String taskId : taskGroup.taskIds()) { - taskClient.updatePendingSegmentMappingAsync( - taskId, - rootPendingSegment, - Collections.singleton(upgradedPendingSegment) - ); + taskClient.registerNewVersionOfPendingSegmentAsync(taskId, basePendingSegment, newSegmentVersion); } } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 00260ea1d254..81cd4db25561 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -69,7 +69,6 @@ import org.joda.time.Interval; import java.io.Closeable; -import java.util.List; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -95,7 +94,7 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker private final Cache cache; private final CacheConfig cacheConfig; private final CachePopulatorStats cachePopulatorStats; - private final ConcurrentMap newIdToRootPendingSegment + private final ConcurrentMap newIdToBasePendingSegment = new ConcurrentHashMap<>(); public SinkQuerySegmentWalker( @@ -188,7 +187,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable> perSegmentRunners = Iterables.transform( specs, newDescriptor -> { - final SegmentDescriptor descriptor = newIdToRootPendingSegment.getOrDefault(newDescriptor, newDescriptor); + final SegmentDescriptor descriptor = newIdToBasePendingSegment.getOrDefault(newDescriptor, newDescriptor); final PartitionChunk chunk = sinkTimeline.findChunk( descriptor.getInterval(), descriptor.getVersion(), @@ -303,17 +302,15 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ); } - public void updatePendingSegmentMapping( - SegmentIdWithShardSpec rootPendingSegment, - List versionsOfPendingSegment + public void registerNewVersionOfPendingSegment( + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newSegmentVersion ) { - for (SegmentIdWithShardSpec versionOfPendingSegment : versionsOfPendingSegment) { - newIdToRootPendingSegment.put( - versionOfPendingSegment.asSegmentId().toDescriptor(), - rootPendingSegment.asSegmentId().toDescriptor() - ); - } + newIdToBasePendingSegment.put( + newSegmentVersion.asSegmentId().toDescriptor(), + basePendingSegment.asSegmentId().toDescriptor() + ); } @VisibleForTesting diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 2209cf082deb..c02e23b21791 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1059,36 +1059,35 @@ private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOExcept } } - public void updatePendingSegmentMapping( - SegmentIdWithShardSpec rootPendingSegment, - List versionsOfPendingSegment + public void registerNewVersionOfPendingSegment( + SegmentIdWithShardSpec basePendingSegment, + SegmentIdWithShardSpec newSegmentVersion ) throws IOException { - if (!sinks.containsKey(rootPendingSegment) || droppingSinks.contains(rootPendingSegment)) { + if (!sinks.containsKey(basePendingSegment) || droppingSinks.contains(basePendingSegment)) { return; } // Update query mapping with SinkQuerySegmentWalker - ((SinkQuerySegmentWalker) texasRanger).updatePendingSegmentMapping(rootPendingSegment, versionsOfPendingSegment); + ((SinkQuerySegmentWalker) texasRanger).registerNewVersionOfPendingSegment(basePendingSegment, newSegmentVersion); // Announce segments - rootPendingSegmentToNewerVersions.putIfAbsent(rootPendingSegment.asSegmentId(), new HashSet<>()); - final DataSegment rootSegment = sinks.get(rootPendingSegment).getSegment(); - for (SegmentIdWithShardSpec idWithShardSpec : versionsOfPendingSegment) { - final DataSegment newSegment = new DataSegment( - idWithShardSpec.getDataSource(), - idWithShardSpec.getInterval(), - idWithShardSpec.getVersion(), - rootSegment.getLoadSpec(), - rootSegment.getDimensions(), - rootSegment.getMetrics(), - idWithShardSpec.getShardSpec(), - rootSegment.getBinaryVersion(), - rootSegment.getSize() - ); - segmentAnnouncer.announceSegment(newSegment); - rootPendingSegmentToNewerVersions.get(rootPendingSegment.asSegmentId()).add(idWithShardSpec); - } + rootPendingSegmentToNewerVersions.putIfAbsent(basePendingSegment.asSegmentId(), new HashSet<>()); + final DataSegment rootSegment = sinks.get(basePendingSegment).getSegment(); + + final DataSegment newSegment = new DataSegment( + newSegmentVersion.getDataSource(), + newSegmentVersion.getInterval(), + newSegmentVersion.getVersion(), + rootSegment.getLoadSpec(), + rootSegment.getDimensions(), + rootSegment.getMetrics(), + newSegmentVersion.getShardSpec(), + rootSegment.getBinaryVersion(), + rootSegment.getSize() + ); + segmentAnnouncer.announceSegment(newSegment); + rootPendingSegmentToNewerVersions.get(basePendingSegment.asSegmentId()).add(newSegmentVersion); } private void lockBasePersistDirectory() diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 879cfc6eb11c..301d9631b7d8 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -871,18 +871,18 @@ public Response isHandOffComplete( final Interval theInterval = Intervals.of(interval); final SegmentDescriptor descriptor = new SegmentDescriptor(theInterval, version, partitionNumber); final DateTime now = DateTimes.nowUtc(); - // dropped means a segment will never be handed off, i.e it completed hand off - // init to true, reset to false only if this segment can be loaded by rules - boolean dropped = true; + + // A segment that is not eligible for load will never be handed off + boolean notEligibleForLoad = true; for (Rule rule : rules) { if (rule.appliesTo(theInterval, now)) { if (rule instanceof LoadRule) { - dropped = false; + notEligibleForLoad = false; } break; } } - if (dropped) { + if (notEligibleForLoad) { return Response.ok(true).build(); } @@ -894,10 +894,11 @@ public Response isHandOffComplete( return Response.ok(false).build(); } - // If the segment being handed off has a lower version than the current chunk's, do not wait. - // This can happen when a concurrent replace occurs and there are multiple versions of segments being appended - if (!timeline.lookup(Intervals.of(interval)).isEmpty() - && timeline.lookup(Intervals.of(interval)).get(0).getVersion().compareTo(version) > 0) { + // A segment with version lower than that of the latest chunk might never get handed off + // If there are multiple versions of this segment (due to a concurrent replace task), + // only the latest version would get handed off + List> timelineObjects = timeline.lookup(Intervals.of(interval)); + if (!timelineObjects.isEmpty() && timelineObjects.get(0).getVersion().compareTo(version) > 0) { return Response.ok(true).build(); } From e893d1a26bb5992490a96d0bcd79e09aa9256d7a Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 13:00:31 +0530 Subject: [PATCH 23/31] Allow intervals with append locks to be chosen by autocompaction with replace locks --- .../druid/indexing/overlord/TaskLockbox.java | 58 ++++++++++++ .../overlord/TaskStorageQueryAdapter.java | 10 +++ .../overlord/http/OverlordResource.java | 15 ++++ .../indexing/overlord/TaskLockboxTest.java | 77 ++++++++++++++++ .../metadata/ConflictingLockRequest.java | 88 +++++++++++++++++++ .../druid/rpc/indexing/OverlordClient.java | 15 +++- .../rpc/indexing/OverlordClientImpl.java | 26 ++++++ .../DataSourceCompactionConfig.java | 6 ++ .../coordinator/duty/CompactSegments.java | 21 ++++- .../client/indexing/NoopOverlordClient.java | 9 ++ .../coordinator/duty/CompactSegmentsTest.java | 10 +++ 11 files changed, 333 insertions(+), 2 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index b7aab8505346..1cb365ce1b5e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -39,12 +39,14 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateRequest; import org.apache.druid.indexing.common.actions.SegmentAllocateResult; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; @@ -957,6 +959,62 @@ private Set getNonRevokedReplaceLocks(List posse return replaceLocks; } + /** + * @param conflictingLockRequests Requests for conflicing lock intervals for various datasources + * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked + */ + public Map> getConflictingLockIntervals(List conflictingLockRequests) + { + final Map> datasourceToIntervals = new HashMap<>(); + + // Take a lock and populate the maps + giant.lock(); + + try { + conflictingLockRequests.forEach( + conflictingLockRequest -> { + final String datasource = conflictingLockRequest.getDatasource(); + final int priority = conflictingLockRequest.getPriority(); + final boolean ignoreAppendLocks = + TaskLockType.REPLACE.name().equals(conflictingLockRequest.getContext().get(Tasks.TASK_LOCK_TYPE)); + if (!running.containsKey(datasource)) { + return; + } + + running.get(datasource).forEach( + (startTime, startTimeLocks) -> startTimeLocks.forEach( + (interval, taskLockPosses) -> taskLockPosses.forEach( + taskLockPosse -> { + if (taskLockPosse.getTaskLock().isRevoked()) { + // do nothing + } else if (ignoreAppendLocks + && TaskLockType.APPEND.equals(taskLockPosse.getTaskLock().getType())) { + // do nothing + } else if (taskLockPosse.getTaskLock().getPriority() == null + || taskLockPosse.getTaskLock().getPriority() < priority) { + // do nothing + } else { + datasourceToIntervals.computeIfAbsent(datasource, k -> new HashSet<>()) + .add(interval); + } + } + ) + ) + ); + } + ); + } + finally { + giant.unlock(); + } + + return datasourceToIntervals.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> new ArrayList<>(entry.getValue()) + )); + } + /** * Gets a List of Intervals locked by higher priority tasks for each datasource. * Here, Segment Locks are being treated the same as Time Chunk Locks i.e. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 3fa570ccb32d..6730e363858f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -60,6 +61,15 @@ public List getActiveTasks() return storage.getActiveTasks(); } + /** + * @param conflictingLockRequests Requests for conflicing lock intervals for various datasources + * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked + */ + public Map> getConflictingLockIntervals(List conflictingLockRequests) + { + return taskLockbox.getConflictingLockIntervals(conflictingLockRequests); + } + /** * Gets a List of Intervals locked by higher priority tasks for each datasource. * diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index fa61f796154d..4aedb3c04d06 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -63,6 +63,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; @@ -274,6 +275,20 @@ public Response getDatasourceLockedIntervals(Map minTaskPriorit return Response.ok(taskStorageQueryAdapter.getLockedIntervals(minTaskPriority)).build(); } + @POST + @Path("/conflictingLockIntervals") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response getConflictingLockIntervals(List conflictingLockRequests) + { + if (conflictingLockRequests == null || conflictingLockRequests.isEmpty()) { + return Response.status(Status.BAD_REQUEST).entity("No Datasource provided").build(); + } + + // Build the response + return Response.ok(taskStorageQueryAdapter.getConflictingLockIntervals(conflictingLockRequests)).build(); + } + @GET @Path("/task/{taskid}") @Produces(MediaType.APPLICATION_JSON) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 4c761a6f71b6..fd74827f4ae4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; @@ -43,6 +44,7 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.overlord.TaskLockbox.TaskLockPosse; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -52,6 +54,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; @@ -1248,6 +1251,80 @@ public void testGetLockedIntervalsForEqualPriorityTask() ); } + @Test + public void testGetConflictingLockIntervalsForHigherPriorityExclusiveLock() + { + final Task task = NoopTask.ofPriority(50); + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + tryTimeChunkLock( + TaskLockType.APPEND, + task, + Intervals.of("2017/2018") + ); + + ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + task.getDataSource(), + 75, + null + ); + + Map> conflictingIntervals = + lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + Assert.assertTrue(conflictingIntervals.isEmpty()); + } + + @Test + public void testGetConflictingLockIntervalsForLowerPriorityExclusiveLock() + { + final Task task = NoopTask.ofPriority(50); + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + tryTimeChunkLock( + TaskLockType.APPEND, + task, + Intervals.of("2017/2018") + ); + + ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + task.getDataSource(), + 25, + null + ); + + Map> conflictingIntervals = + lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + Assert.assertEquals(1, conflictingIntervals.size()); + Assert.assertEquals( + Collections.singletonList(Intervals.of("2017/2018")), + conflictingIntervals.get(task.getDataSource()) + ); + } + + @Test + public void testGetConflictingLockIntervalsForLowerPriorityReplaceLock() + { + final Task task = NoopTask.ofPriority(50); + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + tryTimeChunkLock( + TaskLockType.APPEND, + task, + Intervals.of("2017/2018") + ); + + ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + task.getDataSource(), + 25, + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.REPLACE.name()) + ); + + Map> conflictingIntervals = + lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + Assert.assertTrue(conflictingIntervals.isEmpty()); + } + + @Test public void testExclusiveLockCompatibility() { diff --git a/server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java b/server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java new file mode 100644 index 000000000000..54d44ae9eb03 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +/** + * Request object used by CompactSegments to determine intervals with conflicting locks on the Overlord + */ +public class ConflictingLockRequest +{ + private final String datasource; + private final int priority; + private final Map context; + + @JsonCreator + public ConflictingLockRequest( + @JsonProperty("datasource") String datasource, + @JsonProperty("priority") int priority, + @JsonProperty("context") Map context + ) + { + this.datasource = datasource; + this.priority = priority; + this.context = context == null ? Collections.emptyMap() : context; + } + + @JsonProperty + public String getDatasource() + { + return datasource; + } + + @JsonProperty + public int getPriority() + { + return priority; + } + + @JsonProperty + public Map getContext() + { + return context; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ConflictingLockRequest that = (ConflictingLockRequest) o; + return Objects.equals(datasource, that.datasource) + && priority == that.priority + && Objects.equals(context, that.context); + } + + @Override + public int hashCode() + { + return Objects.hash(datasource, priority, context); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index b4391dc0a8ef..c4e7ab824b37 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -32,6 +32,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.rpc.ServiceRetryPolicy; import org.joda.time.Interval; @@ -190,11 +191,23 @@ ListenableFuture> taskStatuses( * @param minTaskPriority Minimum task priority for each datasource. Only the intervals that are locked by tasks with * equal or higher priority than this are returned. * - * @return Map from dtasource name to list of intervals locked by tasks that have priority greater than or equal to + * @return Map from datasource name to list of intervals locked by tasks that have priority greater than or equal to * the {@code minTaskPriority} for that datasource. */ ListenableFuture>> findLockedIntervals(Map minTaskPriority); + /** + * Returns a list of intervals locked by higher priority conflicting lock types + * + * @param conflictingLockRequests List of all requests for different datasources + * + * @return Map from datasource name to list of intervals locked by tasks that have a conflicting lock type with + * priority greater than or equal to the {@code minTaskPriority} for that datasource. + */ + ListenableFuture>> findConflictingLockIntervals( + List conflictingLockRequests + ); + /** * Deletes pending segment records from the metadata store for a particular datasource. Records with * {@code created_date} within the provided {@code interval} are deleted; other records are left alone. diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 306c909e4048..5349de3618c6 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -39,6 +39,7 @@ import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -210,6 +211,31 @@ public ListenableFuture>> findLockedIntervals(Map>> findConflictingLockIntervals( + List conflictingLockRequests + ) + { + final String path = "/druid/indexer/v1/conflictingLockIntervals"; + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, conflictingLockRequests), + new BytesFullResponseHandler() + ), + holder -> { + final Map> response = JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + new TypeReference>>() {} + ); + + return response == null ? Collections.emptyMap() : response; + } + ); + } + @Override public ListenableFuture> taskReportAsMap(String taskId) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index da89040f50a4..3076c1797e2f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -214,4 +215,9 @@ public int hashCode() result = 31 * result + Arrays.hashCode(metricsSpec); return result; } + + public ConflictingLockRequest toConflictingLockRequest() + { + return new ConflictingLockRequest(dataSource, taskPriority, taskContext); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0c08da7c8ddf..903b11d4362b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -42,6 +42,7 @@ 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.logger.Logger; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -175,7 +176,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Skip all the intervals locked by higher priority tasks for each datasource // This must be done after the invalid compaction tasks are cancelled // in the loop above so that their intervals are not considered locked - getLockedIntervalsToSkip(compactionConfigList).forEach( + getConflictingLockIntervals(compactionConfigList).forEach( (dataSource, intervals) -> intervalsToSkipCompaction .computeIfAbsent(dataSource, ds -> new ArrayList<>()) @@ -245,6 +246,24 @@ private boolean cancelTaskIfGranularityChanged( return true; } + private Map> getConflictingLockIntervals( + List compactionConfigs + ) + { + final List conflictingLockRequests = compactionConfigs + .stream() + .map(DataSourceCompactionConfig::toConflictingLockRequest) + .collect(Collectors.toList()); + final Map> datasourceToLockedIntervals = + new HashMap<>(FutureUtils.getUnchecked(overlordClient.findConflictingLockIntervals(conflictingLockRequests), true)); + LOG.debug( + "Skipping the following intervals for Compaction as they are currently locked: %s", + datasourceToLockedIntervals + ); + + return datasourceToLockedIntervals; + } + /** * Gets a List of Intervals locked by higher priority tasks for each datasource. * Since compaction tasks submitted for these Intervals would have to wait anyway, diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 6882c6b762d2..180d615962c2 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -24,6 +24,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; import org.joda.time.Interval; @@ -100,6 +101,14 @@ public ListenableFuture>> findLockedIntervals(Map>> findConflictingLockIntervals( + List conflictingLockRequests + ) + { + throw new UnsupportedOperationException(); + } + @Override public ListenableFuture killPendingSegments(String dataSource, Interval interval) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 2926dbd6d70b..f7a151332613 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -58,6 +58,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; @@ -1994,6 +1995,15 @@ public ListenableFuture runTask(String taskId, Object taskObject) return Futures.immediateFuture(null); } + + @Override + public ListenableFuture>> findConflictingLockIntervals( + List conflictingLockRequests + ) + { + return Futures.immediateFuture(lockedIntervals); + } + @Override public ListenableFuture>> findLockedIntervals(Map minTaskPriority) { From eb1825b9cca6a13731b6f073040b9e846ac80eb2 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 13:06:06 +0530 Subject: [PATCH 24/31] Deprecate previous API and clean up --- .../overlord/http/OverlordResource.java | 1 + .../druid/rpc/indexing/OverlordClient.java | 11 ------- .../rpc/indexing/OverlordClientImpl.java | 23 --------------- .../coordinator/duty/CompactSegments.java | 29 ++++++++++--------- 4 files changed, 16 insertions(+), 48 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 4aedb3c04d06..06491a47ad53 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -261,6 +261,7 @@ public Response isLeader() } } + @Deprecated @POST @Path("/lockedIntervals") @Produces(MediaType.APPLICATION_JSON) diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index c4e7ab824b37..7e43a43614c8 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -185,17 +185,6 @@ ListenableFuture> taskStatuses( */ ListenableFuture> supervisorStatuses(); - /** - * Returns a list of intervals locked by higher priority tasks for each datasource. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the intervals that are locked by tasks with - * equal or higher priority than this are returned. - * - * @return Map from datasource name to list of intervals locked by tasks that have priority greater than or equal to - * the {@code minTaskPriority} for that datasource. - */ - ListenableFuture>> findLockedIntervals(Map minTaskPriority); - /** * Returns a list of intervals locked by higher priority conflicting lock types * diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 5349de3618c6..7a8c57a4b27b 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -188,29 +188,6 @@ public ListenableFuture taskStatus(final String taskId) ); } - @Override - public ListenableFuture>> findLockedIntervals(Map minTaskPriority) - { - final String path = "/druid/indexer/v1/lockedIntervals"; - - return FutureUtils.transform( - client.asyncRequest( - new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, minTaskPriority), - new BytesFullResponseHandler() - ), - holder -> { - final Map> response = JacksonUtils.readValue( - jsonMapper, - holder.getContent(), - new TypeReference>>() {} - ); - - return response == null ? Collections.emptyMap() : response; - } - ); - } - @Override public ListenableFuture>> findConflictingLockIntervals( List conflictingLockRequests diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 903b11d4362b..edebcdc75270 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -246,6 +246,21 @@ private boolean cancelTaskIfGranularityChanged( return true; } + /** + * Gets a List of Intervals locked by higher priority tasks for each datasource. + * However, when using a REPLACE lock for compaction, intervals locked with any APPEND lock will not be returned + * Since compaction tasks submitted for these Intervals would have to wait anyway, + * we skip these Intervals until the next compaction run. + *

+ * For now, Segment Locks are being treated the same as Time Chunk Locks even + * though they lock only a Segment and not the entire Interval. Thus, + * a compaction task will not be submitted for an Interval if + *

    + *
  • either the whole Interval is locked by a higher priority Task with an incompatible lock type
  • + *
  • or there is atleast one Segment in the Interval that is locked by a + * higher priority Task
  • + *
+ */ private Map> getConflictingLockIntervals( List compactionConfigs ) @@ -264,20 +279,6 @@ private Map> getConflictingLockIntervals( return datasourceToLockedIntervals; } - /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * Since compaction tasks submitted for these Intervals would have to wait anyway, - * we skip these Intervals until the next compaction run. - *

- * For now, Segment Locks are being treated the same as Time Chunk Locks even - * though they lock only a Segment and not the entire Interval. Thus, - * a compaction task will not be submitted for an Interval if - *

    - *
  • either the whole Interval is locked by a higher priority Task
  • - *
  • or there is atleast one Segment in the Interval that is locked by a - * higher priority Task
  • - *
- */ private Map> getLockedIntervalsToSkip( List compactionConfigs ) From 1380b2998c671dcf256e7947f14ca79d3819ea0d Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 14:19:44 +0530 Subject: [PATCH 25/31] Fix compilation --- .../coordinator/duty/CompactSegments.java | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index edebcdc75270..f0a7ababd40a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -279,28 +279,6 @@ private Map> getConflictingLockIntervals( return datasourceToLockedIntervals; } - private Map> getLockedIntervalsToSkip( - List compactionConfigs - ) - { - final Map minTaskPriority = compactionConfigs - .stream() - .collect( - Collectors.toMap( - DataSourceCompactionConfig::getDataSource, - DataSourceCompactionConfig::getTaskPriority - ) - ); - final Map> datasourceToLockedIntervals = - new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(minTaskPriority), true)); - LOG.debug( - "Skipping the following intervals for Compaction as they are currently locked: %s", - datasourceToLockedIntervals - ); - - return datasourceToLockedIntervals; - } - /** * Returns the maximum number of task slots used by one compaction task at any time when the task is issued with * the given tuningConfig. From ad084e744f269dbce199c1b86817d4b1fa996455 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 14:37:39 +0530 Subject: [PATCH 26/31] Fix tests --- .../client/indexing/NoopOverlordClient.java | 6 ------ .../rpc/indexing/OverlordClientImplTest.java | 21 ++++++++++++------- .../coordinator/duty/CompactSegmentsTest.java | 10 ++------- 3 files changed, 15 insertions(+), 22 deletions(-) diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 180d615962c2..23395462bb6d 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -95,12 +95,6 @@ public ListenableFuture> supervisorStatuses( throw new UnsupportedOperationException(); } - @Override - public ListenableFuture>> findLockedIntervals(Map minTaskPriority) - { - throw new UnsupportedOperationException(); - } - @Override public ListenableFuture>> findConflictingLockIntervals( List conflictingLockRequests diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 5b9a88d58410..0698d54b7718 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; @@ -219,13 +220,15 @@ public void test_taskStatuses_null_null_zero() throws Exception @Test public void test_findLockedIntervals() throws Exception { - final Map priorityMap = ImmutableMap.of("foo", 3); final Map> lockMap = ImmutableMap.of("foo", Collections.singletonList(Intervals.of("2000/2001"))); + final List requests = ImmutableList.of( + new ConflictingLockRequest("foo", 3, null) + ); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals") - .jsonContent(jsonMapper, priorityMap), + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/conflictingLockIntervals") + .jsonContent(jsonMapper, requests), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), jsonMapper.writeValueAsBytes(lockMap) @@ -233,18 +236,20 @@ public void test_findLockedIntervals() throws Exception Assert.assertEquals( lockMap, - overlordClient.findLockedIntervals(priorityMap).get() + overlordClient.findConflictingLockIntervals(requests).get() ); } @Test public void test_findLockedIntervals_nullReturn() throws Exception { - final Map priorityMap = ImmutableMap.of("foo", 3); + final List requests = ImmutableList.of( + new ConflictingLockRequest("foo", 3, null) + ); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals") - .jsonContent(jsonMapper, priorityMap), + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/conflictingLockIntervals") + .jsonContent(jsonMapper, requests), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), jsonMapper.writeValueAsBytes(null) @@ -252,7 +257,7 @@ public void test_findLockedIntervals_nullReturn() throws Exception Assert.assertEquals( Collections.emptyMap(), - overlordClient.findLockedIntervals(priorityMap).get() + overlordClient.findConflictingLockIntervals(requests).get() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index f7a151332613..6b34e7a4e2dd 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -1093,7 +1093,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() .thenReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage(ImmutableList.of(runningConflictCompactionTask).iterator()))); - Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) + Mockito.when(mockClient.findConflictingLockIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.cancelTask(conflictTaskId)) .thenReturn(Futures.immediateFuture(null)); @@ -2004,12 +2004,6 @@ public ListenableFuture>> findConflictingLockInterval return Futures.immediateFuture(lockedIntervals); } - @Override - public ListenableFuture>> findLockedIntervals(Map minTaskPriority) - { - return Futures.immediateFuture(lockedIntervals); - } - @Override public ListenableFuture> taskStatuses( @Nullable String state, @@ -2196,7 +2190,7 @@ private static ArgumentCaptor setUpMockClient(final OverlordClient mockC final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); Mockito.when(mockClient.taskStatuses(null, null, 0)) .thenReturn(Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator()))); - Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) + Mockito.when(mockClient.findConflictingLockIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.getTotalWorkerCapacity()) .thenReturn(Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(0, 0))); From 877937115982aed9d7dc8df7084fb5ae2c2d49ab Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 16:27:53 +0530 Subject: [PATCH 27/31] Renaming --- .../druid/indexing/overlord/TaskLockbox.java | 18 ++++++++--------- .../overlord/TaskStorageQueryAdapter.java | 8 ++++---- .../overlord/http/OverlordResource.java | 10 +++++----- .../indexing/overlord/TaskLockboxTest.java | 20 +++++++++---------- ...LockRequest.java => LockFilterPolicy.java} | 8 ++++---- .../druid/rpc/indexing/OverlordClient.java | 9 ++++----- .../rpc/indexing/OverlordClientImpl.java | 10 +++++----- .../DataSourceCompactionConfig.java | 6 ------ .../coordinator/duty/CompactSegments.java | 12 +++++------ .../client/indexing/NoopOverlordClient.java | 6 +++--- .../rpc/indexing/OverlordClientImplTest.java | 18 ++++++++--------- .../coordinator/duty/CompactSegmentsTest.java | 10 +++++----- 12 files changed, 64 insertions(+), 71 deletions(-) rename server/src/main/java/org/apache/druid/metadata/{ConflictingLockRequest.java => LockFilterPolicy.java} (90%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 1cb365ce1b5e..d22d88fa463d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -46,7 +46,7 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; @@ -960,10 +960,10 @@ private Set getNonRevokedReplaceLocks(List posse } /** - * @param conflictingLockRequests Requests for conflicing lock intervals for various datasources - * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked + * @param lockFilterPolicies Lock filters for the given datasources + * @return Map from datasource to intervals locked by tasks satisfying the lock filter condititions */ - public Map> getConflictingLockIntervals(List conflictingLockRequests) + public Map> getLockedIntervalsV2(List lockFilterPolicies) { final Map> datasourceToIntervals = new HashMap<>(); @@ -971,12 +971,12 @@ public Map> getConflictingLockIntervals(List { - final String datasource = conflictingLockRequest.getDatasource(); - final int priority = conflictingLockRequest.getPriority(); + lockFilterPolicies.forEach( + lockFilter -> { + final String datasource = lockFilter.getDatasource(); + final int priority = lockFilter.getPriority(); final boolean ignoreAppendLocks = - TaskLockType.REPLACE.name().equals(conflictingLockRequest.getContext().get(Tasks.TASK_LOCK_TYPE)); + TaskLockType.REPLACE.name().equals(lockFilter.getContext().get(Tasks.TASK_LOCK_TYPE)); if (!running.containsKey(datasource)) { return; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 6730e363858f..ba984401d3c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -28,7 +28,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -62,12 +62,12 @@ public List getActiveTasks() } /** - * @param conflictingLockRequests Requests for conflicing lock intervals for various datasources + * @param lockFilterPolicies Requests for conflicing lock intervals for various datasources * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked */ - public Map> getConflictingLockIntervals(List conflictingLockRequests) + public Map> getLockedIntervalsV2(List lockFilterPolicies) { - return taskLockbox.getConflictingLockIntervals(conflictingLockRequests); + return taskLockbox.getLockedIntervalsV2(lockFilterPolicies); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 06491a47ad53..ff3f7e23b6c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -63,7 +63,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; @@ -277,17 +277,17 @@ public Response getDatasourceLockedIntervals(Map minTaskPriorit } @POST - @Path("/conflictingLockIntervals") + @Path("/getLockedIntervals/v2") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response getConflictingLockIntervals(List conflictingLockRequests) + public Response getDatasourceLockedIntervalsV2(List lockFilterPolicies) { - if (conflictingLockRequests == null || conflictingLockRequests.isEmpty()) { + if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { return Response.status(Status.BAD_REQUEST).entity("No Datasource provided").build(); } // Build the response - return Response.ok(taskStorageQueryAdapter.getConflictingLockIntervals(conflictingLockRequests)).build(); + return Response.ok(taskStorageQueryAdapter.getLockedIntervalsV2(lockFilterPolicies)).build(); } @GET diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index fd74827f4ae4..18e9fa9ccfb1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -54,10 +54,10 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; @@ -1252,7 +1252,7 @@ public void testGetLockedIntervalsForEqualPriorityTask() } @Test - public void testGetConflictingLockIntervalsForHigherPriorityExclusiveLock() + public void testGetLockedIntervalsV2ForHigherPriorityExclusiveLock() { final Task task = NoopTask.ofPriority(50); lockbox.add(task); @@ -1263,19 +1263,19 @@ public void testGetConflictingLockIntervalsForHigherPriorityExclusiveLock() Intervals.of("2017/2018") ); - ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 75, null ); Map> conflictingIntervals = - lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertTrue(conflictingIntervals.isEmpty()); } @Test - public void testGetConflictingLockIntervalsForLowerPriorityExclusiveLock() + public void testGetLockedIntervalsForLowerPriorityExclusiveLock() { final Task task = NoopTask.ofPriority(50); lockbox.add(task); @@ -1286,14 +1286,14 @@ public void testGetConflictingLockIntervalsForLowerPriorityExclusiveLock() Intervals.of("2017/2018") ); - ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, null ); Map> conflictingIntervals = - lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertEquals(1, conflictingIntervals.size()); Assert.assertEquals( Collections.singletonList(Intervals.of("2017/2018")), @@ -1302,7 +1302,7 @@ public void testGetConflictingLockIntervalsForLowerPriorityExclusiveLock() } @Test - public void testGetConflictingLockIntervalsForLowerPriorityReplaceLock() + public void testGetLockedIntervalsV2ForLowerPriorityReplaceLock() { final Task task = NoopTask.ofPriority(50); lockbox.add(task); @@ -1313,14 +1313,14 @@ public void testGetConflictingLockIntervalsForLowerPriorityReplaceLock() Intervals.of("2017/2018") ); - ConflictingLockRequest requestForExclusiveLowerPriorityLock = new ConflictingLockRequest( + LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.REPLACE.name()) ); Map> conflictingIntervals = - lockbox.getConflictingLockIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertTrue(conflictingIntervals.isEmpty()); } diff --git a/server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java similarity index 90% rename from server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java rename to server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java index 54d44ae9eb03..88ab4673aa8a 100644 --- a/server/src/main/java/org/apache/druid/metadata/ConflictingLockRequest.java +++ b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java @@ -27,16 +27,16 @@ import java.util.Objects; /** - * Request object used by CompactSegments to determine intervals with conflicting locks on the Overlord + * Specifies a policy to filter active locks held by a datasource */ -public class ConflictingLockRequest +public class LockFilterPolicy { private final String datasource; private final int priority; private final Map context; @JsonCreator - public ConflictingLockRequest( + public LockFilterPolicy( @JsonProperty("datasource") String datasource, @JsonProperty("priority") int priority, @JsonProperty("context") Map context @@ -74,7 +74,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ConflictingLockRequest that = (ConflictingLockRequest) o; + LockFilterPolicy that = (LockFilterPolicy) o; return Objects.equals(datasource, that.datasource) && priority == that.priority && Objects.equals(context, that.context); diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index 7e43a43614c8..a18af000d476 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -32,7 +32,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.joda.time.Interval; @@ -188,13 +188,12 @@ ListenableFuture> taskStatuses( /** * Returns a list of intervals locked by higher priority conflicting lock types * - * @param conflictingLockRequests List of all requests for different datasources - * + * @param lockFilterPolicies List of all filters for different datasources * @return Map from datasource name to list of intervals locked by tasks that have a conflicting lock type with * priority greater than or equal to the {@code minTaskPriority} for that datasource. */ - ListenableFuture>> findConflictingLockIntervals( - List conflictingLockRequests + ListenableFuture>> findLockedIntervalsV2( + List lockFilterPolicies ); /** diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 7a8c57a4b27b..a3b39b6d3e55 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -39,7 +39,7 @@ import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.java.util.http.client.response.StringFullResponseHandler; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.IgnoreHttpResponseHandler; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -189,16 +189,16 @@ public ListenableFuture taskStatus(final String taskId) } @Override - public ListenableFuture>> findConflictingLockIntervals( - List conflictingLockRequests + public ListenableFuture>> findLockedIntervalsV2( + List lockFilterPolicies ) { - final String path = "/druid/indexer/v1/conflictingLockIntervals"; + final String path = "/druid/indexer/v1/lockedIntervals/v2"; return FutureUtils.transform( client.asyncRequest( new RequestBuilder(HttpMethod.POST, path) - .jsonContent(jsonMapper, conflictingLockRequests), + .jsonContent(jsonMapper, lockFilterPolicies), new BytesFullResponseHandler() ), holder -> { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 3076c1797e2f..da89040f50a4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.metadata.ConflictingLockRequest; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -215,9 +214,4 @@ public int hashCode() result = 31 * result + Arrays.hashCode(metricsSpec); return result; } - - public ConflictingLockRequest toConflictingLockRequest() - { - return new ConflictingLockRequest(dataSource, taskPriority, taskContext); - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index f0a7ababd40a..0ffcc54055ff 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -42,7 +42,7 @@ 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.logger.Logger; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -176,7 +176,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Skip all the intervals locked by higher priority tasks for each datasource // This must be done after the invalid compaction tasks are cancelled // in the loop above so that their intervals are not considered locked - getConflictingLockIntervals(compactionConfigList).forEach( + getLockedIntervalsV2(compactionConfigList).forEach( (dataSource, intervals) -> intervalsToSkipCompaction .computeIfAbsent(dataSource, ds -> new ArrayList<>()) @@ -261,16 +261,16 @@ private boolean cancelTaskIfGranularityChanged( * higher priority Task * */ - private Map> getConflictingLockIntervals( + private Map> getLockedIntervalsV2( List compactionConfigs ) { - final List conflictingLockRequests = compactionConfigs + final List lockFilterPolicies = compactionConfigs .stream() - .map(DataSourceCompactionConfig::toConflictingLockRequest) + .map(config -> new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), config.getTaskContext())) .collect(Collectors.toList()); final Map> datasourceToLockedIntervals = - new HashMap<>(FutureUtils.getUnchecked(overlordClient.findConflictingLockIntervals(conflictingLockRequests), true)); + new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervalsV2(lockFilterPolicies), true)); LOG.debug( "Skipping the following intervals for Compaction as they are currently locked: %s", datasourceToLockedIntervals diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 23395462bb6d..b123945cb5bd 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -24,7 +24,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; import org.joda.time.Interval; @@ -96,8 +96,8 @@ public ListenableFuture> supervisorStatuses( } @Override - public ListenableFuture>> findConflictingLockIntervals( - List conflictingLockRequests + public ListenableFuture>> findLockedIntervalsV2( + List lockFilterPolicies ) { throw new UnsupportedOperationException(); diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 0698d54b7718..e9d6f287b632 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -41,7 +41,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; @@ -222,12 +222,12 @@ public void test_findLockedIntervals() throws Exception { final Map> lockMap = ImmutableMap.of("foo", Collections.singletonList(Intervals.of("2000/2001"))); - final List requests = ImmutableList.of( - new ConflictingLockRequest("foo", 3, null) + final List requests = ImmutableList.of( + new LockFilterPolicy("foo", 3, null) ); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/conflictingLockIntervals") + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals/v2") .jsonContent(jsonMapper, requests), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), @@ -236,19 +236,19 @@ public void test_findLockedIntervals() throws Exception Assert.assertEquals( lockMap, - overlordClient.findConflictingLockIntervals(requests).get() + overlordClient.findLockedIntervalsV2(requests).get() ); } @Test public void test_findLockedIntervals_nullReturn() throws Exception { - final List requests = ImmutableList.of( - new ConflictingLockRequest("foo", 3, null) + final List requests = ImmutableList.of( + new LockFilterPolicy("foo", 3, null) ); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/conflictingLockIntervals") + new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/lockedIntervals/v2") .jsonContent(jsonMapper, requests), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), @@ -257,7 +257,7 @@ public void test_findLockedIntervals_nullReturn() throws Exception Assert.assertEquals( Collections.emptyMap(), - overlordClient.findConflictingLockIntervals(requests).get() + overlordClient.findLockedIntervalsV2(requests).get() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 6b34e7a4e2dd..35e0bf5467f1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -58,7 +58,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.ConflictingLockRequest; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; @@ -1093,7 +1093,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() .thenReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage(ImmutableList.of(runningConflictCompactionTask).iterator()))); - Mockito.when(mockClient.findConflictingLockIntervals(ArgumentMatchers.any())) + Mockito.when(mockClient.findLockedIntervalsV2(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.cancelTask(conflictTaskId)) .thenReturn(Futures.immediateFuture(null)); @@ -1997,8 +1997,8 @@ public ListenableFuture runTask(String taskId, Object taskObject) @Override - public ListenableFuture>> findConflictingLockIntervals( - List conflictingLockRequests + public ListenableFuture>> findLockedIntervalsV2( + List lockFilterPolicies ) { return Futures.immediateFuture(lockedIntervals); @@ -2190,7 +2190,7 @@ private static ArgumentCaptor setUpMockClient(final OverlordClient mockC final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); Mockito.when(mockClient.taskStatuses(null, null, 0)) .thenReturn(Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator()))); - Mockito.when(mockClient.findConflictingLockIntervals(ArgumentMatchers.any())) + Mockito.when(mockClient.findLockedIntervalsV2(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.getTotalWorkerCapacity()) .thenReturn(Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(0, 0))); From 79444257f4bb30682658731fdf2652c6450a0968 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 12 Oct 2023 19:13:01 +0800 Subject: [PATCH 28/31] Minor cleanup --- .../SegmentTransactionalReplaceAction.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 45 ++++++------ ...TestIndexerMetadataStorageCoordinator.java | 2 +- .../IndexerMetadataStorageCoordinator.java | 4 +- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- .../appenderator/StreamAppenderator.java | 69 ++++++++++--------- 6 files changed, 64 insertions(+), 60 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index a9baf10ece40..5a2b3ceec8f6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -145,7 +145,7 @@ private void tryUpgradeOverlappingPendingSegments(Task task, TaskActionToolbox t } Map upgradedPendingSegments = - toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegments(segments); + toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegmentsOverlappingWith(segments); log.info( "Upgraded [%d] pending segments for REPLACE task[%s]: [%s]", upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments 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 537a04f3c84a..769413d6ffce 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 @@ -46,6 +46,8 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -83,7 +85,6 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; @@ -1563,10 +1564,27 @@ public Response registerNewVersionOfPendingSegment( ) { authorizationCheck(req, Action.WRITE); - return registerNewVersionOfPendingSegment( - pendingSegmentVersions.getBaseSegment(), - pendingSegmentVersions.getNewVersion() - ); + try { + ((StreamAppenderator) appenderator).registerNewVersionOfPendingSegment( + pendingSegmentVersions.getBaseSegment(), + pendingSegmentVersions.getNewVersion() + ); + return Response.ok().build(); + } + catch (DruidException e) { + return Response + .status(e.getStatusCode()) + .entity(new ErrorResponse(e)) + .build(); + } + catch (Exception e) { + log.error( + e, + "Could not register new version[%s] of pending segment[%s]", + pendingSegmentVersions.getNewVersion(), pendingSegmentVersions.getBaseSegment() + ); + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } } public Map doGetRowStats() @@ -1774,23 +1792,6 @@ public Response setEndOffsets( return Response.ok(sequenceNumbers).build(); } - private Response registerNewVersionOfPendingSegment( - SegmentIdWithShardSpec basePendingSegment, - SegmentIdWithShardSpec newSegmentVersion - ) - { - try { - ((StreamAppenderator) appenderator).registerNewVersionOfPendingSegment( - basePendingSegment, - newSegmentVersion - ); - return Response.ok().build(); - } - catch (IOException e) { - return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); - } - } - private void resetNextCheckpointTime() { nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index ab3bcc0ae606..d1c72485011f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -238,7 +238,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( } @Override - public Map upgradePendingSegments(Set replaceSegments) + public Map upgradePendingSegmentsOverlappingWith(Set replaceSegments) { return Collections.emptyMap(); } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index fc6bd3c308cb..2c2a6bc0f77b 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -347,7 +347,9 @@ SegmentPublishResult commitReplaceSegments( * @param replaceSegments Segments being committed by a REPLACE task * @return Map from originally allocated pending segment to its new upgraded ID. */ - Map upgradePendingSegments(Set replaceSegments); + Map upgradePendingSegmentsOverlappingWith( + Set replaceSegments + ); /** * Retrieves data source's metadata from the metadata store. Returns null if there is no metadata. diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 39efaf1fb3e5..226663c32330 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -600,7 +600,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( } @Override - public Map upgradePendingSegments( + public Map upgradePendingSegmentsOverlappingWith( Set replaceSegments ) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index c02e23b21791..f21f67ed5041 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -155,7 +155,8 @@ public class StreamAppenderator implements Appenderator private final AtomicBoolean closed = new AtomicBoolean(false); - private final ConcurrentMap> rootPendingSegmentToNewerVersions; + private final ConcurrentHashMap> + baseSegmentToUpgradedVersions = new ConcurrentHashMap<>(); private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; @@ -220,7 +221,6 @@ public class StreamAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); this.useMaxMemoryEstimates = useMaxMemoryEstimates; - rootPendingSegmentToNewerVersions = new ConcurrentHashMap<>(); } @Override @@ -1003,7 +1003,7 @@ public void closeNow() log.debug("Shutting down immediately..."); for (Map.Entry entry : sinks.entrySet()) { try { - unannounceRootSegmentAndUpgradedVersions(entry.getValue()); + unannounceAllVersionsOfSegment(entry.getValue().getSegment()); } catch (Exception e) { log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) @@ -1031,31 +1031,32 @@ public void closeNow() } } - private void unannounceRootSegmentAndUpgradedVersions(Sink sink) throws IOException + /** + * Unannounces the given base segment and all its upgraded versions. + */ + private void unannounceAllVersionsOfSegment(DataSegment baseSegment) throws IOException { - final DataSegment rootSegment = sink.getSegment(); - segmentAnnouncer.unannounceSegment(rootSegment); - if (!rootPendingSegmentToNewerVersions.containsKey(rootSegment.getId())) { + segmentAnnouncer.unannounceSegment(baseSegment); + + final Set upgradedVersionsOfSegment + = baseSegmentToUpgradedVersions.remove(baseSegment.getId()); + if (upgradedVersionsOfSegment == null || upgradedVersionsOfSegment.isEmpty()) { return; } - try { - for (SegmentIdWithShardSpec newId : rootPendingSegmentToNewerVersions.get(rootSegment.getId())) { - final DataSegment newSegment = new DataSegment( - newId.getDataSource(), - newId.getInterval(), - newId.getVersion(), - rootSegment.getLoadSpec(), - rootSegment.getDimensions(), - rootSegment.getMetrics(), - newId.getShardSpec(), - rootSegment.getBinaryVersion(), - rootSegment.getSize() - ); - segmentAnnouncer.unannounceSegment(newSegment); - } - } - finally { - rootPendingSegmentToNewerVersions.remove(rootSegment.getId()); + + for (SegmentIdWithShardSpec newId : upgradedVersionsOfSegment) { + final DataSegment newSegment = new DataSegment( + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + baseSegment.getLoadSpec(), + baseSegment.getDimensions(), + baseSegment.getMetrics(), + newId.getShardSpec(), + baseSegment.getBinaryVersion(), + baseSegment.getSize() + ); + segmentAnnouncer.unannounceSegment(newSegment); } } @@ -1072,22 +1073,22 @@ public void registerNewVersionOfPendingSegment( ((SinkQuerySegmentWalker) texasRanger).registerNewVersionOfPendingSegment(basePendingSegment, newSegmentVersion); // Announce segments - rootPendingSegmentToNewerVersions.putIfAbsent(basePendingSegment.asSegmentId(), new HashSet<>()); - final DataSegment rootSegment = sinks.get(basePendingSegment).getSegment(); + final DataSegment baseSegment = sinks.get(basePendingSegment).getSegment(); final DataSegment newSegment = new DataSegment( newSegmentVersion.getDataSource(), newSegmentVersion.getInterval(), newSegmentVersion.getVersion(), - rootSegment.getLoadSpec(), - rootSegment.getDimensions(), - rootSegment.getMetrics(), + baseSegment.getLoadSpec(), + baseSegment.getDimensions(), + baseSegment.getMetrics(), newSegmentVersion.getShardSpec(), - rootSegment.getBinaryVersion(), - rootSegment.getSize() + baseSegment.getBinaryVersion(), + baseSegment.getSize() ); segmentAnnouncer.announceSegment(newSegment); - rootPendingSegmentToNewerVersions.get(basePendingSegment.asSegmentId()).add(newSegmentVersion); + baseSegmentToUpgradedVersions.computeIfAbsent(basePendingSegment.asSegmentId(), id -> new HashSet<>()) + .add(newSegmentVersion); } private void lockBasePersistDirectory() @@ -1391,7 +1392,7 @@ public Void apply(@Nullable Object input) // Unannounce the segment. try { - unannounceRootSegmentAndUpgradedVersions(sink); + unannounceAllVersionsOfSegment(sink.getSegment()); } catch (Exception e) { log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) From 09911e01f55fe45240328eb108fa172eae83aa77 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 16:48:49 +0530 Subject: [PATCH 29/31] Fix path and remove unnecessary v2 --- .../apache/druid/indexing/overlord/http/OverlordResource.java | 2 +- .../apache/druid/server/coordinator/duty/CompactSegments.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index ff3f7e23b6c7..7eabb73dc2b9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -277,7 +277,7 @@ public Response getDatasourceLockedIntervals(Map minTaskPriorit } @POST - @Path("/getLockedIntervals/v2") + @Path("/lockedIntervals/v2") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) public Response getDatasourceLockedIntervalsV2(List lockFilterPolicies) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0ffcc54055ff..7f57d641bb7b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -176,7 +176,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Skip all the intervals locked by higher priority tasks for each datasource // This must be done after the invalid compaction tasks are cancelled // in the loop above so that their intervals are not considered locked - getLockedIntervalsV2(compactionConfigList).forEach( + getLockedIntervals(compactionConfigList).forEach( (dataSource, intervals) -> intervalsToSkipCompaction .computeIfAbsent(dataSource, ds -> new ArrayList<>()) @@ -261,7 +261,7 @@ private boolean cancelTaskIfGranularityChanged( * higher priority Task * */ - private Map> getLockedIntervalsV2( + private Map> getLockedIntervals( List compactionConfigs ) { From eb96ebbec5fff7818414df55d7ea68d94dab3d01 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 18:20:42 +0530 Subject: [PATCH 30/31] Address feedback --- .../apache/druid/indexing/overlord/TaskLockbox.java | 9 +++++---- .../indexing/overlord/TaskStorageQueryAdapter.java | 4 ++-- .../druid/indexing/overlord/http/OverlordResource.java | 4 ++-- .../overlord/supervisor/SupervisorManager.java | 3 ++- .../druid/indexing/overlord/TaskLockboxTest.java | 10 +++++----- 5 files changed, 16 insertions(+), 14 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index d22d88fa463d..54191adf05df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -963,7 +963,7 @@ private Set getNonRevokedReplaceLocks(List posse * @param lockFilterPolicies Lock filters for the given datasources * @return Map from datasource to intervals locked by tasks satisfying the lock filter condititions */ - public Map> getLockedIntervalsV2(List lockFilterPolicies) + public Map> getLockedIntervals(List lockFilterPolicies) { final Map> datasourceToIntervals = new HashMap<>(); @@ -974,13 +974,14 @@ public Map> getLockedIntervalsV2(List l lockFilterPolicies.forEach( lockFilter -> { final String datasource = lockFilter.getDatasource(); - final int priority = lockFilter.getPriority(); - final boolean ignoreAppendLocks = - TaskLockType.REPLACE.name().equals(lockFilter.getContext().get(Tasks.TASK_LOCK_TYPE)); if (!running.containsKey(datasource)) { return; } + final int priority = lockFilter.getPriority(); + final boolean ignoreAppendLocks = + TaskLockType.REPLACE.name().equals(lockFilter.getContext().get(Tasks.TASK_LOCK_TYPE)); + running.get(datasource).forEach( (startTime, startTimeLocks) -> startTimeLocks.forEach( (interval, taskLockPosses) -> taskLockPosses.forEach( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index ba984401d3c9..140d9b7ac404 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -65,9 +65,9 @@ public List getActiveTasks() * @param lockFilterPolicies Requests for conflicing lock intervals for various datasources * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked */ - public Map> getLockedIntervalsV2(List lockFilterPolicies) + public Map> getLockedIntervals(List lockFilterPolicies) { - return taskLockbox.getLockedIntervalsV2(lockFilterPolicies); + return taskLockbox.getLockedIntervals(lockFilterPolicies); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 7eabb73dc2b9..f9604492dd44 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -283,11 +283,11 @@ public Response getDatasourceLockedIntervals(Map minTaskPriorit public Response getDatasourceLockedIntervalsV2(List lockFilterPolicies) { if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { - return Response.status(Status.BAD_REQUEST).entity("No Datasource provided").build(); + return Response.status(Status.BAD_REQUEST).entity("No filter provided").build(); } // Build the response - return Response.ok(taskStorageQueryAdapter.getLockedIntervalsV2(lockFilterPolicies)).build(); + return Response.ok(taskStorageQueryAdapter.getLockedIntervals(lockFilterPolicies)).build(); } @GET diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 177123c900b6..d55f3cc8bd0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -292,7 +292,8 @@ public boolean registerNewVersionOfPendingSegmentOnSupervisor( return true; } catch (Exception e) { - log.error(e, "Pending segment mapping update request failed"); + log.error(e, "PendingSegment[%s] mapping update request to version[%s] on Supervisor[%s] failed", + basePendingSegment.asSegmentId(), newSegmentVersion.getVersion(), supervisorId); } return false; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 18e9fa9ccfb1..01dfc0f5d67d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -1252,7 +1252,7 @@ public void testGetLockedIntervalsForEqualPriorityTask() } @Test - public void testGetLockedIntervalsV2ForHigherPriorityExclusiveLock() + public void testGetLockedIntervalsForHigherPriorityExclusiveLock() { final Task task = NoopTask.ofPriority(50); lockbox.add(task); @@ -1270,7 +1270,7 @@ public void testGetLockedIntervalsV2ForHigherPriorityExclusiveLock() ); Map> conflictingIntervals = - lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertTrue(conflictingIntervals.isEmpty()); } @@ -1293,7 +1293,7 @@ public void testGetLockedIntervalsForLowerPriorityExclusiveLock() ); Map> conflictingIntervals = - lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertEquals(1, conflictingIntervals.size()); Assert.assertEquals( Collections.singletonList(Intervals.of("2017/2018")), @@ -1302,7 +1302,7 @@ public void testGetLockedIntervalsForLowerPriorityExclusiveLock() } @Test - public void testGetLockedIntervalsV2ForLowerPriorityReplaceLock() + public void testGetLockedIntervalsForLowerPriorityReplaceLock() { final Task task = NoopTask.ofPriority(50); lockbox.add(task); @@ -1320,7 +1320,7 @@ public void testGetLockedIntervalsV2ForLowerPriorityReplaceLock() ); Map> conflictingIntervals = - lockbox.getLockedIntervalsV2(ImmutableList.of(requestForExclusiveLowerPriorityLock)); + lockbox.getLockedIntervals(ImmutableList.of(requestForExclusiveLowerPriorityLock)); Assert.assertTrue(conflictingIntervals.isEmpty()); } From 037390a35fe43bb98ac9c0359cc1608fb30fd073 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 12 Oct 2023 18:26:14 +0530 Subject: [PATCH 31/31] Clean dangling V2s --- .../java/org/apache/druid/rpc/indexing/OverlordClient.java | 2 +- .../org/apache/druid/rpc/indexing/OverlordClientImpl.java | 2 +- .../druid/server/coordinator/duty/CompactSegments.java | 2 +- .../apache/druid/client/indexing/NoopOverlordClient.java | 2 +- .../apache/druid/rpc/indexing/OverlordClientImplTest.java | 4 ++-- .../druid/server/coordinator/duty/CompactSegmentsTest.java | 6 +++--- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index a18af000d476..6a8e515b3270 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -192,7 +192,7 @@ ListenableFuture> taskStatuses( * @return Map from datasource name to list of intervals locked by tasks that have a conflicting lock type with * priority greater than or equal to the {@code minTaskPriority} for that datasource. */ - ListenableFuture>> findLockedIntervalsV2( + ListenableFuture>> findLockedIntervals( List lockFilterPolicies ); diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index a3b39b6d3e55..d7fab4b75fa2 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -189,7 +189,7 @@ public ListenableFuture taskStatus(final String taskId) } @Override - public ListenableFuture>> findLockedIntervalsV2( + public ListenableFuture>> findLockedIntervals( List lockFilterPolicies ) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 7f57d641bb7b..ab92f1801495 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -270,7 +270,7 @@ private Map> getLockedIntervals( .map(config -> new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), config.getTaskContext())) .collect(Collectors.toList()); final Map> datasourceToLockedIntervals = - new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervalsV2(lockFilterPolicies), true)); + new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); LOG.debug( "Skipping the following intervals for Compaction as they are currently locked: %s", datasourceToLockedIntervals diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index b123945cb5bd..42ca59ffee71 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -96,7 +96,7 @@ public ListenableFuture> supervisorStatuses( } @Override - public ListenableFuture>> findLockedIntervalsV2( + public ListenableFuture>> findLockedIntervals( List lockFilterPolicies ) { diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index e9d6f287b632..7ba5916a7710 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -236,7 +236,7 @@ public void test_findLockedIntervals() throws Exception Assert.assertEquals( lockMap, - overlordClient.findLockedIntervalsV2(requests).get() + overlordClient.findLockedIntervals(requests).get() ); } @@ -257,7 +257,7 @@ public void test_findLockedIntervals_nullReturn() throws Exception Assert.assertEquals( Collections.emptyMap(), - overlordClient.findLockedIntervalsV2(requests).get() + overlordClient.findLockedIntervals(requests).get() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 35e0bf5467f1..43b1c50c9698 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -1093,7 +1093,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() .thenReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage(ImmutableList.of(runningConflictCompactionTask).iterator()))); - Mockito.when(mockClient.findLockedIntervalsV2(ArgumentMatchers.any())) + Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.cancelTask(conflictTaskId)) .thenReturn(Futures.immediateFuture(null)); @@ -1997,7 +1997,7 @@ public ListenableFuture runTask(String taskId, Object taskObject) @Override - public ListenableFuture>> findLockedIntervalsV2( + public ListenableFuture>> findLockedIntervals( List lockFilterPolicies ) { @@ -2190,7 +2190,7 @@ private static ArgumentCaptor setUpMockClient(final OverlordClient mockC final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); Mockito.when(mockClient.taskStatuses(null, null, 0)) .thenReturn(Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator()))); - Mockito.when(mockClient.findLockedIntervalsV2(ArgumentMatchers.any())) + Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.getTotalWorkerCapacity()) .thenReturn(Futures.immediateFuture(new IndexingTotalWorkerCapacityInfo(0, 0)));