From fd20a2f86e252dd9a28b4002a13cbc24faf70517 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 15 Nov 2022 19:44:45 +0530 Subject: [PATCH 01/35] Add SegmentAllocationQueue to batch allocation actions --- .../druid/indexing/common/TaskToolbox.java | 10 +- .../common/actions/SegmentAllocateAction.java | 191 +------ .../actions/SegmentAllocateRequest.java | 67 +++ .../common/actions/SegmentAllocateResult.java | 49 ++ .../actions/SegmentAllocationQueue.java | 482 ++++++++++++++++ .../common/actions/TaskActionToolbox.java | 27 + .../druid/indexing/overlord/TaskLockbox.java | 302 +++++++++- ...TestIndexerMetadataStorageCoordinator.java | 14 + .../IndexerMetadataStorageCoordinator.java | 22 + .../overlord/SegmentCreateRequest.java | 89 +++ .../IndexerSQLMetadataStorageCoordinator.java | 516 +++++++++++++++--- 11 files changed, 1484 insertions(+), 285 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java create mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 6b2fad9610b3..897db9750354 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; @@ -329,14 +328,7 @@ public void publishSegments(Iterable segments) throws IOException // Request segment pushes for each set final Multimap segmentMultimap = Multimaps.index( segments, - new Function() - { - @Override - public Interval apply(DataSegment segment) - { - return segment.getInterval(); - } - } + DataSegment::getInterval ); for (final Collection segmentCollection : segmentMultimap.asMap().values()) { getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(segmentCollection))); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index f61dad98168d..c0f3bc9e3bd3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -26,29 +26,16 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.LockRequestForNewSegment; -import org.apache.druid.indexing.overlord.LockResult; -import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Collectors; +import java.util.concurrent.Future; /** * Allocates a pending segment for a given timestamp. The preferredSegmentGranularity is used if there are no prior @@ -180,177 +167,25 @@ public TypeReference getReturnTypeReference() }; } - @Override - public SegmentIdWithShardSpec perform( - final Task task, - final TaskActionToolbox toolbox - ) - { - int attempt = 0; - while (true) { - attempt++; - - if (!task.getDataSource().equals(dataSource)) { - throw new IAE("Task dataSource must match action dataSource, [%s] != [%s].", task.getDataSource(), dataSource); - } - - final IndexerMetadataStorageCoordinator msc = toolbox.getIndexerMetadataStorageCoordinator(); - - // 1) if something overlaps our timestamp, use that - // 2) otherwise try preferredSegmentGranularity & going progressively smaller - - final Interval rowInterval = queryGranularity.bucket(timestamp).withChronology(ISOChronology.getInstanceUTC()); - - final Set usedSegmentsForRow = - new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)); - - final SegmentIdWithShardSpec identifier; - if (usedSegmentsForRow.isEmpty()) { - identifier = tryAllocateFirstSegment(toolbox, task, rowInterval); - } else { - identifier = tryAllocateSubsequentSegment(toolbox, task, rowInterval, usedSegmentsForRow.iterator().next()); - } - if (identifier != null) { - return identifier; - } - - // Could not allocate a pending segment. There's a chance that this is because someone else inserted a segment - // overlapping with this row between when we called "msc.retrieveUsedSegmentsForInterval" and now. Check it again, - // and if it's different, repeat. - - Set newUsedSegmentsForRow = - new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)); - if (!newUsedSegmentsForRow.equals(usedSegmentsForRow)) { - if (attempt < MAX_ATTEMPTS) { - final long shortRandomSleep = 50 + (long) (ThreadLocalRandom.current().nextDouble() * 450); - log.debug( - "Used segment set changed for rowInterval[%s]. Retrying segment allocation in %,dms (attempt = %,d).", - rowInterval, - shortRandomSleep, - attempt - ); - try { - Thread.sleep(shortRandomSleep); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } else { - log.error( - "Used segment set changed for rowInterval[%s]. Not trying again (attempt = %,d).", - rowInterval, - attempt - ); - return null; - } - } else { - return null; - } - } - } - - private SegmentIdWithShardSpec tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval) + public Future performAsync(Task task, TaskActionToolbox toolbox) { - // No existing segments for this row, but there might still be nearby ones that conflict with our preferred - // segment granularity. Try that first, and then progressively smaller ones if it fails. - final List tryIntervals = Granularity.granularitiesFinerThan(preferredSegmentGranularity) - .stream() - .map(granularity -> granularity.bucket(timestamp)) - .collect(Collectors.toList()); - for (Interval tryInterval : tryIntervals) { - if (tryInterval.contains(rowInterval)) { - final SegmentIdWithShardSpec identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); - if (identifier != null) { - return identifier; - } - } - } - return null; + return toolbox.getSegmentAllocationQueue().add( + new SegmentAllocateRequest(task, this, MAX_ATTEMPTS) + ); } - private SegmentIdWithShardSpec tryAllocateSubsequentSegment( - TaskActionToolbox toolbox, - Task task, - Interval rowInterval, - DataSegment usedSegment - ) + @Override + public SegmentIdWithShardSpec perform(Task task, TaskActionToolbox toolbox) { - // Existing segment(s) exist for this row; use the interval of the first one. - if (!usedSegment.getInterval().contains(rowInterval)) { - log.error( - "The interval of existing segment[%s] doesn't contain rowInterval[%s]", - usedSegment.getId(), - rowInterval - ); - return null; - } else { - // If segment allocation failed here, it is highly likely an unrecoverable error. We log here for easier - // debugging. - return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true); + if (!task.getDataSource().equals(dataSource)) { + throw new IAE("Task dataSource must match action dataSource, [%s] != [%s].", task.getDataSource(), dataSource); } - } - - private SegmentIdWithShardSpec tryAllocate( - TaskActionToolbox toolbox, - Task task, - Interval tryInterval, - Interval rowInterval, - boolean logOnFail - ) - { - // This action is always used by appending tasks, so if it is a time_chunk lock then we allow it to be - // shared with other appending tasks as well - final LockResult lockResult = toolbox.getTaskLockbox().tryLock( - task, - new LockRequestForNewSegment( - lockGranularity, - taskLockType, - task.getGroupId(), - dataSource, - tryInterval, - partialShardSpec, - task.getPriority(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ); - if (lockResult.isRevoked()) { - // We had acquired a lock but it was preempted by other locks - throw new ISE("The lock for interval[%s] is preempted and no longer valid", tryInterval); + try { + return performAsync(task, toolbox).get(); } - - if (lockResult.isOk()) { - final SegmentIdWithShardSpec identifier = lockResult.getNewSegmentId(); - if (identifier != null) { - return identifier; - } else { - final String msg = StringUtils.format( - "Could not allocate pending segment for rowInterval[%s], segmentInterval[%s].", - rowInterval, - tryInterval - ); - if (logOnFail) { - log.error(msg); - } else { - log.debug(msg); - } - return null; - } - } else { - final String msg = StringUtils.format( - "Could not acquire lock for rowInterval[%s], segmentInterval[%s].", - rowInterval, - tryInterval - ); - if (logOnFail) { - log.error(msg); - } else { - log.debug(msg); - } - return null; + catch (Exception e) { + throw new RuntimeException(e); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java new file mode 100644 index 000000000000..8d26a27d9910 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.druid.indexing.common.task.Task; + +/** + * Request received by the overlord for segment allocation. + */ +public class SegmentAllocateRequest +{ + private final Task task; + private final SegmentAllocateAction action; + private final int maxAttempts; + + private int attempts; + + public SegmentAllocateRequest(Task task, SegmentAllocateAction action, int maxAttempts) + { + this.task = task; + this.action = action; + this.maxAttempts = maxAttempts; + } + + public Task getTask() + { + return task; + } + + public SegmentAllocateAction getAction() + { + return action; + } + + public void incrementAttempts() + { + ++attempts; + } + + public boolean canRetry() + { + return attempts < maxAttempts; + } + + public int getAttempts() + { + return attempts; + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java new file mode 100644 index 000000000000..995bfb9d31d0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateResult.java @@ -0,0 +1,49 @@ +/* + * 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 org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; + +public class SegmentAllocateResult +{ + private final SegmentIdWithShardSpec segmentId; + private final String errorMessage; + + public SegmentAllocateResult(SegmentIdWithShardSpec segmentId, String errorMessage) + { + this.segmentId = segmentId; + this.errorMessage = errorMessage; + } + + public SegmentIdWithShardSpec getSegmentId() + { + return segmentId; + } + + public String getErrorMessage() + { + return errorMessage; + } + + public boolean isSuccess() + { + return segmentId != null; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java new file mode 100644 index 000000000000..979779b34254 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -0,0 +1,482 @@ +/* + * 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.google.inject.Inject; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * Queue for {@link SegmentAllocateRequest}s. + */ +public class SegmentAllocationQueue implements DruidLeaderSelector.Listener +{ + private static final Logger log = new Logger(SegmentAllocationQueue.class); + private static final long MAX_WAIT_TIME_MILLIS = 5_000; + + private final TaskLockbox taskLockbox; + private final ScheduledExecutorService executor; + private final IndexerMetadataStorageCoordinator metadataStorage; + private final DruidLeaderSelector leaderSelector; + private final ServiceEmitter emitter; + + private final ConcurrentHashMap keyToBatch = new ConcurrentHashMap<>(); + private final Deque processingQueue = new ConcurrentLinkedDeque<>(); + + @Inject + public SegmentAllocationQueue( + TaskLockbox taskLockbox, + IndexerMetadataStorageCoordinator metadataStorage, + @IndexingService DruidLeaderSelector leaderSelector, + ServiceEmitter emitter + ) + { + this.emitter = emitter; + this.taskLockbox = taskLockbox; + this.metadataStorage = metadataStorage; + this.leaderSelector = leaderSelector; + this.executor = ScheduledExecutors.fixed(1, "SegmentAllocationQueue-%s"); + + if (leaderSelector.isLeader()) { + scheduleQueuePoll(MAX_WAIT_TIME_MILLIS); + } + leaderSelector.registerListener(this); + } + + private void scheduleQueuePoll(long delay) + { + executor.schedule(this::processBatchesDue, delay, TimeUnit.MILLISECONDS); + } + + /** + * Queues a SegmentAllocateRequest. The returned future may complete successfully + * with a non-null value or with a non-null value. + */ + public Future add(SegmentAllocateRequest request) + { + if (!leaderSelector.isLeader()) { + throw new ISE("Cannot allocate segment if not leader."); + } + + SegmentAllocateAction action = request.getAction(); + boolean isExclusiveTimeChunkLock = action.getLockGranularity() == LockGranularity.TIME_CHUNK + && action.getTaskLockType() == TaskLockType.EXCLUSIVE; + + final AllocateRequestBatch batch; + final AllocateRequestKey requestKey = AllocateRequestKey.forAction(action); + if (isExclusiveTimeChunkLock) { + // Cannot batch exclusive time chunk locks + batch = new AllocateRequestBatch(requestKey); + processingQueue.add(batch); + } else { + batch = keyToBatch.computeIfAbsent(requestKey, k -> { + AllocateRequestBatch b = new AllocateRequestBatch(k); + processingQueue.add(b); + return b; + }); + } + + return batch.add(request); + } + + private void processBatchesDue() + { + // If not leader, clear the queue and do not schedule any more rounds of processing + if (!leaderSelector.isLeader()) { + log.info("Not leader anymore. Clearing [%d] batches from queue.", processingQueue.size()); + processingQueue.clear(); + keyToBatch.clear(); + return; + } + + // Process all batches which are due + AllocateRequestBatch nextBatch = processingQueue.peek(); + while (nextBatch != null && nextBatch.isDue()) { + processNextBatch(); + nextBatch = processingQueue.peek(); + } + + // Schedule the next round of processing + final long nextScheduleDelay; + if (processingQueue.isEmpty()) { + nextScheduleDelay = MAX_WAIT_TIME_MILLIS; + } else { + nextBatch = processingQueue.peek(); + long timeElapsed = System.currentTimeMillis() - nextBatch.getQueueTime(); + nextScheduleDelay = Math.max(0, MAX_WAIT_TIME_MILLIS - timeElapsed); + } + scheduleQueuePoll(nextScheduleDelay); + } + + private void processNextBatch() + { + final AllocateRequestBatch requestBatch = processingQueue.poll(); + if (requestBatch == null || requestBatch.isEmpty()) { + return; + } + + final AllocateRequestKey requestKey = requestBatch.key; + keyToBatch.remove(requestKey); + + log.info("Processing [%d] requests for batch [%s].", requestBatch.size(), requestKey); + + final long startTimeMillis = System.currentTimeMillis(); + emitBatchMetric("task/action/batch/size", requestBatch.size(), requestKey); + emitBatchMetric("task/action/batch/queueTime", (startTimeMillis - requestBatch.getQueueTime()), requestKey); + + final int batchSize = requestBatch.size(); + final Set usedSegments = retrieveUsedSegments(requestKey); + final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); + + emitBatchMetric("task/action/batch/runTime", (System.currentTimeMillis() - startTimeMillis), requestKey); + log.info("Successfully processed [%d / %d] requests in batch [%s].", successCount, batchSize, requestKey); + + if (requestBatch.isEmpty()) { + log.info("All requests in batch [%s] have been processed.", requestKey); + } else { + // Requeue the batch only if used segments have changed + final Set updatedUsedSegments = retrieveUsedSegments(requestKey); + if (updatedUsedSegments.equals(usedSegments)) { + log.error( + "Used segments have not changed. Not requeueing [%d] failed requests in batch [%s].", + requestBatch.size(), + requestKey + ); + } else { + log.info( + "Used segment set changed from [%d] segments to [%d].", + usedSegments.size(), + updatedUsedSegments.size() + ); + log.info("Requeueing [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); + + requestBatch.resetQueueTime(); + processingQueue.offer(requestBatch); + } + } + } + + private Set retrieveUsedSegments(AllocateRequestKey key) + { + return new HashSet<>( + metadataStorage.retrieveUsedSegmentsForInterval( + key.dataSource, + key.rowInterval, + Segments.ONLY_VISIBLE + ) + ); + } + + private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set usedSegments) + { + final AllocateRequestKey requestKey = requestBatch.key; + final List tryIntervals = getTryIntervals(requestKey, usedSegments); + if (tryIntervals.isEmpty()) { + log.error("Found no valid interval containing the row interval [%s]", requestKey.rowInterval); + return 0; + } + + int successCount = 0; + for (Interval tryInterval : tryIntervals) { + final List requests = new ArrayList<>(requestBatch.requestToFuture.keySet()); + final List results = taskLockbox.allocateSegments( + requests, + requestKey.dataSource, + tryInterval, + requestKey.skipSegmentLineageCheck, + requestKey.lockGranularity + ); + + successCount += updateBatchWithResults(requestBatch, requests, results); + } + + return successCount; + } + + /** + * Gets the intervals for which allocation should be tried. + *

+ * If there are no used segments for this row, first try to allocate segments + * using the preferred segment granularity. If that fails due to other nearby + * segments, try progressively smaller granularities. + *

+ * If there are used segments for this row, try only the interval of those used + * segments (we assume that all of them must have the same interval). + */ + private List getTryIntervals(AllocateRequestKey key, Set usedSegments) + { + final Interval rowInterval = key.rowInterval; + if (usedSegments.isEmpty()) { + return Granularity.granularitiesFinerThan(key.preferredSegmentGranularity) + .stream() + .map(granularity -> granularity.bucket(rowInterval.getStart())) + .filter(interval -> interval.contains(rowInterval)) + .collect(Collectors.toList()); + } else { + Interval existingInterval = usedSegments.iterator().next().getInterval(); + if (existingInterval.contains(rowInterval)) { + return Collections.singletonList(existingInterval); + } else { + return Collections.emptyList(); + } + } + } + + private int updateBatchWithResults( + AllocateRequestBatch requestBatch, + List requests, + List results + ) + { + int successCount = 0; + for (int i = 0; i < requests.size(); ++i) { + SegmentAllocateResult result = results.get(i); + if (result.isSuccess()) { + ++successCount; + } + + requestBatch.handleResult(result, requests.get(i)); + } + return successCount; + } + + private void emitTaskMetric(String metric, long value, SegmentAllocateRequest request) + { + final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, request.getTask()); + metricBuilder.setDimension("taskActionType", SegmentAllocateAction.TYPE); + emitter.emit(metricBuilder.build(metric, value)); + } + + private void emitBatchMetric(String metric, long value, AllocateRequestKey key) + { + final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); + metricBuilder.setDimension("taskActionType", SegmentAllocateAction.TYPE); + metricBuilder.setDimension(DruidMetrics.DATASOURCE, key.dataSource); + emitter.emit(metricBuilder.build(metric, value)); + } + + @Override + public void becomeLeader() + { + log.info("Elected leader. Starting queue processing."); + + // Start polling the queue + scheduleQueuePoll(MAX_WAIT_TIME_MILLIS); + } + + @Override + public void stopBeingLeader() + { + log.info("Not leader anymore. Stopping queue processing."); + } + + /** + * A batch of segment allocation requests. + */ + private class AllocateRequestBatch + { + private final AllocateRequestKey key; + private final Map> + requestToFuture = new HashMap<>(); + private long queueTimeMillis; + + AllocateRequestBatch(AllocateRequestKey key) + { + this.key = key; + } + + Future add(SegmentAllocateRequest request) + { + return requestToFuture.computeIfAbsent(request, req -> new CompletableFuture<>()); + } + + void handleResult(SegmentAllocateResult result, SegmentAllocateRequest request) + { + request.incrementAttempts(); + emitTaskMetric("task/action/attempt/count", request.getAttempts(), request); + + if (result.isSuccess()) { + emitTaskMetric("task/action/success/count", 1L, request); + requestToFuture.remove(request).complete(result.getSegmentId()); + return; + } + + log.info("Failed to allocate segment for action [%s]: %s", request.getAction(), result.getErrorMessage()); + if (request.canRetry()) { + log.debug( + "Can requeue action [%s] after [%d] failed attempts.", + request.getAction(), + request.getAttempts() + ); + } else { + log.error( + "Removing allocation action [%s] from batch after [%d] failed attempts.", + request.getAction(), + request.getAttempts() + ); + requestToFuture.remove(request).complete(null); + } + } + + boolean isEmpty() + { + return requestToFuture.isEmpty(); + } + + int size() + { + return requestToFuture.size(); + } + + void resetQueueTime() + { + queueTimeMillis = System.currentTimeMillis(); + } + + long getQueueTime() + { + return queueTimeMillis; + } + + boolean isDue() + { + return System.currentTimeMillis() - queueTimeMillis > MAX_WAIT_TIME_MILLIS; + } + } + + /** + * Key to identify a batch of allocation requests. + */ + private static class AllocateRequestKey + { + private final String dataSource; + private final Interval rowInterval; + private final Granularity queryGranularity; + private final Granularity preferredSegmentGranularity; + + private final boolean skipSegmentLineageCheck; + private final LockGranularity lockGranularity; + private final TaskLockType taskLockType; + + private final boolean useNonRootGenPartitionSpace; + private final int hash; + + static AllocateRequestKey forAction(SegmentAllocateAction action) + { + return new AllocateRequestKey(action); + } + + AllocateRequestKey(SegmentAllocateAction action) + { + this.dataSource = action.getDataSource(); + this.queryGranularity = action.getQueryGranularity(); + this.preferredSegmentGranularity = action.getPreferredSegmentGranularity(); + this.skipSegmentLineageCheck = action.isSkipSegmentLineageCheck(); + this.lockGranularity = action.getLockGranularity(); + this.taskLockType = action.getTaskLockType(); + + this.useNonRootGenPartitionSpace = action.getPartialShardSpec() + .useNonRootGenerationPartitionSpace(); + this.rowInterval = queryGranularity.bucket(action.getTimestamp()) + .withChronology(ISOChronology.getInstanceUTC()); + + this.hash = Objects.hash( + skipSegmentLineageCheck, + useNonRootGenPartitionSpace, + dataSource, + rowInterval, + queryGranularity, + preferredSegmentGranularity, + lockGranularity, + taskLockType + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AllocateRequestKey that = (AllocateRequestKey) o; + return skipSegmentLineageCheck == that.skipSegmentLineageCheck + && useNonRootGenPartitionSpace == that.useNonRootGenPartitionSpace + && dataSource.equals(that.dataSource) + && rowInterval.equals(that.rowInterval) + && queryGranularity.equals(that.queryGranularity) + && preferredSegmentGranularity.equals(that.preferredSegmentGranularity) + && lockGranularity == that.lockGranularity + && taskLockType == that.taskLockType; + } + + @Override + public int hashCode() + { + return hash; + } + + @Override + public String toString() + { + return "AllocateRequestKey{" + + "dataSource='" + dataSource + '\'' + + ", rowInterval=" + rowInterval + + '}'; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java index 134a9bf6c72e..169bde0f99a2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java @@ -35,6 +35,7 @@ public class TaskActionToolbox { private final TaskLockbox taskLockbox; private final TaskStorage taskStorage; + private final SegmentAllocationQueue segmentAllocationQueue; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final ServiceEmitter emitter; private final SupervisorManager supervisorManager; @@ -46,6 +47,7 @@ public TaskActionToolbox( TaskLockbox taskLockbox, TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + SegmentAllocationQueue segmentAllocationQueue, ServiceEmitter emitter, SupervisorManager supervisorManager, @Json ObjectMapper jsonMapper @@ -57,6 +59,27 @@ public TaskActionToolbox( this.emitter = emitter; this.supervisorManager = supervisorManager; this.jsonMapper = jsonMapper; + this.segmentAllocationQueue = segmentAllocationQueue; + } + + public TaskActionToolbox( + TaskLockbox taskLockbox, + TaskStorage taskStorage, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + ServiceEmitter emitter, + SupervisorManager supervisorManager, + @Json ObjectMapper jsonMapper + ) + { + this( + taskLockbox, + taskStorage, + indexerMetadataStorageCoordinator, + null, + emitter, + supervisorManager, + jsonMapper + ); } public TaskLockbox getTaskLockbox() @@ -103,4 +126,8 @@ public Optional getTaskRunner() return Optional.absent(); } + public SegmentAllocationQueue getSegmentAllocationQueue() + { + return segmentAllocationQueue; + } } 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 41caf0620fb3..362cd768f92e 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 @@ -20,7 +20,6 @@ package org.apache.druid.indexing.overlord; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -34,6 +33,9 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +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.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -446,6 +448,143 @@ public LockResult tryLock(final Task task, final LockRequest request) } } + /** + * Attempts to allocate segments for the given requests. Each request contains + * a {@link Task} and a {@link SegmentAllocateAction}. This method tries to + * acquire the task locks on the required intervals/segments and then performs + * a batch allocation of segments. It is possible that some requests succeed + * successfully and others failed. In that case, only the failed ones should be + * retried. + * + * @param requests List of allocation requests + * @param dataSource Datasource for which segment is to be allocated. + * @param interval Interval for which segment is to be allocated. + * @param skipSegmentLineageCheck Whether lineage check is to be skipped + * (this is true for streaming ingestion) + * @param lockGranularity Granularity of task lock + * @return List of allocation results in the same order as the requests. + */ + public List allocateSegments( + List requests, + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + LockGranularity lockGranularity + ) + { + final boolean isTimeChunkLock = lockGranularity == LockGranularity.TIME_CHUNK; + + final AllocationHolderList holderList = new AllocationHolderList(requests, interval); + holderList.pending.forEach(this::verifyTaskIsActive); + + giant.lock(); + try { + if (isTimeChunkLock) { + holderList.pending.forEach(holder -> acquireTaskLock(holder, true)); + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.pending); + } else { + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.pending); + holderList.pending.forEach(holder -> acquireTaskLock(holder, false)); + } + + // TODO: for failed allocations, cleanup newly created locks from the posse map + + holderList.pending.forEach(holder -> addTaskAndPersistLocks(holder, isTimeChunkLock)); + } + finally { + giant.unlock(); + } + + return holderList.all.stream().map(holder -> holder.result).collect(Collectors.toList()); + } + + /** + * Marks the segment allocation as failed if the underlying task is not active. + */ + private void verifyTaskIsActive(SegmentAllocationHolder holder) + { + final String taskId = holder.task.getId(); + if (!activeTasks.contains(taskId)) { + holder.markFailed("Unable to grant lock to inactive Task [%s]", taskId); + } + } + + /** + * Creates a task lock request and creates or finds the lock for that request. + * Marks the segment allocation as failed if the lock could not be acquired or + * was revoked. + */ + private void acquireTaskLock(SegmentAllocationHolder holder, boolean isTimeChunkLock) + { + final LockRequest lockRequest; + final LockRequestForNewSegment lockRequestForNewSegment = holder.createLockRequest(); + if (isTimeChunkLock) { + lockRequest = new TimeChunkLockRequest(lockRequestForNewSegment); + } else { + lockRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, holder.allocatedSegment); + } + + // Create or find the task lock for the created lock request + final TaskLockPosse posseToUse = createOrFindLockPosse(lockRequest); + final TaskLock acquiredLock = posseToUse == null ? null : posseToUse.getTaskLock(); + if (posseToUse == null) { + holder.markFailed("Could not find or create lock posse."); + } else if (acquiredLock.isRevoked()) { + holder.markFailed("Lock was revoked."); + } else { + // Update the holder + holder.lockRequest = lockRequest; + holder.taskLockPosse = posseToUse; + holder.acquiredLock = acquiredLock; + } + } + + /** + * Adds the task to the found lock posse if not already added and updates + * in the metadata store. Marks the segment allocation as failed if the update + * did not succeed. + */ + private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTimeChunkLock) + { + final Task task = holder.task; + final TaskLock acquiredLock = holder.acquiredLock; + + if (holder.taskLockPosse.addTask(task)) { + log.info("Added task [%s] to TaskLock [%s]", task.getId(), acquiredLock); + + // This can also be batched later + boolean success = updateLockInStorage(task, acquiredLock); + if (!success) { + final Integer partitionId = isTimeChunkLock + ? null : ((SegmentLock) acquiredLock).getPartitionId(); + unlock(task, holder.lockRequest.getInterval(), partitionId); + holder.markFailed("Could not update task lock in metadata store."); + } + } else { + log.info("Task [%s] already present in TaskLock [%s]", task.getId(), acquiredLock.getGroupId()); + } + + holder.markSucceeded(); + } + + private boolean updateLockInStorage(Task task, TaskLock taskLock) + { + try { + taskStorage.addLock(task.getId(), taskLock); + return true; + } + catch (Exception e) { + log.makeAlert("Failed to persist lock in storage") + .addData("task", task.getId()) + .addData("dataSource", taskLock.getDataSource()) + .addData("interval", taskLock.getInterval()) + .addData("version", taskLock.getVersion()) + .emit(); + + return false; + } + } + private TaskLockPosse createOrFindLockPosse(LockRequest request) { Preconditions.checkState(!(request instanceof LockRequestForNewSegment), "Can't handle LockRequestForNewSegment"); @@ -537,7 +676,6 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request) * monotonicity and that callers specifying {@code preferredVersion} are doing the right thing. * * @param request request to lock - * * @return a new {@link TaskLockPosse} */ private TaskLockPosse createNewTaskLockPosse(LockRequest request) @@ -546,7 +684,10 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) try { final TaskLockPosse posseToUse = new TaskLockPosse(request.toLock()); running.computeIfAbsent(request.getDataSource(), k -> new TreeMap<>()) - .computeIfAbsent(request.getInterval().getStart(), k -> new TreeMap<>(Comparators.intervalsByStartThenEnd())) + .computeIfAbsent( + request.getInterval().getStart(), + k -> new TreeMap<>(Comparators.intervalsByStartThenEnd()) + ) .computeIfAbsent(request.getInterval(), k -> new ArrayList<>()) .add(posseToUse); @@ -557,6 +698,39 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) } } + private void allocateSegmentIds( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + Collection holders + ) + { + if (holders.isEmpty()) { + return; + } + + final List createRequests = holders.stream() + .map(SegmentAllocationHolder::getSegmentRequest) + .collect(Collectors.toList()); + + Map allocatedSegments = + metadataStorageCoordinator.allocatePendingSegments( + dataSource, + interval, + skipSegmentLineageCheck, + createRequests + ); + + for (SegmentAllocationHolder holder : holders) { + SegmentIdWithShardSpec segmentId = allocatedSegments.get(holder.getSegmentRequest()); + if (segmentId == null) { + holder.markFailed("Storage coordinator could not allocate segment."); + } else { + holder.setAllocatedSegment(segmentId); + } + } + } + private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment request, String version) { return metadataStorageCoordinator.allocatePendingSegment( @@ -573,7 +747,7 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques /** * Perform the given action with a guarantee that the locks of the task are not revoked in the middle of action. This * method first checks that all locks for the given task and intervals are valid and perform the right action. - * + *

* The given action should be finished as soon as possible because all other methods in this class are blocked until * this method is finished. * @@ -607,7 +781,7 @@ private boolean isTaskLocksValid(Task task, List intervals) .allMatch(interval -> { final List lockPosses = getOnlyTaskLockPosseContainingInterval(task, interval); return lockPosses.stream().map(TaskLockPosse::getTaskLock).noneMatch( - lock -> lock.isRevoked() + TaskLock::isRevoked ); }); } @@ -660,7 +834,9 @@ protected void revokeLock(String taskId, TaskLock lock) final TaskLock revokedLock = lock.revokedCopy(); taskStorage.replaceLock(taskId, lock, revokedLock); - final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval().getStart()).get(lock.getInterval()); + final List possesHolder = running.get(task.getDataSource()) + .get(lock.getInterval().getStart()) + .get(lock.getInterval()); final TaskLockPosse foundPosse = possesHolder.stream() .filter(posse -> posse.getTaskLock().equals(lock)) .findFirst() @@ -688,16 +864,7 @@ public List findLocksForTask(final Task task) giant.lock(); try { - return Lists.transform( - findLockPossesForTask(task), new Function() - { - @Override - public TaskLock apply(TaskLockPosse taskLockPosse) - { - return taskLockPosse.getTaskLock(); - } - } - ); + return Lists.transform(findLockPossesForTask(task), TaskLockPosse::getTaskLock); } finally { giant.unlock(); @@ -774,7 +941,7 @@ public void unlock(final Task task, final Interval interval) * Release lock held for a task on a particular interval. Does nothing if the task does not currently * hold the mentioned lock. * - * @param task task to unlock + * @param task task to unlock * @param interval interval to unlock */ public void unlock(final Task task, final Interval interval, @Nullable Integer partitionId) @@ -1178,7 +1345,6 @@ boolean reusableFor(LockRequest request) } else { throw new ISE("Unknown request type[%s]", request); } - //noinspection SuspiciousIndentAfterControlStatement default: throw new ISE("Unknown lock type[%s]", taskLock.getType()); } @@ -1206,7 +1372,7 @@ public boolean equals(Object o) TaskLockPosse that = (TaskLockPosse) o; return java.util.Objects.equals(taskLock, that.taskLock) && - java.util.Objects.equals(taskIds, that.taskIds); + java.util.Objects.equals(taskIds, that.taskIds); } @Override @@ -1224,4 +1390,102 @@ public String toString() .toString(); } } + + /** + * Maintains a list of pending allocation holders. + */ + private static class AllocationHolderList + { + final List all = new ArrayList<>(); + final Set pending = new HashSet<>(); + + AllocationHolderList(List requests, Interval interval) + { + for (SegmentAllocateRequest request : requests) { + SegmentAllocationHolder holder = new SegmentAllocationHolder(request, interval, this); + all.add(holder); + pending.add(holder); + } + } + + void markCompleted(SegmentAllocationHolder holder) + { + pending.remove(holder); + } + } + + /** + * Contains the task, request, lock and final result for a segment allocation. + */ + private static class SegmentAllocationHolder + { + final AllocationHolderList list; + + final Task task; + final Interval interval; + final SegmentAllocateAction action; + SegmentCreateRequest segmentRequest; + + SegmentIdWithShardSpec allocatedSegment; + SegmentAllocateResult result; + TaskLock acquiredLock; + TaskLockPosse taskLockPosse; + LockRequest lockRequest; + + SegmentAllocationHolder(SegmentAllocateRequest request, Interval interval, AllocationHolderList list) + { + this.list = list; + this.interval = interval; + this.task = request.getTask(); + this.action = request.getAction(); + } + + LockRequestForNewSegment createLockRequest() + { + return new LockRequestForNewSegment( + action.getLockGranularity(), + action.getTaskLockType(), + task.getGroupId(), + action.getDataSource(), + interval, + action.getPartialShardSpec(), + task.getPriority(), + action.getSequenceName(), + action.getPreviousSegmentId(), + action.isSkipSegmentLineageCheck() + ); + } + + SegmentCreateRequest getSegmentRequest() + { + // Initialize the first time this is requested + if (segmentRequest == null) { + segmentRequest = new SegmentCreateRequest( + action.getSequenceName(), + action.getPreviousSegmentId(), + acquiredLock.getVersion(), + action.getPartialShardSpec() + ); + } + + return segmentRequest; + } + + void markFailed(String msgFormat, Object... args) + { + list.markCompleted(this); + result = new SegmentAllocateResult(null, String.format(msgFormat, args)); + } + + void markSucceeded() + { + list.markCompleted(this); + result = new SegmentAllocateResult(allocatedSegment, null); + } + + void setAllocatedSegment(SegmentIdWithShardSpec segmentId) + { + this.allocatedSegment = segmentId; + } + } } 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 2dcff7694678..d64bd1d22263 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 @@ -25,6 +25,7 @@ import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; @@ -36,8 +37,10 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator @@ -124,6 +127,17 @@ public Set announceHistoricalSegments(Set segments) return ImmutableSet.copyOf(added); } + @Override + public Map allocatePendingSegments( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + List requests + ) + { + return Collections.emptyMap(); + } + @Override public SegmentPublishResult announceHistoricalSegments( 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 acb617f78a4c..b3c70f0cdbe9 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 @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -154,6 +155,27 @@ Collection retrieveUsedSegmentsForIntervals( */ Set announceHistoricalSegments(Set segments) throws IOException; + /** + * Allocates pending segments for the given requests in the pending segments table. + * The segment id allocated for a request will not be given out again unless a + * request is made with the same {@link SegmentCreateRequest}. + * + * @param dataSource dataSource for which to allocate a segment + * @param interval interval for which to allocate a segment + * @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence. + * Should be set to false if replica tasks would index events in same order + * @param requests Requests for which to allocate segments. All + * the requests must share the same partition space. + * @return Map from request to allocated segment id. The map does not contain + * entries for failed requests. + */ + Map allocatePendingSegments( + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + List requests + ); + /** * Allocate a new pending segment in the pending segments table. This segment identifier will never be given out * again, unless another call is made with the same dataSource, sequenceName, and previousSegmentId. diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java new file mode 100644 index 000000000000..d322db3b6429 --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java @@ -0,0 +1,89 @@ +/* + * 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.overlord; + +import org.apache.druid.timeline.partition.PartialShardSpec; + +/** + * Contains information used by {@link IndexerMetadataStorageCoordinator} for + * creating a new segment. + *

+ * The {@code sequenceName} and {@code previousSegmentId} fields are meant to + * make it easy for two independent ingestion tasks to produce the same series + * of segments. + */ +public class SegmentCreateRequest +{ + private final String version; + private final String sequenceName; + private final String previousSegmentId; + private final PartialShardSpec partialShardSpec; + + public SegmentCreateRequest( + String sequenceName, + String previousSegmentId, + String version, + PartialShardSpec partialShardSpec + ) + { + this.sequenceName = sequenceName; + this.previousSegmentId = previousSegmentId == null ? "" : previousSegmentId; + this.version = version; + this.partialShardSpec = partialShardSpec; + } + + public String getUniqueSequenceId() + { + return getUniqueSequenceId(sequenceName, previousSegmentId); + } + + public String getSequenceName() + { + return sequenceName; + } + + /** + * Non-null previous segment id. This can be used for persisting to the + * pending segments table in the metadata store. + */ + public String getPreviousSegmentId() + { + return previousSegmentId; + } + + public String getVersion() + { + return version; + } + + public PartialShardSpec getPartialShardSpec() + { + return partialShardSpec; + } + + /** + * Returns a String representing (sequenceName + previousSegmentId) used to + * uniquely identify a segment. + */ + public static String getUniqueSequenceId(String sequenceName, String previousSegmentId) + { + return sequenceName + "####" + previousSegmentId; + } +} 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 cc42d77e1b70..3b19bbf61be8 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -34,6 +34,7 @@ import org.apache.commons.lang.StringEscapeUtils; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; @@ -75,15 +76,19 @@ import javax.validation.constraints.NotNull; import java.io.IOException; import java.sql.ResultSet; +import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -221,6 +226,10 @@ public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interv return numSegmentsMarkedUnused; } + /** + * Fetches all the pending segments, whose interval overlaps with the given + * search interval from the metadata store. + */ private Set getPendingSegmentsForIntervalWithHandle( final Handle handle, final String dataSource, @@ -481,6 +490,42 @@ public int getSqlMetadataMaxRetry() return SQLMetadataConnector.DEFAULT_MAX_TRIES; } + @Override + public Map allocatePendingSegments( + String dataSource, + Interval allocateInterval, + boolean skipSegmentLineageCheck, + List requests + ) + { + Preconditions.checkNotNull(dataSource, "dataSource"); + Preconditions.checkNotNull(allocateInterval, "interval"); + + final Interval interval = allocateInterval.withChronology(ISOChronology.getInstanceUTC()); + + @SuppressWarnings("UnstableApiUsage") + final Function sequenceShaFunction = + request -> + skipSegmentLineageCheck + ? BaseEncoding.base16().encode( + Hashing.sha1().newHasher() + .putBytes(StringUtils.toUtf8(request.getSequenceName())) + .putByte((byte) 0xff) + .putLong(interval.getStartMillis()) + .putLong(interval.getEndMillis()) + .hash().asBytes()) + : BaseEncoding.base16().encode( + Hashing.sha1().newHasher() + .putBytes(StringUtils.toUtf8(request.getSequenceName())) + .putByte((byte) 0xff) + .putBytes(StringUtils.toUtf8(request.getPreviousSegmentId())) + .hash().asBytes()); + + return connector.retryWithHandle( + handle -> allocatePendingSegments(handle, dataSource, interval, requests, sequenceShaFunction) + ); + } + @Override public SegmentIdWithShardSpec allocatePendingSegment( final String dataSource, @@ -599,6 +644,52 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( return newIdentifier; } + private Map allocatePendingSegments( + final Handle handle, + final String dataSource, + final Interval interval, + final List requests, + final Function sequenceShaFunction + ) throws IOException + { + final Map existingSegmentIds = + checkAndGetExistingSegmentIds(handle, dataSource, interval); + + // For every request see if a segment id already exists + final Map allocatedSegmentIds = new HashMap<>(); + final List requestsForNewSegments = new ArrayList<>(); + for (SegmentCreateRequest request : requests) { + SegmentIdWithShardSpec existingSegmentId = existingSegmentIds.get(request.getUniqueSequenceId()); + if (existingSegmentId == null) { + requestsForNewSegments.add(request); + } else { + allocatedSegmentIds.put(request, existingSegmentId); + } + } + + // For each of the remaining requests, create a new segment + final Map createdSegments = + createNewSegments(handle, dataSource, interval, requestsForNewSegments); + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + + // UNIQUE key for the row, ensuring we don't have more than one segment per sequence per interval. + // Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines + // have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319) + insertPendingSegmentsIntoMetastore( + handle, + createdSegments, + dataSource, + interval, + sequenceShaFunction + ); + + allocatedSegmentIds.putAll(createdSegments); + return allocatedSegmentIds; + } + @Nullable private SegmentIdWithShardSpec allocatePendingSegment( final Handle handle, @@ -631,7 +722,6 @@ private SegmentIdWithShardSpec allocatePendingSegment( ); if (result.found) { - // The found existing segment identifier can be null if its interval doesn't match with the given interval return result.segmentIdentifier; } @@ -672,6 +762,47 @@ private SegmentIdWithShardSpec allocatePendingSegment( return newIdentifier; } + /** + * Returns a map from sequenceId (sequenceName + previousSegmentId) to segment id. + */ + private Map checkAndGetExistingSegmentIds( + Handle handle, + String dataSource, + Interval interval + ) throws IOException + { + final Query> query = handle + .createQuery( + StringUtils.format( + "SELECT start, %2$send%2$s, sequence_name, payload " + + "FROM %s WHERE " + + "dataSource = :dataSource AND " + + "start = :start AND " + + "%2$send%2$s = :end", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + ) + ) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + + final ResultIterator dbSegments = query + .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r, jsonMapper)) + .iterator(); + + // key of map should be sequenceName + previousSegmentId in a non-null thingy + final Map sequenceIdToSegmentId = new HashMap<>(); + while (dbSegments.hasNext()) { + final PendingSegmentsRecord record = dbSegments.next(); + final SegmentIdWithShardSpec segmentId = + jsonMapper.readValue(record.getPayload(), SegmentIdWithShardSpec.class); + sequenceIdToSegmentId.put(record.getSequenceId(), segmentId); + } + + return sequenceIdToSegmentId; + } + private CheckExistingSegmentIdResult checkAndGetExistingSegmentId( final Query> query, final Interval interval, @@ -686,50 +817,36 @@ private CheckExistingSegmentIdResult checkAndGetExistingSegmentId( } final List existingBytes = boundQuery.map(ByteArrayMapper.FIRST).list(); - if (!existingBytes.isEmpty()) { + if (existingBytes.isEmpty()) { + return new CheckExistingSegmentIdResult(false, null); + } else { final SegmentIdWithShardSpec existingIdentifier = jsonMapper.readValue( Iterables.getOnlyElement(existingBytes), SegmentIdWithShardSpec.class ); - if (existingIdentifier.getInterval().getStartMillis() == interval.getStartMillis() - && existingIdentifier.getInterval().getEndMillis() == interval.getEndMillis()) { - if (previousSegmentId == null) { - log.info("Found existing pending segment [%s] for sequence[%s] in DB", existingIdentifier, sequenceName); - } else { - log.info( - "Found existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB", - existingIdentifier, - sequenceName, - previousSegmentId - ); - } + if (existingIdentifier.getInterval().isEqual(interval)) { + log.info( + "Found existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB", + existingIdentifier, + sequenceName, + previousSegmentId + ); return new CheckExistingSegmentIdResult(true, existingIdentifier); } else { - if (previousSegmentId == null) { - log.warn( - "Cannot use existing pending segment [%s] for sequence[%s] in DB, " - + "does not match requested interval[%s]", - existingIdentifier, - sequenceName, - interval - ); - } else { - log.warn( - "Cannot use existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB, " - + "does not match requested interval[%s]", - existingIdentifier, - sequenceName, - previousSegmentId, - interval - ); - } + log.warn( + "Cannot use existing pending segment [%s] for sequence[%s] (previous = [%s]) in DB, " + + "does not match requested interval[%s]", + existingIdentifier, + sequenceName, + previousSegmentId, + interval + ); return new CheckExistingSegmentIdResult(true, null); } } - return new CheckExistingSegmentIdResult(false, null); } private static class CheckExistingSegmentIdResult @@ -745,6 +862,39 @@ private static class CheckExistingSegmentIdResult } } + private void insertPendingSegmentsIntoMetastore( + Handle handle, + Map createdSegments, + String dataSource, + Interval interval, + Function sequenceNamePrevIdSha1 + ) throws JsonProcessingException + { + final PreparedBatch insertBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " + + "sequence_name_prev_id_sha1, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " + + ":sequence_name_prev_id_sha1, :payload)", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + )); + + for (Map.Entry entry : createdSegments.entrySet()) { + final SegmentCreateRequest request = entry.getKey(); + final SegmentIdWithShardSpec segmentId = entry.getValue(); + insertBatch.bind("id", segmentId.toString()) + .bind("dataSource", dataSource) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .bind("sequence_name", request.getSequenceName()) + .bind("sequence_prev_id", request.getPreviousSegmentId()) + .bind("sequence_name_prev_id_sha1", sequenceNamePrevIdSha1.apply(request)) + .bind("payload", jsonMapper.writeValueAsBytes(segmentId)); + } + } + private void insertPendingSegmentIntoMetastore( Handle handle, SegmentIdWithShardSpec newIdentifier, @@ -777,6 +927,177 @@ private void insertPendingSegmentIntoMetastore( .execute(); } + private Map createNewSegments( + Handle handle, + String dataSource, + Interval interval, + List requests + ) throws IOException + { + // Get the time chunk and associated data segments for the given interval, if any + final List> existingChunks = + getTimelineForIntervalsWithHandle(handle, dataSource, Collections.singletonList(interval)) + .lookup(interval); + + if (existingChunks.size() > 1) { + // Not possible to expand more than one chunk with a single segment. + log.warn( + "Cannot allocate new segments for dataSource[%s], interval[%s]: already have [%,d] chunks.", + dataSource, + interval, + existingChunks.size() + ); + return Collections.emptyMap(); + } + + // Shard spec of any of the requests (as they are all compatible) used to + // identify existing shard specs that share partition space with the requested ones. + final PartialShardSpec partialShardSpec = requests.get(0).getPartialShardSpec(); + + // max partitionId of published data segments which share the same partition space. + SegmentIdWithShardSpec committedMaxId = null; + + @Nullable + final String versionOfExistingChunk; + if (existingChunks.isEmpty()) { + versionOfExistingChunk = null; + } else { + TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); + versionOfExistingChunk = existingHolder.getVersion(); + + // Don't use the stream API for performance. + for (DataSegment segment : FluentIterable + .from(existingHolder.getObject()) + .transform(PartitionChunk::getObject) + // Here we check only the segments of the shardSpec which shares the same partition space with the given + // partialShardSpec. Note that OverwriteShardSpec doesn't share the partition space with others. + // See PartitionIds. + .filter(segment -> segment.getShardSpec().sharePartitionSpace(partialShardSpec))) { + if (committedMaxId == null + || committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) { + committedMaxId = SegmentIdWithShardSpec.fromDataSegment(segment); + } + } + } + + + // Fetch the pending segments for this interval to determine max partitionId + // 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 pendingSegments = + getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); + if (committedMaxId != null) { + pendingSegments.add(committedMaxId); + } + + Map createdSegments = new HashMap<>(); + for (SegmentCreateRequest request : requests) { + SegmentIdWithShardSpec createdSegment = createNewSegment( + request, + dataSource, + interval, + versionOfExistingChunk, + committedMaxId, + pendingSegments + ); + if (createdSegment != null) { + createdSegments.put(request, createdSegment); + } + } + return createdSegments; + } + + private SegmentIdWithShardSpec createNewSegment( + SegmentCreateRequest request, + String dataSource, + Interval interval, + String versionOfExistingChunk, + SegmentIdWithShardSpec committedMaxId, + Set pendingSegments + ) + { + final PartialShardSpec partialShardSpec = request.getPartialShardSpec(); + final String existingVersion = request.getVersion(); + + // 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); + + // Determine the version of the new segment + final String newSegmentVersion; + if (versionOfExistingChunk != null) { + newSegmentVersion = versionOfExistingChunk; + } else if (overallMaxId != null) { + newSegmentVersion = overallMaxId.getVersion(); + } else { + // this is the first segment for this interval + newSegmentVersion = null; + } + + if (overallMaxId == null) { + // When appending segments, null overallMaxId means that we are allocating the very initial + // segment for this time chunk. + // This code is executed when the Overlord coordinates segment allocation, which is either you append segments + // or you use segment lock. Since the core partitions set is not determined for appended segments, we set + // it 0. When you use segment lock, the core partitions set doesn't work with it. We simply set it 0 so that the + // OvershadowableManager handles the atomic segment update. + final int newPartitionId = partialShardSpec.useNonRootGenerationPartitionSpace() + ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + : PartitionIds.ROOT_GEN_START_PARTITION_ID; + + String version = newSegmentVersion == null ? existingVersion : newSegmentVersion; + return new SegmentIdWithShardSpec( + dataSource, + interval, + version, + partialShardSpec.complete(jsonMapper, newPartitionId, 0) + ); + } else if (!overallMaxId.getInterval().equals(interval) + || overallMaxId.getVersion().compareTo(existingVersion) > 0) { + log.warn( + "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", + dataSource, + interval, + existingVersion, + overallMaxId + ); + return null; + } else if (committedMaxId != null + && committedMaxId.getShardSpec().getNumCorePartitions() + == SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) { + log.warn( + "Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]", + committedMaxId, + committedMaxId.getShardSpec() + ); + return null; + } else { + // The number of core partitions must always be chosen from the set of used segments in the SegmentTimeline. + // When the core partitions have been dropped, using pending segments may lead to an incorrect state + // where the chunk is believed to have core partitions and queries results are incorrect. + + return new SegmentIdWithShardSpec( + dataSource, + interval, + Preconditions.checkNotNull(newSegmentVersion, "newSegmentVersion"), + partialShardSpec.complete( + jsonMapper, + overallMaxId.getShardSpec().getPartitionNum() + 1, + committedMaxId == null ? 0 : committedMaxId.getShardSpec().getNumCorePartitions() + ) + ); + } + } + /** * This function creates a new segment for the given datasource/interval/etc. A critical * aspect of the creation is to make sure that the new version & new partition number will make @@ -818,15 +1139,18 @@ private SegmentIdWithShardSpec createNewSegment( return null; } else { - // max partitionId of the committed shardSpecs which share the same partition space. + // max partitionId of published data segments which share the same partition space. SegmentIdWithShardSpec committedMaxId = null; - // max partitionId of the all shardSpecs including the pending ones which share the same partition space. - SegmentIdWithShardSpec overallMaxId; - if (!existingChunks.isEmpty()) { + @Nullable + final String versionOfExistingChunk; + if (existingChunks.isEmpty()) { + versionOfExistingChunk = null; + } else { TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); + versionOfExistingChunk = existingHolder.getVersion(); - //noinspection ConstantConditions + // Don't use the stream API for performance. for (DataSegment segment : FluentIterable .from(existingHolder.getObject()) .transform(PartitionChunk::getObject) @@ -834,8 +1158,6 @@ private SegmentIdWithShardSpec createNewSegment( // partialShardSpec. Note that OverwriteShardSpec doesn't share the partition space with others. // See PartitionIds. .filter(segment -> segment.getShardSpec().sharePartitionSpace(partialShardSpec))) { - // Don't use the stream API for performance. - // Note that this will compute the max id of existing, visible, data segments in the time chunk: if (committedMaxId == null || committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) { committedMaxId = SegmentIdWithShardSpec.fromDataSegment(segment); @@ -843,63 +1165,41 @@ private SegmentIdWithShardSpec createNewSegment( } } - // Get the version of the existing chunk, we might need it in some of the cases below - // to compute the new identifier's version - @Nullable - final String versionOfExistingChunk; - if (!existingChunks.isEmpty()) { - // remember only one chunk possible for given interval so get the first & only one - versionOfExistingChunk = existingChunks.get(0).getVersion(); - } else { - versionOfExistingChunk = null; - } - - // next, we need to enrich the overallMaxId computed with committed segments with the information of the pending segments - // it is possible that a pending segment has a higher id in which case we need that, it will work, - // and it will avoid clashes when inserting the new pending segment later in the caller of this method + // Fetch the pending segments for this interval to determine max partitionId + // 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 ); - // Make sure we add the committed max id we obtained from the segments table: if (committedMaxId != null) { pendings.add(committedMaxId); } - // Now compute the overallMaxId with all the information: pendings + segments: - // The versionOfExistingChunks filter is ensure that we pick the max id with the version of the existing chunk - // in the case that there may be a pending segment with a higher version but no corresponding used segments + + // 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; overallMaxId = pendings.stream() .filter(id -> id.getShardSpec().sharePartitionSpace(partialShardSpec)) - .filter(id -> versionOfExistingChunk == null || id.getVersion() - .equals(versionOfExistingChunk)) - .max((id1, id2) -> { - final int versionCompare = id1.getVersion().compareTo(id2.getVersion()); - if (versionCompare != 0) { - return versionCompare; - } else { - return Integer.compare( - id1.getShardSpec().getPartitionNum(), - id2.getShardSpec().getPartitionNum() - ); - } - }) + .filter(id -> versionOfExistingChunk == null + || id.getVersion().equals(versionOfExistingChunk)) + .max(Comparator.comparing(SegmentIdWithShardSpec::getVersion) + .thenComparing(id -> id.getShardSpec().getPartitionNum())) .orElse(null); - // The following code attempts to compute the new version, if this - // new version is not null at the end of next block then it will be - // used as the new version in the case for initial or appended segment + + // Determine the version of the new segment final String newSegmentVersion; if (versionOfExistingChunk != null) { - // segment version overrides, so pick that now that we know it exists newSegmentVersion = versionOfExistingChunk; - } else if (!pendings.isEmpty() && overallMaxId != null) { - // there is no visible segments in the time chunk, so pick the max id of pendings, as computed above + } else if (overallMaxId != null) { newSegmentVersion = overallMaxId.getVersion(); } else { - // no segments, no pendings, so this must be the very first segment created for this interval + // this is the first segment for this interval newSegmentVersion = null; } @@ -940,7 +1240,7 @@ private SegmentIdWithShardSpec createNewSegment( ); return null; } else { - // The number of core partitions must always be chosen from the set of used segments in the VersionedIntervalTimeline. + // The number of core partitions must always be chosen from the set of used segments in the SegmentTimeline. // When the core partitions have been dropped, using pending segments may lead to an incorrect state // where the chunk is believed to have core partitions and queries results are incorrect. @@ -1472,4 +1772,62 @@ public int removeDataSourceMetadataOlderThan(long timestamp, @NotNull Set + *

  • start
  • + *
  • end
  • + *
  • sequence_name
  • + *
  • previous_segment_id
  • + *
  • payload
  • + * + */ + static PendingSegmentsRecord fromResultSet(ResultSet resultSet, ObjectMapper mapper) + { + try { + DateTime startTime = DateTimes.of(resultSet.getString(1)); + DateTime endTime = DateTimes.of(resultSet.getString(2)); + return new PendingSegmentsRecord( + new Interval(startTime.toInstant(), endTime.toInstant()), + resultSet.getString(3), + resultSet.getString(4), + resultSet.getBytes(5) + ); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + PendingSegmentsRecord(Interval interval, String sequenceName, String previousSegmentId, byte[] payload) + { + this.interval = interval; + this.payload = payload; + this.sequenceName = sequenceName; + this.previousSegmentId = previousSegmentId; + } + + public byte[] getPayload() + { + return payload; + } + + public Interval getInterval() + { + return interval; + } + + public String getSequenceId() + { + return SegmentCreateRequest.getUniqueSequenceId(sequenceName, previousSegmentId); + } + } + } From 54fa8cc8fc4070895dd42f4f8ac0622613f2e306 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 15 Nov 2022 20:16:58 +0530 Subject: [PATCH 02/35] Retain old code in SegmentAllocateAction --- .../common/actions/SegmentAllocateAction.java | 187 +++++++++++++++++- 1 file changed, 180 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index c0f3bc9e3bd3..1ec60c5974c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -26,16 +26,30 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.LockRequestForNewSegment; +import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; /** * Allocates a pending segment for a given timestamp. The preferredSegmentGranularity is used if there are no prior @@ -175,17 +189,176 @@ public Future performAsync(Task task, TaskActionToolbox } @Override - public SegmentIdWithShardSpec perform(Task task, TaskActionToolbox toolbox) + public SegmentIdWithShardSpec perform( + final Task task, + final TaskActionToolbox toolbox + ) + { + int attempt = 0; + while (true) { + attempt++; + + if (!task.getDataSource().equals(dataSource)) { + throw new IAE("Task dataSource must match action dataSource, [%s] != [%s].", task.getDataSource(), dataSource); + } + + final IndexerMetadataStorageCoordinator msc = toolbox.getIndexerMetadataStorageCoordinator(); + + // 1) if something overlaps our timestamp, use that + // 2) otherwise try preferredSegmentGranularity & going progressively smaller + + final Interval rowInterval = queryGranularity.bucket(timestamp).withChronology(ISOChronology.getInstanceUTC()); + + final Set usedSegmentsForRow = + new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)); + + final SegmentIdWithShardSpec identifier; + if (usedSegmentsForRow.isEmpty()) { + identifier = tryAllocateFirstSegment(toolbox, task, rowInterval); + } else { + identifier = tryAllocateSubsequentSegment(toolbox, task, rowInterval, usedSegmentsForRow.iterator().next()); + } + if (identifier != null) { + return identifier; + } + + // Could not allocate a pending segment. There's a chance that this is because someone else inserted a segment + // overlapping with this row between when we called "msc.retrieveUsedSegmentsForInterval" and now. Check it again, + // and if it's different, repeat. + + Set newUsedSegmentsForRow = + new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)); + if (!newUsedSegmentsForRow.equals(usedSegmentsForRow)) { + if (attempt < MAX_ATTEMPTS) { + final long shortRandomSleep = 50 + (long) (ThreadLocalRandom.current().nextDouble() * 450); + log.debug( + "Used segment set changed for rowInterval[%s]. Retrying segment allocation in %,dms (attempt = %,d).", + rowInterval, + shortRandomSleep, + attempt + ); + try { + Thread.sleep(shortRandomSleep); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } else { + log.error( + "Used segment set changed for rowInterval[%s]. Not trying again (attempt = %,d).", + rowInterval, + attempt + ); + return null; + } + } else { + return null; + } + } + } + + private SegmentIdWithShardSpec tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval) + { + // No existing segments for this row, but there might still be nearby ones that conflict with our preferred + // segment granularity. Try that first, and then progressively smaller ones if it fails. + final List tryIntervals = Granularity.granularitiesFinerThan(preferredSegmentGranularity) + .stream() + .map(granularity -> granularity.bucket(timestamp)) + .collect(Collectors.toList()); + for (Interval tryInterval : tryIntervals) { + if (tryInterval.contains(rowInterval)) { + final SegmentIdWithShardSpec identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); + if (identifier != null) { + return identifier; + } + } + } + return null; + } + + private SegmentIdWithShardSpec tryAllocateSubsequentSegment( + TaskActionToolbox toolbox, + Task task, + Interval rowInterval, + DataSegment usedSegment + ) { - if (!task.getDataSource().equals(dataSource)) { - throw new IAE("Task dataSource must match action dataSource, [%s] != [%s].", task.getDataSource(), dataSource); + // Existing segment(s) exist for this row; use the interval of the first one. + if (!usedSegment.getInterval().contains(rowInterval)) { + log.error( + "The interval of existing segment[%s] doesn't contain rowInterval[%s]", + usedSegment.getId(), + rowInterval + ); + return null; + } else { + // If segment allocation failed here, it is highly likely an unrecoverable error. We log here for easier + // debugging. + return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true); } + } + + private SegmentIdWithShardSpec tryAllocate( + TaskActionToolbox toolbox, + Task task, + Interval tryInterval, + Interval rowInterval, + boolean logOnFail + ) + { + // This action is always used by appending tasks, so if it is a time_chunk lock then we allow it to be + // shared with other appending tasks as well + final LockResult lockResult = toolbox.getTaskLockbox().tryLock( + task, + new LockRequestForNewSegment( + lockGranularity, + taskLockType, + task.getGroupId(), + dataSource, + tryInterval, + partialShardSpec, + task.getPriority(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ); - try { - return performAsync(task, toolbox).get(); + if (lockResult.isRevoked()) { + // We had acquired a lock but it was preempted by other locks + throw new ISE("The lock for interval[%s] is preempted and no longer valid", tryInterval); } - catch (Exception e) { - throw new RuntimeException(e); + + if (lockResult.isOk()) { + final SegmentIdWithShardSpec identifier = lockResult.getNewSegmentId(); + if (identifier != null) { + return identifier; + } else { + final String msg = StringUtils.format( + "Could not allocate pending segment for rowInterval[%s], segmentInterval[%s].", + rowInterval, + tryInterval + ); + if (logOnFail) { + log.error(msg); + } else { + log.debug(msg); + } + return null; + } + } else { + final String msg = StringUtils.format( + "Could not acquire lock for rowInterval[%s], segmentInterval[%s].", + rowInterval, + tryInterval + ); + if (logOnFail) { + log.error(msg); + } else { + log.debug(msg); + } + return null; } } From 401e04ae1adfe4e86a00f873606d75d5abf15428 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 11:00:14 +0530 Subject: [PATCH 03/35] Add tests, fix concurrency bugs --- .../actions/SegmentAllocationQueue.java | 214 +++++++++++++----- .../druid/indexing/overlord/TaskLockbox.java | 54 +++-- .../actions/SegmentAllocateActionTest.java | 39 +++- .../common/actions/TaskActionTestKit.java | 14 +- .../IndexerSQLMetadataStorageCoordinator.java | 2 + 5 files changed, 233 insertions(+), 90 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 979779b34254..04d2d083365b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -19,9 +19,11 @@ package org.apache.druid.indexing.common.actions; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -31,6 +33,8 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -55,15 +59,17 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; /** * Queue for {@link SegmentAllocateRequest}s. */ +@ManageLifecycle public class SegmentAllocationQueue implements DruidLeaderSelector.Listener { private static final Logger log = new Logger(SegmentAllocationQueue.class); - private static final long MAX_WAIT_TIME_MILLIS = 5_000; + private static final long MAX_WAIT_TIME_MILLIS = 1_000; private final TaskLockbox taskLockbox; private final ScheduledExecutorService executor; @@ -86,14 +92,27 @@ public SegmentAllocationQueue( this.taskLockbox = taskLockbox; this.metadataStorage = metadataStorage; this.leaderSelector = leaderSelector; - this.executor = ScheduledExecutors.fixed(1, "SegmentAllocationQueue-%s"); + this.executor = ScheduledExecutors.fixed(1, "SegmentAllocQueue-%s"); + } + @LifecycleStart + public void start() + { + log.info("Starting queue."); if (leaderSelector.isLeader()) { scheduleQueuePoll(MAX_WAIT_TIME_MILLIS); + log.info("Scheduled queue processing."); } leaderSelector.registerListener(this); } + @LifecycleStop + public void stop() + { + log.info("Stopping queue."); + executor.shutdownNow(); + } + private void scheduleQueuePoll(long delay) { executor.schedule(this::processBatchesDue, delay, TimeUnit.MILLISECONDS); @@ -110,24 +129,41 @@ public Future add(SegmentAllocateRequest request) } SegmentAllocateAction action = request.getAction(); + + // Requests for exclusive time chunks cannot be batched with other requests boolean isExclusiveTimeChunkLock = action.getLockGranularity() == LockGranularity.TIME_CHUNK && action.getTaskLockType() == TaskLockType.EXCLUSIVE; + final AllocateRequestKey requestKey = new AllocateRequestKey(action, isExclusiveTimeChunkLock); + + final AtomicReference> requestFuture = new AtomicReference<>(); + keyToBatch.compute(requestKey, (key, existingBatch) -> { + AllocateRequestBatch computedBatch = existingBatch; + if (computedBatch == null) { + computedBatch = new AllocateRequestBatch(key); + computedBatch.resetQueueTime(); + processingQueue.offer(computedBatch); + } - final AllocateRequestBatch batch; - final AllocateRequestKey requestKey = AllocateRequestKey.forAction(action); - if (isExclusiveTimeChunkLock) { - // Cannot batch exclusive time chunk locks - batch = new AllocateRequestBatch(requestKey); - processingQueue.add(batch); - } else { - batch = keyToBatch.computeIfAbsent(requestKey, k -> { - AllocateRequestBatch b = new AllocateRequestBatch(k); - processingQueue.add(b); - return b; - }); - } + requestFuture.set(computedBatch.add(request)); + return computedBatch; + }); - return batch.add(request); + return requestFuture.get(); + } + + private void requeueBatch(AllocateRequestBatch batch) + { + log.info("Requeueing [%d] failed requests in batch [%s].", batch.size(), batch.key); + keyToBatch.compute(batch.key, (key, existingBatch) -> { + if (existingBatch == null) { + batch.resetQueueTime(); + return batch; + } + + // Merge requests from this batch to existing one + existingBatch.merge(batch); + return existingBatch; + }); } private void processBatchesDue() @@ -141,9 +177,25 @@ private void processBatchesDue() } // Process all batches which are due + log.info("Processing all batches which are due for execution."); AllocateRequestBatch nextBatch = processingQueue.peek(); while (nextBatch != null && nextBatch.isDue()) { - processNextBatch(); + processingQueue.poll(); + boolean processed; + try { + processed = processBatch(nextBatch); + } + catch (Throwable t) { + processed = true; + log.error(t, "Error while processing batch [%s]", nextBatch.key); + } + + if (processed) { + nextBatch.markCompleted(); + } else { + requeueBatch(nextBatch); + } + nextBatch = processingQueue.peek(); } @@ -159,23 +211,30 @@ private void processBatchesDue() scheduleQueuePoll(nextScheduleDelay); } - private void processNextBatch() + /** + * Processes the given batch. Returns true if the batch was completely processed + * and should not be requeued. + */ + private boolean processBatch(AllocateRequestBatch requestBatch) { - final AllocateRequestBatch requestBatch = processingQueue.poll(); - if (requestBatch == null || requestBatch.isEmpty()) { - return; - } - final AllocateRequestKey requestKey = requestBatch.key; keyToBatch.remove(requestKey); + if (requestBatch.isEmpty()) { + return true; + } - log.info("Processing [%d] requests for batch [%s].", requestBatch.size(), requestKey); + log.info( + "Processing [%d] requests for batch [%s], queue time [%s].", + requestBatch.size(), + requestKey, + requestBatch.getQueueTime() + ); final long startTimeMillis = System.currentTimeMillis(); - emitBatchMetric("task/action/batch/size", requestBatch.size(), requestKey); + final int batchSize = requestBatch.size(); + emitBatchMetric("task/action/batch/size", batchSize, requestKey); emitBatchMetric("task/action/batch/queueTime", (startTimeMillis - requestBatch.getQueueTime()), requestKey); - final int batchSize = requestBatch.size(); final Set usedSegments = retrieveUsedSegments(requestKey); final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); @@ -184,26 +243,19 @@ private void processNextBatch() if (requestBatch.isEmpty()) { log.info("All requests in batch [%s] have been processed.", requestKey); - } else { - // Requeue the batch only if used segments have changed - final Set updatedUsedSegments = retrieveUsedSegments(requestKey); - if (updatedUsedSegments.equals(usedSegments)) { - log.error( - "Used segments have not changed. Not requeueing [%d] failed requests in batch [%s].", - requestBatch.size(), - requestKey - ); - } else { - log.info( - "Used segment set changed from [%d] segments to [%d].", - usedSegments.size(), - updatedUsedSegments.size() - ); - log.info("Requeueing [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); + return true; + } - requestBatch.resetQueueTime(); - processingQueue.offer(requestBatch); - } + // Requeue the batch only if used segments have changed + log.info("There are [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); + final Set updatedUsedSegments = retrieveUsedSegments(requestKey); + + if (updatedUsedSegments.equals(usedSegments)) { + log.error("Used segments have not changed. Not requeueing failed requests."); + return true; + } else { + log.info("Used segments have changed. Requeuing failed requests"); + return false; } } @@ -229,7 +281,14 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set requests = new ArrayList<>(requestBatch.requestToFuture.keySet()); + final List requests = requestBatch.getPendingRequests(); + log.info( + "Trying allocation for [%d] requests, interval [%s] in batch [%s]", + requests.size(), + tryInterval, + requestKey + ); + final List results = taskLockbox.allocateSegments( requests, requestKey.dataSource, @@ -327,22 +386,50 @@ public void stopBeingLeader() */ private class AllocateRequestBatch { + private long queueTimeMillis; private final AllocateRequestKey key; + + /** + * This must be accessed through methods synchronized on this batch. + * It is to avoid races between a new request being added just when the batch + * is being processed. + */ + @GuardedBy("this") private final Map> requestToFuture = new HashMap<>(); - private long queueTimeMillis; AllocateRequestBatch(AllocateRequestKey key) { + log.info("Creating a new batch with key: %s", key); this.key = key; } - Future add(SegmentAllocateRequest request) + synchronized Future add(SegmentAllocateRequest request) { + log.info("Adding request to batch [%s]: %s", key, request.getAction()); return requestToFuture.computeIfAbsent(request, req -> new CompletableFuture<>()); } - void handleResult(SegmentAllocateResult result, SegmentAllocateRequest request) + synchronized void merge(AllocateRequestBatch batch) + { + requestToFuture.putAll(batch.requestToFuture); + batch.requestToFuture.clear(); + } + + synchronized List getPendingRequests() + { + return new ArrayList<>(requestToFuture.keySet()); + } + + synchronized void markCompleted() + { + if (!requestToFuture.isEmpty()) { + log.info("Marking [%d] requests in batch [%s] as failed.", size(), key); + requestToFuture.values().forEach(future -> future.complete(null)); + } + } + + synchronized void handleResult(SegmentAllocateResult result, SegmentAllocateRequest request) { request.incrementAttempts(); emitTaskMetric("task/action/attempt/count", request.getAttempts(), request); @@ -370,12 +457,12 @@ void handleResult(SegmentAllocateResult result, SegmentAllocateRequest request) } } - boolean isEmpty() + synchronized boolean isEmpty() { return requestToFuture.isEmpty(); } - int size() + synchronized int size() { return requestToFuture.size(); } @@ -401,6 +488,8 @@ boolean isDue() */ private static class AllocateRequestKey { + private final boolean unique; + private final String dataSource; private final Interval rowInterval; private final Granularity queryGranularity; @@ -413,13 +502,13 @@ private static class AllocateRequestKey private final boolean useNonRootGenPartitionSpace; private final int hash; - static AllocateRequestKey forAction(SegmentAllocateAction action) - { - return new AllocateRequestKey(action); - } - - AllocateRequestKey(SegmentAllocateAction action) + /** + * Creates a new key for the given action. The batch for a unique key will + * always contain a single request. + */ + AllocateRequestKey(SegmentAllocateAction action, boolean unique) { + this.unique = unique; this.dataSource = action.getDataSource(); this.queryGranularity = action.getQueryGranularity(); this.preferredSegmentGranularity = action.getPreferredSegmentGranularity(); @@ -432,7 +521,7 @@ static AllocateRequestKey forAction(SegmentAllocateAction action) this.rowInterval = queryGranularity.bucket(action.getTimestamp()) .withChronology(ISOChronology.getInstanceUTC()); - this.hash = Objects.hash( + this.hash = unique ? super.hashCode() : Objects.hash( skipSegmentLineageCheck, useNonRootGenPartitionSpace, dataSource, @@ -447,6 +536,9 @@ static AllocateRequestKey forAction(SegmentAllocateAction action) @Override public boolean equals(Object o) { + if (unique) { + return this == o; + } if (this == o) { return true; } @@ -473,9 +565,9 @@ public int hashCode() @Override public String toString() { - return "AllocateRequestKey{" + - "dataSource='" + dataSource + '\'' + - ", rowInterval=" + rowInterval + + return "{" + + "ds='" + dataSource + '\'' + + ", row=" + rowInterval + '}'; } } 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 362cd768f92e..cb3de6f220c4 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 @@ -472,6 +472,7 @@ public List allocateSegments( LockGranularity lockGranularity ) { + log.info("Allocating [%d] segments for datasource [%s], interval [%s]", requests.size(), dataSource, interval); final boolean isTimeChunkLock = lockGranularity == LockGranularity.TIME_CHUNK; final AllocationHolderList holderList = new AllocationHolderList(requests, interval); @@ -517,11 +518,10 @@ private void verifyTaskIsActive(SegmentAllocationHolder holder) private void acquireTaskLock(SegmentAllocationHolder holder, boolean isTimeChunkLock) { final LockRequest lockRequest; - final LockRequestForNewSegment lockRequestForNewSegment = holder.createLockRequest(); if (isTimeChunkLock) { - lockRequest = new TimeChunkLockRequest(lockRequestForNewSegment); + lockRequest = new TimeChunkLockRequest(holder.lockRequest); } else { - lockRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, holder.allocatedSegment); + lockRequest = new SpecificSegmentLockRequest(holder.lockRequest, holder.allocatedSegment); } // Create or find the task lock for the created lock request @@ -532,10 +532,7 @@ private void acquireTaskLock(SegmentAllocationHolder holder, boolean isTimeChunk } else if (acquiredLock.isRevoked()) { holder.markFailed("Lock was revoked."); } else { - // Update the holder - holder.lockRequest = lockRequest; - holder.taskLockPosse = posseToUse; - holder.acquiredLock = acquiredLock; + holder.setAcquiredLock(posseToUse, lockRequest.getInterval()); } } @@ -557,7 +554,7 @@ private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTi if (!success) { final Integer partitionId = isTimeChunkLock ? null : ((SegmentLock) acquiredLock).getPartitionId(); - unlock(task, holder.lockRequest.getInterval(), partitionId); + unlock(task, holder.lockRequestInterval, partitionId); holder.markFailed("Could not update task lock in metadata store."); } } else { @@ -698,6 +695,11 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) } } + /** + * Makes a call to the {@link #metadataStorageCoordinator} to allocate segments + * for the given requests. Updates the holder with the allocated segment if + * the allocation succeeds, otherwise marks it as failed. + */ private void allocateSegmentIds( String dataSource, Interval interval, @@ -709,9 +711,10 @@ private void allocateSegmentIds( return; } - final List createRequests = holders.stream() - .map(SegmentAllocationHolder::getSegmentRequest) - .collect(Collectors.toList()); + final List createRequests = + holders.stream() + .map(SegmentAllocationHolder::getSegmentRequest) + .collect(Collectors.toList()); Map allocatedSegments = metadataStorageCoordinator.allocatePendingSegments( @@ -1422,32 +1425,30 @@ private static class SegmentAllocationHolder final AllocationHolderList list; final Task task; - final Interval interval; + final Interval allocateInterval; final SegmentAllocateAction action; + final LockRequestForNewSegment lockRequest; SegmentCreateRequest segmentRequest; - SegmentIdWithShardSpec allocatedSegment; - SegmentAllocateResult result; TaskLock acquiredLock; TaskLockPosse taskLockPosse; - LockRequest lockRequest; + Interval lockRequestInterval; + SegmentIdWithShardSpec allocatedSegment; + SegmentAllocateResult result; - SegmentAllocationHolder(SegmentAllocateRequest request, Interval interval, AllocationHolderList list) + SegmentAllocationHolder(SegmentAllocateRequest request, Interval allocateInterval, AllocationHolderList list) { this.list = list; - this.interval = interval; + this.allocateInterval = allocateInterval; this.task = request.getTask(); this.action = request.getAction(); - } - LockRequestForNewSegment createLockRequest() - { - return new LockRequestForNewSegment( + this.lockRequest = new LockRequestForNewSegment( action.getLockGranularity(), action.getTaskLockType(), task.getGroupId(), action.getDataSource(), - interval, + allocateInterval, action.getPartialShardSpec(), task.getPriority(), action.getSequenceName(), @@ -1463,7 +1464,7 @@ SegmentCreateRequest getSegmentRequest() segmentRequest = new SegmentCreateRequest( action.getSequenceName(), action.getPreviousSegmentId(), - acquiredLock.getVersion(), + acquiredLock == null ? lockRequest.getVersion() : acquiredLock.getVersion(), action.getPartialShardSpec() ); } @@ -1487,5 +1488,12 @@ void setAllocatedSegment(SegmentIdWithShardSpec segmentId) { this.allocatedSegment = segmentId; } + + void setAcquiredLock(TaskLockPosse lockPosse, Interval lockRequestInterval) + { + this.taskLockPosse = lockPosse; + this.acquiredLock = lockPosse == null ? null : lockPosse.getTaskLock(); + this.lockRequestInterval = lockRequestInterval; + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index a7e85a027e61..f4776cd37292 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -51,6 +51,7 @@ import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -79,20 +80,26 @@ public class SegmentAllocateActionTest private static final DateTime PARTY_TIME = DateTimes.of("1999"); private static final DateTime THE_DISTANT_FUTURE = DateTimes.of("3000"); + private final boolean useBatch; private final LockGranularity lockGranularity; - @Parameterized.Parameters(name = "{0}") + private SegmentAllocationQueue allocationQueue; + + @Parameterized.Parameters(name = "granularity = {0}, useBatch = {1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.SEGMENT}, - new Object[]{LockGranularity.TIME_CHUNK} + new Object[]{LockGranularity.SEGMENT, true}, + new Object[]{LockGranularity.SEGMENT, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.TIME_CHUNK, false} ); } - public SegmentAllocateActionTest(LockGranularity lockGranularity) + public SegmentAllocateActionTest(LockGranularity lockGranularity, boolean useBatch) { this.lockGranularity = lockGranularity; + this.useBatch = useBatch; } @Before @@ -101,6 +108,18 @@ public void setUp() ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); EmittingLogger.registerEmitter(emitter); EasyMock.replay(emitter); + allocationQueue = taskActionTestKit.getTaskActionToolbox().getSegmentAllocationQueue(); + if (allocationQueue != null) { + allocationQueue.start(); + } + } + + @After + public void tearDown() + { + if (allocationQueue != null) { + allocationQueue.stop(); + } } @Test @@ -990,7 +1009,17 @@ private SegmentIdWithShardSpec allocate( lockGranularity, null ); - return action.perform(task, taskActionTestKit.getTaskActionToolbox()); + + try { + if (useBatch) { + return action.performAsync(task, taskActionTestKit.getTaskActionToolbox()).get(); + } else { + return action.perform(task, taskActionTestKit.getTaskActionToolbox()); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } } private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 2d6b22732a69..3785bd22687c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; +import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; @@ -28,6 +29,7 @@ import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -35,6 +37,7 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.joda.time.Period; @@ -99,11 +102,20 @@ public int getSqlMetadataMaxRetry() Suppliers.ofInstance(metadataStorageTablesConfig), testDerbyConnector ); + final ServiceEmitter noopEmitter = new NoopServiceEmitter(); + final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector(); + leaderSelector.becomeLeader(); taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, metadataStorageCoordinator, - new NoopServiceEmitter(), + new SegmentAllocationQueue( + taskLockbox, + metadataStorageCoordinator, + leaderSelector, + noopEmitter + ), + noopEmitter, EasyMock.createMock(SupervisorManager.class), objectMapper ); 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 3b19bbf61be8..4b75d79d7b88 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -993,6 +993,8 @@ private Map createNewSegments( Map createdSegments = new HashMap<>(); for (SegmentCreateRequest request : requests) { + // TODO: create new segment should maybe also take the segments created so far as argument + // so that we assign the right partition id to all of them SegmentIdWithShardSpec createdSegment = createNewSegment( request, dataSource, From 8882596b5ea87141ec91aa508baa4d122539430d Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 17:11:51 +0530 Subject: [PATCH 04/35] Fix bugs, update test --- .../actions/SegmentAllocateActionTest.java | 47 +++---------------- .../common/actions/TaskActionTestKit.java | 1 - .../IndexerSQLMetadataStorageCoordinator.java | 14 ++++-- 3 files changed, 17 insertions(+), 45 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index f4776cd37292..4758f91b6d25 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -54,6 +53,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -227,6 +227,7 @@ public void testManySegmentsSameInterval() } @Test + @Ignore("fix and enable this!") public void testResumeSequence() { final Task task = NoopTask.create(); @@ -307,29 +308,11 @@ public void testResumeSequence() if (lockGranularity == LockGranularity.TIME_CHUNK) { final TaskLock partyLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) + .filter(input -> input.getInterval().contains(PARTY_TIME)) ); final TaskLock futureLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) ); assertSameIdentifier( @@ -465,29 +448,11 @@ public void testMultipleSequences() if (lockGranularity == LockGranularity.TIME_CHUNK) { final TaskLock partyLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) + .filter(input -> input.getInterval().contains(PARTY_TIME)) ); final TaskLock futureLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) ); assertSameIdentifier( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 3785bd22687c..bbd2d9ea7d57 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; -import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; 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 4b75d79d7b88..2c2e0b91f87d 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -663,6 +663,7 @@ private Map allocatePendingSegment if (existingSegmentId == null) { requestsForNewSegments.add(request); } else { + log.info("Found existing segment [%d] for request.", existingSegmentId); allocatedSegmentIds.put(request, existingSegmentId); } } @@ -774,7 +775,7 @@ private Map checkAndGetExistingSegmentIds( final Query> query = handle .createQuery( StringUtils.format( - "SELECT start, %2$send%2$s, sequence_name, payload " + "SELECT start, %2$send%2$s, sequence_name, sequence_prev_id, payload " + "FROM %s WHERE " + "dataSource = :dataSource AND " + "start = :start AND " @@ -883,7 +884,8 @@ private void insertPendingSegmentsIntoMetastore( for (Map.Entry entry : createdSegments.entrySet()) { final SegmentCreateRequest request = entry.getKey(); final SegmentIdWithShardSpec segmentId = entry.getValue(); - insertBatch.bind("id", segmentId.toString()) + insertBatch.add() + .bind("id", segmentId.toString()) .bind("dataSource", dataSource) .bind("created_date", DateTimes.nowUtc().toString()) .bind("start", interval.getStart().toString()) @@ -893,6 +895,7 @@ private void insertPendingSegmentsIntoMetastore( .bind("sequence_name_prev_id_sha1", sequenceNamePrevIdSha1.apply(request)) .bind("payload", jsonMapper.writeValueAsBytes(segmentId)); } + insertBatch.execute(); } private void insertPendingSegmentIntoMetastore( @@ -934,6 +937,10 @@ private Map createNewSegments( List requests ) throws IOException { + if (requests.isEmpty()) { + return Collections.emptyMap(); + } + // Get the time chunk and associated data segments for the given interval, if any final List> existingChunks = getTimelineForIntervalsWithHandle(handle, dataSource, Collections.singletonList(interval)) @@ -1004,6 +1011,7 @@ private Map createNewSegments( pendingSegments ); if (createdSegment != null) { + log.info("Created new segment [%s]", createdSegment); createdSegments.put(request, createdSegment); } } @@ -1787,7 +1795,7 @@ private static class PendingSegmentsRecord *
  • start
  • *
  • end
  • *
  • sequence_name
  • - *
  • previous_segment_id
  • + *
  • sequence_prev_id
  • *
  • payload
  • * */ From da36e53c8fa94ae6ad613d312460be6fca1a6b32 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 20:02:22 +0530 Subject: [PATCH 05/35] Fix tests and bugs --- .../actions/SegmentAllocationQueue.java | 17 +- .../actions/SegmentAllocateActionTest.java | 2 - .../IndexerSQLMetadataStorageCoordinator.java | 170 +++++++++++------- 3 files changed, 123 insertions(+), 66 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 04d2d083365b..1cd197b4d892 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -282,6 +282,10 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set requests = requestBatch.getPendingRequests(); + if (requests.isEmpty()) { + break; + } + log.info( "Trying allocation for [%d] requests, interval [%s] in batch [%s]", requests.size(), @@ -500,7 +504,9 @@ private static class AllocateRequestKey private final TaskLockType taskLockType; private final boolean useNonRootGenPartitionSpace; + private final int hash; + private final String serialized; /** * Creates a new key for the given action. The batch for a unique key will @@ -531,6 +537,7 @@ private static class AllocateRequestKey lockGranularity, taskLockType ); + this.serialized = serialize(); } @Override @@ -564,10 +571,18 @@ public int hashCode() @Override public String toString() + { + return serialized; + } + + private String serialize() { return "{" + - "ds='" + dataSource + '\'' + + "unique=" + unique + + ", skipLineageCheck=" + skipSegmentLineageCheck + + ", ds='" + dataSource + '\'' + ", row=" + rowInterval + + ", lock=" + lockGranularity + "/" + taskLockType + '}'; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 4758f91b6d25..733798bcc0f9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -53,7 +53,6 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -227,7 +226,6 @@ public void testManySegmentsSameInterval() } @Test - @Ignore("fix and enable this!") public void testResumeSequence() { final Task task = NoopTask.create(); 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 2c2e0b91f87d..0387896148ed 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; import com.google.inject.Inject; @@ -88,7 +89,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -502,27 +502,8 @@ public Map allocatePendingSegments Preconditions.checkNotNull(allocateInterval, "interval"); final Interval interval = allocateInterval.withChronology(ISOChronology.getInstanceUTC()); - - @SuppressWarnings("UnstableApiUsage") - final Function sequenceShaFunction = - request -> - skipSegmentLineageCheck - ? BaseEncoding.base16().encode( - Hashing.sha1().newHasher() - .putBytes(StringUtils.toUtf8(request.getSequenceName())) - .putByte((byte) 0xff) - .putLong(interval.getStartMillis()) - .putLong(interval.getEndMillis()) - .hash().asBytes()) - : BaseEncoding.base16().encode( - Hashing.sha1().newHasher() - .putBytes(StringUtils.toUtf8(request.getSequenceName())) - .putByte((byte) 0xff) - .putBytes(StringUtils.toUtf8(request.getPreviousSegmentId())) - .hash().asBytes()); - return connector.retryWithHandle( - handle -> allocatePendingSegments(handle, dataSource, interval, requests, sequenceShaFunction) + handle -> allocatePendingSegments(handle, dataSource, interval, skipSegmentLineageCheck, requests) ); } @@ -648,23 +629,29 @@ private Map allocatePendingSegment final Handle handle, final String dataSource, final Interval interval, - final List requests, - final Function sequenceShaFunction + final boolean skipSegmentLineageCheck, + final List requests ) throws IOException { - final Map existingSegmentIds = - checkAndGetExistingSegmentIds(handle, dataSource, interval); + final Map existingSegmentIds; + if (skipSegmentLineageCheck) { + existingSegmentIds = getExistingSegmentIdsSkipLineageCheck(handle, dataSource, interval, requests); + } else { + existingSegmentIds = getExistingSegmentIdsWithLineageCheck(handle, dataSource, interval, requests); + } // For every request see if a segment id already exists final Map allocatedSegmentIds = new HashMap<>(); final List requestsForNewSegments = new ArrayList<>(); for (SegmentCreateRequest request : requests) { - SegmentIdWithShardSpec existingSegmentId = existingSegmentIds.get(request.getUniqueSequenceId()); - if (existingSegmentId == null) { + CheckExistingSegmentIdResult existingSegmentId = existingSegmentIds.get(request); + if (existingSegmentId == null || !existingSegmentId.found) { requestsForNewSegments.add(request); + } else if (existingSegmentId.segmentIdentifier != null) { + log.info("Found valid existing segment [%s] for request.", existingSegmentId.segmentIdentifier); + allocatedSegmentIds.put(request, existingSegmentId.segmentIdentifier); } else { - log.info("Found existing segment [%d] for request.", existingSegmentId); - allocatedSegmentIds.put(request, existingSegmentId); + log.info("Found clashing existing segment [%s] for request.", existingSegmentId); } } @@ -684,13 +671,35 @@ private Map allocatePendingSegment createdSegments, dataSource, interval, - sequenceShaFunction + skipSegmentLineageCheck ); allocatedSegmentIds.putAll(createdSegments); return allocatedSegmentIds; } + @SuppressWarnings("UnstableApiUsage") + private String getSequenceNameAndPrevIdSha( + SegmentCreateRequest request, + Interval interval, + boolean skipSegmentLineageCheck + ) + { + final Hasher hasher = Hashing.sha1().newHasher() + .putBytes(StringUtils.toUtf8(request.getSequenceName())) + .putByte((byte) 0xff); + if (skipSegmentLineageCheck) { + hasher + .putLong(interval.getStartMillis()) + .putLong(interval.getEndMillis()); + } else { + hasher + .putBytes(StringUtils.toUtf8(request.getPreviousSegmentId())); + } + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + @Nullable private SegmentIdWithShardSpec allocatePendingSegment( final Handle handle, @@ -764,18 +773,19 @@ private SegmentIdWithShardSpec allocatePendingSegment( } /** - * Returns a map from sequenceId (sequenceName + previousSegmentId) to segment id. + * Returns a map from sequenceName to segment id. */ - private Map checkAndGetExistingSegmentIds( + private Map getExistingSegmentIdsSkipLineageCheck( Handle handle, String dataSource, - Interval interval + Interval interval, + List requests ) throws IOException { final Query> query = handle .createQuery( StringUtils.format( - "SELECT start, %2$send%2$s, sequence_name, sequence_prev_id, payload " + "SELECT sequence_name, payload " + "FROM %s WHERE " + "dataSource = :dataSource AND " + "start = :start AND " @@ -789,19 +799,66 @@ private Map checkAndGetExistingSegmentIds( .bind("end", interval.getEnd().toString()); final ResultIterator dbSegments = query - .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r, jsonMapper)) + .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r)) .iterator(); - // key of map should be sequenceName + previousSegmentId in a non-null thingy - final Map sequenceIdToSegmentId = new HashMap<>(); + // Map from sequenceName to segment id + final Map sequenceToSegmentId = new HashMap<>(); while (dbSegments.hasNext()) { final PendingSegmentsRecord record = dbSegments.next(); final SegmentIdWithShardSpec segmentId = jsonMapper.readValue(record.getPayload(), SegmentIdWithShardSpec.class); - sequenceIdToSegmentId.put(record.getSequenceId(), segmentId); + sequenceToSegmentId.put(record.getSequenceName(), segmentId); + } + + final Map requestToResult = new HashMap<>(); + for (SegmentCreateRequest request : requests) { + SegmentIdWithShardSpec segmentId = sequenceToSegmentId.get(request.getSequenceName()); + requestToResult.put(request, new CheckExistingSegmentIdResult(segmentId != null, segmentId)); + } + + return requestToResult; + } + + /** + * Returns a map from sequenceName to segment id. + */ + private Map getExistingSegmentIdsWithLineageCheck( + Handle handle, + String dataSource, + Interval interval, + List requests + ) throws IOException + { + // This cannot be batched because there doesn't seem to be a clean option: + // 1. WHERE must have sequence_name and sequence_prev_id but not start or end. + // (sequence columns are used to find the matching segment whereas start and + // end are used to determine if the found segment is valid or not) + // 2. IN filters on sequence_name and sequence_prev_id might perform worse than individual SELECTs? + // 3. IN filter on sequence_name alone might be a feasible option worth evaluating + final String sql = String.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "sequence_prev_id = :sequence_prev_id", + dbTables.getPendingSegmentsTable() + ); + + final Map requestToResult = new HashMap<>(); + for (SegmentCreateRequest request : requests) { + CheckExistingSegmentIdResult result = checkAndGetExistingSegmentId( + handle.createQuery(sql) + .bind("dataSource", dataSource) + .bind("sequence_name", request.getSequenceName()) + .bind("sequence_prev_id", request.getPreviousSegmentId()), + interval, + request.getSequenceName(), + request.getPreviousSegmentId() + ); + requestToResult.put(request, result); } - return sequenceIdToSegmentId; + return requestToResult; } private CheckExistingSegmentIdResult checkAndGetExistingSegmentId( @@ -868,7 +925,7 @@ private void insertPendingSegmentsIntoMetastore( Map createdSegments, String dataSource, Interval interval, - Function sequenceNamePrevIdSha1 + boolean skipSegmentLineageCheck ) throws JsonProcessingException { final PreparedBatch insertBatch = handle.prepareBatch( @@ -892,7 +949,10 @@ private void insertPendingSegmentsIntoMetastore( .bind("end", interval.getEnd().toString()) .bind("sequence_name", request.getSequenceName()) .bind("sequence_prev_id", request.getPreviousSegmentId()) - .bind("sequence_name_prev_id_sha1", sequenceNamePrevIdSha1.apply(request)) + .bind( + "sequence_name_prev_id_sha1", + getSequenceNameAndPrevIdSha(request, interval, skipSegmentLineageCheck) + ) .bind("payload", jsonMapper.writeValueAsBytes(segmentId)); } insertBatch.execute(); @@ -1785,30 +1845,21 @@ public int removeDataSourceMetadataOlderThan(long timestamp, @NotNull Set - *
  • start
  • - *
  • end
  • *
  • sequence_name
  • - *
  • sequence_prev_id
  • *
  • payload
  • * */ - static PendingSegmentsRecord fromResultSet(ResultSet resultSet, ObjectMapper mapper) + static PendingSegmentsRecord fromResultSet(ResultSet resultSet) { try { - DateTime startTime = DateTimes.of(resultSet.getString(1)); - DateTime endTime = DateTimes.of(resultSet.getString(2)); return new PendingSegmentsRecord( - new Interval(startTime.toInstant(), endTime.toInstant()), - resultSet.getString(3), - resultSet.getString(4), - resultSet.getBytes(5) + resultSet.getString(1), + resultSet.getBytes(2) ); } catch (SQLException e) { @@ -1816,12 +1867,10 @@ static PendingSegmentsRecord fromResultSet(ResultSet resultSet, ObjectMapper map } } - PendingSegmentsRecord(Interval interval, String sequenceName, String previousSegmentId, byte[] payload) + PendingSegmentsRecord(String sequenceName, byte[] payload) { - this.interval = interval; this.payload = payload; this.sequenceName = sequenceName; - this.previousSegmentId = previousSegmentId; } public byte[] getPayload() @@ -1829,14 +1878,9 @@ public byte[] getPayload() return payload; } - public Interval getInterval() - { - return interval; - } - - public String getSequenceId() + public String getSequenceName() { - return SegmentCreateRequest.getUniqueSequenceId(sequenceName, previousSegmentId); + return sequenceName; } } From d4de20b5f17d386caa55b273e0021c6dac4dbf2c Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 22:04:02 +0530 Subject: [PATCH 06/35] Fix bug --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 5 ++--- 1 file changed, 2 insertions(+), 3 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 0387896148ed..939a6e054511 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1017,7 +1017,7 @@ private Map createNewSegments( return Collections.emptyMap(); } - // Shard spec of any of the requests (as they are all compatible) used to + // Shard spec of any of the requests (as they are all compatible) can be used to // identify existing shard specs that share partition space with the requested ones. final PartialShardSpec partialShardSpec = requests.get(0).getPartialShardSpec(); @@ -1060,8 +1060,6 @@ private Map createNewSegments( Map createdSegments = new HashMap<>(); for (SegmentCreateRequest request : requests) { - // TODO: create new segment should maybe also take the segments created so far as argument - // so that we assign the right partition id to all of them SegmentIdWithShardSpec createdSegment = createNewSegment( request, dataSource, @@ -1073,6 +1071,7 @@ private Map createNewSegments( if (createdSegment != null) { log.info("Created new segment [%s]", createdSegment); createdSegments.put(request, createdSegment); + pendingSegments.add(createdSegment); } } return createdSegments; From 8f45a9cbb12a7c0c5d66f1856b88f33c0bd4b3ae Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 22:43:10 +0530 Subject: [PATCH 07/35] Put new flow behind a feature flag --- .../common/actions/LocalTaskActionClient.java | 25 ++++++++++++++++++- .../actions/SegmentAllocationQueue.java | 24 +++++++++++++----- .../common/actions/TaskActionToolbox.java | 5 ++++ .../overlord/config/TaskLockConfig.java | 16 ++++++++++++ .../common/actions/TaskActionTestKit.java | 18 +++++++++++++ 5 files changed, 81 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java index 27e0bcbaa29d..e33782403ec9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java @@ -76,11 +76,34 @@ public RetType submit(TaskAction taskAction) } final long performStartTime = System.currentTimeMillis(); - final RetType result = taskAction.perform(task, toolbox); + final RetType result; + if (isBatchAllocateAction(taskAction)) { + result = performBatchAllocateAction(taskAction); + } else { + result = taskAction.perform(task, toolbox); + } emitTimerMetric("task/action/run/time", taskAction, System.currentTimeMillis() - performStartTime); return result; } + private boolean isBatchAllocateAction(TaskAction taskAction) + { + return toolbox.canBatchSegmentAllocation() + && taskAction instanceof SegmentAllocateAction; + } + + @SuppressWarnings("unchecked") + private R performBatchAllocateAction(TaskAction taskAction) + { + try { + SegmentAllocateAction allocateAction = (SegmentAllocateAction) taskAction; + return (R) allocateAction.performAsync(task, toolbox).get(); + } + catch (Throwable t) { + throw new RuntimeException(t); + } + } + private void emitTimerMetric(final String metric, final TaskAction action, final long time) { final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 1cd197b4d892..aa9d4ae985da 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; @@ -69,7 +70,9 @@ public class SegmentAllocationQueue implements DruidLeaderSelector.Listener { private static final Logger log = new Logger(SegmentAllocationQueue.class); - private static final long MAX_WAIT_TIME_MILLIS = 1_000; + + private final long maxWaitTimeMillis; + private final boolean enabled; private final TaskLockbox taskLockbox; private final ScheduledExecutorService executor; @@ -83,6 +86,7 @@ public class SegmentAllocationQueue implements DruidLeaderSelector.Listener @Inject public SegmentAllocationQueue( TaskLockbox taskLockbox, + TaskLockConfig taskLockConfig, IndexerMetadataStorageCoordinator metadataStorage, @IndexingService DruidLeaderSelector leaderSelector, ServiceEmitter emitter @@ -92,6 +96,9 @@ public SegmentAllocationQueue( this.taskLockbox = taskLockbox; this.metadataStorage = metadataStorage; this.leaderSelector = leaderSelector; + this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationMaxWaitTime(); + this.enabled = taskLockConfig.isBatchSegmentAllocation(); + this.executor = ScheduledExecutors.fixed(1, "SegmentAllocQueue-%s"); } @@ -100,7 +107,7 @@ public void start() { log.info("Starting queue."); if (leaderSelector.isLeader()) { - scheduleQueuePoll(MAX_WAIT_TIME_MILLIS); + scheduleQueuePoll(maxWaitTimeMillis); log.info("Scheduled queue processing."); } leaderSelector.registerListener(this); @@ -113,6 +120,11 @@ public void stop() executor.shutdownNow(); } + public boolean isEnabled() + { + return enabled; + } + private void scheduleQueuePoll(long delay) { executor.schedule(this::processBatchesDue, delay, TimeUnit.MILLISECONDS); @@ -202,11 +214,11 @@ private void processBatchesDue() // Schedule the next round of processing final long nextScheduleDelay; if (processingQueue.isEmpty()) { - nextScheduleDelay = MAX_WAIT_TIME_MILLIS; + nextScheduleDelay = maxWaitTimeMillis; } else { nextBatch = processingQueue.peek(); long timeElapsed = System.currentTimeMillis() - nextBatch.getQueueTime(); - nextScheduleDelay = Math.max(0, MAX_WAIT_TIME_MILLIS - timeElapsed); + nextScheduleDelay = Math.max(0, maxWaitTimeMillis - timeElapsed); } scheduleQueuePoll(nextScheduleDelay); } @@ -376,7 +388,7 @@ public void becomeLeader() log.info("Elected leader. Starting queue processing."); // Start polling the queue - scheduleQueuePoll(MAX_WAIT_TIME_MILLIS); + scheduleQueuePoll(maxWaitTimeMillis); } @Override @@ -483,7 +495,7 @@ long getQueueTime() boolean isDue() { - return System.currentTimeMillis() - queueTimeMillis > MAX_WAIT_TIME_MILLIS; + return System.currentTimeMillis() - queueTimeMillis > maxWaitTimeMillis; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java index 169bde0f99a2..7d001ecbcb9a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java @@ -130,4 +130,9 @@ public SegmentAllocationQueue getSegmentAllocationQueue() { return segmentAllocationQueue; } + + public boolean canBatchSegmentAllocation() + { + return segmentAllocationQueue != null && segmentAllocationQueue.isEnabled(); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index 96f14759f78a..0b4c3cbcf1c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -30,8 +30,24 @@ public class TaskLockConfig @JsonProperty private boolean forceTimeChunkLock = true; + @JsonProperty + private boolean batchSegmentAllocation = false; + + @JsonProperty + private long batchAllocationMaxWaitTime = 1_000L; + public boolean isForceTimeChunkLock() { return forceTimeChunkLock; } + + public boolean isBatchSegmentAllocation() + { + return batchSegmentAllocation; + } + + public long getBatchAllocationMaxWaitTime() + { + return batchAllocationMaxWaitTime; + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index bbd2d9ea7d57..ad032082b460 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; @@ -104,12 +105,29 @@ public int getSqlMetadataMaxRetry() final ServiceEmitter noopEmitter = new NoopServiceEmitter(); final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector(); leaderSelector.becomeLeader(); + + final TaskLockConfig taskLockConfig = new TaskLockConfig() + { + @Override + public boolean isBatchSegmentAllocation() + { + return true; + } + + @Override + public long getBatchAllocationMaxWaitTime() + { + return 10L; + } + }; + taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, metadataStorageCoordinator, new SegmentAllocationQueue( taskLockbox, + taskLockConfig, metadataStorageCoordinator, leaderSelector, noopEmitter From ab2665809deb26dc1852a82e87180827c7ebfbf7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 16 Nov 2022 23:42:57 +0530 Subject: [PATCH 08/35] Fix forbidden api usage --- .../druid/indexing/common/actions/SegmentAllocationQueue.java | 4 +++- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index aa9d4ae985da..58d147f19c66 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -312,6 +312,7 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set getExistingSegme // end are used to determine if the found segment is valid or not) // 2. IN filters on sequence_name and sequence_prev_id might perform worse than individual SELECTs? // 3. IN filter on sequence_name alone might be a feasible option worth evaluating - final String sql = String.format( + final String sql = StringUtils.format( "SELECT payload FROM %s WHERE " + "dataSource = :dataSource AND " + "sequence_name = :sequence_name AND " From 5c687cec402939f36fafb6d011c1299fca71da48 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 00:04:42 +0530 Subject: [PATCH 09/35] Remove forbidden API usage --- .../java/org/apache/druid/indexing/overlord/TaskLockbox.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 cb3de6f220c4..739667309af6 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,6 +39,7 @@ import org.apache.druid.indexing.common.task.Task; 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.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -1475,7 +1476,7 @@ SegmentCreateRequest getSegmentRequest() void markFailed(String msgFormat, Object... args) { list.markCompleted(this); - result = new SegmentAllocateResult(null, String.format(msgFormat, args)); + result = new SegmentAllocateResult(null, StringUtils.format(msgFormat, args)); } void markSucceeded() From a9574f57d1bd2aa0c3f2f428a52cb7e3590789e9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 00:46:46 +0530 Subject: [PATCH 10/35] Fix compilation error due to GuardedBy --- .../druid/indexing/common/actions/SegmentAllocationQueue.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 58d147f19c66..105d6530eb8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.actions; -import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.discovery.DruidLeaderSelector; @@ -412,7 +411,6 @@ private class AllocateRequestBatch * It is to avoid races between a new request being added just when the batch * is being processed. */ - @GuardedBy("this") private final Map> requestToFuture = new HashMap<>(); From 0524e8378a476f8328a49f38fb45cfe4a394fa35 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 01:44:17 +0530 Subject: [PATCH 11/35] Revert suspicious indentation --- .../java/org/apache/druid/indexing/overlord/TaskLockbox.java | 1 + 1 file changed, 1 insertion(+) 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 739667309af6..c417931bccbe 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 @@ -1349,6 +1349,7 @@ boolean reusableFor(LockRequest request) } else { throw new ISE("Unknown request type[%s]", request); } + //noinspection SuspiciousIndentAfterControlStatement default: throw new ISE("Unknown lock type[%s]", taskLock.getType()); } From 22ca1f3dd64d19dc27e2bcbea0cf96d245455ec9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 07:49:01 +0530 Subject: [PATCH 12/35] Fix problematic use of leaderSelector.registerListener --- .../actions/SegmentAllocationQueue.java | 11 ++-------- .../druid/indexing/overlord/TaskMaster.java | 20 ++++++++++++++++++- .../indexing/overlord/http/OverlordTest.java | 4 +++- 3 files changed, 24 insertions(+), 11 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 105d6530eb8b..0e693881e59a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -66,7 +66,7 @@ * Queue for {@link SegmentAllocateRequest}s. */ @ManageLifecycle -public class SegmentAllocationQueue implements DruidLeaderSelector.Listener +public class SegmentAllocationQueue { private static final Logger log = new Logger(SegmentAllocationQueue.class); @@ -105,11 +105,6 @@ public SegmentAllocationQueue( public void start() { log.info("Starting queue."); - if (leaderSelector.isLeader()) { - scheduleQueuePoll(maxWaitTimeMillis); - log.info("Scheduled queue processing."); - } - leaderSelector.registerListener(this); } @LifecycleStop @@ -188,7 +183,7 @@ private void processBatchesDue() } // Process all batches which are due - log.info("Processing all batches which are due for execution."); + log.debug("Processing all batches which are due for execution."); AllocateRequestBatch nextBatch = processingQueue.peek(); while (nextBatch != null && nextBatch.isDue()) { processingQueue.poll(); @@ -383,7 +378,6 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) emitter.emit(metricBuilder.build(metric, value)); } - @Override public void becomeLeader() { log.info("Elected leader. Starting queue processing."); @@ -392,7 +386,6 @@ public void becomeLeader() scheduleQueuePoll(maxWaitTimeMillis); } - @Override public void stopBeingLeader() { log.info("Not leader anymore. Stopping queue processing."); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java index 7b9101cf1f22..28c623fdadb4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java @@ -26,6 +26,7 @@ import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.discovery.DruidLeaderSelector.Listener; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; @@ -91,7 +92,8 @@ public TaskMaster( final ServiceEmitter emitter, final SupervisorManager supervisorManager, final OverlordHelperManager overlordHelperManager, - @IndexingService final DruidLeaderSelector overlordLeaderSelector + @IndexingService final DruidLeaderSelector overlordLeaderSelector, + final SegmentAllocationQueue segmentAllocationQueue ) { this.supervisorManager = supervisorManager; @@ -136,6 +138,22 @@ public void becomeLeader() leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(supervisorManager); leaderLifecycle.addManagedInstance(overlordHelperManager); + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() + { + segmentAllocationQueue.becomeLeader(); + } + + @Override + public void stop() + { + segmentAllocationQueue.stopBeingLeader(); + } + } + ); leaderLifecycle.addHandler( new Lifecycle.Handler() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index 6f9a01c25b0b..3ada645ff88a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -40,6 +40,7 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.NoopTask; @@ -226,7 +227,8 @@ public MockTaskRunner get() serviceEmitter, supervisorManager, EasyMock.createNiceMock(OverlordHelperManager.class), - new TestDruidLeaderSelector() + new TestDruidLeaderSelector(), + EasyMock.createNiceMock(SegmentAllocationQueue.class) ); EmittingLogger.registerEmitter(serviceEmitter); } From bdad64d785c7d4cfac90c5137c51f011cdd675b4 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 08:13:56 +0530 Subject: [PATCH 13/35] Schedule queue processing only if batching is enabled --- .../common/actions/SegmentAllocationQueue.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 0e693881e59a..c8e229e5f203 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -132,6 +132,8 @@ public Future add(SegmentAllocateRequest request) { if (!leaderSelector.isLeader()) { throw new ISE("Cannot allocate segment if not leader."); + } else if (!isEnabled()) { + throw new ISE("Batched segment allocation is disabled."); } SegmentAllocateAction action = request.getAction(); @@ -380,10 +382,13 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) public void becomeLeader() { - log.info("Elected leader. Starting queue processing."); - - // Start polling the queue - scheduleQueuePoll(maxWaitTimeMillis); + if (isEnabled()) { + // Start polling the queue + log.info("Elected leader. Starting queue processing."); + scheduleQueuePoll(maxWaitTimeMillis); + } else { + log.info("Elected leader but batched segment allocation is disabled."); + } } public void stopBeingLeader() From b546f6c8b9b88e4dbdd6bda84d2b6b23b613854b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 08:31:28 +0530 Subject: [PATCH 14/35] Remove unused method --- .../indexing/overlord/SegmentCreateRequest.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java index d322db3b6429..89908990cd63 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java @@ -49,11 +49,6 @@ public SegmentCreateRequest( this.partialShardSpec = partialShardSpec; } - public String getUniqueSequenceId() - { - return getUniqueSequenceId(sequenceName, previousSegmentId); - } - public String getSequenceName() { return sequenceName; @@ -77,13 +72,4 @@ public PartialShardSpec getPartialShardSpec() { return partialShardSpec; } - - /** - * Returns a String representing (sequenceName + previousSegmentId) used to - * uniquely identify a segment. - */ - public static String getUniqueSequenceId(String sequenceName, String previousSegmentId) - { - return sequenceName + "####" + previousSegmentId; - } } From ea0b96f7a22f7addebd8b148f29a919c763d4c7e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 17 Nov 2022 23:25:06 +0530 Subject: [PATCH 15/35] Batch requests of same task group, same allocation interval --- .../actions/SegmentAllocateRequest.java | 10 + .../actions/SegmentAllocationQueue.java | 258 +++++++++++------- 2 files changed, 174 insertions(+), 94 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java index 8d26a27d9910..adac7523f442 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateRequest.java @@ -20,6 +20,8 @@ package org.apache.druid.indexing.common.actions; import org.apache.druid.indexing.common.task.Task; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; /** * Request received by the overlord for segment allocation. @@ -29,6 +31,7 @@ public class SegmentAllocateRequest private final Task task; private final SegmentAllocateAction action; private final int maxAttempts; + private final Interval rowInterval; private int attempts; @@ -37,6 +40,9 @@ public SegmentAllocateRequest(Task task, SegmentAllocateAction action, int maxAt this.task = task; this.action = action; this.maxAttempts = maxAttempts; + this.rowInterval = action.getQueryGranularity() + .bucket(action.getTimestamp()) + .withChronology(ISOChronology.getInstanceUTC()); } public Task getTask() @@ -64,4 +70,8 @@ public int getAttempts() return attempts; } + public Interval getRowInterval() + { + return rowInterval; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index c8e229e5f203..79423c585c95 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -24,8 +24,8 @@ import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.indexing.common.LockGranularity; -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.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskLockbox; @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; @@ -42,9 +43,9 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; -import org.joda.time.chrono.ISOChronology; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Deque; import java.util.HashMap; @@ -53,6 +54,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; @@ -60,7 +62,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; /** * Queue for {@link SegmentAllocateRequest}s. @@ -136,13 +137,7 @@ public Future add(SegmentAllocateRequest request) throw new ISE("Batched segment allocation is disabled."); } - SegmentAllocateAction action = request.getAction(); - - // Requests for exclusive time chunks cannot be batched with other requests - boolean isExclusiveTimeChunkLock = action.getLockGranularity() == LockGranularity.TIME_CHUNK - && action.getTaskLockType() == TaskLockType.EXCLUSIVE; - final AllocateRequestKey requestKey = new AllocateRequestKey(action, isExclusiveTimeChunkLock); - + final AllocateRequestKey requestKey = new AllocateRequestKey(request, false); final AtomicReference> requestFuture = new AtomicReference<>(); keyToBatch.compute(requestKey, (key, existingBatch) -> { AllocateRequestBatch computedBatch = existingBatch; @@ -272,7 +267,7 @@ private Set retrieveUsedSegments(AllocateRequestKey key) return new HashSet<>( metadataStorage.retrieveUsedSegmentsForInterval( key.dataSource, - key.rowInterval, + key.preferredAllocationInterval, Segments.ONLY_VISIBLE ) ); @@ -280,87 +275,165 @@ private Set retrieveUsedSegments(AllocateRequestKey key) private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set usedSegments) { - final AllocateRequestKey requestKey = requestBatch.key; - final List tryIntervals = getTryIntervals(requestKey, usedSegments); - if (tryIntervals.isEmpty()) { - log.error("Found no valid interval containing the row interval [%s]", requestKey.rowInterval); - return 0; - } - int successCount = 0; - for (Interval tryInterval : tryIntervals) { - final List requests = requestBatch.getPendingRequests(); - if (requests.isEmpty()) { - break; - } - log.info( - "Trying allocation for [%d] requests, interval [%s] in batch [%s]", - requests.size(), - tryInterval, - requestKey - ); + final List allRequests = requestBatch.getRequests(); + final Set pendingRequests = new HashSet<>(); - final List results = taskLockbox.allocateSegments( - requests, - requestKey.dataSource, - tryInterval, - requestKey.skipSegmentLineageCheck, - requestKey.lockGranularity - ); - emitBatchMetric("task/action/batch/retries", 1L, requestKey); + if (usedSegments.isEmpty()) { + pendingRequests.addAll(allRequests); + } else { + final Interval[] sortedUsedSegmentIntervals = getSortedIntervals(usedSegments); + final Map> usedIntervalToRequests = new HashMap<>(); + + for (SegmentAllocateRequest request : allRequests) { + // If there is an overlapping used segment interval, that interval is + // the only candidate for allocation + Interval overlappingInterval = findOverlappingInterval( + request.getRowInterval(), + sortedUsedSegmentIntervals + ); + + if (overlappingInterval == null) { + pendingRequests.add(request); + } else if (overlappingInterval.contains(request.getRowInterval())) { + // Found an enclosing interval, use this for allocation + usedIntervalToRequests.computeIfAbsent(overlappingInterval, i -> new ArrayList<>()) + .add(request); + } + } + + // Try to allocate segments for the identified used segment intervals + // Do not retry the failed requests with other intervals unless the batch is requeued + for (Map.Entry> entry : usedIntervalToRequests.entrySet()) { + List successfulRequests = allocateSegmentsForInterval( + entry.getKey(), + entry.getValue(), + requestBatch + ); + successCount += successfulRequests.size(); + } + } - successCount += updateBatchWithResults(requestBatch, requests, results); + // For requests that do not overlap with a used segment, first try to allocate + // using the preferred granularity, then smaller granularities + for (Granularity granularity : + Granularity.granularitiesFinerThan(requestBatch.key.preferredSegmentGranularity)) { + Map> requestsByInterval = + getRequestsByInterval(pendingRequests, requestBatch.key, granularity); + + for (Map.Entry> entry : requestsByInterval.entrySet()) { + List successfulRequests = allocateSegmentsForInterval( + entry.getKey(), + entry.getValue(), + requestBatch + ); + successCount += successfulRequests.size(); + pendingRequests.removeAll(successfulRequests); + } } return successCount; } - /** - * Gets the intervals for which allocation should be tried. - *

    - * If there are no used segments for this row, first try to allocate segments - * using the preferred segment granularity. If that fails due to other nearby - * segments, try progressively smaller granularities. - *

    - * If there are used segments for this row, try only the interval of those used - * segments (we assume that all of them must have the same interval). - */ - private List getTryIntervals(AllocateRequestKey key, Set usedSegments) + private Interval findOverlappingInterval(Interval searchInterval, Interval[] sortedIntervals) { - final Interval rowInterval = key.rowInterval; - if (usedSegments.isEmpty()) { - return Granularity.granularitiesFinerThan(key.preferredSegmentGranularity) - .stream() - .map(granularity -> granularity.bucket(rowInterval.getStart())) - .filter(interval -> interval.contains(rowInterval)) - .collect(Collectors.toList()); - } else { - Interval existingInterval = usedSegments.iterator().next().getInterval(); - if (existingInterval.contains(rowInterval)) { - return Collections.singletonList(existingInterval); - } else { - return Collections.emptyList(); + int index = Arrays.binarySearch( + sortedIntervals, + searchInterval, + Comparators.intervalsByStartThenEnd() + ); + + // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either + if (index < sortedIntervals.length) { + if (sortedIntervals[index].overlaps(searchInterval)) { + return sortedIntervals[index]; } } + + // If the interval at (index - 1) doesn't overlap, (index - 2) wouldn't overlap either + if (index > 0) { + if (sortedIntervals[index - 1].overlaps(searchInterval)) { + return sortedIntervals[index - 1]; + } + } + + return null; } - private int updateBatchWithResults( - AllocateRequestBatch requestBatch, + private Interval[] getSortedIntervals(Set usedSegments) + { + TreeSet sortedSet = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + usedSegments.forEach(segment -> sortedSet.add(segment.getInterval())); + return sortedSet.toArray(new Interval[0]); + } + + /** + * Tries to allocate segments for the given requests over the specified interval. + * Returns the list of requests for which segments were successfully allocated. + */ + private List allocateSegmentsForInterval( + Interval tryInterval, List requests, - List results + AllocateRequestBatch requestBatch ) { - int successCount = 0; + if (requests.isEmpty()) { + return Collections.emptyList(); + } + + final AllocateRequestKey requestKey = requestBatch.key; + log.info( + "Trying allocation for [%d] requests, interval [%s] in batch [%s]", + requests.size(), + tryInterval, + requestKey + ); + + final List results = taskLockbox.allocateSegments( + requests, + requestKey.dataSource, + tryInterval, + requestKey.skipSegmentLineageCheck, + requestKey.lockGranularity + ); + emitBatchMetric("task/action/batch/retries", 1L, requestKey); + + final List successfulRequests = new ArrayList<>(); for (int i = 0; i < requests.size(); ++i) { + SegmentAllocateRequest request = requests.get(i); SegmentAllocateResult result = results.get(i); if (result.isSuccess()) { - ++successCount; + successfulRequests.add(request); } - requestBatch.handleResult(result, requests.get(i)); + requestBatch.handleResult(result, request); } - return successCount; + + return successfulRequests; + } + + private Map> getRequestsByInterval( + Set requests, + AllocateRequestKey requestKey, + Granularity tryGranularity + ) + { + if (tryGranularity.equals(requestKey.preferredSegmentGranularity)) { + return Collections.singletonMap( + requestKey.preferredAllocationInterval, + new ArrayList<>(requests) + ); + } + + final Map> tryIntervalToRequests = new HashMap<>(); + for (SegmentAllocateRequest request : requests) { + Interval tryInterval = tryGranularity.bucket(request.getAction().getTimestamp()); + if (tryInterval.contains(request.getRowInterval())) { + tryIntervalToRequests.computeIfAbsent(tryInterval, i -> new ArrayList<>()).add(request); + } + } + return tryIntervalToRequests; } private void emitTaskMetric(String metric, long value, SegmentAllocateRequest request) @@ -376,7 +449,7 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); metricBuilder.setDimension("taskActionType", SegmentAllocateAction.TYPE); metricBuilder.setDimension(DruidMetrics.DATASOURCE, key.dataSource); - metricBuilder.setDimension(DruidMetrics.INTERVAL, key.rowInterval.toString()); + metricBuilder.setDimension(DruidMetrics.INTERVAL, key.preferredAllocationInterval.toString()); emitter.emit(metricBuilder.build(metric, value)); } @@ -430,7 +503,7 @@ synchronized void merge(AllocateRequestBatch batch) batch.requestToFuture.clear(); } - synchronized List getPendingRequests() + synchronized List getRequests() { return new ArrayList<>(requestToFuture.keySet()); } @@ -505,13 +578,12 @@ private static class AllocateRequestKey private final boolean unique; private final String dataSource; - private final Interval rowInterval; - private final Granularity queryGranularity; + private final String groupId; + private final Interval preferredAllocationInterval; private final Granularity preferredSegmentGranularity; private final boolean skipSegmentLineageCheck; private final LockGranularity lockGranularity; - private final TaskLockType taskLockType; private final boolean useNonRootGenPartitionSpace; @@ -519,33 +591,32 @@ private static class AllocateRequestKey private final String serialized; /** - * Creates a new key for the given action. The batch for a unique key will + * Creates a new key for the given request. The batch for a unique key will * always contain a single request. */ - AllocateRequestKey(SegmentAllocateAction action, boolean unique) + AllocateRequestKey(SegmentAllocateRequest request, boolean unique) { + final SegmentAllocateAction action = request.getAction(); + final Task task = request.getTask(); + this.unique = unique; this.dataSource = action.getDataSource(); - this.queryGranularity = action.getQueryGranularity(); - this.preferredSegmentGranularity = action.getPreferredSegmentGranularity(); + this.groupId = task.getGroupId(); this.skipSegmentLineageCheck = action.isSkipSegmentLineageCheck(); this.lockGranularity = action.getLockGranularity(); - this.taskLockType = action.getTaskLockType(); - this.useNonRootGenPartitionSpace = action.getPartialShardSpec() .useNonRootGenerationPartitionSpace(); - this.rowInterval = queryGranularity.bucket(action.getTimestamp()) - .withChronology(ISOChronology.getInstanceUTC()); + this.preferredSegmentGranularity = action.getPreferredSegmentGranularity(); + this.preferredAllocationInterval = action.getPreferredSegmentGranularity() + .bucket(action.getTimestamp()); this.hash = unique ? super.hashCode() : Objects.hash( skipSegmentLineageCheck, useNonRootGenPartitionSpace, dataSource, - rowInterval, - queryGranularity, - preferredSegmentGranularity, - lockGranularity, - taskLockType + groupId, + preferredAllocationInterval, + lockGranularity ); this.serialized = serialize(); } @@ -566,11 +637,9 @@ public boolean equals(Object o) return skipSegmentLineageCheck == that.skipSegmentLineageCheck && useNonRootGenPartitionSpace == that.useNonRootGenPartitionSpace && dataSource.equals(that.dataSource) - && rowInterval.equals(that.rowInterval) - && queryGranularity.equals(that.queryGranularity) - && preferredSegmentGranularity.equals(that.preferredSegmentGranularity) - && lockGranularity == that.lockGranularity - && taskLockType == that.taskLockType; + && groupId.equals(that.groupId) + && preferredAllocationInterval.equals(that.preferredAllocationInterval) + && lockGranularity == that.lockGranularity; } @Override @@ -591,8 +660,9 @@ private String serialize() "unique=" + unique + ", skipLineageCheck=" + skipSegmentLineageCheck + ", ds='" + dataSource + '\'' + - ", row=" + rowInterval + - ", lock=" + lockGranularity + "/" + taskLockType + + ", groupId='" + groupId + '\'' + + ", interval=" + preferredAllocationInterval + + ", lock=" + lockGranularity + '}'; } } From f6b8f7785444ed7959881c130e003cd56e4927c9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 00:16:13 +0530 Subject: [PATCH 16/35] Add failed metric --- .../actions/SegmentAllocationQueue.java | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 79423c585c95..28b8c4b2b4fe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -139,16 +139,25 @@ public Future add(SegmentAllocateRequest request) final AllocateRequestKey requestKey = new AllocateRequestKey(request, false); final AtomicReference> requestFuture = new AtomicReference<>(); + keyToBatch.compute(requestKey, (key, existingBatch) -> { - AllocateRequestBatch computedBatch = existingBatch; - if (computedBatch == null) { - computedBatch = new AllocateRequestBatch(key); - computedBatch.resetQueueTime(); - processingQueue.offer(computedBatch); + AllocateRequestBatch batch = existingBatch; + if (batch == null) { + batch = new AllocateRequestBatch(key); + batch.resetQueueTime(); + processingQueue.offer(batch); } - requestFuture.set(computedBatch.add(request)); - return computedBatch; + // Possible race condition: + // t1 -> new batch is added to queue or batch already exists in queue + // t2 -> executor pops batch, processes all requests in it + // t1 -> new request is added to dangling batch and is never picked up + // Solution: For existing batch, call keyToBatch.remove() on the key to + // wait on keyToBatch.compute() to finish before proceeding with processBatch(). + // For new batch, keyToBatch.remove() would not wait as key is not in map yet + // but a new batch is unlikely to be due immediately, so it won't get popped right away. + requestFuture.set(batch.add(request)); + return batch; }); return requestFuture.get(); @@ -160,6 +169,7 @@ private void requeueBatch(AllocateRequestBatch batch) keyToBatch.compute(batch.key, (key, existingBatch) -> { if (existingBatch == null) { batch.resetQueueTime(); + processingQueue.offer(batch); return batch; } @@ -513,6 +523,10 @@ synchronized void markCompleted() if (!requestToFuture.isEmpty()) { log.info("Marking [%d] requests in batch [%s] as failed.", size(), key); requestToFuture.values().forEach(future -> future.complete(null)); + requestToFuture.keySet().forEach( + request -> emitTaskMetric("task/action/failed/count", 1L, request) + ); + requestToFuture.clear(); } } From c5862e72cb510ade69165b8d69a1e0b047f2d4f9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 00:38:07 +0530 Subject: [PATCH 17/35] Fix possible concurrent modification bug --- .../druid/indexing/overlord/TaskLockbox.java | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 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 c417931bccbe..037a04c3c656 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 @@ -477,27 +477,26 @@ public List allocateSegments( final boolean isTimeChunkLock = lockGranularity == LockGranularity.TIME_CHUNK; final AllocationHolderList holderList = new AllocationHolderList(requests, interval); - holderList.pending.forEach(this::verifyTaskIsActive); + holderList.getPending().forEach(this::verifyTaskIsActive); giant.lock(); try { if (isTimeChunkLock) { - holderList.pending.forEach(holder -> acquireTaskLock(holder, true)); - allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.pending); + holderList.getPending().forEach(holder -> acquireTaskLock(holder, true)); + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending()); } else { - allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.pending); - holderList.pending.forEach(holder -> acquireTaskLock(holder, false)); + allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending()); + holderList.getPending().forEach(holder -> acquireTaskLock(holder, false)); } // TODO: for failed allocations, cleanup newly created locks from the posse map - - holderList.pending.forEach(holder -> addTaskAndPersistLocks(holder, isTimeChunkLock)); + holderList.getPending().forEach(holder -> addTaskAndPersistLocks(holder, isTimeChunkLock)); } finally { giant.unlock(); } - return holderList.all.stream().map(holder -> holder.result).collect(Collectors.toList()); + return holderList.getResults(); } /** @@ -1403,6 +1402,7 @@ private static class AllocationHolderList { final List all = new ArrayList<>(); final Set pending = new HashSet<>(); + final Set recentlyCompleted = new HashSet<>(); AllocationHolderList(List requests, Interval interval) { @@ -1415,7 +1415,20 @@ private static class AllocationHolderList void markCompleted(SegmentAllocationHolder holder) { - pending.remove(holder); + recentlyCompleted.add(holder); + } + + Set getPending() + { + pending.removeAll(recentlyCompleted); + recentlyCompleted.clear(); + return pending; + } + + + List getResults() + { + return all.stream().map(holder -> holder.result).collect(Collectors.toList()); } } From 4c66afd2f11365b234936ea59a1050364a07edce Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 01:04:19 +0530 Subject: [PATCH 18/35] Fix interval search --- .../indexing/common/actions/SegmentAllocationQueue.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 28b8c4b2b4fe..4e1eedb8dbb4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -251,6 +251,7 @@ private boolean processBatch(AllocateRequestBatch requestBatch) final Set usedSegments = retrieveUsedSegments(requestKey); final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); + emitBatchMetric("task/action/batch/retries", 1L, requestKey); emitBatchMetric("task/action/batch/runTime", (System.currentTimeMillis() - startTimeMillis), requestKey); log.info("Successfully processed [%d / %d] requests in batch [%s].", successCount, batchSize, requestKey); @@ -353,6 +354,12 @@ private Interval findOverlappingInterval(Interval searchInterval, Interval[] sor searchInterval, Comparators.intervalsByStartThenEnd() ); + if (index >= 0) { + return sortedIntervals[index]; + } + + // Key was not found, returned index is (-(insertionPoint) - 1) + index = -(index + 1); // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either if (index < sortedIntervals.length) { @@ -407,7 +414,6 @@ private List allocateSegmentsForInterval( requestKey.skipSegmentLineageCheck, requestKey.lockGranularity ); - emitBatchMetric("task/action/batch/retries", 1L, requestKey); final List successfulRequests = new ArrayList<>(); for (int i = 0; i < requests.size(); ++i) { From e07e8526ac4d27d5e9e968174e2a51ef6c1efdc1 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 08:14:00 +0530 Subject: [PATCH 19/35] Deduplicate requests while creating new segments and updating metadata store --- .../overlord/SegmentCreateRequest.java | 3 ++ .../IndexerSQLMetadataStorageCoordinator.java | 53 +++++++++++++------ 2 files changed, 41 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java index 89908990cd63..b43e46d8e7a5 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/SegmentCreateRequest.java @@ -31,6 +31,9 @@ */ public class SegmentCreateRequest { + // DO NOT IMPLEMENT equals or hashCode for this class as each request must be + // treated as unique even if it is for the same parameters + private final String version; private final String sequenceName; private final String previousSegmentId; 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 76448021759e..11d7846892d2 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -657,7 +657,7 @@ private Map allocatePendingSegment // For each of the remaining requests, create a new segment final Map createdSegments = - createNewSegments(handle, dataSource, interval, requestsForNewSegments); + createNewSegments(handle, dataSource, interval, skipSegmentLineageCheck, requestsForNewSegments); // SELECT -> INSERT can fail due to races; callers must be prepared to retry. // Avoiding ON DUPLICATE KEY since it's not portable. @@ -938,9 +938,13 @@ private void insertPendingSegmentsIntoMetastore( connector.getQuoteString() )); - for (Map.Entry entry : createdSegments.entrySet()) { - final SegmentCreateRequest request = entry.getKey(); - final SegmentIdWithShardSpec segmentId = entry.getValue(); + // Deduplicate the segment ids by inverting the map + Map segmentIdToRequest = new HashMap<>(); + createdSegments.forEach((request, segmentId) -> segmentIdToRequest.put(segmentId, request)); + + for (Map.Entry entry : segmentIdToRequest.entrySet()) { + final SegmentCreateRequest request = entry.getValue(); + final SegmentIdWithShardSpec segmentId = entry.getKey(); insertBatch.add() .bind("id", segmentId.toString()) .bind("dataSource", dataSource) @@ -994,6 +998,7 @@ private Map createNewSegments( Handle handle, String dataSource, Interval interval, + boolean skipSegmentLineageCheck, List requests ) throws IOException { @@ -1058,22 +1063,40 @@ private Map createNewSegments( pendingSegments.add(committedMaxId); } - Map createdSegments = new HashMap<>(); + final Map createdSegments = new HashMap<>(); + final Map sequenceHashToSegment = new HashMap<>(); + for (SegmentCreateRequest request : requests) { - SegmentIdWithShardSpec createdSegment = createNewSegment( - request, - dataSource, - interval, - versionOfExistingChunk, - committedMaxId, - pendingSegments - ); + // Check if the required segment has already been created in this batch + final String sequenceHash = getSequenceNameAndPrevIdSha(request, interval, skipSegmentLineageCheck); + + final SegmentIdWithShardSpec createdSegment; + if (sequenceHashToSegment.containsKey(sequenceHash)) { + createdSegment = sequenceHashToSegment.get(sequenceHash); + } else { + createdSegment = createNewSegment( + request, + dataSource, + interval, + versionOfExistingChunk, + committedMaxId, + pendingSegments + ); + + // Add to pendingSegments to consider for partitionId + if (createdSegment != null) { + pendingSegments.add(createdSegment); + sequenceHashToSegment.put(sequenceHash, createdSegment); + log.info("Created new segment [%s]", createdSegment); + } + } + if (createdSegment != null) { - log.info("Created new segment [%s]", createdSegment); createdSegments.put(request, createdSegment); - pendingSegments.add(createdSegment); } } + + log.info("Created [%d] new segments for [%d] allocate requests.", sequenceHashToSegment.size(), requests.size()); return createdSegments; } From 829628c95d4ae02ad4a7699c915ce280a195e2e6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 10:35:06 +0530 Subject: [PATCH 20/35] Mark success and failure correctly --- .../org/apache/druid/indexing/overlord/TaskLockbox.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 037a04c3c656..c62950d32581 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 @@ -551,7 +551,9 @@ private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTi // This can also be batched later boolean success = updateLockInStorage(task, acquiredLock); - if (!success) { + if (success) { + holder.markSucceeded(); + } else { final Integer partitionId = isTimeChunkLock ? null : ((SegmentLock) acquiredLock).getPartitionId(); unlock(task, holder.lockRequestInterval, partitionId); @@ -559,9 +561,8 @@ private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTi } } else { log.info("Task [%s] already present in TaskLock [%s]", task.getId(), acquiredLock.getGroupId()); + holder.markSucceeded(); } - - holder.markSucceeded(); } private boolean updateLockInStorage(Task task, TaskLock taskLock) From 3a8556811295ec67bb62b45f38cb8b743890b771 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 18 Nov 2022 11:08:54 +0530 Subject: [PATCH 21/35] Minor fixes --- .../actions/SegmentAllocationQueue.java | 36 +++++++++---------- .../druid/indexing/overlord/TaskLockbox.java | 2 +- .../actions/SegmentAllocateActionTest.java | 1 + 3 files changed, 18 insertions(+), 21 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 4e1eedb8dbb4..15e4536c826c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -141,11 +141,11 @@ public Future add(SegmentAllocateRequest request) final AtomicReference> requestFuture = new AtomicReference<>(); keyToBatch.compute(requestKey, (key, existingBatch) -> { - AllocateRequestBatch batch = existingBatch; - if (batch == null) { - batch = new AllocateRequestBatch(key); - batch.resetQueueTime(); - processingQueue.offer(batch); + AllocateRequestBatch computedBatch = existingBatch; + if (computedBatch == null) { + computedBatch = new AllocateRequestBatch(key); + computedBatch.resetQueueTime(); + processingQueue.offer(computedBatch); } // Possible race condition: @@ -156,8 +156,8 @@ public Future add(SegmentAllocateRequest request) // wait on keyToBatch.compute() to finish before proceeding with processBatch(). // For new batch, keyToBatch.remove() would not wait as key is not in map yet // but a new batch is unlikely to be due immediately, so it won't get popped right away. - requestFuture.set(batch.add(request)); - return batch; + requestFuture.set(computedBatch.add(request)); + return computedBatch; }); return requestFuture.get(); @@ -171,11 +171,11 @@ private void requeueBatch(AllocateRequestBatch batch) batch.resetQueueTime(); processingQueue.offer(batch); return batch; + } else { + // Merge requests from this batch to existing one + existingBatch.merge(batch); + return existingBatch; } - - // Merge requests from this batch to existing one - existingBatch.merge(batch); - return existingBatch; }); } @@ -191,12 +191,15 @@ private void processBatchesDue() // Process all batches which are due log.debug("Processing all batches which are due for execution."); + int numProcessedBatches = 0; + AllocateRequestBatch nextBatch = processingQueue.peek(); while (nextBatch != null && nextBatch.isDue()) { processingQueue.poll(); boolean processed; try { processed = processBatch(nextBatch); + ++numProcessedBatches; } catch (Throwable t) { processed = true; @@ -222,6 +225,7 @@ private void processBatchesDue() nextScheduleDelay = Math.max(0, maxWaitTimeMillis - timeElapsed); } scheduleQueuePoll(nextScheduleDelay); + log.debug("Processed [%d] batches, next execution in [%d ms]", numProcessedBatches, nextScheduleDelay); } /** @@ -331,7 +335,7 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set> requestsByInterval = - getRequestsByInterval(pendingRequests, requestBatch.key, granularity); + getRequestsByInterval(pendingRequests, granularity); for (Map.Entry> entry : requestsByInterval.entrySet()) { List successfulRequests = allocateSegmentsForInterval( @@ -431,17 +435,9 @@ private List allocateSegmentsForInterval( private Map> getRequestsByInterval( Set requests, - AllocateRequestKey requestKey, Granularity tryGranularity ) { - if (tryGranularity.equals(requestKey.preferredSegmentGranularity)) { - return Collections.singletonMap( - requestKey.preferredAllocationInterval, - new ArrayList<>(requests) - ); - } - final Map> tryIntervalToRequests = new HashMap<>(); for (SegmentAllocateRequest request : requests) { Interval tryInterval = tryGranularity.bucket(request.getAction().getTimestamp()); 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 c62950d32581..81ae29e6d72b 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 @@ -553,7 +553,7 @@ private void addTaskAndPersistLocks(SegmentAllocationHolder holder, boolean isTi boolean success = updateLockInStorage(task, acquiredLock); if (success) { holder.markSucceeded(); - } else { + } else { final Integer partitionId = isTimeChunkLock ? null : ((SegmentLock) acquiredLock).getPartitionId(); unlock(task, holder.lockRequestInterval, partitionId); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 733798bcc0f9..30fde9eddd0c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -110,6 +110,7 @@ public void setUp() allocationQueue = taskActionTestKit.getTaskActionToolbox().getSegmentAllocationQueue(); if (allocationQueue != null) { allocationQueue.start(); + allocationQueue.becomeLeader(); } } From b6da3c48a525adef2d4f08841511084855308074 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 21 Nov 2022 15:12:13 +0530 Subject: [PATCH 22/35] Limit queue size --- .../common/actions/LocalTaskActionClient.java | 3 +- .../actions/SegmentAllocationQueue.java | 128 +++++++++++------- 2 files changed, 80 insertions(+), 51 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java index e33782403ec9..e7ce9b204894 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java @@ -30,6 +30,7 @@ import javax.annotation.Nullable; import java.util.Map; +import java.util.concurrent.TimeUnit; public class LocalTaskActionClient implements TaskActionClient { @@ -97,7 +98,7 @@ private R performBatchAllocateAction(TaskAction taskAction) { try { SegmentAllocateAction allocateAction = (SegmentAllocateAction) taskAction; - return (R) allocateAction.performAsync(task, toolbox).get(); + return (R) allocateAction.performAsync(task, toolbox).get(5, TimeUnit.MINUTES); } catch (Throwable t) { throw new RuntimeException(t); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 15e4536c826c..1302b32baa13 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -47,7 +47,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -55,10 +54,11 @@ import java.util.Objects; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.BlockingDeque; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -71,6 +71,8 @@ public class SegmentAllocationQueue { private static final Logger log = new Logger(SegmentAllocationQueue.class); + private static final int MAX_QUEUE_SIZE = 5000; + private final long maxWaitTimeMillis; private final boolean enabled; @@ -81,7 +83,7 @@ public class SegmentAllocationQueue private final ServiceEmitter emitter; private final ConcurrentHashMap keyToBatch = new ConcurrentHashMap<>(); - private final Deque processingQueue = new ConcurrentLinkedDeque<>(); + private final BlockingDeque processingQueue = new LinkedBlockingDeque<>(MAX_QUEUE_SIZE); @Inject public SegmentAllocationQueue( @@ -105,13 +107,13 @@ public SegmentAllocationQueue( @LifecycleStart public void start() { - log.info("Starting queue."); + log.info("Initializing segment allocation queue."); } @LifecycleStop public void stop() { - log.info("Stopping queue."); + log.info("Tearing down segment allocation queue."); executor.shutdownNow(); } @@ -140,40 +142,60 @@ public Future add(SegmentAllocateRequest request) final AllocateRequestKey requestKey = new AllocateRequestKey(request, false); final AtomicReference> requestFuture = new AtomicReference<>(); + // Possible race condition: + // t1 -> new batch is added to queue or batch already exists in queue + // t2 -> executor pops batch, processes all requests in it + // t1 -> new request is added to dangling batch and is never picked up + // Solution: For existing batch, call keyToBatch.remove() on the key to + // wait on keyToBatch.compute() to finish before proceeding with processBatch(). + // For new batch, keyToBatch.remove() would not wait as key is not in map yet + // but a new batch is unlikely to be due immediately, so it won't get popped right away. keyToBatch.compute(requestKey, (key, existingBatch) -> { - AllocateRequestBatch computedBatch = existingBatch; - if (computedBatch == null) { - computedBatch = new AllocateRequestBatch(key); - computedBatch.resetQueueTime(); - processingQueue.offer(computedBatch); + if (existingBatch == null) { + AllocateRequestBatch newBatch = new AllocateRequestBatch(key); + requestFuture.set(newBatch.add(request)); + return addBatchToQueue(newBatch) ? newBatch : null; + } else { + requestFuture.set(existingBatch.add(request)); + return existingBatch; } - - // Possible race condition: - // t1 -> new batch is added to queue or batch already exists in queue - // t2 -> executor pops batch, processes all requests in it - // t1 -> new request is added to dangling batch and is never picked up - // Solution: For existing batch, call keyToBatch.remove() on the key to - // wait on keyToBatch.compute() to finish before proceeding with processBatch(). - // For new batch, keyToBatch.remove() would not wait as key is not in map yet - // but a new batch is unlikely to be due immediately, so it won't get popped right away. - requestFuture.set(computedBatch.add(request)); - return computedBatch; }); return requestFuture.get(); } + /** + * Tries to add the given batch to the processing queue. If the queue is full, + * marks the batch as completed failing all the pending requests. + */ + private boolean addBatchToQueue(AllocateRequestBatch batch) + { + batch.resetQueueTime(); + if (processingQueue.offer(batch)) { + log.debug("Added a new batch [%s] to queue.", batch.key); + return true; + } else { + log.warn("Cannot add batch [%s] as queue is full. Failing [%d] requests.", batch.key, batch.size()); + batch.markCompleted(); + return false; + } + } + + /** + * Tries to add the given batch to the processing queue. If a batch already + * exists for this key, transfers all the requests from this batch to the + * existing one. + */ private void requeueBatch(AllocateRequestBatch batch) { log.info("Requeueing [%d] failed requests in batch [%s].", batch.size(), batch.key); keyToBatch.compute(batch.key, (key, existingBatch) -> { if (existingBatch == null) { batch.resetQueueTime(); - processingQueue.offer(batch); - return batch; + return addBatchToQueue(batch) ? batch : null; } else { // Merge requests from this batch to existing one - existingBatch.merge(batch); + existingBatch.transferRequestsFrom(batch); return existingBatch; } }); @@ -183,19 +205,25 @@ private void processBatchesDue() { // If not leader, clear the queue and do not schedule any more rounds of processing if (!leaderSelector.isLeader()) { - log.info("Not leader anymore. Clearing [%d] batches from queue.", processingQueue.size()); - processingQueue.clear(); + log.info("Not leader anymore. Failing [%d] batches in queue.", processingQueue.size()); + + AllocateRequestBatch nextBatch = processingQueue.pollFirst(); + while (nextBatch != null) { + nextBatch.markCompleted(); + nextBatch = processingQueue.pollFirst(); + } + keyToBatch.clear(); return; } // Process all batches which are due - log.debug("Processing all batches which are due for execution."); + log.debug("Processing all batches which are due for execution. Overall queue size [%d].", processingQueue.size()); int numProcessedBatches = 0; - AllocateRequestBatch nextBatch = processingQueue.peek(); + AllocateRequestBatch nextBatch = processingQueue.peekFirst(); while (nextBatch != null && nextBatch.isDue()) { - processingQueue.poll(); + processingQueue.pollFirst(); boolean processed; try { processed = processBatch(nextBatch); @@ -362,7 +390,7 @@ private Interval findOverlappingInterval(Interval searchInterval, Interval[] sor return sortedIntervals[index]; } - // Key was not found, returned index is (-(insertionPoint) - 1) + // Key was not found, index returned from binarySearch is (-(insertionPoint) - 1) index = -(index + 1); // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either @@ -472,13 +500,20 @@ public void becomeLeader() log.info("Elected leader. Starting queue processing."); scheduleQueuePoll(maxWaitTimeMillis); } else { - log.info("Elected leader but batched segment allocation is disabled."); + log.info( + "Elected leader but batched segment allocation is disabled. " + + "Segment allocation queue will not be used." + ); } } public void stopBeingLeader() { - log.info("Not leader anymore. Stopping queue processing."); + if (isEnabled()) { + log.info("Not leader anymore. Stopping queue processing."); + } else { + log.info("Not leader anymore. Segment allocation queue is already disabled."); + } } /** @@ -490,6 +525,9 @@ private class AllocateRequestBatch private final AllocateRequestKey key; /** + * Map from allocate requests (represents a single SegmentAllocateAction) + * to the future of allocated segment id. + *

    * This must be accessed through methods synchronized on this batch. * It is to avoid races between a new request being added just when the batch * is being processed. @@ -499,7 +537,6 @@ private class AllocateRequestBatch AllocateRequestBatch(AllocateRequestKey key) { - log.info("Creating a new batch with key: %s", key); this.key = key; } @@ -509,7 +546,7 @@ synchronized Future add(SegmentAllocateRequest request) return requestToFuture.computeIfAbsent(request, req -> new CompletableFuture<>()); } - synchronized void merge(AllocateRequestBatch batch) + synchronized void transferRequestsFrom(AllocateRequestBatch batch) { requestToFuture.putAll(batch.requestToFuture); batch.requestToFuture.clear(); @@ -539,22 +576,20 @@ synchronized void handleResult(SegmentAllocateResult result, SegmentAllocateRequ if (result.isSuccess()) { emitTaskMetric("task/action/success/count", 1L, request); requestToFuture.remove(request).complete(result.getSegmentId()); - return; - } - - log.info("Failed to allocate segment for action [%s]: %s", request.getAction(), result.getErrorMessage()); - if (request.canRetry()) { + } else if (request.canRetry()) { log.debug( - "Can requeue action [%s] after [%d] failed attempts.", + "Action [%s] failed in attempt [%d]. Can still retry. Latest error: %s", request.getAction(), - request.getAttempts() + request.getAttempts(), + result.getErrorMessage() ); } else { emitTaskMetric("task/action/failed/count", 1L, request); log.error( - "Removing allocation action [%s] from batch after [%d] failed attempts.", + "Failing allocate action [%s] after [%d] attempts. Latest error: %s", request.getAction(), - request.getAttempts() + request.getAttempts(), + result.getErrorMessage() ); requestToFuture.remove(request).complete(null); } @@ -604,7 +639,6 @@ private static class AllocateRequestKey private final boolean useNonRootGenPartitionSpace; private final int hash; - private final String serialized; /** * Creates a new key for the given request. The batch for a unique key will @@ -634,7 +668,6 @@ private static class AllocateRequestKey preferredAllocationInterval, lockGranularity ); - this.serialized = serialize(); } @Override @@ -666,11 +699,6 @@ public int hashCode() @Override public String toString() - { - return serialized; - } - - private String serialize() { return "{" + "unique=" + unique + From 1d7a688bb4b9d0edffe37c1e235a677277c8ae2c Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 23 Nov 2022 18:06:24 +0530 Subject: [PATCH 23/35] Add IntervalsTest and test for IndexerSQLMSC --- .../druid/java/util/common/Intervals.java | 48 ++++ .../druid/java/util/common/IntervalsTest.java | 72 +++++ .../actions/SegmentAllocationQueue.java | 38 +-- ...exerSQLMetadataStorageCoordinatorTest.java | 269 +++++++++--------- 4 files changed, 262 insertions(+), 165 deletions(-) create mode 100644 core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java index b7a1f37cf1c3..338e520d63b3 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -20,10 +20,14 @@ package org.apache.druid.java.util.common; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.guava.Comparators; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; +import javax.annotation.Nullable; +import java.util.Arrays; + public final class Intervals { public static final Interval ETERNITY = utc(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT); @@ -68,6 +72,50 @@ public static boolean isEternity(final Interval interval) return ETERNITY.equals(interval); } + /** + * Finds an interval from the given set of sortedIntervals which overlaps with + * the searchInterval. If multiple candidate intervals overlap with the + * searchInterval, the "smallest" interval based on the + * {@link Comparators#intervalsByStartThenEnd()} is returned. + * + * @param searchInterval Interval which should overlap with the result + * @param sortedIntervals Candidate overlapping intervals, sorted in ascending + * order, using {@link Comparators#intervalsByStartThenEnd()}. + * @return The first overlapping interval, if one exists, otherwise null. + */ + @Nullable + public static Interval findOverlappingInterval(Interval searchInterval, Interval[] sortedIntervals) + { + Arrays.sort(sortedIntervals, Comparators.intervalsByStartThenEnd()); + int index = Arrays.binarySearch( + sortedIntervals, + searchInterval, + Comparators.intervalsByStartThenEnd() + ); + if (index >= 0) { + return sortedIntervals[index]; + } + + // Key was not found, index returned from binarySearch is (-(insertionPoint) - 1) + index = -(index + 1); + + // If the interval at (index - 1) doesn't overlap, (index - 2) wouldn't overlap either + if (index > 0) { + if (sortedIntervals[index - 1].overlaps(searchInterval)) { + return sortedIntervals[index - 1]; + } + } + + // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either + if (index < sortedIntervals.length) { + if (sortedIntervals[index].overlaps(searchInterval)) { + return sortedIntervals[index]; + } + } + + return null; + } + private Intervals() { } diff --git a/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java new file mode 100644 index 000000000000..c66f4a9cb35e --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java @@ -0,0 +1,72 @@ +/* + * 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.java.util.common; + +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +public class IntervalsTest +{ + + @Test + public void testFindOverlappingInterval() + { + final Interval[] sortedIntervals = new Interval[]{ + Intervals.of("2019/2020"), + Intervals.of("2021/2022"), + Intervals.of("2021-01-01/2021-02-01"), + Intervals.of("2021/2023") + }; + + // Search interval outside the bounds of the sorted intervals + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2018/2019"), sortedIntervals) + ); + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2023/2024"), sortedIntervals) + ); + + // Search interval within bounds, overlap exists + // Fully overlapping interval + Assert.assertEquals( + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021/2022"), sortedIntervals) + ); + + // Partially overlapping interval + Assert.assertEquals( + Intervals.of("2021/2023"), + Intervals.findOverlappingInterval(Intervals.of("2022-01-01/2022-01-02"), sortedIntervals) + ); + + // Overlap with multiple intervals, "smallest" one is returned + Assert.assertEquals( + Intervals.of("2021-01-01/2021-02-01"), + Intervals.findOverlappingInterval(Intervals.of("2021-01-01/2021-01-02"), sortedIntervals) + ); + + // Search interval within bounds, no overlap + Assert.assertNull( + Intervals.findOverlappingInterval(Intervals.of("2020-01-02/2020-03-03"), sortedIntervals) + ); + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 1302b32baa13..4edcdb4b72c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; @@ -45,7 +46,6 @@ import org.joda.time.Interval; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -71,7 +71,7 @@ public class SegmentAllocationQueue { private static final Logger log = new Logger(SegmentAllocationQueue.class); - private static final int MAX_QUEUE_SIZE = 5000; + private static final int MAX_QUEUE_SIZE = 2000; private final long maxWaitTimeMillis; private final boolean enabled; @@ -191,7 +191,6 @@ private void requeueBatch(AllocateRequestBatch batch) log.info("Requeueing [%d] failed requests in batch [%s].", batch.size(), batch.key); keyToBatch.compute(batch.key, (key, existingBatch) -> { if (existingBatch == null) { - batch.resetQueueTime(); return addBatchToQueue(batch) ? batch : null; } else { // Merge requests from this batch to existing one @@ -332,7 +331,7 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set= 0) { - return sortedIntervals[index]; - } - - // Key was not found, index returned from binarySearch is (-(insertionPoint) - 1) - index = -(index + 1); - - // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either - if (index < sortedIntervals.length) { - if (sortedIntervals[index].overlaps(searchInterval)) { - return sortedIntervals[index]; - } - } - - // If the interval at (index - 1) doesn't overlap, (index - 2) wouldn't overlap either - if (index > 0) { - if (sortedIntervals[index - 1].overlaps(searchInterval)) { - return sortedIntervals[index - 1]; - } - } - - return null; - } - private Interval[] getSortedIntervals(Set usedSegments) { TreeSet sortedSet = new TreeSet<>(Comparators.intervalsByStartThenEnd()); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index d7000f8f4d20..f7af9611e51a 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; +import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; @@ -59,7 +60,6 @@ import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; -import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.util.StringMapper; import java.io.IOException; @@ -371,40 +371,12 @@ private void markAllSegmentsUnused(Set segments) Assert.assertEquals( 1, (int) derbyConnector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Integer withHandle(Handle handle) - { - String request = StringUtils.format( - "UPDATE %s SET used = false WHERE id = :id", - derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() - ); - return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); - } - } - ) - ); - } - } - - private void markAllSegmentsUsed(Set segments) - { - for (final DataSegment segment : segments) { - Assert.assertEquals( - 1, - (int) derbyConnector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Integer withHandle(Handle handle) - { - String request = StringUtils.format( - "UPDATE %s SET used = true WHERE id = :id", - derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() - ); - return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); - } + handle -> { + String request = StringUtils.format( + "UPDATE %s SET used = false WHERE id = :id", + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable() + ); + return handle.createStatement(request).bind("id", segment.getId().toString()).execute(); } ) ); @@ -415,32 +387,19 @@ private List retrievePendingSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getPendingSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } + private List retrieveUsedSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } @@ -448,16 +407,9 @@ private List retrieveUnusedSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") - .map(StringMapper.FIRST) - .list(); - } - } + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") + .map(StringMapper.FIRST) + .list() ); } @@ -466,39 +418,34 @@ private Boolean insertUsedSegments(Set dataSegments) { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); return derbyConnector.retryWithHandle( - new HandleCallback() - { - @Override - public Boolean withHandle(Handle handle) throws Exception - { - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - table, - derbyConnector.getQuoteString() - ) - ); - for (DataSegment segment : dataSegments) { - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", mapper.writeValueAsBytes(segment)); - } - - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); - if (!succeeded) { - throw new ISE("Failed to publish segments to DB"); - } - return true; + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + table, + derbyConnector.getQuoteString() + ) + ); + for (DataSegment segment : dataSegments) { + preparedBatch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", mapper.writeValueAsBytes(segment)); } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; } ); } @@ -561,12 +508,12 @@ public void testAnnounceHistoricalSegments() throws IOException ); } - List segmentIds = segments.stream().map(segment -> segment.getId().toString()).collect(Collectors.toList()); - segmentIds.sort(Comparator.naturalOrder()); - Assert.assertEquals( - segmentIds, - retrieveUsedSegmentIds() - ); + List segmentIds = segments.stream() + .map(segment -> segment.getId().toString()) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds()); // Should not update dataSource metadata. Assert.assertEquals(0, metadataUpdateCounter.get()); @@ -823,10 +770,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc retrieveUsedSegmentIds() ); - DataSegment nonExistingSegment = defaultSegment4; - - Set dropSegments = ImmutableSet.of(existingSegment1, nonExistingSegment); - + Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( SEGMENTS, dropSegments, @@ -1576,17 +1520,19 @@ public void testAllocatePendingSegment() } /** - * This test simulates an issue detected on the field consisting of the following sequence of events: - * - A kafka stream segment was created on a given interval - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later a compaction was issued for the three segments above - * - Later, after the above was published, another segment on same interval was created by the stream - * - Later, the compacted segment got dropped due to a drop rule + * This test verifies the behaviour in the following sequence of events: + * - create segment1 for an interval and publish + * - create segment2 for same interval and publish + * - create segment3 for same interval and publish + * - compact all segments above and publish new segments + * - create segment4 for the same interval + * - drop the compacted segment + * - create segment5 for the same interval + * - verify that the id for segment5 is correct * - Later, after the above was dropped, another segment on same interval was created by the stream but this - * time there was an integrity violation in the pending segments table because the - * {@link IndexerSQLMetadataStorageCoordinator#createNewSegment(Handle, String, Interval, PartialShardSpec, String)} - * method returned an segment id that already existed in the pending segments table + * time there was an integrity violation in the pending segments table because the + * {@link IndexerSQLMetadataStorageCoordinator#createNewSegment(Handle, String, Interval, PartialShardSpec, String)} + * method returned an segment id that already existed in the pending segments table */ @Test public void testAllocatePendingSegmentAfterDroppingExistingSegment() @@ -1690,13 +1636,13 @@ public void testAllocatePendingSegmentAfterDroppingExistingSegment() /** * Slightly different that the above test but that involves reverted compaction - 1) used segments of version = A, id = 0, 1, 2 - 2) overwrote segments of version = B, id = 0 <= compaction - 3) marked segments unused for version = A, id = 0, 1, 2 <= overshadowing - 4) pending segment of version = B, id = 1 <= appending new data, aborted - 5) reverted compaction, mark segments used for version = A, id = 0, 1, 2, and mark compacted segments unused - 6) used segments of version = A, id = 0, 1, 2 - 7) pending segment of version = B, id = 1 + * 1) used segments of version = A, id = 0, 1, 2 + * 2) overwrote segments of version = B, id = 0 <= compaction + * 3) marked segments unused for version = A, id = 0, 1, 2 <= overshadowing + * 4) pending segment of version = B, id = 1 <= appending new data, aborted + * 5) reverted compaction, mark segments used for version = A, id = 0, 1, 2, and mark compacted segments unused + * 6) used segments of version = A, id = 0, 1, 2 + * 7) pending segment of version = B, id = 1 */ @Test public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() @@ -1842,13 +1788,13 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // used segment: version = A, id = 0,1,2 // unused segment: version = B, id = 0 List pendings = retrievePendingSegmentIds(); - Assert.assertTrue(pendings.size() == 4); + Assert.assertEquals(4, pendings.size()); List used = retrieveUsedSegmentIds(); - Assert.assertTrue(used.size() == 3); + Assert.assertEquals(3, used.size()); List unused = retrieveUnusedSegmentIds(); - Assert.assertTrue(unused.size() == 1); + Assert.assertEquals(1, unused.size()); // Simulate one more append load final SegmentIdWithShardSpec identifier4 = coordinator.allocatePendingSegment( @@ -1886,7 +1832,71 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_3", ids.get(3)); } - + + @Test + public void testAllocatePendingSegments() + { + final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + final String sequenceName = "seq"; + + final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec); + final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request) + ).get(request); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString()); + + final SegmentCreateRequest request1 = + new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request1) + ).get(request1); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString()); + + final SegmentCreateRequest request2 = + new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request2) + ).get(request2); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString()); + + final SegmentCreateRequest request3 = + new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec); + final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request3) + ).get(request3); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId3.toString()); + Assert.assertEquals(segmentId2, segmentId3); + + final SegmentCreateRequest request4 = + new SegmentCreateRequest("seq1", null, "v1", partialShardSpec); + final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments( + dataSource, + interval, + false, + Collections.singletonList(request4) + ).get(request4); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_3", segmentId4.toString()); + } + @Test public void testNoPendingSegmentsAndOneUsedSegment() { @@ -1923,9 +1933,8 @@ public void testNoPendingSegmentsAndOneUsedSegment() true ); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_1", identifier.toString()); - - } + } @Test From 55676cc2d35d1fbd8e5bef5f1ba32c232861b34f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 10:57:05 +0530 Subject: [PATCH 24/35] Add tests, fix success bug --- .../actions/SegmentAllocationQueue.java | 76 +++-- .../actions/SegmentAllocateActionBuilder.java | 138 ++++++++ .../actions/SegmentAllocateActionTest.java | 15 +- .../actions/SegmentAllocationQueueTest.java | 314 ++++++++++++++++++ .../common/actions/TaskActionTestKit.java | 8 +- .../overlord/SegmentCreateRequestTest.java | 46 +++ 6 files changed, 549 insertions(+), 48 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java create mode 100644 server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 4edcdb4b72c9..281e786a5f87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; @@ -61,6 +62,7 @@ import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; /** @@ -79,7 +81,7 @@ public class SegmentAllocationQueue private final TaskLockbox taskLockbox; private final ScheduledExecutorService executor; private final IndexerMetadataStorageCoordinator metadataStorage; - private final DruidLeaderSelector leaderSelector; + private final AtomicBoolean isLeader = new AtomicBoolean(false); private final ServiceEmitter emitter; private final ConcurrentHashMap keyToBatch = new ConcurrentHashMap<>(); @@ -90,18 +92,17 @@ public SegmentAllocationQueue( TaskLockbox taskLockbox, TaskLockConfig taskLockConfig, IndexerMetadataStorageCoordinator metadataStorage, - @IndexingService DruidLeaderSelector leaderSelector, - ServiceEmitter emitter + ServiceEmitter emitter, + ScheduledExecutorFactory executorFactory ) { this.emitter = emitter; this.taskLockbox = taskLockbox; this.metadataStorage = metadataStorage; - this.leaderSelector = leaderSelector; this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationMaxWaitTime(); this.enabled = taskLockConfig.isBatchSegmentAllocation(); - this.executor = ScheduledExecutors.fixed(1, "SegmentAllocQueue-%s"); + this.executor = executorFactory.create(1, "SegmentAllocQueue-%s"); } @LifecycleStart @@ -127,13 +128,21 @@ private void scheduleQueuePoll(long delay) executor.schedule(this::processBatchesDue, delay, TimeUnit.MILLISECONDS); } + /** + * Gets the number of batches currently in the queue. + */ + public int size() + { + return processingQueue.size(); + } + /** * Queues a SegmentAllocateRequest. The returned future may complete successfully * with a non-null value or with a non-null value. */ public Future add(SegmentAllocateRequest request) { - if (!leaderSelector.isLeader()) { + if (!isLeader.get()) { throw new ISE("Cannot allocate segment if not leader."); } else if (!isEnabled()) { throw new ISE("Batched segment allocation is disabled."); @@ -203,7 +212,7 @@ private void requeueBatch(AllocateRequestBatch batch) private void processBatchesDue() { // If not leader, clear the queue and do not schedule any more rounds of processing - if (!leaderSelector.isLeader()) { + if (!isLeader.get()) { log.info("Not leader anymore. Failing [%d] batches in queue.", processingQueue.size()); AllocateRequestBatch nextBatch = processingQueue.pollFirst(); @@ -226,13 +235,13 @@ private void processBatchesDue() boolean processed; try { processed = processBatch(nextBatch); - ++numProcessedBatches; } catch (Throwable t) { processed = true; log.error(t, "Error while processing batch [%s]", nextBatch.key); } + ++numProcessedBatches; if (processed) { nextBatch.markCompleted(); } else { @@ -252,7 +261,7 @@ private void processBatchesDue() nextScheduleDelay = Math.max(0, maxWaitTimeMillis - timeElapsed); } scheduleQueuePoll(nextScheduleDelay); - log.debug("Processed [%d] batches, next execution in [%d ms]", numProcessedBatches, nextScheduleDelay); + log.info("Processed [%d] batches, next execution in [%d ms]", numProcessedBatches, nextScheduleDelay); } /** @@ -267,7 +276,7 @@ private boolean processBatch(AllocateRequestBatch requestBatch) return true; } - log.info( + log.debug( "Processing [%d] requests for batch [%s], queue time [%s].", requestBatch.size(), requestKey, @@ -287,19 +296,19 @@ private boolean processBatch(AllocateRequestBatch requestBatch) log.info("Successfully processed [%d / %d] requests in batch [%s].", successCount, batchSize, requestKey); if (requestBatch.isEmpty()) { - log.info("All requests in batch [%s] have been processed.", requestKey); + log.debug("All requests in batch [%s] have been processed.", requestKey); return true; } // Requeue the batch only if used segments have changed - log.info("There are [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); + log.debug("There are [%d] failed requests in batch [%s].", requestBatch.size(), requestKey); final Set updatedUsedSegments = retrieveUsedSegments(requestKey); if (updatedUsedSegments.equals(usedSegments)) { log.error("Used segments have not changed. Not requeueing failed requests."); return true; } else { - log.info("Used segments have changed. Requeuing failed requests"); + log.debug("Used segments have changed. Requeuing failed requests"); return false; } } @@ -319,7 +328,7 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set allRequests = requestBatch.getRequests(); + final Set allRequests = requestBatch.getRequests(); final Set pendingRequests = new HashSet<>(); if (usedSegments.isEmpty()) { @@ -348,12 +357,11 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set> entry : usedIntervalToRequests.entrySet()) { - List successfulRequests = allocateSegmentsForInterval( + successCount += allocateSegmentsForInterval( entry.getKey(), entry.getValue(), requestBatch ); - successCount += successfulRequests.size(); } } @@ -365,13 +373,12 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set> entry : requestsByInterval.entrySet()) { - List successfulRequests = allocateSegmentsForInterval( + successCount += allocateSegmentsForInterval( entry.getKey(), entry.getValue(), requestBatch ); - successCount += successfulRequests.size(); - pendingRequests.removeAll(successfulRequests); + pendingRequests.retainAll(requestBatch.getRequests()); } } @@ -387,20 +394,20 @@ private Interval[] getSortedIntervals(Set usedSegments) /** * Tries to allocate segments for the given requests over the specified interval. - * Returns the list of requests for which segments were successfully allocated. + * Returns the number of requests for which segments were successfully allocated. */ - private List allocateSegmentsForInterval( + private int allocateSegmentsForInterval( Interval tryInterval, List requests, AllocateRequestBatch requestBatch ) { if (requests.isEmpty()) { - return Collections.emptyList(); + return 0; } final AllocateRequestKey requestKey = requestBatch.key; - log.info( + log.debug( "Trying allocation for [%d] requests, interval [%s] in batch [%s]", requests.size(), tryInterval, @@ -415,12 +422,12 @@ private List allocateSegmentsForInterval( requestKey.lockGranularity ); - final List successfulRequests = new ArrayList<>(); + int successfulRequests = 0; for (int i = 0; i < requests.size(); ++i) { SegmentAllocateRequest request = requests.get(i); SegmentAllocateResult result = results.get(i); if (result.isSuccess()) { - successfulRequests.add(request); + ++successfulRequests; } requestBatch.handleResult(result, request); @@ -463,7 +470,10 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) public void becomeLeader() { - if (isEnabled()) { + if (!isLeader.compareAndSet(false, true)) { + // Already leader + log.info("Already the leader. Queue processing is already scheduled."); + } else if (isEnabled()) { // Start polling the queue log.info("Elected leader. Starting queue processing."); scheduleQueuePoll(maxWaitTimeMillis); @@ -477,7 +487,9 @@ public void becomeLeader() public void stopBeingLeader() { - if (isEnabled()) { + if (!isLeader.compareAndSet(true, false)) { + log.info("Already surrendered leadership. Queue processing is stopped."); + } else if (isEnabled()) { log.info("Not leader anymore. Stopping queue processing."); } else { log.info("Not leader anymore. Segment allocation queue is already disabled."); @@ -510,7 +522,7 @@ private class AllocateRequestBatch synchronized Future add(SegmentAllocateRequest request) { - log.info("Adding request to batch [%s]: %s", key, request.getAction()); + log.debug("Adding request to batch [%s]: %s", key, request.getAction()); return requestToFuture.computeIfAbsent(request, req -> new CompletableFuture<>()); } @@ -520,9 +532,9 @@ synchronized void transferRequestsFrom(AllocateRequestBatch batch) batch.requestToFuture.clear(); } - synchronized List getRequests() + synchronized Set getRequests() { - return new ArrayList<>(requestToFuture.keySet()); + return new HashSet<>(requestToFuture.keySet()); } synchronized void markCompleted() @@ -545,7 +557,7 @@ synchronized void handleResult(SegmentAllocateResult result, SegmentAllocateRequ emitTaskMetric("task/action/success/count", 1L, request); requestToFuture.remove(request).complete(result.getSegmentId()); } else if (request.canRetry()) { - log.debug( + log.info( "Action [%s] failed in attempt [%d]. Can still retry. Latest error: %s", request.getAction(), request.getAttempts(), @@ -585,7 +597,7 @@ long getQueueTime() boolean isDue() { - return System.currentTimeMillis() - queueTimeMillis > maxWaitTimeMillis; + return System.currentTimeMillis() - queueTimeMillis >= maxWaitTimeMillis; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java new file mode 100644 index 000000000000..2e9cd70bbdbc --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionBuilder.java @@ -0,0 +1,138 @@ +/* + * 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 org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.partition.PartialShardSpec; +import org.joda.time.DateTime; + +public class SegmentAllocateActionBuilder +{ + private String dataSource; + private DateTime timestamp; + private Granularity queryGranularity; + private Granularity preferredSegmentGranularity; + private String sequenceName; + private String previousSegmentId; + private boolean skipSegmentLineageCheck; + private PartialShardSpec partialShardSpec; + private LockGranularity lockGranularity; + private TaskLockType taskLockType; + private Task task; + + public SegmentAllocateActionBuilder forDatasource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public SegmentAllocateActionBuilder forTimestamp(DateTime timestamp) + { + this.timestamp = timestamp; + return this; + } + + public SegmentAllocateActionBuilder forTimestamp(String instant) + { + this.timestamp = DateTimes.of(instant); + return this; + } + + public SegmentAllocateActionBuilder withQueryGranularity(Granularity queryGranularity) + { + this.queryGranularity = queryGranularity; + return this; + } + + public SegmentAllocateActionBuilder withSegmentGranularity(Granularity segmentGranularity) + { + this.preferredSegmentGranularity = segmentGranularity; + return this; + } + + public SegmentAllocateActionBuilder withSequenceName(String sequenceName) + { + this.sequenceName = sequenceName; + return this; + } + + public SegmentAllocateActionBuilder withPreviousSegmentId(String previousSegmentId) + { + this.previousSegmentId = previousSegmentId; + return this; + } + + public SegmentAllocateActionBuilder withSkipLineageCheck(boolean skipLineageCheck) + { + this.skipSegmentLineageCheck = skipLineageCheck; + return this; + } + + public SegmentAllocateActionBuilder withPartialShardSpec(PartialShardSpec partialShardSpec) + { + this.partialShardSpec = partialShardSpec; + return this; + } + + public SegmentAllocateActionBuilder withLockGranularity(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + return this; + } + + public SegmentAllocateActionBuilder withTaskLockType(TaskLockType taskLockType) + { + this.taskLockType = taskLockType; + return this; + } + + public SegmentAllocateActionBuilder forTask(Task task) + { + this.dataSource = task.getDataSource(); + this.sequenceName = task.getId(); + this.task = task; + return this; + } + + public SegmentAllocateRequest build() + { + return new SegmentAllocateRequest(task, buildAction(), 1); + } + + public SegmentAllocateAction buildAction() + { + return new SegmentAllocateAction( + dataSource, + timestamp, + queryGranularity, + preferredSegmentGranularity, + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + partialShardSpec, + lockGranularity, + taskLockType + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 30fde9eddd0c..b498834fbaba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -64,6 +64,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @RunWith(Parameterized.class) @@ -976,7 +977,8 @@ private SegmentIdWithShardSpec allocate( try { if (useBatch) { - return action.performAsync(task, taskActionTestKit.getTaskActionToolbox()).get(); + return action.performAsync(task, taskActionTestKit.getTaskActionToolbox()) + .get(5, TimeUnit.SECONDS); } else { return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } @@ -989,15 +991,6 @@ private SegmentIdWithShardSpec allocate( private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) { Assert.assertEquals(expected, actual); - Assert.assertEquals(expected.getShardSpec().getPartitionNum(), actual.getShardSpec().getPartitionNum()); - Assert.assertEquals(expected.getShardSpec().getClass(), actual.getShardSpec().getClass()); - - if (expected.getShardSpec().getClass() == NumberedShardSpec.class - && actual.getShardSpec().getClass() == NumberedShardSpec.class) { - Assert.assertEquals(expected.getShardSpec().getNumCorePartitions(), actual.getShardSpec().getNumCorePartitions()); - } else if (expected.getShardSpec().getClass() == LinearShardSpec.class - && actual.getShardSpec().getClass() == LinearShardSpec.class) { - // do nothing - } + Assert.assertEquals(expected.getShardSpec(), actual.getShardSpec()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java new file mode 100644 index 000000000000..721ce83e7774 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -0,0 +1,314 @@ +/* + * 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 org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +public class SegmentAllocationQueueTest +{ + @Rule + public TaskActionTestKit taskActionTestKit = new TaskActionTestKit(); + + private static final String DS_WIKI = "wiki"; + private static final String DS_KOALA = "koala"; + + private SegmentAllocationQueue allocationQueue; + + private StubServiceEmitter emitter; + private BlockingExecutorService executor; + + @Before + public void setUp() + { + executor = new BlockingExecutorService("alloc-test-exec"); + emitter = new StubServiceEmitter("overlord", "alloc-test"); + + final TaskLockConfig lockConfig = new TaskLockConfig() + { + @Override + public boolean isBatchSegmentAllocation() + { + return true; + } + + @Override + public long getBatchAllocationMaxWaitTime() + { + return 0; + } + }; + + allocationQueue = new SegmentAllocationQueue( + taskActionTestKit.getTaskLockbox(), + lockConfig, + taskActionTestKit.getMetadataStorageCoordinator(), + emitter, + (corePoolSize, nameFormat) + -> new WrappingScheduledExecutorService(nameFormat, executor, false) + ); + allocationQueue.becomeLeader(); + } + + @After + public void tearDown() + { + if (allocationQueue != null) { + allocationQueue.stop(); + } + if (executor != null) { + executor.shutdownNow(); + } + emitter.flush(); + } + + @Test + public void testCriteriaForBatching() + { + // Requests with different timestamps and sequence names can batched together + // as long as the other params are the same + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-01T01:00:00") + .withSegmentGranularity(Granularities.DAY) + .withQueryGranularity(Granularities.SECOND) + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withSequenceName("seq_1") + .build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .forTimestamp("2022-01-01T02:00:00") + .withSegmentGranularity(Granularities.DAY) + .withQueryGranularity(Granularities.SECOND) + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withSequenceName("seq_2") + .build(), + true + ); + + // Requests for different lock types can be batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.EXCLUSIVE).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.SHARED).build(), + true + ); + } + + @Test(timeout = 60_000) + public void testCriteriaForBatchingNegative() + { + // Requests for different datasources are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), + allocateRequest().forTask(createTask(DS_KOALA, "group_1")).build(), + false + ); + + // Requests for different groupIds are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_2")).build(), + false + ); + + // Requests for different lock granularities are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withLockGranularity(LockGranularity.TIME_CHUNK).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withLockGranularity(LockGranularity.SEGMENT).build(), + false + ); + + // Requests for different query granularities are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "1")) + .withQueryGranularity(Granularities.SECOND).build(), + allocateRequest().forTask(createTask(DS_WIKI, "2")) + .withQueryGranularity(Granularities.MINUTE).build(), + false + ); + + // Requests for different segment granularities are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "1")) + .withSegmentGranularity(Granularities.HOUR).build(), + allocateRequest().forTask(createTask(DS_WIKI, "2")) + .withSegmentGranularity(Granularities.THIRTY_MINUTE).build(), + false + ); + + // Requests for different allocate intervals are not batched together + verifyRequestsCanBeBatched( + allocateRequest().forTask(createTask(DS_WIKI, "1")) + .forTimestamp("2022-01-01") + .withSegmentGranularity(Granularities.DAY) + .withSequenceName("seq_1").build(), + allocateRequest().forTask(createTask(DS_WIKI, "2")) + .forTimestamp("2022-01-02") + .withSegmentGranularity(Granularities.DAY).build(), + false + ); + } + + private void verifyRequestsCanBeBatched( + SegmentAllocateRequest a, + SegmentAllocateRequest b, + boolean canBatch + ) + { + // Disable queue and cleanup state + allocationQueue.stopBeingLeader(); + executor.finishAllPendingTasks(); + emitter.flush(); + + // Enable queue again + allocationQueue.becomeLeader(); + + Assert.assertEquals(0, allocationQueue.size()); + allocationQueue.add(a); + allocationQueue.add(b); + + final int expectedCount = canBatch ? 1 : 2; + Assert.assertEquals(expectedCount, allocationQueue.size()); + + executor.finishNextPendingTask(); + emitter.verifyEmitted("task/action/batch/size", expectedCount); + } + + @Test + public void testBatchNotDue() + { + + } + + @Test + public void testAllocationWithBatch() + { + // try out different combos of these: + // lock granularity, lock type, skip lineage, + } + + // now we need to test out a bunch of scenarios + // possibly with all combos above? + + @Test + public void testMultipleRequestsForSameSegment() + { + + } + + @Test + public void testMultipleInOrderRequests() + { + // multiple requests which are requesting for segments in the right prev_segment_sequence whatever + } + + @Test + public void testMultipleOutOfOrderRequests() + { + // multiple out of order thingies + } + + @Test + public void testRequestsWithDifferentRowIntervals() + { + + } + + @Test + public void testPartialSuccess() + { + + } + + @Test + public void testRetry() + { + + } + + + @Test + public void testMaxWaitTime() + { + + } + + @Test + public void testEnableDisableQueue() + { + + } + + @Test + public void testLeadershipChanges() + { + + } + + @Test + public void testDifferentGranualarities() + { + + } + + @Test + public void testWeekGranularity() + { + // This test should probably be included in the old SegmentAllocateActionTest as well + } + + private SegmentAllocateActionBuilder allocateRequest() + { + return new SegmentAllocateActionBuilder() + .forDatasource(DS_WIKI) + .forTimestamp("2022-01-01") + .withQueryGranularity(Granularities.SECOND) + .withSegmentGranularity(Granularities.HOUR); + } + + private void assertEquals(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) + { + Assert.assertEquals(expected, actual); + Assert.assertEquals(expected.getShardSpec(), actual.getShardSpec()); + } + + private Task createTask(String datasource, String groupId) + { + Task task = new NoopTask(null, groupId, datasource, 0, 0, null, null, null); + taskActionTestKit.getTaskLockbox().add(task); + return task; + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index ad032082b460..2475ac1a0be8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataStorageConnectorConfig; @@ -103,9 +104,6 @@ public int getSqlMetadataMaxRetry() testDerbyConnector ); final ServiceEmitter noopEmitter = new NoopServiceEmitter(); - final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector(); - leaderSelector.becomeLeader(); - final TaskLockConfig taskLockConfig = new TaskLockConfig() { @Override @@ -129,8 +127,8 @@ public long getBatchAllocationMaxWaitTime() taskLockbox, taskLockConfig, metadataStorageCoordinator, - leaderSelector, - noopEmitter + noopEmitter, + ScheduledExecutors::fixed ), noopEmitter, EasyMock.createMock(SupervisorManager.class), diff --git a/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java b/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java new file mode 100644 index 000000000000..33641a8417da --- /dev/null +++ b/server/src/test/java/org/apache/druid/indexing/overlord/SegmentCreateRequestTest.java @@ -0,0 +1,46 @@ +/* + * 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.overlord; + +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.druid.timeline.partition.PartialShardSpec; +import org.junit.Assert; +import org.junit.Test; + +public class SegmentCreateRequestTest +{ + + @Test + public void testNullPreviousSegmentId() + { + final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); + SegmentCreateRequest request = new SegmentCreateRequest( + "sequence", + null, + "version", + partialShardSpec + ); + Assert.assertEquals("sequence", request.getSequenceName()); + Assert.assertEquals("", request.getPreviousSegmentId()); + Assert.assertEquals("version", request.getVersion()); + Assert.assertEquals(partialShardSpec, request.getPartialShardSpec()); + } + +} From 39a97fd04e3ca5c3e10c738709f18b7b5dace8e8 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 11:08:08 +0530 Subject: [PATCH 25/35] Improve leadership check --- .../common/actions/SegmentAllocationQueue.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 281e786a5f87..c9644bf3eddc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -215,13 +215,18 @@ private void processBatchesDue() if (!isLeader.get()) { log.info("Not leader anymore. Failing [%d] batches in queue.", processingQueue.size()); - AllocateRequestBatch nextBatch = processingQueue.pollFirst(); - while (nextBatch != null) { + // Keep removing items from the queue as long as not leader + AllocateRequestBatch nextBatch = processingQueue.peekFirst(); + while (nextBatch != null && !isLeader.get()) { + processingQueue.pollFirst(); + keyToBatch.remove(nextBatch.key); nextBatch.markCompleted(); - nextBatch = processingQueue.pollFirst(); + nextBatch = processingQueue.peekFirst(); } + } - keyToBatch.clear(); + // Check once again for leadership + if (!isLeader.get()) { return; } From 5e0cdca49dd4027a2c2089c284ad1a7c13f16081 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 12:06:33 +0530 Subject: [PATCH 26/35] Add more tests --- .../actions/SegmentAllocationQueueTest.java | 249 ++++++++++-------- 1 file changed, 132 insertions(+), 117 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index 721ce83e7774..9626e231cdb8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -35,6 +35,10 @@ import org.junit.Rule; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Future; + public class SegmentAllocationQueueTest { @Rule @@ -93,11 +97,9 @@ public void tearDown() } @Test - public void testCriteriaForBatching() + public void testBatchWithMultipleTimestamps() { - // Requests with different timestamps and sequence names can batched together - // as long as the other params are the same - verifyRequestsCanBeBatched( + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .forTimestamp("2022-01-01T01:00:00") .withSegmentGranularity(Granularities.DAY) @@ -114,180 +116,197 @@ public void testCriteriaForBatching() .build(), true ); + } - // Requests for different lock types can be batched together - verifyRequestsCanBeBatched( + @Test + public void testBatchWithExclusiveLocks() + { + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .withTaskLockType(TaskLockType.EXCLUSIVE).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.EXCLUSIVE).build(), + true + ); + } + + @Test + public void testBatchWithSharedLocks() + { + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .withTaskLockType(TaskLockType.SHARED).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withTaskLockType(TaskLockType.SHARED).build(), + true + ); + } + + @Test + public void testBatchWithMultipleQueryGranularities() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withQueryGranularity(Granularities.SECOND).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withQueryGranularity(Granularities.MINUTE).build(), true ); } - @Test(timeout = 60_000) - public void testCriteriaForBatchingNegative() + @Test + public void testMultipleDatasourcesCannotBatch() { - // Requests for different datasources are not batched together - verifyRequestsCanBeBatched( + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), allocateRequest().forTask(createTask(DS_KOALA, "group_1")).build(), false ); + } - // Requests for different groupIds are not batched together - verifyRequestsCanBeBatched( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), + @Test + public void testMultipleGroupIdsCannotBatch() + { + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_2")).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_3")).build(), false ); + } - // Requests for different lock granularities are not batched together - verifyRequestsCanBeBatched( + @Test + public void testMultipleLockGranularitiesCannotBatch() + { + verifyAllocationWithBatching( allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .withLockGranularity(LockGranularity.TIME_CHUNK).build(), allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .withLockGranularity(LockGranularity.SEGMENT).build(), false ); + } - // Requests for different query granularities are not batched together - verifyRequestsCanBeBatched( - allocateRequest().forTask(createTask(DS_WIKI, "1")) - .withQueryGranularity(Granularities.SECOND).build(), - allocateRequest().forTask(createTask(DS_WIKI, "2")) - .withQueryGranularity(Granularities.MINUTE).build(), - false - ); - - // Requests for different segment granularities are not batched together - verifyRequestsCanBeBatched( - allocateRequest().forTask(createTask(DS_WIKI, "1")) - .withSegmentGranularity(Granularities.HOUR).build(), - allocateRequest().forTask(createTask(DS_WIKI, "2")) - .withSegmentGranularity(Granularities.THIRTY_MINUTE).build(), - false - ); - - // Requests for different allocate intervals are not batched together - verifyRequestsCanBeBatched( - allocateRequest().forTask(createTask(DS_WIKI, "1")) + @Test + public void testMultipleAllocateIntervalsCannotBatch() + { + verifyAllocationWithBatching( + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .forTimestamp("2022-01-01") - .withSegmentGranularity(Granularities.DAY) - .withSequenceName("seq_1").build(), - allocateRequest().forTask(createTask(DS_WIKI, "2")) + .withSegmentGranularity(Granularities.DAY).build(), + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) .forTimestamp("2022-01-02") .withSegmentGranularity(Granularities.DAY).build(), false ); } - private void verifyRequestsCanBeBatched( - SegmentAllocateRequest a, - SegmentAllocateRequest b, - boolean canBatch - ) - { - // Disable queue and cleanup state - allocationQueue.stopBeingLeader(); - executor.finishAllPendingTasks(); - emitter.flush(); - - // Enable queue again - allocationQueue.becomeLeader(); - - Assert.assertEquals(0, allocationQueue.size()); - allocationQueue.add(a); - allocationQueue.add(b); - - final int expectedCount = canBatch ? 1 : 2; - Assert.assertEquals(expectedCount, allocationQueue.size()); - - executor.finishNextPendingTask(); - emitter.verifyEmitted("task/action/batch/size", expectedCount); - } - @Test - public void testBatchNotDue() + public void testConflictingPendingSegment() throws Exception { + SegmentAllocateRequest hourSegmentRequest = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withSegmentGranularity(Granularities.HOUR) + .build(); + Future hourSegmentFuture = allocationQueue.add(hourSegmentRequest); - } - - @Test - public void testAllocationWithBatch() - { - // try out different combos of these: - // lock granularity, lock type, skip lineage, - } - - // now we need to test out a bunch of scenarios - // possibly with all combos above? - - @Test - public void testMultipleRequestsForSameSegment() - { + SegmentAllocateRequest halfHourSegmentRequest = + allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + .withSegmentGranularity(Granularities.THIRTY_MINUTE) + .build(); + Future halfHourSegmentFuture = allocationQueue.add(halfHourSegmentRequest); - } + executor.finishNextPendingTask(); - @Test - public void testMultipleInOrderRequests() - { - // multiple requests which are requesting for segments in the right prev_segment_sequence whatever + Assert.assertNotNull(hourSegmentFuture.get()); + Assert.assertNull(halfHourSegmentFuture.get()); } @Test - public void testMultipleOutOfOrderRequests() + public void testFullAllocationQueue() throws Exception { - // multiple out of order thingies - } + for (int i = 0; i < 2000; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + allocationQueue.add(request); + } - @Test - public void testRequestsWithDifferentRowIntervals() - { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "next_group")).build(); + Future segmentIdFuture = allocationQueue.add(request); + // Verify that the future is already complete and segment allocation has failed + Assert.assertNull(segmentIdFuture.get()); } @Test - public void testPartialSuccess() + public void testMultipleRequestsForSameSegment() throws Exception { + final List> segmentFutures = new ArrayList<>(); + for (int i = 0; i < 10; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)) + .withSequenceName("sequence_1") + .withPreviousSegmentId("segment_1") + .build(); + segmentFutures.add(allocationQueue.add(request)); + } - } + executor.finishNextPendingTask(); - @Test - public void testRetry() - { + SegmentIdWithShardSpec segmentId1 = segmentFutures.get(0).get(); + for (Future segmentFuture : segmentFutures) { + Assert.assertEquals(segmentFuture.get(), segmentId1); + } } - @Test public void testMaxWaitTime() { - + // Verify that the batch is due yet } @Test - public void testEnableDisableQueue() + public void testRequestsFailOnLeaderChange() throws Exception { + final List> segmentFutures = new ArrayList<>(); + for (int i = 0; i < 10; ++i) { + SegmentAllocateRequest request = + allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + segmentFutures.add(allocationQueue.add(request)); + } - } - - @Test - public void testLeadershipChanges() - { + allocationQueue.stopBeingLeader(); + executor.finishNextPendingTask(); + for (Future segmentFuture : segmentFutures) { + Assert.assertNull(segmentFuture.get()); + } } - @Test - public void testDifferentGranualarities() + private void verifyAllocationWithBatching( + SegmentAllocateRequest a, + SegmentAllocateRequest b, + boolean canBatch + ) { + Assert.assertEquals(0, allocationQueue.size()); + final Future segmentIdA = allocationQueue.add(a); + final Future segmentIdB = allocationQueue.add(b); - } + final int expectedCount = canBatch ? 1 : 2; + Assert.assertEquals(expectedCount, allocationQueue.size()); - @Test - public void testWeekGranularity() - { - // This test should probably be included in the old SegmentAllocateActionTest as well + executor.finishNextPendingTask(); + emitter.verifyEmitted("task/action/batch/size", expectedCount); + + try { + Assert.assertNotNull(segmentIdA.get()); + Assert.assertNotNull(segmentIdB.get()); + } + catch (Exception e) { + throw new RuntimeException("Error while getting segment ids from future", e); + } } private SegmentAllocateActionBuilder allocateRequest() @@ -295,16 +314,12 @@ private SegmentAllocateActionBuilder allocateRequest() return new SegmentAllocateActionBuilder() .forDatasource(DS_WIKI) .forTimestamp("2022-01-01") + .withLockGranularity(LockGranularity.TIME_CHUNK) + .withTaskLockType(TaskLockType.SHARED) .withQueryGranularity(Granularities.SECOND) .withSegmentGranularity(Granularities.HOUR); } - private void assertEquals(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) - { - Assert.assertEquals(expected, actual); - Assert.assertEquals(expected.getShardSpec(), actual.getShardSpec()); - } - private Task createTask(String datasource, String groupId) { Task task = new NoopTask(null, groupId, datasource, 0, 0, null, null, null); From 0ed6562fbb995bca5e8eac6b1873b206dc9a3dbb Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 12:21:13 +0530 Subject: [PATCH 27/35] Fix checkstyle --- .../druid/indexing/common/actions/SegmentAllocationQueue.java | 4 ---- .../druid/indexing/common/actions/TaskActionTestKit.java | 1 - 2 files changed, 5 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index c9644bf3eddc..786a256c9416 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -20,8 +20,6 @@ package org.apache.druid.indexing.common.actions; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -33,7 +31,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; @@ -47,7 +44,6 @@ import org.joda.time.Interval; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 2475ac1a0be8..189d3fe8779c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -38,7 +38,6 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; -import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.joda.time.Period; From 92479da5235b0aeea90d3116fc0a268f400016fd Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 12:34:46 +0530 Subject: [PATCH 28/35] Add TaskAction.canPerformAsync() --- .../common/actions/LocalTaskActionClient.java | 26 +++++------- .../common/actions/SegmentAllocateAction.java | 7 ++++ .../indexing/common/actions/TaskAction.java | 12 ++++++ .../actions/SegmentAllocationQueueTest.java | 42 +++++++++++-------- 4 files changed, 53 insertions(+), 34 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java index e7ce9b204894..a07977c6a564 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java @@ -77,28 +77,22 @@ public RetType submit(TaskAction taskAction) } final long performStartTime = System.currentTimeMillis(); - final RetType result; - if (isBatchAllocateAction(taskAction)) { - result = performBatchAllocateAction(taskAction); - } else { - result = taskAction.perform(task, toolbox); - } + final RetType result = performAction(taskAction); emitTimerMetric("task/action/run/time", taskAction, System.currentTimeMillis() - performStartTime); return result; } - private boolean isBatchAllocateAction(TaskAction taskAction) - { - return toolbox.canBatchSegmentAllocation() - && taskAction instanceof SegmentAllocateAction; - } - - @SuppressWarnings("unchecked") - private R performBatchAllocateAction(TaskAction taskAction) + private R performAction(TaskAction taskAction) { try { - SegmentAllocateAction allocateAction = (SegmentAllocateAction) taskAction; - return (R) allocateAction.performAsync(task, toolbox).get(5, TimeUnit.MINUTES); + final R result; + if (taskAction.canPerformAsync(task, toolbox)) { + result = taskAction.performAsync(task, toolbox).get(5, TimeUnit.MINUTES); + } else { + result = taskAction.perform(task, toolbox); + } + + return result; } catch (Throwable t) { throw new RuntimeException(t); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index 1ec60c5974c8..7c3f87e424eb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -181,6 +181,13 @@ public TypeReference getReturnTypeReference() }; } + @Override + public boolean canPerformAsync(Task task, TaskActionToolbox toolbox) + { + return toolbox.canBatchSegmentAllocation(); + } + + @Override public Future performAsync(Task task, TaskActionToolbox toolbox) { return toolbox.getSegmentAllocationQueue().add( 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 559039d96ef6..18e373727790 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 @@ -24,6 +24,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.indexing.common.task.Task; +import java.util.concurrent.Future; + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = TaskAction.TYPE_FIELD) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "lockAcquire", value = TimeChunkLockAcquireAction.class), @@ -58,6 +60,16 @@ public interface TaskAction boolean isAudited(); + default boolean canPerformAsync(Task task, TaskActionToolbox toolbox) + { + return false; + } + + default Future performAsync(Task task, TaskActionToolbox toolbox) + { + throw new UnsupportedOperationException(); + } + @Override String toString(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index 9626e231cdb8..cc413043ffa8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -38,6 +38,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; public class SegmentAllocationQueueTest { @@ -201,7 +202,7 @@ public void testMultipleAllocateIntervalsCannotBatch() } @Test - public void testConflictingPendingSegment() throws Exception + public void testConflictingPendingSegment() { SegmentAllocateRequest hourSegmentRequest = allocateRequest().forTask(createTask(DS_WIKI, "group_1")) @@ -217,12 +218,12 @@ public void testConflictingPendingSegment() throws Exception executor.finishNextPendingTask(); - Assert.assertNotNull(hourSegmentFuture.get()); - Assert.assertNull(halfHourSegmentFuture.get()); + Assert.assertNotNull(getSegmentId(hourSegmentFuture)); + Assert.assertNull(getSegmentId(halfHourSegmentFuture)); } @Test - public void testFullAllocationQueue() throws Exception + public void testFullAllocationQueue() { for (int i = 0; i < 2000; ++i) { SegmentAllocateRequest request = @@ -232,14 +233,14 @@ public void testFullAllocationQueue() throws Exception SegmentAllocateRequest request = allocateRequest().forTask(createTask(DS_WIKI, "next_group")).build(); - Future segmentIdFuture = allocationQueue.add(request); + Future future = allocationQueue.add(request); // Verify that the future is already complete and segment allocation has failed - Assert.assertNull(segmentIdFuture.get()); + Assert.assertNull(getSegmentId(future)); } @Test - public void testMultipleRequestsForSameSegment() throws Exception + public void testMultipleRequestsForSameSegment() { final List> segmentFutures = new ArrayList<>(); for (int i = 0; i < 10; ++i) { @@ -253,10 +254,10 @@ public void testMultipleRequestsForSameSegment() throws Exception executor.finishNextPendingTask(); - SegmentIdWithShardSpec segmentId1 = segmentFutures.get(0).get(); + SegmentIdWithShardSpec segmentId1 = getSegmentId(segmentFutures.get(0)); - for (Future segmentFuture : segmentFutures) { - Assert.assertEquals(segmentFuture.get(), segmentId1); + for (Future future : segmentFutures) { + Assert.assertEquals(getSegmentId(future), segmentId1); } } @@ -267,7 +268,7 @@ public void testMaxWaitTime() } @Test - public void testRequestsFailOnLeaderChange() throws Exception + public void testRequestsFailOnLeaderChange() { final List> segmentFutures = new ArrayList<>(); for (int i = 0; i < 10; ++i) { @@ -279,8 +280,8 @@ public void testRequestsFailOnLeaderChange() throws Exception allocationQueue.stopBeingLeader(); executor.finishNextPendingTask(); - for (Future segmentFuture : segmentFutures) { - Assert.assertNull(segmentFuture.get()); + for (Future future : segmentFutures) { + Assert.assertNull(getSegmentId(future)); } } @@ -291,8 +292,8 @@ private void verifyAllocationWithBatching( ) { Assert.assertEquals(0, allocationQueue.size()); - final Future segmentIdA = allocationQueue.add(a); - final Future segmentIdB = allocationQueue.add(b); + final Future futureA = allocationQueue.add(a); + final Future futureB = allocationQueue.add(b); final int expectedCount = canBatch ? 1 : 2; Assert.assertEquals(expectedCount, allocationQueue.size()); @@ -300,12 +301,17 @@ private void verifyAllocationWithBatching( executor.finishNextPendingTask(); emitter.verifyEmitted("task/action/batch/size", expectedCount); + Assert.assertNotNull(getSegmentId(futureA)); + Assert.assertNotNull(getSegmentId(futureB)); + } + + private SegmentIdWithShardSpec getSegmentId(Future future) + { try { - Assert.assertNotNull(segmentIdA.get()); - Assert.assertNotNull(segmentIdB.get()); + return future.get(5, TimeUnit.SECONDS); } catch (Exception e) { - throw new RuntimeException("Error while getting segment ids from future", e); + throw new RuntimeException(e); } } From 295e96d5d21017aad1e342f68db306e420ba40dc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Nov 2022 19:50:22 +0530 Subject: [PATCH 29/35] Fix interval search --- .../druid/java/util/common/Intervals.java | 31 ++++--------------- .../druid/java/util/common/IntervalsTest.java | 19 +++++++++--- 2 files changed, 20 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java index 338e520d63b3..96f858fd4be2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/core/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -26,7 +26,6 @@ import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; -import java.util.Arrays; public final class Intervals { @@ -86,30 +85,12 @@ public static boolean isEternity(final Interval interval) @Nullable public static Interval findOverlappingInterval(Interval searchInterval, Interval[] sortedIntervals) { - Arrays.sort(sortedIntervals, Comparators.intervalsByStartThenEnd()); - int index = Arrays.binarySearch( - sortedIntervals, - searchInterval, - Comparators.intervalsByStartThenEnd() - ); - if (index >= 0) { - return sortedIntervals[index]; - } - - // Key was not found, index returned from binarySearch is (-(insertionPoint) - 1) - index = -(index + 1); - - // If the interval at (index - 1) doesn't overlap, (index - 2) wouldn't overlap either - if (index > 0) { - if (sortedIntervals[index - 1].overlaps(searchInterval)) { - return sortedIntervals[index - 1]; - } - } - - // If the interval at index doesn't overlap, (index + 1) wouldn't overlap either - if (index < sortedIntervals.length) { - if (sortedIntervals[index].overlaps(searchInterval)) { - return sortedIntervals[index]; + for (Interval interval : sortedIntervals) { + if (interval.overlaps(searchInterval)) { + return interval; + } else if (interval.getStart().isAfter(searchInterval.getEnd())) { + // Intervals after this cannot have an overlap + return null; } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java index c66f4a9cb35e..59eac8d5a991 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java @@ -19,10 +19,13 @@ package org.apache.druid.java.util.common; +import org.apache.druid.java.util.common.guava.Comparators; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; + public class IntervalsTest { @@ -32,9 +35,10 @@ public void testFindOverlappingInterval() final Interval[] sortedIntervals = new Interval[]{ Intervals.of("2019/2020"), Intervals.of("2021/2022"), - Intervals.of("2021-01-01/2021-02-01"), - Intervals.of("2021/2023") + Intervals.of("2021-04-01/2021-05-01"), + Intervals.of("2022/2023") }; + Arrays.sort(sortedIntervals, Comparators.intervalsByStartThenEnd()); // Search interval outside the bounds of the sorted intervals Assert.assertNull( @@ -53,14 +57,19 @@ public void testFindOverlappingInterval() // Partially overlapping interval Assert.assertEquals( - Intervals.of("2021/2023"), + Intervals.of("2022/2023"), Intervals.findOverlappingInterval(Intervals.of("2022-01-01/2022-01-02"), sortedIntervals) ); + Assert.assertEquals( + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021-06-01/2021-07-01"), sortedIntervals) + ); + // Overlap with multiple intervals, "smallest" one is returned Assert.assertEquals( - Intervals.of("2021-01-01/2021-02-01"), - Intervals.findOverlappingInterval(Intervals.of("2021-01-01/2021-01-02"), sortedIntervals) + Intervals.of("2021/2022"), + Intervals.findOverlappingInterval(Intervals.of("2021-03-01/2021-04-01"), sortedIntervals) ); // Search interval within bounds, no overlap From 0d09a6ecc6382358d176ec0dead09d543dcd1242 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 1 Dec 2022 13:06:23 +0530 Subject: [PATCH 30/35] Fix behaviour on being elected leader --- .../common/actions/SegmentAllocateAction.java | 3 + .../actions/SegmentAllocationQueue.java | 217 ++++++++++-------- .../actions/SegmentAllocateActionTest.java | 4 + .../actions/SegmentAllocationQueueTest.java | 14 +- .../IndexerSQLMetadataStorageCoordinator.java | 8 +- 5 files changed, 140 insertions(+), 106 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index 7c3f87e424eb..f0fae4a8617d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -190,6 +190,9 @@ public boolean canPerformAsync(Task task, TaskActionToolbox toolbox) @Override public Future performAsync(Task task, TaskActionToolbox toolbox) { + if (!toolbox.canBatchSegmentAllocation()) { + throw new ISE("Batched segment allocation is disabled"); + } return toolbox.getSegmentAllocationQueue().add( new SegmentAllocateRequest(task, this, MAX_ATTEMPTS) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 786a256c9416..80a8ffcdcb06 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -72,7 +72,6 @@ public class SegmentAllocationQueue private static final int MAX_QUEUE_SIZE = 2000; private final long maxWaitTimeMillis; - private final boolean enabled; private final TaskLockbox taskLockbox; private final ScheduledExecutorService executor; @@ -96,27 +95,57 @@ public SegmentAllocationQueue( this.taskLockbox = taskLockbox; this.metadataStorage = metadataStorage; this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationMaxWaitTime(); - this.enabled = taskLockConfig.isBatchSegmentAllocation(); - this.executor = executorFactory.create(1, "SegmentAllocQueue-%s"); + this.executor = taskLockConfig.isBatchSegmentAllocation() + ? executorFactory.create(1, "SegmentAllocQueue-%s") : null; } @LifecycleStart public void start() { - log.info("Initializing segment allocation queue."); + if (isEnabled()) { + log.info("Initializing segment allocation queue."); + scheduleQueuePoll(maxWaitTimeMillis); + } } @LifecycleStop public void stop() { - log.info("Tearing down segment allocation queue."); - executor.shutdownNow(); + if (isEnabled()) { + log.info("Tearing down segment allocation queue."); + executor.shutdownNow(); + } + } + + public void becomeLeader() + { + if (!isLeader.compareAndSet(false, true)) { + log.info("Already the leader. Queue processing has started."); + } else if (isEnabled()) { + log.info("Elected leader. Starting queue processing."); + } else { + log.info( + "Elected leader but batched segment allocation is disabled. " + + "Segment allocation queue will not be used." + ); + } + } + + public void stopBeingLeader() + { + if (!isLeader.compareAndSet(true, false)) { + log.info("Already surrendered leadership. Queue processing is stopped."); + } else if (isEnabled()) { + log.info("Not leader anymore. Stopping queue processing."); + } else { + log.info("Not leader anymore. Segment allocation queue is already disabled."); + } } public boolean isEnabled() { - return enabled; + return executor != null && !executor.isShutdown(); } private void scheduleQueuePoll(long delay) @@ -144,7 +173,7 @@ public Future add(SegmentAllocateRequest request) throw new ISE("Batched segment allocation is disabled."); } - final AllocateRequestKey requestKey = new AllocateRequestKey(request, false); + final AllocateRequestKey requestKey = new AllocateRequestKey(request); final AtomicReference> requestFuture = new AtomicReference<>(); // Possible race condition: @@ -170,18 +199,23 @@ public Future add(SegmentAllocateRequest request) } /** - * Tries to add the given batch to the processing queue. If the queue is full, - * marks the batch as completed failing all the pending requests. + * Tries to add the given batch to the processing queue. Fails all the pending + * requests in the batch if we are not leader or if the queue is full. */ private boolean addBatchToQueue(AllocateRequestBatch batch) { batch.resetQueueTime(); - if (processingQueue.offer(batch)) { + if (!isLeader.get()) { + batch.failPendingRequests("Cannot allocate segment if not leader"); + return false; + } else if (processingQueue.offer(batch)) { log.debug("Added a new batch [%s] to queue.", batch.key); return true; } else { - log.warn("Cannot add batch [%s] as queue is full. Failing [%d] requests.", batch.key, batch.size()); - batch.markCompleted(); + batch.failPendingRequests( + "Segment allocation queue is full. Check the metric `task/action/batch/runTime` " + + "to determine if metadata operations are slow." + ); return false; } } @@ -207,27 +241,10 @@ private void requeueBatch(AllocateRequestBatch batch) private void processBatchesDue() { - // If not leader, clear the queue and do not schedule any more rounds of processing - if (!isLeader.get()) { - log.info("Not leader anymore. Failing [%d] batches in queue.", processingQueue.size()); - - // Keep removing items from the queue as long as not leader - AllocateRequestBatch nextBatch = processingQueue.peekFirst(); - while (nextBatch != null && !isLeader.get()) { - processingQueue.pollFirst(); - keyToBatch.remove(nextBatch.key); - nextBatch.markCompleted(); - nextBatch = processingQueue.peekFirst(); - } - } - - // Check once again for leadership - if (!isLeader.get()) { - return; - } + clearQueueIfNotLeader(); // Process all batches which are due - log.debug("Processing all batches which are due for execution. Overall queue size [%d].", processingQueue.size()); + log.debug("Processing batches which are due. Queue size [%d].", processingQueue.size()); int numProcessedBatches = 0; AllocateRequestBatch nextBatch = processingQueue.peekFirst(); @@ -238,13 +255,14 @@ private void processBatchesDue() processed = processBatch(nextBatch); } catch (Throwable t) { + nextBatch.failPendingRequests(t); processed = true; log.error(t, "Error while processing batch [%s]", nextBatch.key); } - ++numProcessedBatches; + // Requeue if not fully processed yet if (processed) { - nextBatch.markCompleted(); + ++numProcessedBatches; } else { requeueBatch(nextBatch); } @@ -265,6 +283,26 @@ private void processBatchesDue() log.info("Processed [%d] batches, next execution in [%d ms]", numProcessedBatches, nextScheduleDelay); } + /** + * Removes items from the queue as long as we are not leader. + */ + private void clearQueueIfNotLeader() + { + int failedBatches = 0; + AllocateRequestBatch nextBatch = processingQueue.peekFirst(); + while (nextBatch != null && !isLeader.get()) { + processingQueue.pollFirst(); + keyToBatch.remove(nextBatch.key); + nextBatch.failPendingRequests("Cannot allocate segment if not leader"); + ++failedBatches; + + nextBatch = processingQueue.peekFirst(); + } + if (failedBatches > 0) { + log.info("Not leader. Failed [%d] batches, remaining in queue [%d].", failedBatches, processingQueue.size()); + } + } + /** * Processes the given batch. Returns true if the batch was completely processed * and should not be requeued. @@ -275,6 +313,9 @@ private boolean processBatch(AllocateRequestBatch requestBatch) keyToBatch.remove(requestKey); if (requestBatch.isEmpty()) { return true; + } else if (!isLeader.get()) { + requestBatch.failPendingRequests("Cannot allocate segment if not leader"); + return true; } log.debug( @@ -292,7 +333,7 @@ private boolean processBatch(AllocateRequestBatch requestBatch) final Set usedSegments = retrieveUsedSegments(requestKey); final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); - emitBatchMetric("task/action/batch/retries", 1L, requestKey); + emitBatchMetric("task/action/batch/attempts", 1L, requestKey); emitBatchMetric("task/action/batch/runTime", (System.currentTimeMillis() - startTimeMillis), requestKey); log.info("Successfully processed [%d / %d] requests in batch [%s].", successCount, batchSize, requestKey); @@ -306,10 +347,10 @@ private boolean processBatch(AllocateRequestBatch requestBatch) final Set updatedUsedSegments = retrieveUsedSegments(requestKey); if (updatedUsedSegments.equals(usedSegments)) { - log.error("Used segments have not changed. Not requeueing failed requests."); + requestBatch.failPendingRequests("Allocation failed probably due to conflicting segments."); return true; } else { - log.debug("Used segments have changed. Requeuing failed requests"); + log.debug("Used segments have changed. Requeuing failed requests."); return false; } } @@ -329,35 +370,40 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set allRequests = requestBatch.getRequests(); - final Set pendingRequests = new HashSet<>(); + final Set requestsWithNoOverlappingSegment = new HashSet<>(); if (usedSegments.isEmpty()) { - pendingRequests.addAll(allRequests); + requestsWithNoOverlappingSegment.addAll(allRequests); } else { final Interval[] sortedUsedSegmentIntervals = getSortedIntervals(usedSegments); - final Map> usedIntervalToRequests = new HashMap<>(); + final Map> overlapIntervalToRequests = new HashMap<>(); for (SegmentAllocateRequest request : allRequests) { - // If there is an overlapping used segment interval, that interval is - // the only candidate for allocation - Interval overlappingInterval = Intervals.findOverlappingInterval( + // If there is an overlapping used segment, the interval of the used segment + // is the only candidate for allocation for this request + final Interval overlappingInterval = Intervals.findOverlappingInterval( request.getRowInterval(), sortedUsedSegmentIntervals ); if (overlappingInterval == null) { - pendingRequests.add(request); + requestsWithNoOverlappingSegment.add(request); } else if (overlappingInterval.contains(request.getRowInterval())) { // Found an enclosing interval, use this for allocation - usedIntervalToRequests.computeIfAbsent(overlappingInterval, i -> new ArrayList<>()) - .add(request); + overlapIntervalToRequests.computeIfAbsent(overlappingInterval, i -> new ArrayList<>()) + .add(request); + } else { + // There is no valid allocation interval for this request due to a + // partially overlapping used segment. Need not do anything right now. + // The request will be retried upon requeueing the batch. } } - // Try to allocate segments for the identified used segment intervals - // Do not retry the failed requests with other intervals unless the batch is requeued - for (Map.Entry> entry : usedIntervalToRequests.entrySet()) { + // Try to allocate segments for the identified used segment intervals. + // Do not retry the failed requests with other intervals unless the batch is requeued. + for (Map.Entry> entry : overlapIntervalToRequests.entrySet()) { successCount += allocateSegmentsForInterval( entry.getKey(), entry.getValue(), @@ -368,6 +414,7 @@ private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set pendingRequests = new HashSet<>(requestsWithNoOverlappingSegment); for (Granularity granularity : Granularity.granularitiesFinerThan(requestBatch.key.preferredSegmentGranularity)) { Map> requestsByInterval = @@ -469,34 +516,6 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) emitter.emit(metricBuilder.build(metric, value)); } - public void becomeLeader() - { - if (!isLeader.compareAndSet(false, true)) { - // Already leader - log.info("Already the leader. Queue processing is already scheduled."); - } else if (isEnabled()) { - // Start polling the queue - log.info("Elected leader. Starting queue processing."); - scheduleQueuePoll(maxWaitTimeMillis); - } else { - log.info( - "Elected leader but batched segment allocation is disabled. " - + "Segment allocation queue will not be used." - ); - } - } - - public void stopBeingLeader() - { - if (!isLeader.compareAndSet(true, false)) { - log.info("Already surrendered leadership. Queue processing is stopped."); - } else if (isEnabled()) { - log.info("Not leader anymore. Stopping queue processing."); - } else { - log.info("Not leader anymore. Segment allocation queue is already disabled."); - } - } - /** * A batch of segment allocation requests. */ @@ -538,11 +557,16 @@ synchronized Set getRequests() return new HashSet<>(requestToFuture.keySet()); } - synchronized void markCompleted() + synchronized void failPendingRequests(String reason) + { + failPendingRequests(new ISE(reason)); + } + + synchronized void failPendingRequests(Throwable cause) { if (!requestToFuture.isEmpty()) { - log.info("Marking [%d] requests in batch [%s] as failed.", size(), key); - requestToFuture.values().forEach(future -> future.complete(null)); + log.warn("Failing [%d] requests in batch due to [%s]. Batch key: %s", size(), cause.getMessage(), key); + requestToFuture.values().forEach(future -> future.completeExceptionally(cause)); requestToFuture.keySet().forEach( request -> emitTaskMetric("task/action/failed/count", 1L, request) ); @@ -559,20 +583,20 @@ synchronized void handleResult(SegmentAllocateResult result, SegmentAllocateRequ requestToFuture.remove(request).complete(result.getSegmentId()); } else if (request.canRetry()) { log.info( - "Action [%s] failed in attempt [%d]. Can still retry. Latest error: %s", - request.getAction(), + "Allocation failed in attempt [%d] due to error [%s]. Can still retry. Action: %s", request.getAttempts(), - result.getErrorMessage() + result.getErrorMessage(), + request.getAction() ); } else { emitTaskMetric("task/action/failed/count", 1L, request); log.error( - "Failing allocate action [%s] after [%d] attempts. Latest error: %s", - request.getAction(), + "Failing allocate action after [%d] attempts. Latest error [%s]. Action: %s", request.getAttempts(), - result.getErrorMessage() + result.getErrorMessage(), + request.getAction() ); - requestToFuture.remove(request).complete(null); + requestToFuture.remove(request).completeExceptionally(new ISE(result.getErrorMessage())); } } @@ -607,8 +631,6 @@ boolean isDue() */ private static class AllocateRequestKey { - private final boolean unique; - private final String dataSource; private final String groupId; private final Interval preferredAllocationInterval; @@ -625,12 +647,11 @@ private static class AllocateRequestKey * Creates a new key for the given request. The batch for a unique key will * always contain a single request. */ - AllocateRequestKey(SegmentAllocateRequest request, boolean unique) + AllocateRequestKey(SegmentAllocateRequest request) { final SegmentAllocateAction action = request.getAction(); final Task task = request.getTask(); - this.unique = unique; this.dataSource = action.getDataSource(); this.groupId = task.getGroupId(); this.skipSegmentLineageCheck = action.isSkipSegmentLineageCheck(); @@ -641,7 +662,7 @@ private static class AllocateRequestKey this.preferredAllocationInterval = action.getPreferredSegmentGranularity() .bucket(action.getTimestamp()); - this.hash = unique ? super.hashCode() : Objects.hash( + this.hash = Objects.hash( skipSegmentLineageCheck, useNonRootGenPartitionSpace, dataSource, @@ -654,9 +675,6 @@ private static class AllocateRequestKey @Override public boolean equals(Object o) { - if (unique) { - return this == o; - } if (this == o) { return true; } @@ -682,12 +700,11 @@ public int hashCode() public String toString() { return "{" + - "unique=" + unique + - ", skipLineageCheck=" + skipSegmentLineageCheck + - ", ds='" + dataSource + '\'' + - ", groupId='" + groupId + '\'' + - ", interval=" + preferredAllocationInterval + + "ds='" + dataSource + '\'' + + ", gr='" + groupId + '\'' + ", lock=" + lockGranularity + + ", invl=" + preferredAllocationInterval + + ", slc=" + skipSegmentLineageCheck + '}'; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index b498834fbaba..c8861a92cd3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -64,6 +64,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -983,6 +984,9 @@ private SegmentIdWithShardSpec allocate( return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } } + catch (ExecutionException e) { + return null; + } catch (Exception e) { throw new RuntimeException(e); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index cc413043ffa8..d6559a0532b2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -24,6 +24,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -37,8 +38,10 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class SegmentAllocationQueueTest { @@ -236,7 +239,8 @@ public void testFullAllocationQueue() Future future = allocationQueue.add(request); // Verify that the future is already complete and segment allocation has failed - Assert.assertNull(getSegmentId(future)); + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(future)); + Assert.assertEquals("Segment allocation queue is full", t.getMessage()); } @Test @@ -281,7 +285,8 @@ public void testRequestsFailOnLeaderChange() executor.finishNextPendingTask(); for (Future future : segmentFutures) { - Assert.assertNull(getSegmentId(future)); + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(future)); + Assert.assertEquals("Cannot allocate segment if not leader", t.getMessage()); } } @@ -310,7 +315,10 @@ private SegmentIdWithShardSpec getSegmentId(Future futur try { return future.get(5, TimeUnit.SECONDS); } - catch (Exception e) { + catch (ExecutionException e) { + throw new ISE(e.getCause().getMessage()); + } + catch (InterruptedException | TimeoutException e) { throw new RuntimeException(e); } } 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 11d7846892d2..bb7759a8b5d6 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1059,9 +1059,6 @@ private Map createNewSegments( // to avoid clashes when inserting the pending segment created here. final Set pendingSegments = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); - if (committedMaxId != null) { - pendingSegments.add(committedMaxId); - } final Map createdSegments = new HashMap<>(); final Map sequenceHashToSegment = new HashMap<>(); @@ -1112,6 +1109,11 @@ private SegmentIdWithShardSpec createNewSegment( final PartialShardSpec partialShardSpec = request.getPartialShardSpec(); final String existingVersion = request.getVersion(); + // Include the committedMaxId while computing the overallMaxId + if (committedMaxId != null) { + pendingSegments.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 From c13274890d730086916aeb02a509dd8175a08a94 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 2 Dec 2022 09:20:40 +0530 Subject: [PATCH 31/35] Fix tests --- .../common/actions/SegmentAllocationQueueTest.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index d6559a0532b2..536e9ffac24c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -85,6 +85,7 @@ public long getBatchAllocationMaxWaitTime() (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService(nameFormat, executor, false) ); + allocationQueue.start(); allocationQueue.becomeLeader(); } @@ -222,7 +223,8 @@ public void testConflictingPendingSegment() executor.finishNextPendingTask(); Assert.assertNotNull(getSegmentId(hourSegmentFuture)); - Assert.assertNull(getSegmentId(halfHourSegmentFuture)); + Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(halfHourSegmentFuture)); + Assert.assertEquals("Storage coordinator could not allocate segment.", t.getMessage()); } @Test @@ -240,7 +242,11 @@ public void testFullAllocationQueue() // Verify that the future is already complete and segment allocation has failed Throwable t = Assert.assertThrows(ISE.class, () -> getSegmentId(future)); - Assert.assertEquals("Segment allocation queue is full", t.getMessage()); + Assert.assertEquals( + "Segment allocation queue is full. Check the metric `task/action/batch/runTime` " + + "to determine if metadata operations are slow.", + t.getMessage() + ); } @Test From d472152dcdf3754288b02b199a997d2500ca58fe Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 2 Dec 2022 18:26:12 +0530 Subject: [PATCH 32/35] Keep keys in queue rather than the batch --- .../actions/SegmentAllocationQueue.java | 81 ++++++++++--------- 1 file changed, 43 insertions(+), 38 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 80a8ffcdcb06..9ed53d99faf5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -80,7 +80,7 @@ public class SegmentAllocationQueue private final ServiceEmitter emitter; private final ConcurrentHashMap keyToBatch = new ConcurrentHashMap<>(); - private final BlockingDeque processingQueue = new LinkedBlockingDeque<>(MAX_QUEUE_SIZE); + private final BlockingDeque processingQueue = new LinkedBlockingDeque<>(MAX_QUEUE_SIZE); @Inject public SegmentAllocationQueue( @@ -173,8 +173,8 @@ public Future add(SegmentAllocateRequest request) throw new ISE("Batched segment allocation is disabled."); } - final AllocateRequestKey requestKey = new AllocateRequestKey(request); - final AtomicReference> requestFuture = new AtomicReference<>(); + final AllocateRequestKey requestKey = new AllocateRequestKey(request, maxWaitTimeMillis); + final AtomicReference> futureReference = new AtomicReference<>(); // Possible race condition: // t1 -> new batch is added to queue or batch already exists in queue @@ -187,15 +187,15 @@ public Future add(SegmentAllocateRequest request) keyToBatch.compute(requestKey, (key, existingBatch) -> { if (existingBatch == null) { AllocateRequestBatch newBatch = new AllocateRequestBatch(key); - requestFuture.set(newBatch.add(request)); + futureReference.set(newBatch.add(request)); return addBatchToQueue(newBatch) ? newBatch : null; } else { - requestFuture.set(existingBatch.add(request)); + futureReference.set(existingBatch.add(request)); return existingBatch; } }); - return requestFuture.get(); + return futureReference.get(); } /** @@ -204,11 +204,11 @@ public Future add(SegmentAllocateRequest request) */ private boolean addBatchToQueue(AllocateRequestBatch batch) { - batch.resetQueueTime(); + batch.key.resetQueueTime(); if (!isLeader.get()) { batch.failPendingRequests("Cannot allocate segment if not leader"); return false; - } else if (processingQueue.offer(batch)) { + } else if (processingQueue.offer(batch.key)) { log.debug("Added a new batch [%s] to queue.", batch.key); return true; } else { @@ -247,9 +247,11 @@ private void processBatchesDue() log.debug("Processing batches which are due. Queue size [%d].", processingQueue.size()); int numProcessedBatches = 0; - AllocateRequestBatch nextBatch = processingQueue.peekFirst(); - while (nextBatch != null && nextBatch.isDue()) { + AllocateRequestKey nextKey = processingQueue.peekFirst(); + while (nextKey != null && nextKey.isDue()) { processingQueue.pollFirst(); + AllocateRequestBatch nextBatch = keyToBatch.remove(nextKey); + boolean processed; try { processed = processBatch(nextBatch); @@ -257,7 +259,7 @@ private void processBatchesDue() catch (Throwable t) { nextBatch.failPendingRequests(t); processed = true; - log.error(t, "Error while processing batch [%s]", nextBatch.key); + log.error(t, "Error while processing batch [%s]", nextKey); } // Requeue if not fully processed yet @@ -267,7 +269,7 @@ private void processBatchesDue() requeueBatch(nextBatch); } - nextBatch = processingQueue.peek(); + nextKey = processingQueue.peek(); } // Schedule the next round of processing @@ -275,8 +277,8 @@ private void processBatchesDue() if (processingQueue.isEmpty()) { nextScheduleDelay = maxWaitTimeMillis; } else { - nextBatch = processingQueue.peek(); - long timeElapsed = System.currentTimeMillis() - nextBatch.getQueueTime(); + nextKey = processingQueue.peek(); + long timeElapsed = System.currentTimeMillis() - nextKey.getQueueTime(); nextScheduleDelay = Math.max(0, maxWaitTimeMillis - timeElapsed); } scheduleQueuePoll(nextScheduleDelay); @@ -289,14 +291,14 @@ private void processBatchesDue() private void clearQueueIfNotLeader() { int failedBatches = 0; - AllocateRequestBatch nextBatch = processingQueue.peekFirst(); - while (nextBatch != null && !isLeader.get()) { + AllocateRequestKey nextKey = processingQueue.peekFirst(); + while (nextKey != null && !isLeader.get()) { processingQueue.pollFirst(); - keyToBatch.remove(nextBatch.key); + AllocateRequestBatch nextBatch = keyToBatch.remove(nextKey); nextBatch.failPendingRequests("Cannot allocate segment if not leader"); ++failedBatches; - nextBatch = processingQueue.peekFirst(); + nextKey = processingQueue.peekFirst(); } if (failedBatches > 0) { log.info("Not leader. Failed [%d] batches, remaining in queue [%d].", failedBatches, processingQueue.size()); @@ -310,7 +312,6 @@ private void clearQueueIfNotLeader() private boolean processBatch(AllocateRequestBatch requestBatch) { final AllocateRequestKey requestKey = requestBatch.key; - keyToBatch.remove(requestKey); if (requestBatch.isEmpty()) { return true; } else if (!isLeader.get()) { @@ -322,13 +323,13 @@ private boolean processBatch(AllocateRequestBatch requestBatch) "Processing [%d] requests for batch [%s], queue time [%s].", requestBatch.size(), requestKey, - requestBatch.getQueueTime() + requestKey.getQueueTime() ); final long startTimeMillis = System.currentTimeMillis(); final int batchSize = requestBatch.size(); emitBatchMetric("task/action/batch/size", batchSize, requestKey); - emitBatchMetric("task/action/batch/queueTime", (startTimeMillis - requestBatch.getQueueTime()), requestKey); + emitBatchMetric("task/action/batch/queueTime", (startTimeMillis - requestKey.getQueueTime()), requestKey); final Set usedSegments = retrieveUsedSegments(requestKey); final int successCount = allocateSegmentsForBatch(requestBatch, usedSegments); @@ -521,7 +522,6 @@ private void emitBatchMetric(String metric, long value, AllocateRequestKey key) */ private class AllocateRequestBatch { - private long queueTimeMillis; private final AllocateRequestKey key; /** @@ -609,21 +609,6 @@ synchronized int size() { return requestToFuture.size(); } - - void resetQueueTime() - { - queueTimeMillis = System.currentTimeMillis(); - } - - long getQueueTime() - { - return queueTimeMillis; - } - - boolean isDue() - { - return System.currentTimeMillis() - queueTimeMillis >= maxWaitTimeMillis; - } } /** @@ -631,6 +616,9 @@ boolean isDue() */ private static class AllocateRequestKey { + private long queueTimeMillis; + private final long maxWaitTimeMillis; + private final String dataSource; private final String groupId; private final Interval preferredAllocationInterval; @@ -647,7 +635,7 @@ private static class AllocateRequestKey * Creates a new key for the given request. The batch for a unique key will * always contain a single request. */ - AllocateRequestKey(SegmentAllocateRequest request) + AllocateRequestKey(SegmentAllocateRequest request, long maxWaitTimeMillis) { final SegmentAllocateAction action = request.getAction(); final Task task = request.getTask(); @@ -670,6 +658,23 @@ private static class AllocateRequestKey preferredAllocationInterval, lockGranularity ); + + this.maxWaitTimeMillis = maxWaitTimeMillis; + } + + void resetQueueTime() + { + queueTimeMillis = System.currentTimeMillis(); + } + + long getQueueTime() + { + return queueTimeMillis; + } + + boolean isDue() + { + return System.currentTimeMillis() - queueTimeMillis >= maxWaitTimeMillis; } @Override From 29bc0d4d3e3c67c0f84bf431a9d39778d204d35e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 2 Dec 2022 19:00:14 +0530 Subject: [PATCH 33/35] Set default batchAllocationMaxWaitTime to 500ms --- .../apache/druid/indexing/overlord/config/TaskLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index 0b4c3cbcf1c6..acbc318baabc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -34,7 +34,7 @@ public class TaskLockConfig private boolean batchSegmentAllocation = false; @JsonProperty - private long batchAllocationMaxWaitTime = 1_000L; + private long batchAllocationMaxWaitTime = 500L; public boolean isForceTimeChunkLock() { From 784e6d21f80247ca193b29811084075c5837944f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 2 Dec 2022 19:00:55 +0530 Subject: [PATCH 34/35] Set batchAllocation to true for testing --- .../apache/druid/indexing/overlord/config/TaskLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index acbc318baabc..331187ca9a52 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -31,7 +31,7 @@ public class TaskLockConfig private boolean forceTimeChunkLock = true; @JsonProperty - private boolean batchSegmentAllocation = false; + private boolean batchSegmentAllocation = true; @JsonProperty private long batchAllocationMaxWaitTime = 500L; From 602c39e32f754fa698b1daa2eb2fb9195d47e45c Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 2 Dec 2022 19:03:08 +0530 Subject: [PATCH 35/35] Reduce max wait time to 50ms --- .../apache/druid/indexing/overlord/config/TaskLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java index 331187ca9a52..3f117b2bac1e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskLockConfig.java @@ -34,7 +34,7 @@ public class TaskLockConfig private boolean batchSegmentAllocation = true; @JsonProperty - private long batchAllocationMaxWaitTime = 500L; + private long batchAllocationMaxWaitTime = 50L; public boolean isForceTimeChunkLock() {