From 1f32283bb4af01ce7cd7e41d6a2204e93ca4cd77 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 15 Jul 2017 17:39:15 +0900 Subject: [PATCH 01/20] Implementation of prioritized locking --- docs/content/ingestion/tasks.md | 37 +- .../druid/indexing/kafka/KafkaIndexTask.java | 10 +- .../io/druid/indexing/common/TaskLock.java | 103 ++- .../druid/indexing/common/TaskLockType.java | 26 + .../common/actions/LockAcquireAction.java | 26 +- .../common/actions/LockTryAcquireAction.java | 27 +- .../common/actions/SegmentAllocateAction.java | 24 +- .../actions/SegmentMetadataUpdateAction.java | 2 +- .../common/actions/SegmentNukeAction.java | 2 +- .../SegmentTransactionalInsertAction.java | 12 +- .../actions/TaskActionPreconditions.java | 86 +++ .../common/actions/TaskActionToolbox.java | 53 -- .../task/AbstractFixedIntervalTask.java | 3 +- .../indexing/common/task/AbstractTask.java | 14 +- .../indexing/common/task/ArchiveTask.java | 3 +- .../common/task/ConvertSegmentTask.java | 1 - .../indexing/common/task/HadoopIndexTask.java | 20 +- .../druid/indexing/common/task/IndexTask.java | 74 +- .../druid/indexing/common/task/KillTask.java | 2 +- .../indexing/common/task/MergeTaskBase.java | 8 +- .../druid/indexing/common/task/MoveTask.java | 2 +- .../druid/indexing/common/task/NoopTask.java | 14 + .../common/task/RealtimeIndexTask.java | 27 +- .../indexing/common/task/RestoreTask.java | 2 +- .../io/druid/indexing/common/task/Task.java | 26 +- .../io/druid/indexing/common/task/Tasks.java | 90 +++ .../overlord/HeapMemoryTaskStorage.java | 23 +- .../druid/indexing/overlord/LockResult.java | 83 ++ .../overlord/MetadataTaskStorage.java | 20 + .../druid/indexing/overlord/TaskLockbox.java | 714 +++++++++++++----- .../druid/indexing/overlord/TaskStorage.java | 31 +- .../actions/RemoteTaskActionClientTest.java | 5 +- .../actions/SegmentAllocateActionTest.java | 6 +- .../actions/SegmentInsertActionTest.java | 7 +- .../SegmentTransactionalInsertActionTest.java | 7 +- .../actions/TaskActionPreconditionsTest.java | 106 +++ .../indexing/common/task/IndexTaskTest.java | 217 +++--- .../task/SameIntervalMergeTaskTest.java | 5 +- .../indexing/common/task/TaskSerdeTest.java | 6 +- .../indexing/overlord/RealtimeishTask.java | 13 +- .../indexing/overlord/TaskLifecycleTest.java | 9 +- .../overlord/TaskLockBoxConcurrencyTest.java | 150 ++++ .../indexing/overlord/TaskLockboxTest.java | 374 +++++++-- .../overlord/SegmentPublishResult.java | 6 + .../SQLMetadataStorageActionHandler.java | 2 - .../appenderator/AppenderatorDriver.java | 3 +- .../SQLMetadataStorageActionHandlerTest.java | 2 - 47 files changed, 1950 insertions(+), 533 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/TaskLockType.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 76f1268dd4f2..576520b09ccc 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -88,9 +88,32 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed |property|description|required?| |--------|-----------|---------| |type|The task type, this should always be "index".|yes| -|id|The task ID. If this is not explicitly specified, Druid generates the task ID using the name of the task file and date-time stamp. |no| +|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| |spec|The ingestion spec. See below for more details. |yes| +#### Task Priority + +Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking) + +Each task has a priority which is used for lock acquisition. Higher-priority tasks can preempt lower-priority tasks if they try to write on the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted. + +Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority. + +|task type|default priority| +|---------|----------------| +|Realtime index task|75| +|Batch index task|50| +|Merge/Append task|25| +|Other tasks|0| + +You can override the task priority by setting your priority in the task context like below. + +```json +"context" : { + "priority" : 100 +} +``` + #### DataSchema This field is required. @@ -308,7 +331,17 @@ These tasks start, sleep for a time and are used only for testing. The available Locking ------- -Once an overlord node accepts a task, a lock is created for the data source and interval specified in the task. +Once an overlord node accepts a task, the task acquires locks for the data source and intervals specified in the task. + +There are two locks types, i.e., _shared lock_ and _exclusive lock_. + +- A task needs to acquire a shared lock before it reads segments of an interval. Multiple shared locks can be acquired for the same dataSource and interval. Shared locks are always preemptable, but they don't preempt each other. +- A task needs to acquire an exclusive lock before it writes segemtns for an interval. An exclusive lock is acquired as preemptable and can be upgraded as non-preemptable when publishing segments. + +Each task can have different lock priorities. The locks of higher-priority tasks can preempt the locks of lower-priority tasks. The lock preemption works based on _optimistic locking_. When a lock is preempted, it is not notified to the owner task immediately. Instead, it's notified when the owner task tries to acquire the same lock again or upgrade it. (Note that lock acquisition is idempotent unless the lock is preempted.) In general, tasks don't content to acquire locks because they usually targets different dataSources or intervals. + +A task writing data into a dataSource must acquire exclusive locks for target intervals. Note that execlusive locks are still preemptable. As a result, the task must _upgrade_ its locks as non-preemptable when it executes a critical operation, _publishing segments_. Once the lock is upgraded, it can't be preempted by even higher-priority locks. After publishing segments, the task downgrades its locks as preemptable. + Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks. diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index c9bcc5954c2e..7d416b3d32c3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -99,6 +99,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; @@ -543,13 +544,12 @@ public void run() sequenceNames.values() ).get(); + final Future handoffFuture = driver.registerHandoff(published); final SegmentsAndMetadata handedOff; if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOff = driver.registerHandoff(published) - .get(); + handedOff = handoffFuture.get(); } else { - handedOff = driver.registerHandoff(published) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + handedOff = handoffFuture.get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); } if (handedOff == null) { @@ -888,7 +888,7 @@ private AppenderatorDriver newDriver( { return new AppenderatorDriver( appenderator, - new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), + new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema, true), toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java index ccbf721bb6f7..25d43fa41a5d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; +import com.google.common.base.Preconditions; import org.joda.time.Interval; /** @@ -29,23 +30,77 @@ */ public class TaskLock { + private final TaskLockType type; private final String groupId; private final String dataSource; private final Interval interval; private final String version; + private final int priority; + private final boolean upgraded; + private final boolean revoked; @JsonCreator public TaskLock( + @JsonProperty("type") TaskLockType type, @JsonProperty("groupId") String groupId, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, - @JsonProperty("version") String version + @JsonProperty("version") String version, + @JsonProperty("priority") int priority, + @JsonProperty("upgraded") boolean upgraded, + @JsonProperty("revoked") boolean revoked ) { - this.groupId = groupId; - this.dataSource = dataSource; - this.interval = interval; - this.version = version; + Preconditions.checkArgument(!type.equals(TaskLockType.SHARED) || !upgraded, "lock[%s] cannot be upgraded", type); + Preconditions.checkArgument(!upgraded || !revoked, "Upgraded locks cannot be revoked"); + + this.type = type; + this.groupId = Preconditions.checkNotNull(groupId); + this.dataSource = Preconditions.checkNotNull(dataSource); + this.interval = Preconditions.checkNotNull(interval); + this.version = Preconditions.checkNotNull(version); + this.priority = priority; + this.upgraded = upgraded; + this.revoked = revoked; + } + + public TaskLock( + TaskLockType type, + String groupId, + String dataSource, + Interval interval, + String version, + int priority + ) + { + this(type, groupId, dataSource, interval, version, priority, false, false); + } + + public TaskLock upgrade() + { + Preconditions.checkState(!revoked, "Revoked locks cannot be upgraded"); + Preconditions.checkState(!upgraded, "Already upgraded"); + return new TaskLock(type, groupId, dataSource, interval, version, priority, true, revoked); + } + + public TaskLock downgrade() + { + Preconditions.checkState(!revoked, "Revoked locks cannot be downgraded"); + Preconditions.checkState(upgraded, "Already downgraded"); + return new TaskLock(type, groupId, dataSource, interval, version, priority, false, revoked); + } + + public TaskLock revoke() + { + Preconditions.checkState(!revoked, "Already revoked"); + Preconditions.checkState(!upgraded, "Upgraded locks cannot be revoked"); + return new TaskLock(type, groupId, dataSource, interval, version, priority, upgraded, true); + } + + @JsonProperty + public TaskLockType getType() + { + return type; } @JsonProperty @@ -72,34 +127,60 @@ public String getVersion() return version; } + @JsonProperty + public int getPriority() + { + return priority; + } + + @JsonProperty + public boolean isUpgraded() + { + return upgraded; + } + + @JsonProperty + public boolean isRevoked() + { + return revoked; + } + @Override public boolean equals(Object o) { if (!(o instanceof TaskLock)) { return false; } else { - final TaskLock x = (TaskLock) o; - return Objects.equal(this.groupId, x.groupId) && - Objects.equal(this.dataSource, x.dataSource) && - Objects.equal(this.interval, x.interval) && - Objects.equal(this.version, x.version); + final TaskLock that = (TaskLock) o; + return this.type.equals(that.type) && + this.groupId.equals(that.groupId) && + this.dataSource.equals(that.dataSource) && + this.interval.equals(that.interval) && + this.version.equals(that.version) && + this.priority == that.priority && + this.upgraded == that.upgraded && + this.revoked == that.revoked; } } @Override public int hashCode() { - return Objects.hashCode(groupId, dataSource, interval, version); + return Objects.hashCode(type, groupId, dataSource, interval, version, priority, upgraded, revoked); } @Override public String toString() { return Objects.toStringHelper(this) + .add("type", type) .add("groupId", groupId) .add("dataSource", dataSource) .add("interval", interval) .add("version", version) + .add("priority", priority) + .add("upgraded", upgraded) + .add("revoked", revoked) .toString(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLockType.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLockType.java new file mode 100644 index 000000000000..b0057517b4c6 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLockType.java @@ -0,0 +1,26 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +public enum TaskLockType +{ + SHARED, + EXCLUSIVE +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 2d61f0639242..74ea94342efc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -24,23 +24,34 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Throwables; -import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.LockResult; import org.joda.time.Interval; -public class LockAcquireAction implements TaskAction +public class LockAcquireAction implements TaskAction { + private final TaskLockType type; + @JsonIgnore private final Interval interval; @JsonCreator public LockAcquireAction( + @JsonProperty("lockType") TaskLockType type, @JsonProperty("interval") Interval interval ) { + this.type = type; this.interval = interval; } + @JsonProperty("lockType") + public TaskLockType getType() + { + return type; + } + @JsonProperty public Interval getInterval() { @@ -48,18 +59,18 @@ public Interval getInterval() } @Override - public TypeReference getReturnTypeReference() + public TypeReference getReturnTypeReference() { - return new TypeReference() + return new TypeReference() { }; } @Override - public TaskLock perform(Task task, TaskActionToolbox toolbox) + public LockResult perform(Task task, TaskActionToolbox toolbox) { try { - return toolbox.getTaskLockbox().lock(task, interval); + return toolbox.getTaskLockbox().lock(type, task, interval); } catch (InterruptedException e) { throw Throwables.propagate(e); @@ -76,7 +87,8 @@ public boolean isAudited() public String toString() { return "LockAcquireAction{" + - "interval=" + interval + + "lockType=" + type + + ", interval=" + interval + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java index 28cfd6ef22ae..9d60a0a61f76 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java @@ -23,23 +23,35 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; -import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.LockResult; import org.joda.time.Interval; -public class LockTryAcquireAction implements TaskAction +public class LockTryAcquireAction implements TaskAction { + @JsonIgnore + private final TaskLockType type; + @JsonIgnore private final Interval interval; @JsonCreator public LockTryAcquireAction( + @JsonProperty("lockType") TaskLockType type, @JsonProperty("interval") Interval interval ) { + this.type = type; this.interval = interval; } + @JsonProperty("lockType") + public TaskLockType getType() + { + return type; + } + @JsonProperty public Interval getInterval() { @@ -47,17 +59,17 @@ public Interval getInterval() } @Override - public TypeReference getReturnTypeReference() + public TypeReference getReturnTypeReference() { - return new TypeReference() + return new TypeReference() { }; } @Override - public TaskLock perform(Task task, TaskActionToolbox toolbox) + public LockResult perform(Task task, TaskActionToolbox toolbox) { - return toolbox.getTaskLockbox().tryLock(task, interval).orNull(); + return toolbox.getTaskLockbox().tryLock(type, task, interval); } @Override @@ -70,7 +82,8 @@ public boolean isAudited() public String toString() { return "LockTryAcquireAction{" + - "interval=" + interval + + "lockType=" + type + + ", interval=" + interval + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java index c966131ab3c1..5e22b2736a2e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java @@ -25,12 +25,13 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; - -import io.druid.java.util.common.granularity.Granularity; -import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.DataSegment; @@ -174,16 +175,27 @@ public SegmentIdentifier perform( rowInterval, tryInterval ); - final TaskLock tryLock = toolbox.getTaskLockbox().tryLock(task, tryInterval).orNull(); - if (tryLock != null) { + final LockResult lockResult = toolbox.getTaskLockbox().tryLock(TaskLockType.EXCLUSIVE, task, tryInterval); + if (lockResult.isWasRevoked()) { + // 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); + } + + if (lockResult.isOk()) { final SegmentIdentifier identifier = msc.allocatePendingSegment( dataSource, sequenceName, previousSegmentId, tryInterval, - tryLock.getVersion() + lockResult.getTaskLock().getVersion() ); if (identifier != null) { + log.debug( + "A new segment identifier[%s] is allocated for rowInterval[%s], segmentInterval[%s]", + identifier, + rowInterval, + tryInterval + ); return identifier; } else { log.debug( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index 6162dce4e8ce..519862a47e39 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -62,7 +62,7 @@ public Void perform( Task task, TaskActionToolbox toolbox ) throws IOException { - toolbox.verifyTaskLocks(task, segments); + TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); // Emit metrics diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index 2d078ebfaeed..e9dde04721e1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -62,7 +62,7 @@ public TypeReference getReturnTypeReference() @Override public Void perform(Task task, TaskActionToolbox toolbox) throws IOException { - toolbox.verifyTaskLocks(task, segments); + TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); // Emit metrics diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 806969e860cd..a1dadf62bd22 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -99,7 +99,13 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws IOException { - toolbox.verifyTaskLocks(task, segments); + for (DataSegment segment : segments) { + if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { + return SegmentPublishResult.fail(); + } + } + + TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); final SegmentPublishResult retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( segments, @@ -107,6 +113,10 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws endMetadata ); + for (DataSegment segment : segments) { + toolbox.getTaskLockbox().downgrade(task, segment.getInterval()); + } + // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java new file mode 100644 index 000000000000..bbcfac9d4ce3 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.google.common.annotations.VisibleForTesting; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.java.util.common.ISE; +import io.druid.timeline.DataSegment; + +import java.util.List; +import java.util.Set; + +public class TaskActionPreconditions +{ + public static void checkTaskLocksUpgraded( + final Task task, + final TaskLockbox taskLockbox, + final Set segments + ) + { + if (!checkLockCoversSegments(task, taskLockbox, segments, true)) { + throw new ISE("Segments not covered by locks or locks need upgrade for task: %s", task.getId()); + } + } + + public static void checkTaskLocks( + final Task task, + final TaskLockbox taskLockbox, + final Set segments + ) + { + if (!checkLockCoversSegments(task, taskLockbox, segments, false)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); + } + } + + @VisibleForTesting + static boolean checkLockCoversSegments( + final Task task, + final TaskLockbox taskLockbox, + final Set segments, + final boolean checkUpgraded + ) + { + // Verify that each of these segments falls under some lock + + // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check + // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we + // NOTE: insert some segments from the task but not others. + + final List taskLocks = taskLockbox.findLocksForTask(task); + for (final DataSegment segment : segments) { + final boolean ok = taskLocks.stream().anyMatch( + taskLock -> taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()) + && taskLock.getVersion().compareTo(segment.getVersion()) >= 0 + && (!checkUpgraded || taskLock.isUpgraded()) + ); + + if (!ok) { + return false; + } + } + + return true; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index c24e1fbb2656..59ca22942159 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -19,20 +19,11 @@ package io.druid.indexing.common.actions; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.indexing.common.TaskLock; -import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.supervisor.SupervisorManager; -import io.druid.java.util.common.ISE; -import io.druid.timeline.DataSegment; - -import java.util.List; -import java.util.Set; public class TaskActionToolbox { @@ -74,48 +65,4 @@ public SupervisorManager getSupervisorManager() { return supervisorManager; } - - public void verifyTaskLocks( - final Task task, - final Set segments - ) - { - if (!taskLockCoversSegments(task, segments)) { - throw new ISE("Segments not covered by locks for task: %s", task.getId()); - } - } - - public boolean taskLockCoversSegments( - final Task task, - final Set segments - ) - { - // Verify that each of these segments falls under some lock - - // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check - // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we - // NOTE: insert some segments from the task but not others. - - final List taskLocks = getTaskLockbox().findLocksForTask(task); - for (final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()) - && taskLock.getVersion().compareTo(segment.getVersion()) >= 0; - } - } - ); - - if (!ok) { - return false; - } - } - - return true; - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java index a82c5d8264d4..d9b0c6b55539 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import org.joda.time.Interval; @@ -89,7 +90,7 @@ protected AbstractFixedIntervalTask( @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - return taskActionClient.submit(new LockTryAcquireAction(interval)) != null; + return taskActionClient.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null; } @JsonProperty diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index 67bf7227d3e4..8bd0870cd256 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -26,14 +26,15 @@ import com.google.common.base.Preconditions; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockListAction; +import io.druid.indexing.common.actions.TaskActionClient; import io.druid.query.Query; import io.druid.query.QueryRunner; import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.IOException; +import java.util.List; import java.util.Map; public abstract class AbstractTask implements Task @@ -201,9 +202,9 @@ public int hashCode() return id.hashCode(); } - protected Iterable getTaskLocks(TaskToolbox toolbox) throws IOException + protected List getTaskLocks(TaskActionClient client) throws IOException { - return toolbox.getTaskActionClient().submit(new LockListAction()); + return client.submit(new LockListAction()); } @Override @@ -212,11 +213,4 @@ public Map getContext() { return context; } - - @Override - public Object getContextValue(String key) - { - return context == null ? null : context.get(key); - } - } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java index ffb9ddfe536d..0a45a4f27da8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; - import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -65,7 +64,7 @@ public String getType() public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); if (!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java index b223bdabcf11..18d42fd248c5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java @@ -27,7 +27,6 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentInsertAction; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 5f4833ee5392..2a1ef48288e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -29,7 +29,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import io.druid.common.utils.JodaUtils; import io.druid.indexer.HadoopDruidDetermineConfigurationJob; import io.druid.indexer.HadoopDruidIndexerConfig; @@ -38,12 +37,14 @@ import io.druid.indexer.Jobby; import io.druid.indexer.MetadataStorageUpdaterJobHandler; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.hadoop.OverlordActionBasedUsedSegmentLister; +import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.timeline.DataSegment; @@ -120,6 +121,12 @@ public HadoopIndexTask( this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "null ObjectMappper"); } + @Override + public int getPriority() + { + return Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY; + } + @Override public String getType() { @@ -136,7 +143,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception intervals.get() ) ); - return taskActionClient.submit(new LockTryAcquireAction(interval)) != null; + return taskActionClient.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null; } else { return true; } @@ -198,10 +205,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception indexerSchema.getDataSchema().getGranularitySpec().bucketIntervals().get() ) ); - TaskLock lock = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval)); - version = lock.getVersion(); + final LockResult lockResult = toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, interval) + ); + Tasks.checkLockResult(lockResult, interval); + version = lockResult.getTaskLock().getVersion(); } else { - Iterable locks = getTaskLocks(toolbox); + Iterable locks = getTaskLocks(toolbox.getTaskActionClient()); final TaskLock myLock = Iterables.getOnlyElement(locks); version = myLock.getVersion(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index a091d4b84b2b..6b39b225ced6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -19,7 +19,6 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -43,15 +42,12 @@ import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.data.input.Rows; -import io.druid.guice.annotations.Smile; import io.druid.hll.HyperLogLogCollector; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockAcquireAction; -import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; @@ -100,8 +96,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.SortedSet; import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -126,21 +124,24 @@ private static String makeDataSource(IndexIngestionSpec ingestionSchema) @JsonIgnore private final IndexIngestionSpec ingestionSchema; - private final ObjectMapper smileMapper; @JsonCreator public IndexTask( @JsonProperty("id") final String id, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("spec") final IndexIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context, - @Smile @JacksonInject final ObjectMapper smileMapper + @JsonProperty("context") final Map context ) { super(makeId(id, ingestionSchema), null, taskResource, makeDataSource(ingestionSchema), context); this.ingestionSchema = ingestionSchema; - this.smileMapper = smileMapper; + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); } @Override @@ -152,11 +153,21 @@ public String getType() @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - Optional> intervals = ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals(); + final Optional> intervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals(); if (intervals.isPresent()) { - Interval interval = JodaUtils.umbrellaInterval(intervals.get()); - return taskActionClient.submit(new LockTryAcquireAction(interval)) != null; + final List locks = getTaskLocks(taskActionClient); + if (locks.size() == 0) { + try { + Tasks.acquireLocks(taskActionClient, intervals.get()); + } + catch (Exception e) { + return false; + } + } + return true; } else { return true; } @@ -189,12 +200,15 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); - final String version; final DataSchema dataSchema; + final Map versions; if (determineIntervals) { - Interval interval = JodaUtils.umbrellaInterval(shardSpecs.getIntervals()); - TaskLock lock = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval)); - version = lock.getVersion(); + final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + intervals.addAll(shardSpecs.getIntervals()); + final Map locks = Tasks.acquireLocks(toolbox.getTaskActionClient(), intervals); + versions = locks.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); + dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( ingestionSchema.getDataSchema() .getGranularitySpec() @@ -205,17 +219,32 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ) ); } else { - version = Iterables.getOnlyElement(getTaskLocks(toolbox)).getVersion(); + versions = getTaskLocks(toolbox.getTaskActionClient()).stream() + .collect( + Collectors.toMap( + TaskLock::getInterval, + TaskLock::getVersion + ) + ); dataSchema = ingestionSchema.getDataSchema(); } - if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory, firehoseTempDir)) { + if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, versions, firehoseFactory, firehoseTempDir)) { return TaskStatus.success(getId()); } else { return TaskStatus.failure(getId()); } } + private static String findVersion(Map versions, Interval interval) + { + return versions.entrySet().stream() + .filter(entry -> entry.getKey().contains(interval)) + .map(Entry::getValue) + .findFirst() + .orElse(null); + } + private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { Preconditions.checkState( @@ -565,7 +594,7 @@ private boolean generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, final ShardSpecs shardSpecs, - final String version, + Map versions, final FirehoseFactory firehoseFactory, final File firehoseTempDir ) throws IOException, InterruptedException @@ -597,7 +626,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final SegmentAllocator segmentAllocator; if (ioConfig.isAppendToExisting()) { - segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); + segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema, false); } else { segmentAllocator = (row, sequenceName, previousSegmentId) -> { final DateTime timestamp = row.getTimestamp(); @@ -612,12 +641,12 @@ dataSchema, new RealtimeIOConfig(null, null, null), null throw new ISE("Could not find shardSpec for interval[%s]", interval); } - return new SegmentIdentifier(getDataSource(), interval, version, shardSpec); + return new SegmentIdentifier(getDataSource(), interval, findVersion(versions, interval), shardSpec); }; } final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments, null, null); + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); return toolbox.getTaskActionClient().submit(action).isSuccess(); }; @@ -654,7 +683,8 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final Interval interval = optInterval.get(); final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow); - final String sequenceName = Appenderators.getSequenceName(interval, version, shardSpec); + // TODO: validate versions if appendToExisting? => need to test + final String sequenceName = Appenderators.getSequenceName(interval, findVersion(versions, interval), shardSpec); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); if (addResult.isOk()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java index 9b1b6b736efe..6873f665f128 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java @@ -69,7 +69,7 @@ public String getType() public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); if (!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 6d9292db6249..d5a91a5c1584 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -124,10 +124,16 @@ public boolean apply(@Nullable DataSegment segment) ); } + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_MERGE_TASK_PRIORITY); + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 1680726de791..e1b221f2bfb3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -73,7 +73,7 @@ public String getType() public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java index 487b8e401eee..823e96da1542 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.indexing.common.TaskStatus; @@ -152,4 +154,16 @@ public static NoopTask create() { return new NoopTask(null, 0, 0, null, null, null); } + + @VisibleForTesting + public static NoopTask create(int priority) + { + return new NoopTask(null, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); + } + + @VisibleForTesting + public static NoopTask create(String id, int priority) + { + return new NoopTask(id, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f55b7f2471fd..e93c55f7f17e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -31,12 +31,13 @@ import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; -import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockReleaseAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.query.DruidMetrics; @@ -153,6 +154,12 @@ public RealtimeIndexTask( this.spec = fireDepartment; } + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + } + @Override public String getType() { @@ -213,7 +220,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception public void announceSegment(final DataSegment segment) throws IOException { // Side effect: Calling announceSegment causes a lock to be acquired - toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); + final LockResult lockResult = toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval()) + ); + Tasks.checkLockResult(lockResult, segment.getInterval()); toolbox.getSegmentAnnouncer().announceSegment(segment); } @@ -233,7 +243,10 @@ public void announceSegments(Iterable segments) throws IOException { // Side effect: Calling announceSegments causes locks to be acquired for (DataSegment segment : segments) { - toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); + final LockResult lockResult = toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval()) + ); + Tasks.checkLockResult(lockResult, segment.getInterval()); } toolbox.getSegmentAnnouncer().announceSegments(segments); } @@ -265,10 +278,10 @@ public String getVersion(final Interval interval) { try { // Side effect: Calling getVersion causes a lock to be acquired - final TaskLock myLock = toolbox.getTaskActionClient() - .submit(new LockAcquireAction(interval)); - - return myLock.getVersion(); + final LockResult lockResult = toolbox.getTaskActionClient() + .submit(new LockAcquireAction(TaskLockType.EXCLUSIVE, interval)); + Tasks.checkLockResult(lockResult, interval); + return lockResult.getTaskLock().getVersion(); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java index 736273a29192..680e50954cf7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java @@ -65,7 +65,7 @@ public String getType() public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); if (!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 7f57c0e7662b..38f4bc804bae 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -27,6 +27,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; +import javax.annotation.Nullable; import java.util.Map; /** @@ -78,6 +79,19 @@ public interface Task */ public String getGroupId(); + /** + * Returns task priority. The task priority is currently used only for prioritized locking, but, in the future, it can + * be used for task scheduling, cluster resource management, etc. + * + * @return task priority + * + * @see Tasks for default task priorities + */ + default int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_TASK_PRIORITY); + } + /** * Returns a {@link io.druid.indexing.common.task.TaskResource} for this task. Task resources define specific * worker requirements a task may require. @@ -166,6 +180,16 @@ public interface Task public Map getContext(); - public Object getContextValue(String key); + @Nullable + default ContextValueType getContextValue(String key) + { + return getContext() == null ? null : (ContextValueType) getContext().get(key); + } + + default ContextValueType getContextValue(String key, ContextValueType defaultValue) + { + final ContextValueType value = getContextValue(key); + return value == null ? defaultValue : value; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java new file mode 100644 index 000000000000..1309333edf6c --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java @@ -0,0 +1,90 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import io.druid.common.utils.JodaUtils; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.actions.LockTryAcquireAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.overlord.LockResult; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.Comparators; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +public class Tasks +{ + public static final int DEFAULT_REALTIME_TASK_PRIORITY = 75; + public static final int DEFAULT_BATCH_INDEX_TASK_PRIORITY = 50; + public static final int DEFAULT_MERGE_TASK_PRIORITY = 25; + public static final int DEFAULT_TASK_PRIORITY = 0; + + public static final String PRIORITY_KEY = "priority"; + + public static void checkLockResult(LockResult result, Interval interval) + { + if (!result.isOk()) { + throw new ISE("Failed to lock for interval[%s]", interval); + } + } + + public static Map acquireLocks(TaskActionClient client, SortedSet intervals) + throws IOException + { + final Map lockMap = new HashMap<>(); + for (Interval interval : computeCompactIntervals(intervals)) { + final LockResult lockResult = client.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)); + checkLockResult(lockResult, interval); + lockMap.put(interval, lockResult.getTaskLock()); + } + return lockMap; + } + + public static SortedSet computeCompactIntervals(SortedSet intervals) + { + final SortedSet compactIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + List toBeAccumulated = null; + for (Interval interval : intervals) { + if (toBeAccumulated == null) { + toBeAccumulated = new ArrayList<>(); + toBeAccumulated.add(interval); + } else { + if (toBeAccumulated.get(toBeAccumulated.size() - 1).abuts(interval)) { + toBeAccumulated.add(interval); + } else { + compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated)); + toBeAccumulated = null; + } + } + } + if (toBeAccumulated != null) { + compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated)); + } + return compactIntervals; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java index 52237f654a77..4cef406ce1eb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -29,7 +29,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; import com.google.inject.Inject; - import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskAction; @@ -193,6 +192,7 @@ public void addLock(final String taskid, final TaskLock taskLock) giant.lock(); try { + Preconditions.checkNotNull(taskid, "taskid"); Preconditions.checkNotNull(taskLock, "taskLock"); taskLocks.put(taskid, taskLock); } finally { @@ -200,6 +200,27 @@ public void addLock(final String taskid, final TaskLock taskLock) } } + @Override + public void replaceLock(String taskid, TaskLock oldLock, TaskLock newLock) + { + giant.lock(); + + try { + Preconditions.checkNotNull(taskid, "taskid"); + Preconditions.checkNotNull(oldLock, "oldLock"); + Preconditions.checkNotNull(newLock, "newLock"); + + if (!taskLocks.remove(taskid, oldLock)) { + log.warn("taskLock[%s] for replacement is not found for task[%s]", oldLock, taskid); + } + + taskLocks.put(taskid, newLock); + } + finally { + giant.unlock(); + } + } + @Override public void removeLock(final String taskid, final TaskLock taskLock) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java b/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java new file mode 100644 index 000000000000..4c3ec8b61f83 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java @@ -0,0 +1,83 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.overlord; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.Task; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +/** + * This class represents the result of {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}. If the lock + * acquisition fails, the callers can tell that it was failed because it was preempted by other locks of higher + * priorities or not by checking the {@link #wasRevoked} flag. + * + * The {@link #wasRevoked} flag means that consecutive lock acquisitions for the same dataSource and interval are + * returning different locks because another lock of a higher priority preempted your lock at some point. In this case, + * the lock acquisition must fail. + * + * @see TaskLockbox#tryLock(TaskLockType, Task, Interval) + */ +public class LockResult +{ + private final TaskLock taskLock; + private final boolean wasRevoked; + + public static LockResult ok(TaskLock taskLock) + { + return new LockResult(taskLock, false); + } + + public static LockResult fail(boolean wasRevoked) + { + return new LockResult(null, wasRevoked); + } + + @JsonCreator + public LockResult( + @JsonProperty("taskLock") @Nullable TaskLock taskLock, + @JsonProperty("wasRevoked") boolean wasRevoked + ) + { + this.taskLock = taskLock; + this.wasRevoked = wasRevoked; + } + + @JsonProperty("taskLock") + public TaskLock getTaskLock() + { + return taskLock; + } + + @JsonProperty("wasRevoked") + public boolean isWasRevoked() + { + return wasRevoked; + } + + public boolean isOk() + { + return taskLock != null; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java index 95cff9a35fb8..1e9684973689 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java @@ -246,6 +246,26 @@ public void addLock(final String taskid, final TaskLock taskLock) handler.addLock(taskid, taskLock); } + @Override + public void replaceLock(String taskid, TaskLock oldLock, TaskLock newLock) + { + Preconditions.checkNotNull(taskid, "taskid"); + Preconditions.checkNotNull(oldLock, "oldLock"); + Preconditions.checkNotNull(newLock, "newLock"); + + log.info( + "Replacing lock on interval[%s] version[%s] for task: %s", + oldLock.getInterval(), + oldLock.getVersion(), + taskid + ); + + // Even though these two operations are not atomically executed, the caller of replaceLock() is thread-safe and + // guarantees that two or more threads never call replaceLock() at the same time + removeLock(taskid, oldLock); + addLock(taskid, newLock); + } + @Override public void removeLock(String taskid, TaskLock taskLockToRemove) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 7d00532d1f72..72933c7b4f48 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -22,9 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Objects; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -36,17 +34,20 @@ import com.metamx.emitter.EmittingLogger; import io.druid.common.utils.JodaUtils; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Comparators; -import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.server.initialization.ServerConfig; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -56,6 +57,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * Remembers which activeTasks have locked which intervals. Tasks are permitted to lock an interval if no other task @@ -64,8 +68,11 @@ */ public class TaskLockbox { - // Datasource -> Interval -> Tasks + TaskLock - private final Map> running = Maps.newHashMap(); + // Datasource -> Interval -> list of (Tasks + TaskLock) + // Multiple shared locks can be acquired for the same dataSource and interval + // Note that revoked locks are also maintained in this map to notify that those locks are revoked to the callers when + // they acquire the same locks again or request lock upgrade/downgrade. + private final Map>> running = Maps.newHashMap(); private final TaskStorage taskStorage; private final ReentrantLock giant = new ReentrantLock(true); private final Condition lockReleaseCondition = giant.newCondition(); @@ -140,13 +147,14 @@ public int compare(Pair left, Pair right) continue; } - final TaskLockPosse taskLockPosse = tryAddTaskToLockPosse( + final TaskLockPosse taskLockPosse = createOrFindLockPosse( task, savedTaskLock.getInterval(), - Optional.of(savedTaskLock.getVersion()) + savedTaskLock.getVersion(), + savedTaskLock.getType() ); if (taskLockPosse != null) { - taskLockPosse.getTaskIds().add(task.getId()); + taskLockPosse.addTask(task); final TaskLock taskLock = taskLockPosse.getTaskLock(); @@ -198,36 +206,39 @@ public int compare(Pair left, Pair right) * * @throws java.lang.InterruptedException if the lock cannot be acquired */ - public TaskLock lock(final Task task, final Interval interval) throws InterruptedException + public LockResult lock( + final TaskLockType lockType, + final Task task, + final Interval interval + ) throws InterruptedException { long timeout = lockTimeoutMillis; - giant.lock(); + giant.lockInterruptibly(); try { - Optional taskLock; - while (!(taskLock = tryLock(task, interval)).isPresent()) { + LockResult lockResult; + while (!(lockResult = tryLock(lockType, task, interval)).isOk()) { long startTime = System.currentTimeMillis(); - lockReleaseCondition.await(timeout, TimeUnit.MILLISECONDS); + if (lockTimeoutMillis == 0) { + lockReleaseCondition.await(); + } else { + lockReleaseCondition.await(timeout, TimeUnit.MILLISECONDS); + } long timeDelta = System.currentTimeMillis() - startTime; if (timeDelta >= timeout) { - log.error( + log.info( "Task [%s] can not acquire lock for interval [%s] within [%s] ms", task.getId(), interval, lockTimeoutMillis ); - throw new InterruptedException(String.format( - "Task [%s] can not acquire lock for interval [%s] within [%s] ms", - task.getId(), - interval, - lockTimeoutMillis - )); + return lockResult; } else { timeout -= timeDelta; } } - return taskLock.get(); + return lockResult; } finally { giant.unlock(); @@ -235,35 +246,21 @@ public TaskLock lock(final Task task, final Interval interval) throws Interrupte } /** - * Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock} - * with no preferred version. - * - * @param task task that wants a lock - * @param interval interval to lock + * Attempt to acquire a lock for a task, without removing it from the queue. Can safely be called multiple times on + * the same task until the lock is preempted. * - * @return lock version if lock was acquired, absent otherwise - * @throws IllegalStateException if the task is not a valid active task - */ - public Optional tryLock(final Task task, final Interval interval) - { - return tryLock(task, interval, Optional.absent()); - } - - /** - * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. - * This method will attempt to assign version strings that obey the invariant that every version string is - * lexicographically greater than any other version string previously assigned to the same interval. This invariant - * is only mostly guaranteed, however; we assume clock monotonicity and we assume that callers specifying - * {@code preferredVersion} are doing the right thing. - * - * @param task task that wants a lock - * @param interval interval to lock - * @param preferredVersion use this version string if one has not yet been assigned + * @param task task that wants a lock + * @param interval interval to lock + * @param lockType type of lock to be acquired * * @return lock version if lock was acquired, absent otherwise * @throws IllegalStateException if the task is not a valid active task */ - private Optional tryLock(final Task task, final Interval interval, final Optional preferredVersion) + public LockResult tryLock( + final TaskLockType lockType, + final Task task, + final Interval interval + ) { giant.lock(); @@ -273,16 +270,18 @@ private Optional tryLock(final Task task, final Interval interval, fin } Preconditions.checkArgument(interval.toDurationMillis() > 0, "interval empty"); - final TaskLockPosse posseToUse = tryAddTaskToLockPosse(task, interval, preferredVersion); - if (posseToUse != null) { + final TaskLockPosse posseToUse = createOrFindLockPosse(task, interval, lockType); + if (posseToUse != null && !posseToUse.getTaskLock().isRevoked()) { // Add to existing TaskLockPosse, if necessary - if (posseToUse.getTaskIds().add(task.getId())) { + if (posseToUse.addTask(task)) { log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); // Update task storage facility. If it fails, revoke the lock. try { taskStorage.addLock(task.getId(), posseToUse.getTaskLock()); - return Optional.of(posseToUse.getTaskLock()); + // The task newly acquired a lock even though the lock itself might not be a new one. + // The revokedState is maintained per task, so it should be updated here. + return LockResult.ok(posseToUse.getTaskLock()); } catch(Exception e) { log.makeAlert("Failed to persist lock in storage") .addData("task", task.getId()) @@ -291,90 +290,352 @@ private Optional tryLock(final Task task, final Interval interval, fin .addData("version", posseToUse.getTaskLock().getVersion()) .emit(); unlock(task, interval); - return Optional.absent(); + return LockResult.fail(false); } } else { log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); - return Optional.of(posseToUse.getTaskLock()); + return LockResult.ok(posseToUse.getTaskLock()); } - } else { - return Optional.absent(); + final boolean lockRevoked = posseToUse != null && posseToUse.getTaskLock().isRevoked(); + return LockResult.fail(lockRevoked); } } finally { giant.unlock(); } + } + private TaskLockPosse createOrFindLockPosse( + final Task task, + final Interval interval, + final TaskLockType lockType + ) + { + return createOrFindLockPosse(task, interval, null, lockType); } - private TaskLockPosse tryAddTaskToLockPosse( + /** + * Create a new {@link TaskLockPosse} or find an existing one for the given task and interval. + * + * @param task task acquiring a lock + * @param interval interval to be locked + * @param preferredVersion a preferred version string + * @param lockType type of lock to be acquired + * + * @return a lock posse + * + * @see #createNewTaskLockPosse(TaskLockType, String, String, Interval, String, int) + */ + private TaskLockPosse createOrFindLockPosse( final Task task, final Interval interval, - final Optional preferredVersion + @Nullable final String preferredVersion, // TODO: check this + final TaskLockType lockType ) { giant.lock(); try { final String dataSource = task.getDataSource(); - final List foundPosses = findLockPossesForInterval(dataSource, interval); - final TaskLockPosse posseToUse; + final int priority = task.getPriority(); + final List foundPosses = findLockPossesOverlapsInterval(dataSource, interval); + + // If we have some locks for dataSource and interval, check they can be reused. + // If they can't be reused, check lock priority and revoke existing locks. + if (foundPosses.size() > 0) { + + final List possesForReusableLocks = foundPosses.stream() + .filter(lockPosse -> + matchGroupIdAndInterval( + lockPosse.getTaskLock(), + task, + interval + ) + ).collect(Collectors.toList()); + + if (lockType.equals(TaskLockType.SHARED) && isAllSharedLocks(foundPosses)) { + + if (possesForReusableLocks.size() == 0) { + // Any number of shared locks can be acquired for the same dataSource and interval. + return createNewTaskLockPosse( + lockType, + task.getGroupId(), + dataSource, + interval, + preferredVersion, + priority + ); + } else if (possesForReusableLocks.size() == 1) { + return possesForReusableLocks.get(0); + } else { + throw new ISE( + "Task group[%s] has multiple locks for the same interval[%s]?", + task.getGroupId(), + interval + ); + } + } else { + if (possesForReusableLocks.size() == 1 && + matchGroupIdAndInterval(possesForReusableLocks.get(0).taskLock, task, interval)) { + final TaskLockPosse foundPosse = possesForReusableLocks.get(0); + if (lockType.equals(foundPosse.getTaskLock().getType())) { + return foundPosse; + } else { + throw new ISE( + "Task[%s] already acquired a lock for interval[%s] but different type[%s]", + task.getId(), + interval, + foundPosse.getTaskLock().getType() + ); + } + } else if (isAllRevocable(foundPosses, priority)) { + // Revoke all existing locks + foundPosses.forEach(this::revokeLock); + + return createNewTaskLockPosse( + lockType, + task.getGroupId(), + dataSource, + interval, + preferredVersion, + priority + ); + } else { + log.info("Cannot create a new taskLockPosse because some locks of higher priorities exist"); + return null; + } + } + } else { + // We don't have any locks for dataSource and interval. + // Let's make a new one. + return createNewTaskLockPosse( + lockType, + task.getGroupId(), + dataSource, + interval, + preferredVersion, + priority + ); + } + } + finally { + giant.unlock(); + } + } + + /** + * Create a new {@link TaskLockPosse} for a new {@link TaskLock}. This method will attempt to assign version strings + * that obey the invariant that every version string is lexicographically greater than any other version string + * previously assigned to the same interval. This invariant is only mostly guaranteed, however; we assume clock + * monotonicity and that callers specifying {@code preferredVersion} are doing the right thing. + * + * @param lockType lock type + * @param groupId group id of task + * @param dataSource data source of task + * @param interval interval to be locked + * @param preferredVersion preferred version string + * @param priority lock priority + * + * @return a new {@link TaskLockPosse} + */ + private TaskLockPosse createNewTaskLockPosse( + TaskLockType lockType, + String groupId, + String dataSource, + Interval interval, + @Nullable String preferredVersion, + int priority + ) + { + giant.lock(); + try { + // Create new TaskLock and assign it a version. + // Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This + // may not always be true, unfortunately. See below.) - if (foundPosses.size() > 1) { + final String version; + + if (preferredVersion != null) { + // We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it. + version = preferredVersion; + } else { + // We are running under an interval lock right now, so just using the current time works as long as we can trust + // our clock to be monotonic and have enough resolution since the last time we created a TaskLock for the same + // interval. This may not always be true; to assure it we would need to use some method of timekeeping other + // than the wall clock. + version = new DateTime().toString(); + } - // Too many existing locks. - return null; + final TaskLockPosse posseToUse = new TaskLockPosse( + new TaskLock(lockType, groupId, dataSource, interval, version, priority) + ); + running.computeIfAbsent(dataSource, k -> new TreeMap<>(Comparators.intervalsByStartThenEnd())) + .computeIfAbsent(interval, k -> new ArrayList<>()) + .add(posseToUse); - } else if (foundPosses.size() == 1) { + return posseToUse; + } + finally { + giant.unlock(); + } + } + + /** + * Upgrade a lock for the given task and interval if possible. Shared locks and revoked locks cannot be upgraded. + * + * @return a succeeded {@link LockResult} if the lock is upgraded. If an upgrade for a revoked lock is requested, a + * failed {@link LockResult} is returned. + * + * @throws IllegalStateException if an upgrade for a shared lock is requested + */ + public LockResult upgrade(Task task, Interval interval) + { + giant.lock(); + + try { + final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosse( + task, + interval, + findLockPossesContainingInterval(task.getDataSource(), interval) + ); + + final TaskLock lock = taskLockPosseToUpdate.getTaskLock(); + + Preconditions.checkState( + lock.getType().equals(TaskLockType.EXCLUSIVE), + "Shared lock cannot be upgraded" + ); - // One existing lock -- check if we can add to it. + if (lock.isRevoked()) { + return LockResult.fail(true); + } else { + log.info("Upgrading taskLock[%s] for task[%s] and interval[%s]", lock, task.getId(), interval); - final TaskLockPosse foundPosse = Iterables.getOnlyElement(foundPosses); - if (foundPosse.getTaskLock().getInterval().contains(interval) && foundPosse.getTaskLock().getGroupId().equals(task.getGroupId())) { - posseToUse = foundPosse; + if (lock.isUpgraded()) { + log.warn("TaskLock[%s] is already upgraded", lock); + return LockResult.ok(lock); } else { - //Could be a deadlock for LockAcquireAction: same task trying to acquire lock for overlapping interval - if (foundPosse.getTaskIds().contains(task.getId())) { - log.makeAlert("Same Task is trying to acquire lock for overlapping interval") - .addData("task", task.getId()) - .addData("interval", interval); - } - return null; + final TaskLock upgradedLock = lock.upgrade(); + taskStorage.replaceLock(task.getId(), lock, upgradedLock); + + // Possible exceptions are already checked above + final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); + + possesHolder.remove(taskLockPosseToUpdate); + possesHolder.add(taskLockPosseToUpdate.withTaskLock(upgradedLock)); + log.info("Upgraded taskLock[%s]"); + return LockResult.ok(upgradedLock); } + } + } + finally { + giant.unlock(); + } + } + /** + * Downgrade a lock for the given task and interval if possible. Shared locks cannot be downgraded. + * + * @return a downgraded lock if succeeded. + * + * @throws IllegalStateException if a downgrade for a shared lock is requested + */ + public TaskLock downgrade(Task task, Interval interval) + { + giant.lock(); + + try { + final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosse( + task, + interval, + findLockPossesContainingInterval(task.getDataSource(), interval) + ); + final TaskLock lock = taskLockPosseToUpdate.taskLock; + + Preconditions.checkState( + lock.getType().equals(TaskLockType.EXCLUSIVE), + "Shared lock cannot be downgraded" + ); + + // Upgraded locks cannot be revoked, so ignore the check to verify the lock is revoked. + log.info("Downgrading taskLock[%s] for task[%s] and interval[%s]", lock, task.getId(), interval); + + if (!lock.isUpgraded()) { + log.warn("TaskLock[%s] is already downgraded", lock); + return lock; } else { + final TaskLock downgradedLock = lock.downgrade(); + taskStorage.replaceLock(task.getId(), lock, downgradedLock); - // No existing locks. We can make a new one. - if (!running.containsKey(dataSource)) { - running.put(dataSource, new TreeMap(Comparators.intervalsByStartThenEnd())); - } + // Possible exceptions are already checked above + final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); - // Create new TaskLock and assign it a version. - // Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This - // may not always be true, unfortunately. See below.) + possesHolder.remove(taskLockPosseToUpdate); + possesHolder.add(taskLockPosseToUpdate.withTaskLock(downgradedLock)); + log.info("Downgraded taskLock[%s]"); + return downgradedLock; + } + } + finally { + giant.unlock(); + } + } - final String version; + private void revokeLock(TaskLockPosse lockPosse) + { + giant.lock(); - if (preferredVersion.isPresent()) { - // We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it. - version = preferredVersion.get(); - } else { - // We are running under an interval lock right now, so just using the current time works as long as we can trust - // our clock to be monotonic and have enough resolution since the last time we created a TaskLock for the same - // interval. This may not always be true; to assure it we would need to use some method of timekeeping other - // than the wall clock. - version = new DateTime().toString(); - } + try { + lockPosse.forEachTask(taskId -> revokeLock(taskId, lockPosse.getTaskLock())); + } + finally { + giant.unlock(); + } + } - posseToUse = new TaskLockPosse(new TaskLock(task.getGroupId(), dataSource, interval, version)); - running.get(dataSource) - .put(interval, posseToUse); + /** + * Mark the lock as revoked. Note that revoked locks are NOT removed. Instead, they are maintained in {@link #running} + * and {@link #taskStorage} as the normal locks do. This is to check locks are revoked when they are requested to be + * acquired or upgraded and notify to the callers if revoked. Revoked locks are removed by calling + * {@link #unlock(Task, Interval)}. + * + * @param taskId an id of the task holding the lock + * @param lock lock to be revoked + */ + private void revokeLock(String taskId, TaskLock lock) + { + giant.lock(); - log.info("Created new TaskLockPosse: %s", posseToUse); + try { + if (!activeTasks.contains(taskId)) { + throw new ISE("Cannot revoke lock for inactive task[%s]", taskId); } - return posseToUse; + final Task task = taskStorage.getTask(taskId).orNull(); + if (task == null) { + throw new ISE("Cannot revoke lock for unknown task[%s]", taskId); + } + + log.info("Revoking task lock[%s] for task[%s]", lock, taskId); + + if (lock.isRevoked()) { + log.warn("TaskLock[%s] is already revoked", lock); + } else { + final TaskLock revokedLock = lock.revoke(); + taskStorage.replaceLock(taskId, lock, revokedLock); + + final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); + final TaskLockPosse foundPosse = possesHolder.stream() + .filter(posse -> posse.getTaskLock().equals(lock)) + .findFirst() + .orElseThrow( + () -> new ISE("Failed to find lock posse for lock[%s]", lock) + ); + possesHolder.remove(foundPosse); + possesHolder.add(foundPosse.withTaskLock(revokedLock)); + log.info("Revoked taskLock[%s]", lock); + } } finally { giant.unlock(); @@ -420,51 +681,62 @@ public void unlock(final Task task, final Interval interval) try { final String dataSource = task.getDataSource(); - final NavigableMap dsRunning = running.get(dataSource); + final NavigableMap> dsRunning = running.get(task.getDataSource()); + + if (dsRunning == null || dsRunning.isEmpty()) { + return; + } - // So we can alert if activeTasks try to release stuff they don't have - boolean removed = false; + final List possesHolder = dsRunning.get(interval); + if (possesHolder == null || possesHolder.isEmpty()) { + return; + } - if(dsRunning != null) { - final TaskLockPosse taskLockPosse = dsRunning.get(interval); - if(taskLockPosse != null) { - final TaskLock taskLock = taskLockPosse.getTaskLock(); + final List posses = possesHolder.stream() + .filter(posse -> posse.containsTask(task)) + .collect(Collectors.toList()); - // Remove task from live list - log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId()); - removed = taskLockPosse.getTaskIds().remove(task.getId()); + for (TaskLockPosse taskLockPosse : posses) { + final TaskLock taskLock = taskLockPosse.getTaskLock(); - if (taskLockPosse.getTaskIds().isEmpty()) { - log.info("TaskLock is now empty: %s", taskLock); - running.get(dataSource).remove(taskLock.getInterval()); - } + // Remove task from live list + log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId()); + final boolean removed = taskLockPosse.removeTask(task); - if (running.get(dataSource).size() == 0) { - running.remove(dataSource); - } + if (taskLockPosse.isTasksEmpty()) { + log.info("TaskLock is now empty: %s", taskLock); + possesHolder.remove(taskLockPosse); + } - // Wake up blocking-lock waiters - lockReleaseCondition.signalAll(); + if (possesHolder.size() == 0) { + dsRunning.remove(interval); + } - // Remove lock from storage. If it cannot be removed, just ignore the failure. - try { - taskStorage.removeLock(task.getId(), taskLock); - } catch(Exception e) { - log.makeAlert(e, "Failed to clean up lock from storage") - .addData("task", task.getId()) - .addData("dataSource", taskLock.getDataSource()) - .addData("interval", taskLock.getInterval()) - .addData("version", taskLock.getVersion()) - .emit(); - } + if (running.get(dataSource).size() == 0) { + running.remove(dataSource); } - } - if(!removed) { - log.makeAlert("Lock release without acquire") - .addData("task", task.getId()) - .addData("interval", interval) - .emit(); + // Wake up blocking-lock waiters + lockReleaseCondition.signalAll(); + + // Remove lock from storage. If it cannot be removed, just ignore the failure. + try { + taskStorage.removeLock(task.getId(), taskLock); + } catch(Exception e) { + log.makeAlert(e, "Failed to clean up lock from storage") + .addData("task", task.getId()) + .addData("dataSource", taskLock.getDataSource()) + .addData("interval", taskLock.getInterval()) + .addData("version", taskLock.getVersion()) + .emit(); + } + + if(!removed) { + log.makeAlert("Lock release without acquire") + .addData("task", task.getId()) + .addData("interval", interval) + .emit(); + } } } finally { giant.unlock(); @@ -505,28 +777,31 @@ private List findLockPossesForTask(final Task task) giant.lock(); try { - final Iterable searchSpace; - // Scan through all locks for this datasource - final NavigableMap dsRunning = running.get(task.getDataSource()); + final NavigableMap> dsRunning = running.get(task.getDataSource()); if(dsRunning == null) { - searchSpace = ImmutableList.of(); + return ImmutableList.of(); } else { - searchSpace = dsRunning.values(); + return dsRunning.values().stream() + .flatMap(Collection::stream) + .filter(taskLockPosse -> taskLockPosse.containsTask(task)) + .collect(Collectors.toList()); } + } + finally { + giant.unlock(); + } + } - return ImmutableList.copyOf( - Iterables.filter( - searchSpace, new Predicate() - { - @Override - public boolean apply(TaskLockPosse taskLock) - { - return taskLock.getTaskIds().contains(task.getId()); - } - } - ) - ); + private List findLockPossesContainingInterval(final String dataSource, final Interval interval) + { + giant.lock(); + + try { + final List intervalOverlapsPosses = findLockPossesOverlapsInterval(dataSource, interval); + return intervalOverlapsPosses.stream() + .filter(taskLockPosse -> taskLockPosse.taskLock.getInterval().contains(interval)) + .collect(Collectors.toList()); } finally { giant.unlock(); @@ -536,12 +811,12 @@ public boolean apply(TaskLockPosse taskLock) /** * Return all locks that overlap some search interval. */ - private List findLockPossesForInterval(final String dataSource, final Interval interval) + private List findLockPossesOverlapsInterval(final String dataSource, final Interval interval) { giant.lock(); try { - final NavigableMap dsRunning = running.get(dataSource); + final NavigableMap> dsRunning = running.get(dataSource); if (dsRunning == null) { // No locks at all return Collections.emptyList(); @@ -561,30 +836,10 @@ private List findLockPossesForInterval(final String dataSource, f ) ); - return Lists.newArrayList( - FunctionalIterable - .create(searchIntervals) - .filter( - new Predicate() - { - @Override - public boolean apply(@Nullable Interval searchInterval) - { - return searchInterval != null && searchInterval.overlaps(interval); - } - } - ) - .transform( - new Function() - { - @Override - public TaskLockPosse apply(Interval interval) - { - return dsRunning.get(interval); - } - } - ) - ); + return StreamSupport.stream(searchIntervals.spliterator(), false) + .filter(searchInterval -> searchInterval != null && searchInterval.overlaps(interval)) + .flatMap(searchInterval -> dsRunning.get(searchInterval).stream()) + .collect(Collectors.toList()); } } finally { @@ -603,6 +858,69 @@ public void add(Task task) } } + private static boolean matchGroupIdAndInterval(TaskLock existingLock, Task task, Interval interval) + { + return existingLock.getInterval().contains(interval) && + existingLock.getGroupId().equals(task.getGroupId()); + } + + private static boolean isAllSharedLocks(List lockPosses) + { + return lockPosses.stream() + .allMatch(taskLockPosse -> taskLockPosse.getTaskLock().getType().equals(TaskLockType.SHARED)); + } + + private static boolean isAllRevocable(List lockPosses, int tryLockPriority) + { + return lockPosses.stream().allMatch(taskLockPosse -> isRevocable(taskLockPosse, tryLockPriority)); + } + + private static boolean isRevocable(TaskLockPosse lockPosse, int tryLockPriority) + { + final TaskLock existingLock = lockPosse.getTaskLock(); + return existingLock.getPriority() < tryLockPriority && !existingLock.isUpgraded(); + } + + private static TaskLockPosse getOnlyTaskLockPosse(Task task, Interval interval, List lockPosses) + { + final List filteredPosses = lockPosses.stream() + .filter(lockPosse -> lockPosse.containsTask(task)) + .collect(Collectors.toList()); + if (filteredPosses.isEmpty()) { + throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval); + } else if (filteredPosses.size() > 1) { + throw new ISE("There are multiple lockPosses for task[%s] and interval[%s]?", task.getId(), interval); + } else { + final TaskLockPosse foundPosse = filteredPosses.get(0); + if (foundPosse.containsTask(task)) { + return foundPosse; + } else { + throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval); + } + } + } + + /** + * Find a {@link TaskLockPosse} for a task and an exact interval. + */ + private List findLockPosse(Task task, Interval interval) + { + final NavigableMap> dsRunning = running.get(task.getDataSource()); + + if (dsRunning == null || dsRunning.isEmpty()) { + throw new ISE("Cannot find lock for dataSource[%s]", task.getDataSource()); + } + + final List possesHolder = dsRunning.get(interval); + if (possesHolder == null || possesHolder.isEmpty()) { + throw new ISE("Not found lock for task[%s] and interval[%s]", task.getId(), interval); + } + + return possesHolder.stream() + .filter(posse -> posse.containsTask(task)) + .collect(Collectors.toList()); + } + @VisibleForTesting Set getActiveTasks() { @@ -610,7 +928,7 @@ Set getActiveTasks() } @VisibleForTesting - Map> getAllLocks() + public Map>> getAllLocks() { return running; } @@ -620,20 +938,56 @@ static class TaskLockPosse final private TaskLock taskLock; final private Set taskIds; - public TaskLockPosse(TaskLock taskLock) + TaskLockPosse(TaskLock taskLock) { this.taskLock = taskLock; - taskIds = Sets.newHashSet(); + this.taskIds = new HashSet<>(); + } + + private TaskLockPosse(TaskLock taskLock, Set taskIds) + { + this.taskLock = taskLock; + this.taskIds = new HashSet<>(taskIds); + } + + TaskLockPosse withTaskLock(TaskLock taskLock) + { + return new TaskLockPosse(taskLock, taskIds); } - public TaskLock getTaskLock() + TaskLock getTaskLock() { return taskLock; } - public Set getTaskIds() + boolean addTask(Task task) + { + Preconditions.checkArgument(taskLock.getGroupId().equals(task.getGroupId())); + Preconditions.checkArgument(taskLock.getPriority() == task.getPriority()); + return taskIds.add(task.getId()); + } + + boolean containsTask(Task task) + { + Preconditions.checkNotNull(task, "task"); + return taskIds.contains(task.getId()); + } + + boolean removeTask(Task task) + { + Preconditions.checkNotNull(task, "task"); + return taskIds.remove(task.getId()); + } + + boolean isTasksEmpty() + { + return taskIds.isEmpty(); + } + + void forEachTask(Consumer action) { - return taskIds; + Preconditions.checkNotNull(action); + taskIds.forEach(action); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index dc5986092e00..d154989f4635 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -37,7 +37,7 @@ public interface TaskStorage * @param status task status * @throws io.druid.metadata.EntryExistsException if the task ID already exists */ - public void insert(Task task, TaskStatus status) throws EntryExistsException; + void insert(Task task, TaskStatus status) throws EntryExistsException; /** * Persists task status in the storage facility. This method should throw an exception if the task status lifecycle @@ -45,14 +45,23 @@ public interface TaskStorage * * @param status task status */ - public void setStatus(TaskStatus status); + void setStatus(TaskStatus status); /** * Persists lock state in the storage facility. * @param taskid task ID * @param taskLock lock state */ - public void addLock(String taskid, TaskLock taskLock); + void addLock(String taskid, TaskLock taskLock); + + /** + * Replace the old lock with the new lock. This method is not thread-safe. + * + * @param taskid an id of the task holding the old lock and new lock + * @param oldLock old lock + * @param newLock new lock + */ + void replaceLock(String taskid, TaskLock oldLock, TaskLock newLock); /** * Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but @@ -61,7 +70,7 @@ public interface TaskStorage * @param taskid task ID * @param taskLock lock state */ - public void removeLock(String taskid, TaskLock taskLock); + void removeLock(String taskid, TaskLock taskLock); /** * Returns task as stored in the storage facility. If the task ID does not exist, this will return an @@ -72,7 +81,7 @@ public interface TaskStorage * @param taskid task ID * @return optional task */ - public Optional getTask(String taskid); + Optional getTask(String taskid); /** * Returns task status as stored in the storage facility. If the task ID does not exist, this will return @@ -81,7 +90,7 @@ public interface TaskStorage * @param taskid task ID * @return task status */ - public Optional getStatus(String taskid); + Optional getStatus(String taskid); /** * Add an action taken by a task to the audit log. @@ -91,7 +100,7 @@ public interface TaskStorage * * @param task action return type */ - public void addAuditLog(Task task, TaskAction taskAction); + void addAuditLog(Task task, TaskAction taskAction); /** * Returns all actions taken by a task. @@ -99,7 +108,7 @@ public interface TaskStorage * @param taskid task ID * @return list of task actions */ - public List getAuditLogs(String taskid); + List getAuditLogs(String taskid); /** * Returns a list of currently running or pending tasks as stored in the storage facility. No particular order @@ -107,7 +116,7 @@ public interface TaskStorage * * @return list of active tasks */ - public List getActiveTasks(); + List getActiveTasks(); /** * Returns a list of recently finished task statuses as stored in the storage facility. No particular order @@ -116,7 +125,7 @@ public interface TaskStorage * * @return list of recently finished tasks */ - public List getRecentlyFinishedTaskStatuses(); + List getRecentlyFinishedTaskStatuses(); /** * Returns a list of locks for a particular task. @@ -124,5 +133,5 @@ public interface TaskStorage * @param taskid task ID * @return list of TaskLocks for the given task */ - public List getLocks(String taskid); + List getLocks(String taskid); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java index 50d756086317..a599319df6b9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java @@ -31,6 +31,7 @@ import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyFactory; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.jackson.DefaultObjectMapper; @@ -98,10 +99,12 @@ public String getAddress() long now = System.currentTimeMillis(); result = Collections.singletonList(new TaskLock( + TaskLockType.SHARED, "groupId", "dataSource", new Interval(now - 30 * 1000, now), - "version" + "version", + 0 )); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java index a587b38e1c14..e86d828e2345 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -689,7 +689,8 @@ public void testSerde() throws Exception Granularities.MINUTE, Granularities.HOUR, "s1", - "prev" + "prev", + true ); final ObjectMapper objectMapper = new DefaultObjectMapper(); @@ -721,7 +722,8 @@ private SegmentIdentifier allocate( queryGranularity, preferredSegmentGranularity, sequenceName, - sequencePreviousId + sequencePreviousId, + true ); return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index f189356d0dfd..1bf16dc14ba6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.timeline.DataSegment; @@ -90,7 +91,8 @@ public void testSimple() throws Exception final Task task = new NoopTask(null, 0, 0, null, null, null); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().upgrade(task, new Interval(INTERVAL)); action.perform(task, actionTestKit.getTaskActionToolbox()); Assert.assertEquals( @@ -108,7 +110,8 @@ public void testFailBadVersion() throws Exception final Task task = new NoopTask(null, 0, 0, null, null, null); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().upgrade(task, new Interval(INTERVAL)); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index c52daae392b4..6b3888cfd926 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.ObjectMetadata; @@ -89,7 +90,7 @@ public void testTransactional() throws Exception { final Task task = new NoopTask(null, 0, 0, null, null, null); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, new Interval(INTERVAL)); SegmentPublishResult result1 = new SegmentTransactionalInsertAction( ImmutableSet.of(SEGMENT1), @@ -130,7 +131,7 @@ public void testFailTransactional() throws Exception { final Task task = new NoopTask(null, 0, 0, null, null, null); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, new Interval(INTERVAL)); SegmentPublishResult result = new SegmentTransactionalInsertAction( ImmutableSet.of(SEGMENT1), @@ -150,7 +151,7 @@ public void testFailBadVersion() throws Exception final Task task = new NoopTask(null, 0, 0, null, null, null); final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, new Interval(INTERVAL)); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java new file mode 100644 index 000000000000..9e5baf497f3c --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class TaskActionPreconditionsTest +{ + private TaskLockbox lockbox; + private Task task; + private Set segments; + + @Before + public void setup() + { + lockbox = new TaskLockbox(new HeapMemoryTaskStorage(new TaskStorageConfig(null)), 5000); + task = NoopTask.create(); + lockbox.add(task); + + segments = ImmutableSet.of( + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(new Interval("2017-01-01/2017-01-02")) + .version(new DateTime().toString()) + .shardSpec(new LinearShardSpec(2)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(new Interval("2017-01-02/2017-01-03")) + .version(new DateTime().toString()) + .shardSpec(new LinearShardSpec(2)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(new Interval("2017-01-03/2017-01-04")) + .version(new DateTime().toString()) + .shardSpec(new LinearShardSpec(2)) + .build() + ); + } + + @Test + public void testCheckLockCoversSegments() throws Exception + { + final List intervals = ImmutableList.of( + new Interval("2017-01-01/2017-01-02"), + new Interval("2017-01-02/2017-01-03"), + new Interval("2017-01-03/2017-01-04") + ); + + final Map locks = intervals.stream().collect( + Collectors.toMap( + Function.identity(), + interval -> { + final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ) + ); + + Assert.assertTrue(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, false)); + Assert.assertFalse(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, true)); + + locks.forEach((interval, lock) -> lockbox.upgrade(task, interval)); + Assert.assertTrue(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, true)); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 0ce369cd1680..388def48eed0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -30,16 +30,19 @@ import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockListAction; +import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.overlord.LockResult; import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; @@ -143,8 +146,7 @@ public void testDeterminePartitions() throws Exception createTuningConfig(2, null, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -187,8 +189,7 @@ public void testForceExtendableShardSpecs() throws Exception createTuningConfig(2, null, true, false), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -227,13 +228,12 @@ public void testWithArbitraryGranularity() throws Exception null, new ArbitraryGranularitySpec( Granularities.MINUTE, - Collections.singletonList(new Interval("2014/2015")) + Collections.singletonList(new Interval("2014-01-01/2014-01-02")) ), createTuningConfig(10, null, false, true), false ), - null, - jsonMapper + null ); List segments = runTask(indexTask); @@ -249,8 +249,8 @@ public void testIntervalBucketing() throws Exception File tmpFile = File.createTempFile("druid", "index", tmpDir); try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { - writer.write("2015-03-01T07:59:59.977Z,a,1\n"); - writer.write("2015-03-01T08:00:00.000Z,b,1\n"); + writer.write("2014-01-01T07:59:59.977Z,a,1\n"); + writer.write("2014-01-01T08:00:00.000Z,b,1\n"); } IndexTask indexTask = new IndexTask( @@ -262,13 +262,12 @@ public void testIntervalBucketing() throws Exception new UniformGranularitySpec( Granularities.HOUR, Granularities.HOUR, - Collections.singletonList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z")) + Collections.singletonList(new Interval("2014-01-01T08:00:00Z/2014-01-01T09:00:00Z")) ), createTuningConfig(50, null, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -298,8 +297,7 @@ public void testNumShardsProvided() throws Exception createTuningConfig(null, 1, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -335,8 +333,7 @@ public void testAppendToExisting() throws Exception createTuningConfig(2, null, false, false), true ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -381,8 +378,7 @@ public void testIntervalNotSpecified() throws Exception createTuningConfig(2, null, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -442,8 +438,7 @@ public void testCSVFileWithHeader() throws Exception createTuningConfig(2, null, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -492,8 +487,7 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception createTuningConfig(2, null, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -537,8 +531,7 @@ public void testWithSmallMaxTotalRows() throws Exception createTuningConfig(2, 2, 2, null, false, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -580,8 +573,7 @@ public void testPerfectRollup() throws Exception createTuningConfig(3, 2, 2, null, false, true, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -622,8 +614,7 @@ public void testBestEffortRollup() throws Exception createTuningConfig(3, 2, 2, null, false, false, true), false ), - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -698,8 +689,7 @@ public void testIgnoreParseException() throws Exception null, null, parseExceptionIgnoreSpec, - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -752,8 +742,7 @@ public void testReportParseException() throws Exception null, null, parseExceptionIgnoreSpec, - null, - jsonMapper + null ); runTask(indexTask); @@ -812,8 +801,7 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception null, null, parseExceptionIgnoreSpec, - null, - jsonMapper + null ); final List segments = runTask(indexTask); @@ -883,8 +871,7 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception null, null, parseExceptionIgnoreSpec, - null, - jsonMapper + null ); runTask(indexTask); @@ -893,79 +880,101 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception private final List runTask(final IndexTask indexTask) throws Exception { final List segments = Lists.newArrayList(); - - indexTask.run( - new TaskToolbox( - null, new TaskActionClient() - { - @Override - public RetType submit(TaskAction taskAction) throws IOException - { - if (taskAction instanceof LockListAction) { - return (RetType) Collections.singletonList( - new TaskLock( - "", "", null, new DateTime().toString() - ) - ); - } - - if (taskAction instanceof LockAcquireAction) { - return (RetType) new TaskLock( + final TaskToolbox box = new TaskToolbox( + null, new TaskActionClient() + { + @Override + public RetType submit(TaskAction taskAction) throws IOException + { + if (taskAction instanceof LockListAction) { + return (RetType) Collections.singletonList( + new TaskLock( + TaskLockType.EXCLUSIVE, + "", + "", + new Interval("2014/P1D"), + new DateTime().toString(), + Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY + ) + ); + } + + if (taskAction instanceof LockAcquireAction) { + return (RetType) LockResult.ok( + new TaskLock( + TaskLockType.EXCLUSIVE, "groupId", "test", ((LockAcquireAction) taskAction).getInterval(), - new DateTime().toString() - ); - } - - if (taskAction instanceof SegmentTransactionalInsertAction) { - return (RetType) new SegmentPublishResult( - ((SegmentTransactionalInsertAction) taskAction).getSegments(), - true - ); - } - - if (taskAction instanceof SegmentAllocateAction) { - SegmentAllocateAction action = (SegmentAllocateAction) taskAction; - Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp()); - ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0); - return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec); - } - - return null; - } - }, null, new DataSegmentPusher() - { - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - return null; - } - - @Override - public DataSegment push(File file, DataSegment segment) throws IOException - { - segments.add(segment); - return segment; - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), - indexIO, null, null, indexMergerV9 - ) + new DateTime().toString(), + Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY + ) + ); + } + + if (taskAction instanceof LockTryAcquireAction) { + return (RetType) LockResult.ok( + new TaskLock( + TaskLockType.EXCLUSIVE, + "groupId", + "test", + ((LockTryAcquireAction) taskAction).getInterval(), + new DateTime().toString(), + Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY + ) + ); + } + + if (taskAction instanceof SegmentTransactionalInsertAction) { + return (RetType) new SegmentPublishResult( + ((SegmentTransactionalInsertAction) taskAction).getSegments(), + true + ); + } + + if (taskAction instanceof SegmentAllocateAction) { + SegmentAllocateAction action = (SegmentAllocateAction) taskAction; + Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp()); + ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0); + return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec); + } + + return null; + } + }, null, new DataSegmentPusher() + { + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } + + @Override + public String getPathForHadoop() + { + return null; + } + + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + segments.add(segment); + return segment; + } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), + indexIO, null, null, indexMergerV9 ); + indexTask.isReady(box.getTaskActionClient()); + indexTask.run(box); + Collections.sort(segments); return segments; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 4dfb87641576..9477b175729e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.actions.LockListAction; @@ -124,10 +125,12 @@ public RetType submit(TaskAction taskAction) throws IOExcepti Assert.assertEquals(mergeTask.getInterval(), ((LockTryAcquireAction) taskAction).getInterval()); isRedayCountDown.countDown(); taskLock = new TaskLock( + TaskLockType.EXCLUSIVE, mergeTask.getGroupId(), mergeTask.getDataSource(), mergeTask.getInterval(), - version + version, + Tasks.DEFAULT_TASK_PRIORITY ); return (RetType) taskLock; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 7b6b473b7e6c..aaf3de4be0d9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -187,8 +187,7 @@ public void testIndexTaskSerde() throws Exception new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null) ), - null, - jsonMapper + null ); final String json = jsonMapper.writeValueAsString(task); @@ -250,8 +249,7 @@ public void testIndexTaskwithResourceSerde() throws Exception new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), - null, - jsonMapper + null ); for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index 3761ad051101..92ac4bb3dd59 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; @@ -72,7 +73,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final TaskLock lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); + final LockResult lockResult1 = toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, interval1) + ); + Assert.assertTrue(lockResult1.isOk() && !lockResult1.isWasRevoked()); + final TaskLock lock1 = lockResult1.getTaskLock(); final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) @@ -80,7 +85,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1); // Acquire lock for second interval - final TaskLock lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); + final LockResult lockResult2 = toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, interval2) + ); + Assert.assertTrue(lockResult2.isOk() && !lockResult2.isWasRevoked()); + final TaskLock lock2 = lockResult2.getTaskLock(); final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 6c8138838090..932a04bf4d2e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -657,8 +657,7 @@ public void testIndexTask() throws Exception new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), - null, - MAPPER + null ); final Optional preRunTaskStatus = tsqa.getStatus(indexTask.getId()); @@ -715,8 +714,7 @@ public void testIndexTaskFailure() throws Exception new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), - null, - MAPPER + null ); final TaskStatus status = runTask(indexTask); @@ -1080,8 +1078,7 @@ public void testResumeTasks() throws Exception new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null) ), - null, - MAPPER + null ); final long startTime = System.currentTimeMillis(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java new file mode 100644 index 000000000000..425f8e89bafd --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -0,0 +1,150 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.metadata.EntryExistsException; +import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; +import io.druid.metadata.TestDerbyConnector; +import io.druid.server.initialization.ServerConfig; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +public class TaskLockBoxConcurrencyTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + + private final ObjectMapper objectMapper = new DefaultObjectMapper(); + private ExecutorService service; + private ServerConfig serverConfig; + private TaskStorage taskStorage; + private TaskLockbox lockbox; + + @Before + public void setup() + { + final TestDerbyConnector derbyConnector = derby.getConnector(); + derbyConnector.createTaskTables(); + taskStorage = new MetadataTaskStorage( + derbyConnector, + new TaskStorageConfig(null), + new SQLMetadataStorageActionHandlerFactory( + derbyConnector, + derby.metadataTablesConfigSupplier().get(), + objectMapper + ) + ); + serverConfig = EasyMock.niceMock(ServerConfig.class); + EasyMock.expect(serverConfig.getMaxIdleTime()).andReturn(new Period(5000)).anyTimes(); + EasyMock.replay(serverConfig); + + lockbox = new TaskLockbox(taskStorage, serverConfig); + service = Executors.newFixedThreadPool(2); + } + + @After + public void teardown() + { + service.shutdownNow(); + } + + @Test(timeout = 5000L) + public void testTryExclusiveLock() throws ExecutionException, InterruptedException, EntryExistsException + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task lowPriorityTask = NoopTask.create(10); + final Task highPriorityTask = NoopTask.create(100); + lockbox.add(lowPriorityTask); + lockbox.add(highPriorityTask); + taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); + taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + + final CountDownLatch latch = new CountDownLatch(1); + final Future lowPriorityFuture = service.submit(() -> { + final LockResult lock = lockbox.tryLock( + TaskLockType.EXCLUSIVE, + lowPriorityTask, + interval + ); + latch.countDown(); + return lock; + }); + final Future highPriorityFuture = service.submit(() -> { + latch.await(); + return lockbox.tryLock( + TaskLockType.EXCLUSIVE, + highPriorityTask, + interval + ); + }); + + final TaskLock lowLock = lowPriorityFuture.get().getTaskLock(); + final TaskLock highLock = highPriorityFuture.get().getTaskLock(); + + Assert.assertNotNull(lowLock); + Assert.assertNotNull(highLock); + + final Future lowUpgradeFuture = service.submit( + () -> lockbox.upgrade(lowPriorityTask, interval) + ); + final Future highUpgradeFuture = service.submit( + () -> lockbox.upgrade(highPriorityTask, interval) + ); + + final LockResult resultOfHighPriorityLock = highUpgradeFuture.get(); + Assert.assertTrue(resultOfHighPriorityLock.isOk()); + Assert.assertTrue(resultOfHighPriorityLock.getTaskLock().isUpgraded()); + assertEqualsExceptUpgraded(highLock, resultOfHighPriorityLock.getTaskLock()); + + final LockResult resultOfLowPriorityLock = lowUpgradeFuture.get(); + Assert.assertFalse(resultOfLowPriorityLock.isOk()); + Assert.assertTrue(resultOfLowPriorityLock.isWasRevoked()); + } + + private static void assertEqualsExceptUpgraded(TaskLock expected, TaskLock actual) + { + Assert.assertEquals(expected.getGroupId(), actual.getGroupId()); + Assert.assertEquals(expected.getDataSource(), actual.getDataSource()); + Assert.assertEquals(expected.getType(), actual.getType()); + Assert.assertEquals(expected.getInterval(), actual.getInterval()); + Assert.assertEquals(expected.getVersion(), actual.getVersion()); + Assert.assertEquals(expected.getPriority(), actual.getPriority()); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index e10a6250b13d..d87f02eee22e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -19,14 +19,12 @@ package io.druid.indexing.overlord; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.data.input.FirehoseFactory; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.NoopTask; @@ -47,8 +45,10 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; -import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class TaskLockboxTest @@ -56,6 +56,9 @@ public class TaskLockboxTest @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private final ObjectMapper objectMapper = new DefaultObjectMapper(); private ServerConfig serverConfig; private TaskStorage taskStorage; @@ -94,13 +97,13 @@ public void testLock() throws InterruptedException { Task task = NoopTask.create(); lockbox.add(task); - Assert.assertNotNull(lockbox.lock(task, new Interval("2015-01-01/2015-01-02"))); + Assert.assertNotNull(lockbox.lock(TaskLockType.EXCLUSIVE, task, new Interval("2015-01-01/2015-01-02"))); } @Test(expected = IllegalStateException.class) public void testLockForInactiveTask() throws InterruptedException { - lockbox.lock(NoopTask.create(), new Interval("2015-01-01/2015-01-02")); + lockbox.lock(TaskLockType.EXCLUSIVE, NoopTask.create(), new Interval("2015-01-01/2015-01-02")); } @Test @@ -111,57 +114,97 @@ public void testLockAfterTaskComplete() throws InterruptedException exception.expectMessage("Unable to grant lock to inactive Task"); lockbox.add(task); lockbox.remove(task); - lockbox.lock(task, new Interval("2015-01-01/2015-01-02")); + lockbox.lock(TaskLockType.EXCLUSIVE, task, new Interval("2015-01-01/2015-01-02")); } @Test - public void testTryLock() + public void testTrySharedLock() + { + final Interval interval = new Interval("2017-01/2017-02"); + final List tasks = new ArrayList<>(); + final Set actualLocks = new HashSet<>(); + + // test creating new locks + for (int i = 0; i < 5; i++) { + final Task task = NoopTask.create(Math.min(0, (i - 1) * 10)); // the first two tasks have the same priority + tasks.add(task); + lockbox.add(task); + final TaskLock lock = lockbox.tryLock(TaskLockType.SHARED, task, interval).getTaskLock(); + Assert.assertNotNull(lock); + actualLocks.add(lock); + } + + Assert.assertEquals(5, getAllLocks(tasks).size()); + Assert.assertEquals(getAllLocks(tasks), actualLocks); + } + + @Test + public void testTryMixedLocks() throws EntryExistsException + { + final Task lowPriorityTask = NoopTask.create(0); + final Task lowPriorityTask2 = NoopTask.create(0); + final Task highPiorityTask = NoopTask.create(10); + final Interval interval1 = new Interval("2017-01-01/2017-01-02"); + final Interval interval2 = new Interval("2017-01-02/2017-01-03"); + final Interval interval3 = new Interval("2017-01-03/2017-01-04"); + + taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); + taskStorage.insert(lowPriorityTask2, TaskStatus.running(lowPriorityTask2.getId())); + taskStorage.insert(highPiorityTask, TaskStatus.running(highPiorityTask.getId())); + + lockbox.add(lowPriorityTask); + lockbox.add(lowPriorityTask2); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval1).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask, interval2).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask2, interval2).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); + + lockbox.add(highPiorityTask); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, highPiorityTask, interval1).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval2).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); + + Assert.assertTrue(lockbox.findLocksForTask(lowPriorityTask).stream().allMatch(TaskLock::isRevoked)); + Assert.assertTrue(lockbox.findLocksForTask(lowPriorityTask2).stream().allMatch(TaskLock::isRevoked)); + + lockbox.remove(lowPriorityTask); + lockbox.remove(lowPriorityTask2); + lockbox.remove(highPiorityTask); + + lockbox.add(highPiorityTask); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval1).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, highPiorityTask, interval2).isOk()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); + + lockbox.add(lowPriorityTask); + Assert.assertFalse(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask, interval1).isOk()); + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval2).isOk()); + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); + } + + @Test + public void testTryExclusiveLock() { Task task = NoopTask.create(); lockbox.add(task); - Assert.assertTrue(lockbox.tryLock(task, new Interval("2015-01-01/2015-01-03")).isPresent()); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, task, new Interval("2015-01-01/2015-01-03")).isOk()); // try to take lock for task 2 for overlapping interval Task task2 = NoopTask.create(); lockbox.add(task2); - Assert.assertFalse(lockbox.tryLock(task2, new Interval("2015-01-01/2015-01-02")).isPresent()); + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, task2, new Interval("2015-01-01/2015-01-02")).isOk()); // task 1 unlocks the lock lockbox.remove(task); // Now task2 should be able to get the lock - Assert.assertTrue(lockbox.tryLock(task2, new Interval("2015-01-01/2015-01-02")).isPresent()); - } - - @Test - public void testTrySmallerLock() - { - Task task = NoopTask.create(); - lockbox.add(task); - Optional lock1 = lockbox.tryLock(task, new Interval("2015-01-01/2015-01-03")); - Assert.assertTrue(lock1.isPresent()); - Assert.assertEquals(new Interval("2015-01-01/2015-01-03"), lock1.get().getInterval()); - - // same task tries to take partially overlapping interval; should fail - Assert.assertFalse(lockbox.tryLock(task, new Interval("2015-01-02/2015-01-04")).isPresent()); - - // same task tries to take contained interval; should succeed and should match the original lock - Optional lock2 = lockbox.tryLock(task, new Interval("2015-01-01/2015-01-02")); - Assert.assertTrue(lock2.isPresent()); - Assert.assertEquals(new Interval("2015-01-01/2015-01-03"), lock2.get().getInterval()); - - // only the first lock should actually exist - Assert.assertEquals( - ImmutableList.of(lock1.get()), - lockbox.findLocksForTask(task) - ); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, task2, new Interval("2015-01-01/2015-01-02")).isOk()); } - @Test(expected = IllegalStateException.class) public void testTryLockForInactiveTask() { - Assert.assertFalse(lockbox.tryLock(NoopTask.create(), new Interval("2015-01-01/2015-01-02")).isPresent()); + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, NoopTask.create(), new Interval("2015-01-01/2015-01-02")).isOk()); } @Test @@ -172,21 +215,19 @@ public void testTryLockAfterTaskComplete() exception.expectMessage("Unable to grant lock to inactive Task"); lockbox.add(task); lockbox.remove(task); - Assert.assertFalse(lockbox.tryLock(task, new Interval("2015-01-01/2015-01-02")).isPresent()); + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, task, new Interval("2015-01-01/2015-01-02")).isOk()); } @Test public void testTimeoutForLock() throws InterruptedException { Task task1 = NoopTask.create(); - Task task2 = new SomeTask(null, 0, 0, null, null, null); + Task task2 = NoopTask.create(); lockbox.add(task1); lockbox.add(task2); - exception.expect(InterruptedException.class); - exception.expectMessage("can not acquire lock for interval"); - lockbox.lock(task1, new Interval("2015-01-01/2015-01-02")); - lockbox.lock(task2, new Interval("2015-01-01/2015-01-15")); + Assert.assertTrue(lockbox.lock(TaskLockType.EXCLUSIVE, task1, new Interval("2015-01-01/2015-01-02")).isOk()); + Assert.assertFalse(lockbox.lock(TaskLockType.EXCLUSIVE, task2, new Interval("2015-01-01/2015-01-15")).isOk()); } @Test @@ -198,8 +239,11 @@ public void testSyncFromStorage() throws EntryExistsException taskStorage.insert(task, TaskStatus.running(task.getId())); originalBox.add(task); Assert.assertTrue( - originalBox.tryLock(task, new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2)))) - .isPresent() + originalBox.tryLock( + TaskLockType.EXCLUSIVE, + task, + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ).isOk() ); } @@ -220,28 +264,232 @@ public void testSyncFromStorage() throws EntryExistsException Assert.assertEquals(beforeLocksInStorage, afterLocksInStorage); } - public static class SomeTask extends NoopTask { + @Test + public void testUpgradeSharedLock() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Shared lock cannot be upgraded"); + + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task task = NoopTask.create(); + lockbox.add(task); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); + + lockbox.upgrade(task, interval); + } + + @Test + public void testDowngradeSharedLock() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Shared lock cannot be downgraded"); + + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task task = NoopTask.create(); + lockbox.add(task); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); + + lockbox.downgrade(task, interval); + } + + @Test + public void testUpgradeAndDowngradeExclusiveLock() + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task task = NoopTask.create(); + lockbox.add(task); + final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + Assert.assertNotNull(lock); + + final LockResult upgradeResult = lockbox.upgrade(task, interval); + Assert.assertTrue(upgradeResult.isOk()); + Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); + + final TaskLock downgradedLock = lockbox.downgrade(task, interval); + Assert.assertNotNull(downgradedLock); + Assert.assertFalse(downgradedLock.isUpgraded()); + } + + @Test + public void testUpgradeDownGradeWithSmallerInterval() + { + final Interval interval = new Interval("2017-01-01/2017-02-01"); + final Interval smallInterval1 = new Interval("2017-01-01/2017-01-02"); + final Interval smallInterval2 = new Interval("2017-01-10/2017-01-11"); + final Task task = NoopTask.create(); + lockbox.add(task); + final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + Assert.assertNotNull(lock); + + final LockResult upgradeResult = lockbox.upgrade(task, smallInterval1); + Assert.assertTrue(upgradeResult.isOk()); + Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); + + final TaskLock downgradedLock = lockbox.downgrade(task, smallInterval2); + Assert.assertNotNull(downgradedLock); + Assert.assertFalse(downgradedLock.isUpgraded()); + } + + @Test + public void testPreemptAndUpgradeExclusiveLock() throws EntryExistsException + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + for (int i = 0; i < 5; i++) { + final Task task = NoopTask.create(); + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); + } + + final Task highPriorityTask = NoopTask.create(100); + lockbox.add(highPriorityTask); + taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).getTaskLock(); + Assert.assertNotNull(lock); + + final LockResult upgradeResult = lockbox.upgrade(highPriorityTask, interval); + Assert.assertTrue(upgradeResult.isOk()); + Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); + } - public SomeTask( - @JsonProperty("id") String id, - @JsonProperty("runTime") long runTime, - @JsonProperty("isReadyTime") long isReadyTime, - @JsonProperty("isReadyResult") String isReadyResult, - @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("context") Map context - ) - { - super(id, runTime, isReadyTime, isReadyResult, firehoseFactory, context); + @Test + public void testPreemption() throws EntryExistsException + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task lowPriorityTask = NoopTask.create(10); + final Task highPriorityTask = NoopTask.create(100); + lockbox.add(lowPriorityTask); + lockbox.add(highPriorityTask); + taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); + taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).isOk()); + Assert.assertNotNull(lockbox.upgrade(lowPriorityTask, interval)); + + Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + Assert.assertNotNull(lockbox.downgrade(lowPriorityTask, interval)); + + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + + final List lowPriorityLocks = taskStorage.getLocks(lowPriorityTask.getId()); + Assert.assertTrue(lowPriorityLocks.stream().allMatch(TaskLock::isRevoked)); + } + + @Test + public void testUpgradeRevokedLock() throws EntryExistsException + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task lowPriorityTask = NoopTask.create("task1", 0); + final Task highPriorityTask = NoopTask.create("task2", 10); + lockbox.add(lowPriorityTask); + lockbox.add(highPriorityTask); + taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); + taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + + final TaskLock lowPriorityLock = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); + Assert.assertNotNull(lowPriorityLock); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); + + final LockResult upgradeResult = lockbox.upgrade(lowPriorityTask, interval); + Assert.assertFalse(upgradeResult.isOk()); + Assert.assertTrue(upgradeResult.isWasRevoked()); + } + + @Test + public void testAcquireLockAfterRevoked() throws EntryExistsException + { + final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Task lowPriorityTask = NoopTask.create("task1", 0); + final Task highPriorityTask = NoopTask.create("task2", 10); + lockbox.add(lowPriorityTask); + lockbox.add(highPriorityTask); + taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); + taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + + final TaskLock lowPriorityLock = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); + Assert.assertNotNull(lowPriorityLock); + Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); + + lockbox.unlock(highPriorityTask, interval); + + // Acquire again + final LockResult lockResult = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + Assert.assertFalse(lockResult.isOk()); + Assert.assertTrue(lockResult.isWasRevoked()); + Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); + } + + @Test + public void testUnlock() throws EntryExistsException + { + final List lowPriorityTasks = new ArrayList<>(); + final List highPriorityTasks = new ArrayList<>(); + + for (int i = 0; i < 8; i++) { + final Task task = NoopTask.create(10); + lowPriorityTasks.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + lockbox.add(task); + Assert.assertTrue( + lockbox.tryLock( + TaskLockType.EXCLUSIVE, + task, + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ).isOk() + ); + } + + // Revoke some locks + for (int i = 0; i < 4; i++) { + final Task task = NoopTask.create(100); + highPriorityTasks.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + lockbox.add(task); + Assert.assertTrue( + lockbox.tryLock( + TaskLockType.EXCLUSIVE, + task, + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ).isOk() + ); + } + + for (int i = 0; i < 4; i++) { + Assert.assertTrue(taskStorage.getLocks(lowPriorityTasks.get(i).getId()).stream().allMatch(TaskLock::isRevoked)); + Assert.assertFalse(taskStorage.getLocks(highPriorityTasks.get(i).getId()).stream().allMatch(TaskLock::isRevoked)); + } + + for (int i = 4; i < 8; i++) { + Assert.assertFalse(taskStorage.getLocks(lowPriorityTasks.get(i).getId()).stream().allMatch(TaskLock::isRevoked)); + } + + for (int i = 0; i < 4; i++) { + lockbox.unlock( + lowPriorityTasks.get(i), + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ); + lockbox.unlock( + highPriorityTasks.get(i), + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ); } - @Override - public String getType() - { - return "someTask"; + for (int i = 4; i < 8; i++) { + lockbox.unlock( + lowPriorityTasks.get(i), + new Interval(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + ); } - @Override - public String getGroupId() { return "someGroupId";} + Assert.assertTrue(lockbox.getAllLocks().isEmpty()); + } + private Set getAllLocks(List tasks) + { + return tasks.stream() + .flatMap(task -> taskStorage.getLocks(task.getId()).stream()) + .collect(Collectors.toSet()); } } diff --git a/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java b/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java index 02545222c9d8..ae96db8b481d 100644 --- a/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java +++ b/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; import io.druid.timeline.DataSegment; import java.util.Objects; @@ -42,6 +43,11 @@ public class SegmentPublishResult private final Set segments; private final boolean success; + public static SegmentPublishResult fail() + { + return new SegmentPublishResult(ImmutableSet.of(), false); + } + @JsonCreator public SegmentPublishResult( @JsonProperty("segments") Set segments, diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index 3fcc211d0d31..c7edc2746121 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -27,9 +27,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.metamx.emitter.EmittingLogger; - import io.druid.java.util.common.Pair; - import io.druid.java.util.common.StringUtils; import org.joda.time.DateTime; import org.skife.jdbi.v2.FoldController; diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java index f003c01c5bf6..178534451e49 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -564,8 +564,7 @@ private ListenableFuture publish( .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { log.info("Our segments really do exist, awaiting handoff."); } else { - log.warn("Our segments don't exist, giving up."); - return null; + throw new ISE("Failed to publish segments[%s]", segmentIdentifiers); } } } diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 63f6e668ce93..38acfaba94a8 100644 --- a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -25,10 +25,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; - import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Pair; - import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Before; From df933494485688b33484c1e5f7ce522315e4a5e9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 15 Jul 2017 17:56:50 +0900 Subject: [PATCH 02/20] Fix build failure --- .../main/java/io/druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../java/io/druid/indexing/common/task/HadoopIndexTask.java | 2 +- .../main/java/io/druid/indexing/common/task/IndexTask.java | 2 +- .../indexing/common/actions/SegmentAllocateActionTest.java | 6 ++---- 4 files changed, 5 insertions(+), 7 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 7d416b3d32c3..57edf836c769 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -888,7 +888,7 @@ private AppenderatorDriver newDriver( { return new AppenderatorDriver( appenderator, - new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema, true), + new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 2a1ef48288e3..69f8604ed7e1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -124,7 +124,7 @@ public HadoopIndexTask( @Override public int getPriority() { - return Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY; + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 6b39b225ced6..8df0a7b5980c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -626,7 +626,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final SegmentAllocator segmentAllocator; if (ioConfig.isAppendToExisting()) { - segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema, false); + segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); } else { segmentAllocator = (row, sequenceName, previousSegmentId) -> { final DateTime timestamp = row.getTimestamp(); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java index e86d828e2345..a587b38e1c14 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -689,8 +689,7 @@ public void testSerde() throws Exception Granularities.MINUTE, Granularities.HOUR, "s1", - "prev", - true + "prev" ); final ObjectMapper objectMapper = new DefaultObjectMapper(); @@ -722,8 +721,7 @@ private SegmentIdentifier allocate( queryGranularity, preferredSegmentGranularity, sequenceName, - sequencePreviousId, - true + sequencePreviousId ); return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } From 14bfaa46308406e8d8b7be5bd48cf242e4c3fefb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 18 Jul 2017 09:51:08 +0900 Subject: [PATCH 03/20] Fix tc fail --- processing/src/main/java/io/druid/guice/ModulesConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/guice/ModulesConfig.java b/processing/src/main/java/io/druid/guice/ModulesConfig.java index 5dbacfb85612..ce044f766fe5 100644 --- a/processing/src/main/java/io/druid/guice/ModulesConfig.java +++ b/processing/src/main/java/io/druid/guice/ModulesConfig.java @@ -28,7 +28,7 @@ public class ModulesConfig { /** * Canonical class names of modules, which should not be loaded despite they are founded in extensions from {@link - * io.druid.guice.ExtensionsConfig#loadList} or the standard list of modules loaded by some node type, e. g. {@code + * ExtensionsConfig#loadList} or the standard list of modules loaded by some node type, e. g. {@code * CliPeon}. */ @JsonProperty From c57a9c58552841799590c98d5a51135d80b8e87f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 21 Jul 2017 10:45:56 +0900 Subject: [PATCH 04/20] Fix typos --- docs/content/ingestion/tasks.md | 6 +++--- .../common/actions/SegmentAllocateAction.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 1 - .../druid/indexing/overlord/LockResult.java | 20 +++++++++---------- .../indexing/overlord/RealtimeishTask.java | 4 ++-- .../overlord/TaskLockBoxConcurrencyTest.java | 2 +- .../indexing/overlord/TaskLockboxTest.java | 4 ++-- 7 files changed, 19 insertions(+), 20 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 576520b09ccc..7ff3d31b6452 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -336,11 +336,11 @@ Once an overlord node accepts a task, the task acquires locks for the data sourc There are two locks types, i.e., _shared lock_ and _exclusive lock_. - A task needs to acquire a shared lock before it reads segments of an interval. Multiple shared locks can be acquired for the same dataSource and interval. Shared locks are always preemptable, but they don't preempt each other. -- A task needs to acquire an exclusive lock before it writes segemtns for an interval. An exclusive lock is acquired as preemptable and can be upgraded as non-preemptable when publishing segments. +- A task needs to acquire an exclusive lock before it writes segments for an interval. An exclusive lock is acquired as preemptable and can be upgraded as non-preemptable when publishing segments. -Each task can have different lock priorities. The locks of higher-priority tasks can preempt the locks of lower-priority tasks. The lock preemption works based on _optimistic locking_. When a lock is preempted, it is not notified to the owner task immediately. Instead, it's notified when the owner task tries to acquire the same lock again or upgrade it. (Note that lock acquisition is idempotent unless the lock is preempted.) In general, tasks don't content to acquire locks because they usually targets different dataSources or intervals. +Each task can have different lock priorities. The locks of higher-priority tasks can preempt the locks of lower-priority tasks. The lock preemption works based on _optimistic locking_. When a lock is preempted, it is not notified to the owner task immediately. Instead, it's notified when the owner task tries to acquire the same lock again or upgrade it. (Note that lock acquisition is idempotent unless the lock is preempted.) In general, tasks don't contend to acquire locks because they usually targets different dataSources or intervals. -A task writing data into a dataSource must acquire exclusive locks for target intervals. Note that execlusive locks are still preemptable. As a result, the task must _upgrade_ its locks as non-preemptable when it executes a critical operation, _publishing segments_. Once the lock is upgraded, it can't be preempted by even higher-priority locks. After publishing segments, the task downgrades its locks as preemptable. +A task writing data into a dataSource must acquire exclusive locks for target intervals. Note that exclusive locks are still preemptable. As a result, the task must _upgrade_ its locks as non-preemptable when it executes a critical operation, _publishing segments_. Once the lock is upgraded, it can't be preempted by even higher-priority locks. After publishing segments, the task downgrades its locks as preemptable. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java index 5e22b2736a2e..5ba6869f0f5a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java @@ -176,7 +176,7 @@ public SegmentIdentifier perform( tryInterval ); final LockResult lockResult = toolbox.getTaskLockbox().tryLock(TaskLockType.EXCLUSIVE, task, tryInterval); - if (lockResult.isWasRevoked()) { + 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); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 8df0a7b5980c..2e1fd043c0d4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -683,7 +683,6 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final Interval interval = optInterval.get(); final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow); - // TODO: validate versions if appendToExisting? => need to test final String sequenceName = Appenderators.getSequenceName(interval, findVersion(versions, interval), shardSpec); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java b/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java index 4c3ec8b61f83..f8a057d6d8c8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/LockResult.java @@ -31,9 +31,9 @@ /** * This class represents the result of {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}. If the lock * acquisition fails, the callers can tell that it was failed because it was preempted by other locks of higher - * priorities or not by checking the {@link #wasRevoked} flag. + * priorities or not by checking the {@link #revoked} flag. * - * The {@link #wasRevoked} flag means that consecutive lock acquisitions for the same dataSource and interval are + * The {@link #revoked} flag means that consecutive lock acquisitions for the same dataSource and interval are * returning different locks because another lock of a higher priority preempted your lock at some point. In this case, * the lock acquisition must fail. * @@ -42,26 +42,26 @@ public class LockResult { private final TaskLock taskLock; - private final boolean wasRevoked; + private final boolean revoked; public static LockResult ok(TaskLock taskLock) { return new LockResult(taskLock, false); } - public static LockResult fail(boolean wasRevoked) + public static LockResult fail(boolean revoked) { - return new LockResult(null, wasRevoked); + return new LockResult(null, revoked); } @JsonCreator public LockResult( @JsonProperty("taskLock") @Nullable TaskLock taskLock, - @JsonProperty("wasRevoked") boolean wasRevoked + @JsonProperty("revoked") boolean revoked ) { this.taskLock = taskLock; - this.wasRevoked = wasRevoked; + this.revoked = revoked; } @JsonProperty("taskLock") @@ -70,10 +70,10 @@ public TaskLock getTaskLock() return taskLock; } - @JsonProperty("wasRevoked") - public boolean isWasRevoked() + @JsonProperty("revoked") + public boolean isRevoked() { - return wasRevoked; + return revoked; } public boolean isOk() diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index 92ac4bb3dd59..d53854925b61 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -76,7 +76,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final LockResult lockResult1 = toolbox.getTaskActionClient().submit( new LockAcquireAction(TaskLockType.EXCLUSIVE, interval1) ); - Assert.assertTrue(lockResult1.isOk() && !lockResult1.isWasRevoked()); + Assert.assertTrue(lockResult1.isOk() && !lockResult1.isRevoked()); final TaskLock lock1 = lockResult1.getTaskLock(); final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); @@ -88,7 +88,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final LockResult lockResult2 = toolbox.getTaskActionClient().submit( new LockAcquireAction(TaskLockType.EXCLUSIVE, interval2) ); - Assert.assertTrue(lockResult2.isOk() && !lockResult2.isWasRevoked()); + Assert.assertTrue(lockResult2.isOk() && !lockResult2.isRevoked()); final TaskLock lock2 = lockResult2.getTaskLock(); final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 425f8e89bafd..0b2515fbdfe3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -135,7 +135,7 @@ public void testTryExclusiveLock() throws ExecutionException, InterruptedExcepti final LockResult resultOfLowPriorityLock = lowUpgradeFuture.get(); Assert.assertFalse(resultOfLowPriorityLock.isOk()); - Assert.assertTrue(resultOfLowPriorityLock.isWasRevoked()); + Assert.assertTrue(resultOfLowPriorityLock.isRevoked()); } private static void assertEqualsExceptUpgraded(TaskLock expected, TaskLock actual) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index d87f02eee22e..5ab38440711b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -393,7 +393,7 @@ public void testUpgradeRevokedLock() throws EntryExistsException final LockResult upgradeResult = lockbox.upgrade(lowPriorityTask, interval); Assert.assertFalse(upgradeResult.isOk()); - Assert.assertTrue(upgradeResult.isWasRevoked()); + Assert.assertTrue(upgradeResult.isRevoked()); } @Test @@ -417,7 +417,7 @@ public void testAcquireLockAfterRevoked() throws EntryExistsException // Acquire again final LockResult lockResult = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); Assert.assertFalse(lockResult.isOk()); - Assert.assertTrue(lockResult.isWasRevoked()); + Assert.assertTrue(lockResult.isRevoked()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); } From 700a9686e38d0d60a6d7660bc2811b1e102e499d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 25 Jul 2017 10:03:28 +0900 Subject: [PATCH 05/20] Fix IndexTaskTest --- .../java/io/druid/indexing/common/task/IndexTask.java | 10 +++------- .../io/druid/indexing/common/task/IndexTaskTest.java | 2 +- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 0bec195766e8..e17ae352b621 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -238,13 +238,9 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ) ); } else { - versions = getTaskLocks(toolbox.getTaskActionClient()).stream() - .collect( - Collectors.toMap( - TaskLock::getInterval, - TaskLock::getVersion - ) - ); + versions = getTaskLocks(toolbox.getTaskActionClient()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); dataSchema = ingestionSchema.getDataSchema(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index cb9228d4ec19..f6b698a49a8d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -896,7 +896,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti TaskLockType.EXCLUSIVE, "", "", - new Interval("2014/P1D"), + new Interval("2014/P1Y"), new DateTime().toString(), Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY ) From 1dc9941a7beda84d7a62dd0c60984545611e52c8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 27 Jul 2017 16:32:32 +0900 Subject: [PATCH 06/20] Addressed comments --- .../actions/SegmentMetadataUpdateAction.java | 14 ++++++++- .../common/actions/SegmentNukeAction.java | 14 ++++++++- .../SegmentTransactionalInsertAction.java | 15 ++++++---- .../actions/TaskActionPreconditions.java | 21 +++----------- .../druid/indexing/common/task/IndexTask.java | 4 +-- .../io/druid/indexing/common/task/Tasks.java | 2 +- .../druid/indexing/overlord/TaskLockbox.java | 29 +++++++------------ .../actions/TaskActionPreconditionsTest.java | 6 ++-- 8 files changed, 57 insertions(+), 48 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index 519862a47e39..14ddd718b966 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -62,9 +63,20 @@ public Void perform( Task task, TaskActionToolbox toolbox ) throws IOException { - TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); + TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + + for (DataSegment segment : segments) { + if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { + // Acquired locks don't have to be released explicitly here because the fail result will make the index task + // calling this action failed, and the acquired locks should be released automatically when the task completes. + throw new ISE("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); + } + } + toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); + segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); + // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index e9dde04721e1..41af97a773e0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -62,9 +63,20 @@ public TypeReference getReturnTypeReference() @Override public Void perform(Task task, TaskActionToolbox toolbox) throws IOException { - TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); + TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + + for (DataSegment segment : segments) { + if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { + // Acquired locks don't have to be released explicitly here because the fail result will make the index task + // calling this action failed, and the acquired locks should be released automatically when the task completes. + throw new ISE("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); + } + } + toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); + segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); + // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index a1dadf62bd22..4e214e949795 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -27,6 +27,7 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.SegmentPublishResult; +import io.druid.java.util.common.logger.Logger; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -43,6 +44,8 @@ */ public class SegmentTransactionalInsertAction implements TaskAction { + private static final Logger LOG = new Logger(SegmentTransactionalInsertAction.class); + private final Set segments; private final DataSourceMetadata startMetadata; private final DataSourceMetadata endMetadata; @@ -99,23 +102,25 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws IOException { + TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + for (DataSegment segment : segments) { if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { + LOG.info("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); + + // Acquired locks don't have to be released explicitly here because the fail result will make the index task + // calling this action failed, and the acquired locks should be released automatically when the task completes. return SegmentPublishResult.fail(); } } - TaskActionPreconditions.checkTaskLocks(task, toolbox.getTaskLockbox(), segments); - final SegmentPublishResult retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( segments, startMetadata, endMetadata ); - for (DataSegment segment : segments) { - toolbox.getTaskLockbox().downgrade(task, segment.getInterval()); - } + segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java index bbcfac9d4ce3..268cc6f7b8a5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java @@ -31,34 +31,22 @@ public class TaskActionPreconditions { - public static void checkTaskLocksUpgraded( + public static void checkLockCoversSegments( final Task task, final TaskLockbox taskLockbox, final Set segments ) { - if (!checkLockCoversSegments(task, taskLockbox, segments, true)) { - throw new ISE("Segments not covered by locks or locks need upgrade for task: %s", task.getId()); - } - } - - public static void checkTaskLocks( - final Task task, - final TaskLockbox taskLockbox, - final Set segments - ) - { - if (!checkLockCoversSegments(task, taskLockbox, segments, false)) { + if (!isLockCoversSegments(task, taskLockbox, segments)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); } } @VisibleForTesting - static boolean checkLockCoversSegments( + static boolean isLockCoversSegments( final Task task, final TaskLockbox taskLockbox, - final Set segments, - final boolean checkUpgraded + final Set segments ) { // Verify that each of these segments falls under some lock @@ -73,7 +61,6 @@ static boolean checkLockCoversSegments( taskLock -> taskLock.getDataSource().equals(segment.getDataSource()) && taskLock.getInterval().contains(segment.getInterval()) && taskLock.getVersion().compareTo(segment.getVersion()) >= 0 - && (!checkUpgraded || taskLock.isUpgraded()) ); if (!ok) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index e17ae352b621..3020a583a742 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -180,7 +180,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception final List locks = getTaskLocks(taskActionClient); if (locks.size() == 0) { try { - Tasks.acquireLocks(taskActionClient, intervals.get()); + Tasks.acquireExclusiveLocks(taskActionClient, intervals.get()); } catch (Exception e) { return false; @@ -224,7 +224,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception if (determineIntervals) { final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); intervals.addAll(shardSpecs.getIntervals()); - final Map locks = Tasks.acquireLocks(toolbox.getTaskActionClient(), intervals); + final Map locks = Tasks.acquireExclusiveLocks(toolbox.getTaskActionClient(), intervals); versions = locks.entrySet().stream() .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java index 1309333edf6c..5dfda8e9ee23 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java @@ -53,7 +53,7 @@ public static void checkLockResult(LockResult result, Interval interval) } } - public static Map acquireLocks(TaskActionClient client, SortedSet intervals) + public static Map acquireExclusiveLocks(TaskActionClient client, SortedSet intervals) throws IOException { final Map lockMap = new HashMap<>(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 5f111f71f7ed..074395721d9d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -69,7 +69,7 @@ public class TaskLockbox { // Datasource -> Interval -> list of (Tasks + TaskLock) - // Multiple shared locks can be acquired for the same dataSource and interval + // Multiple shared locks can be acquired for the same dataSource and interval. // Note that revoked locks are also maintained in this map to notify that those locks are revoked to the callers when // they acquire the same locks again or request lock upgrade/downgrade. private final Map>> running = Maps.newHashMap(); @@ -280,8 +280,6 @@ public LockResult tryLock( // Update task storage facility. If it fails, revoke the lock. try { taskStorage.addLock(task.getId(), posseToUse.getTaskLock()); - // The task newly acquired a lock even though the lock itself might not be a new one. - // The revokedState is maintained per task, so it should be updated here. return LockResult.ok(posseToUse.getTaskLock()); } catch (Exception e) { @@ -332,7 +330,7 @@ private TaskLockPosse createOrFindLockPosse( private TaskLockPosse createOrFindLockPosse( final Task task, final Interval interval, - @Nullable final String preferredVersion, // TODO: check this + @Nullable final String preferredVersion, final TaskLockType lockType ) { @@ -344,19 +342,14 @@ private TaskLockPosse createOrFindLockPosse( final List foundPosses = findLockPossesOverlapsInterval(dataSource, interval); // If we have some locks for dataSource and interval, check they can be reused. - // If they can't be reused, check lock priority and revoke existing locks. + // If they can't be reused, check lock priority and revoke existing locks if possible. if (foundPosses.size() > 0) { - final List possesForReusableLocks = foundPosses.stream() - .filter(lockPosse -> - matchGroupIdAndInterval( - lockPosse.getTaskLock(), - task, - interval - ) - ).collect(Collectors.toList()); - if (lockType.equals(TaskLockType.SHARED) && isAllSharedLocks(foundPosses)) { + final List possesForReusableLocks = foundPosses + .stream() + .filter(lockPosse -> matchGroupIdAndContainInterval(lockPosse.getTaskLock(), task, interval)) + .collect(Collectors.toList()); if (possesForReusableLocks.size() == 0) { // Any number of shared locks can be acquired for the same dataSource and interval. @@ -378,9 +371,9 @@ private TaskLockPosse createOrFindLockPosse( ); } } else { - if (possesForReusableLocks.size() == 1 && - matchGroupIdAndInterval(possesForReusableLocks.get(0).taskLock, task, interval)) { - final TaskLockPosse foundPosse = possesForReusableLocks.get(0); + if (foundPosses.size() == 1 && + matchGroupIdAndContainInterval(foundPosses.get(0).taskLock, task, interval)) { + final TaskLockPosse foundPosse = foundPosses.get(0); if (lockType.equals(foundPosse.getTaskLock().getType())) { return foundPosse; } else { @@ -864,7 +857,7 @@ public void add(Task task) } } - private static boolean matchGroupIdAndInterval(TaskLock existingLock, Task task, Interval interval) + private static boolean matchGroupIdAndContainInterval(TaskLock existingLock, Task task, Interval interval) { return existingLock.getInterval().contains(interval) && existingLock.getGroupId().equals(task.getGroupId()); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java index 9e5baf497f3c..63b3f2fa8172 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -97,10 +97,10 @@ public void testCheckLockCoversSegments() throws Exception ) ); - Assert.assertTrue(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, false)); - Assert.assertFalse(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, true)); + Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); + Assert.assertFalse(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); locks.forEach((interval, lock) -> lockbox.upgrade(task, interval)); - Assert.assertTrue(TaskActionPreconditions.checkLockCoversSegments(task, lockbox, segments, true)); + Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); } } From 2eeaec548dc6624e216fc7b3187d7a7a2a895fb7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 27 Jul 2017 17:07:45 +0900 Subject: [PATCH 07/20] Fix test --- .../indexing/common/actions/TaskActionPreconditionsTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java index 63b3f2fa8172..44c502726ae2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -98,9 +98,5 @@ public void testCheckLockCoversSegments() throws Exception ); Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); - Assert.assertFalse(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); - - locks.forEach((interval, lock) -> lockbox.upgrade(task, interval)); - Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); } } From 2c394a2c6815ca938768afa96ddeb18492b4b4d2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 6 Aug 2017 23:48:39 +0900 Subject: [PATCH 08/20] Fix spacing --- .../main/java/io/druid/indexing/overlord/TaskLockbox.java | 2 +- .../java/io/druid/indexing/overlord/TaskLockboxTest.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 8278341ca0cb..5a5f1a3ce161 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -684,7 +684,7 @@ public void unlock(final Task task, final Interval interval) } final List possesHolder = dsRunning.get(interval); - if(possesHolder == null|| possesHolder.isEmpty()) { + if (possesHolder == null || possesHolder.isEmpty()) { return; } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index 0e33f754609c..b547a78796f6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -267,7 +267,7 @@ public void testSyncFromStorage() throws EntryExistsException @Test public void testUpgradeSharedLock() { - expectedException.expect(IllegalStateException. class ); + expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Shared lock cannot be upgraded"); final Interval interval = new Interval("2017-01-01/2017-01-02"); @@ -310,8 +310,8 @@ public void testUpgradeAndDowngradeExclusiveLock() Assert.assertFalse(downgradedLock.isUpgraded()); } - @Test - public void testUpgradeDownGradeWithSmallerInterval() + @Test + public void testUpgradeDownGradeWithSmallerInterval() { final Interval interval = new Interval("2017-01-01/2017-02-01"); final Interval smallInterval1 = new Interval("2017-01-01/2017-01-02"); From db3195f6a7f66f7fb58a1f7d50b81e68deb20ee2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 22 Aug 2017 09:24:06 +0900 Subject: [PATCH 09/20] Fix build error --- .../actions/SegmentInsertActionTest.java | 4 ++-- .../actions/TaskActionPreconditionsTest.java | 24 +++++++++---------- .../indexing/common/task/IndexTaskTest.java | 2 +- .../overlord/TaskLockBoxConcurrencyTest.java | 3 ++- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index e6e5afe00852..3198713bc709 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -93,7 +93,7 @@ public void testSimple() throws Exception final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); actionTestKit.getTaskLockbox().add(task); actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - actionTestKit.getTaskLockbox().upgrade(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().upgrade(task, INTERVAL); action.perform(task, actionTestKit.getTaskActionToolbox()); Assert.assertEquals( @@ -112,7 +112,7 @@ public void testFailBadVersion() throws Exception final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - actionTestKit.getTaskLockbox().upgrade(task, new Interval(INTERVAL)); + actionTestKit.getTaskLockbox().upgrade(task, INTERVAL); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java index dead46951ec5..ac494f10a95f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -28,16 +28,16 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.TaskLockbox; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Intervals; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -58,20 +58,20 @@ public void setup() segments = ImmutableSet.of( new DataSegment.Builder() .dataSource(task.getDataSource()) - .interval(new Interval("2017-01-01/2017-01-02")) - .version(new DateTime().toString()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) - .interval(new Interval("2017-01-02/2017-01-03")) - .version(new DateTime().toString()) + .interval(Intervals.of("2017-01-02/2017-01-03")) + .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) - .interval(new Interval("2017-01-03/2017-01-04")) - .version(new DateTime().toString()) + .interval(Intervals.of("2017-01-03/2017-01-04")) + .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) .build() ); @@ -81,12 +81,12 @@ public void setup() public void testCheckLockCoversSegments() throws Exception { final List intervals = ImmutableList.of( - new Interval("2017-01-01/2017-01-02"), - new Interval("2017-01-02/2017-01-03"), - new Interval("2017-01-03/2017-01-04") + Intervals.of("2017-01-01/2017-01-02"), + Intervals.of("2017-01-02/2017-01-03"), + Intervals.of("2017-01-03/2017-01-04") ); - final Map locks = intervals.stream().collect( + intervals.stream().collect( Collectors.toMap( Function.identity(), interval -> { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 887b40077c89..be12f0e1be41 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -897,7 +897,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti TaskLockType.EXCLUSIVE, "", "", - new Interval("2014/P1Y"), + Intervals.of("2014/P1Y"), DateTimes.nowUtc().toString(), Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY ) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index a76c968c9751..6fe55e8f5d14 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -27,6 +27,7 @@ import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Intervals; import io.druid.metadata.EntryExistsException; import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; @@ -81,7 +82,7 @@ public void teardown() @Test(timeout = 5000L) public void testTryExclusiveLock() throws ExecutionException, InterruptedException, EntryExistsException { - final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task lowPriorityTask = NoopTask.create(10); final Task highPriorityTask = NoopTask.create(100); lockbox.add(lowPriorityTask); From 51b69627f3bc5ddb0c81401ef4511df67dec0485 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 22 Aug 2017 10:00:23 +0900 Subject: [PATCH 10/20] Fix build error --- .../indexing/common/actions/TaskActionPreconditionsTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java index ac494f10a95f..6baa17c00c01 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -38,6 +38,7 @@ import org.junit.Test; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -86,7 +87,7 @@ public void testCheckLockCoversSegments() throws Exception Intervals.of("2017-01-03/2017-01-04") ); - intervals.stream().collect( + final Map locks = intervals.stream().collect( Collectors.toMap( Function.identity(), interval -> { @@ -97,6 +98,7 @@ public void testCheckLockCoversSegments() throws Exception ) ); + Assert.assertEquals(3, locks.size()); Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); } } From c5576d7ff06000d4412b19a87e42fdfc57080239 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 29 Aug 2017 17:52:04 +0900 Subject: [PATCH 11/20] Add lock status --- .../io/druid/indexing/common/TaskLock.java | 88 ++++++++++++------- .../druid/indexing/common/TaskLockStatus.java | 63 +++++++++++++ 2 files changed, 118 insertions(+), 33 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java index 25d43fa41a5d..836b1d076077 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java @@ -25,6 +25,8 @@ import com.google.common.base.Preconditions; import org.joda.time.Interval; +import javax.annotation.Nullable; + /** * Represents a lock held by some task. Immutable. */ @@ -36,32 +38,32 @@ public class TaskLock private final Interval interval; private final String version; private final int priority; - private final boolean upgraded; - private final boolean revoked; + private final TaskLockStatus lockStatus; @JsonCreator public TaskLock( - @JsonProperty("type") TaskLockType type, + @JsonProperty("type") @Nullable TaskLockType type, // nullable for backward compatibility @JsonProperty("groupId") String groupId, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JsonProperty("priority") int priority, - @JsonProperty("upgraded") boolean upgraded, - @JsonProperty("revoked") boolean revoked + @JsonProperty("lockStatus") @Nullable TaskLockStatus lockStatus // nullable for backward compatibility ) { - Preconditions.checkArgument(!type.equals(TaskLockType.SHARED) || !upgraded, "lock[%s] cannot be upgraded", type); - Preconditions.checkArgument(!upgraded || !revoked, "Upgraded locks cannot be revoked"); - - this.type = type; - this.groupId = Preconditions.checkNotNull(groupId); - this.dataSource = Preconditions.checkNotNull(dataSource); - this.interval = Preconditions.checkNotNull(interval); - this.version = Preconditions.checkNotNull(version); + this.type = type == null ? TaskLockType.EXCLUSIVE : type; + this.groupId = Preconditions.checkNotNull(groupId, "groupId"); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.interval = Preconditions.checkNotNull(interval, "interval"); + this.version = Preconditions.checkNotNull(version, "version"); this.priority = priority; - this.upgraded = upgraded; - this.revoked = revoked; + this.lockStatus = lockStatus == null ? TaskLockStatus.NON_PREEMPTIBLE : lockStatus; + + Preconditions.checkArgument( + !this.type.equals(TaskLockType.SHARED) || this.lockStatus != TaskLockStatus.NON_PREEMPTIBLE, + "lock[%s] cannot be upgraded to non-preemptible", + this.type + ); } public TaskLock( @@ -73,28 +75,46 @@ public TaskLock( int priority ) { - this(type, groupId, dataSource, interval, version, priority, false, false); + this(type, groupId, dataSource, interval, version, priority, TaskLockStatus.PREEMPTIBLE); } public TaskLock upgrade() { - Preconditions.checkState(!revoked, "Revoked locks cannot be upgraded"); - Preconditions.checkState(!upgraded, "Already upgraded"); - return new TaskLock(type, groupId, dataSource, interval, version, priority, true, revoked); + return new TaskLock( + type, + groupId, + dataSource, + interval, + version, + priority, + lockStatus.transitTo(TaskLockStatus.NON_PREEMPTIBLE) + ); } public TaskLock downgrade() { - Preconditions.checkState(!revoked, "Revoked locks cannot be downgraded"); - Preconditions.checkState(upgraded, "Already downgraded"); - return new TaskLock(type, groupId, dataSource, interval, version, priority, false, revoked); + return new TaskLock( + type, + groupId, + dataSource, + interval, + version, + priority, + lockStatus.transitTo(TaskLockStatus.PREEMPTIBLE) + ); } public TaskLock revoke() { - Preconditions.checkState(!revoked, "Already revoked"); - Preconditions.checkState(!upgraded, "Upgraded locks cannot be revoked"); - return new TaskLock(type, groupId, dataSource, interval, version, priority, upgraded, true); + return new TaskLock( + type, + groupId, + dataSource, + interval, + version, + priority, + lockStatus.transitTo(TaskLockStatus.REVOKED) + ); } @JsonProperty @@ -134,15 +154,19 @@ public int getPriority() } @JsonProperty + public TaskLockStatus getLockStatus() + { + return lockStatus; + } + public boolean isUpgraded() { - return upgraded; + return lockStatus == TaskLockStatus.NON_PREEMPTIBLE; } - @JsonProperty public boolean isRevoked() { - return revoked; + return lockStatus == TaskLockStatus.REVOKED; } @Override @@ -158,15 +182,14 @@ public boolean equals(Object o) this.interval.equals(that.interval) && this.version.equals(that.version) && this.priority == that.priority && - this.upgraded == that.upgraded && - this.revoked == that.revoked; + this.lockStatus == that.lockStatus; } } @Override public int hashCode() { - return Objects.hashCode(type, groupId, dataSource, interval, version, priority, upgraded, revoked); + return Objects.hashCode(type, groupId, dataSource, interval, version, priority, lockStatus); } @Override @@ -179,8 +202,7 @@ public String toString() .add("interval", interval) .add("version", version) .add("priority", priority) - .add("upgraded", upgraded) - .add("revoked", revoked) + .add("lockStatus", lockStatus) .toString(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java new file mode 100644 index 000000000000..f85d514bd8ec --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java @@ -0,0 +1,63 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import io.druid.java.util.common.ISE; + +public enum TaskLockStatus +{ + PREEMPTIBLE, + NON_PREEMPTIBLE, + REVOKED; + + @JsonCreator + public static TaskLockStatus fromString(String status) + { + return TaskLockStatus.valueOf(status); + } + + @Override + @JsonValue + public String toString() + { + return this.name(); + } + + public TaskLockStatus transitTo(TaskLockStatus toStatus) + { + // The allowed transitions are + // PREEMPTIBLE -> NON_PREEMPTIBLE -> PREEMPTIBLE + // PREEMPTIBLE -> REVOKED + + if (this == toStatus) { + throw new ISE("Invalid transition from [%s] to [%s]", this, toStatus); + } + + if (this == NON_PREEMPTIBLE && toStatus == PREEMPTIBLE) { + return toStatus; + } else if (this == PREEMPTIBLE) { + return toStatus; + } else { + throw new ISE("Invalid transition from [%s] to [%s]", this, toStatus); + } + } +} From a9ceaceda047fabe214a7ae671211f4d3a59a6e8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 29 Aug 2017 18:07:58 +0900 Subject: [PATCH 12/20] Cleanup suspicious method --- .../common/actions/LockAcquireAction.java | 5 ++-- .../druid/indexing/overlord/TaskLockbox.java | 30 ++++++------------- 2 files changed, 12 insertions(+), 23 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 40e1b8d21c18..0ae84492bfd9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; @@ -46,8 +47,8 @@ public LockAcquireAction( @JsonProperty("timeoutMs") long timeoutMs ) { - this.type = type; - this.interval = interval; + this.type = Preconditions.checkNotNull(type, "lockType"); + this.interval = Preconditions.checkNotNull(interval, "interval"); this.timeoutMs = timeoutMs; } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 163fceeacf0b..0114265111a5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -491,12 +491,7 @@ public LockResult upgrade(Task task, Interval interval) giant.lock(); try { - final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosse( - task, - interval, - findLockPossesContainingInterval(task.getDataSource(), interval) - ); - + final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosseContainingInterval(task, interval); final TaskLock lock = taskLockPosseToUpdate.getTaskLock(); Preconditions.checkState( @@ -543,11 +538,7 @@ public TaskLock downgrade(Task task, Interval interval) giant.lock(); try { - final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosse( - task, - interval, - findLockPossesContainingInterval(task.getDataSource(), interval) - ); + final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosseContainingInterval(task, interval); final TaskLock lock = taskLockPosseToUpdate.taskLock; Preconditions.checkState( @@ -882,22 +873,19 @@ private static boolean isRevocable(TaskLockPosse lockPosse, int tryLockPriority) return existingLock.getPriority() < tryLockPriority && !existingLock.isUpgraded(); } - private static TaskLockPosse getOnlyTaskLockPosse(Task task, Interval interval, List lockPosses) + private TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) { - final List filteredPosses = lockPosses.stream() - .filter(lockPosse -> lockPosse.containsTask(task)) - .collect(Collectors.toList()); + final List filteredPosses = findLockPossesContainingInterval(task.getDataSource(), interval) + .stream() + .filter(lockPosse -> lockPosse.containsTask(task)) + .collect(Collectors.toList()); + if (filteredPosses.isEmpty()) { throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval); } else if (filteredPosses.size() > 1) { throw new ISE("There are multiple lockPosses for task[%s] and interval[%s]?", task.getId(), interval); } else { - final TaskLockPosse foundPosse = filteredPosses.get(0); - if (foundPosse.containsTask(task)) { - return foundPosse; - } else { - throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval); - } + return filteredPosses.get(0); } } From c04bc36759c889fad54c7f561a7778cc5e8104e2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 29 Aug 2017 19:17:32 +0900 Subject: [PATCH 13/20] Add nullables --- .../io/druid/indexing/common/actions/LockAcquireAction.java | 6 ++++-- .../druid/indexing/common/actions/LockTryAcquireAction.java | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 0ae84492bfd9..612281e78f2e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -30,6 +30,8 @@ import io.druid.indexing.overlord.LockResult; import org.joda.time.Interval; +import javax.annotation.Nullable; + public class LockAcquireAction implements TaskAction { private final TaskLockType type; @@ -42,12 +44,12 @@ public class LockAcquireAction implements TaskAction @JsonCreator public LockAcquireAction( - @JsonProperty("lockType") TaskLockType type, + @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility @JsonProperty("interval") Interval interval, @JsonProperty("timeoutMs") long timeoutMs ) { - this.type = Preconditions.checkNotNull(type, "lockType"); + this.type = type == null ? TaskLockType.EXCLUSIVE : type; this.interval = Preconditions.checkNotNull(interval, "interval"); this.timeoutMs = timeoutMs; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java index 9d60a0a61f76..ebbedcff81b2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java @@ -28,6 +28,8 @@ import io.druid.indexing.overlord.LockResult; import org.joda.time.Interval; +import javax.annotation.Nullable; + public class LockTryAcquireAction implements TaskAction { @JsonIgnore @@ -38,11 +40,11 @@ public class LockTryAcquireAction implements TaskAction @JsonCreator public LockTryAcquireAction( - @JsonProperty("lockType") TaskLockType type, + @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility @JsonProperty("interval") Interval interval ) { - this.type = type; + this.type = type == null ? TaskLockType.EXCLUSIVE : type; this.interval = interval; } From 7ad8720a125f3fd0e54c40c3561c06d254e0b51e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 7 Sep 2017 20:26:35 +0900 Subject: [PATCH 14/20] add doInCriticalSection to TaskLockBox and revert return type of task actions --- docs/content/ingestion/tasks.md | 10 +- .../io/druid/indexing/common/TaskLock.java | 60 +---- .../druid/indexing/common/TaskLockStatus.java | 63 ----- .../common/actions/LockAcquireAction.java | 18 +- .../common/actions/LockTryAcquireAction.java | 12 +- .../actions/SegmentMetadataUpdateAction.java | 30 ++- .../common/actions/SegmentNukeAction.java | 30 ++- .../SegmentTransactionalInsertAction.java | 58 ++--- .../indexing/common/task/HadoopIndexTask.java | 11 +- .../common/task/RealtimeIndexTask.java | 30 ++- .../io/druid/indexing/common/task/Tasks.java | 9 +- .../indexing/overlord/CriticalAction.java | 42 ++++ .../druid/indexing/overlord/TaskLockbox.java | 235 +++++++----------- .../common/actions/LockAcquireActionTest.java | 103 ++++++++ .../actions/LockTryAcquireActionTest.java | 97 ++++++++ .../actions/SegmentInsertActionTest.java | 24 +- .../indexing/common/task/IndexTaskTest.java | 33 ++- .../indexing/overlord/RealtimeishTask.java | 10 +- .../overlord/TaskLockBoxConcurrencyTest.java | 139 ++++++++--- .../indexing/overlord/TaskLockboxTest.java | 102 +++----- 20 files changed, 630 insertions(+), 486 deletions(-) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/LockAcquireActionTest.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/LockTryAcquireActionTest.java diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index fd5d5111d528..be39454b5244 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -96,7 +96,7 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking) -Each task has a priority which is used for lock acquisition. Higher-priority tasks can preempt lower-priority tasks if they try to write on the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted. +Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted. Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority. @@ -350,12 +350,12 @@ Once an overlord node accepts a task, the task acquires locks for the data sourc There are two lock types, i.e., _shared lock_ and _exclusive lock_. - A task needs to acquire a shared lock before it reads segments of an interval. Multiple shared locks can be acquired for the same dataSource and interval. Shared locks are always preemptable, but they don't preempt each other. -- A task needs to acquire an exclusive lock before it writes segments for an interval. An exclusive lock is acquired as preemptable and can be upgraded as non-preemptable when publishing segments. +- A task needs to acquire an exclusive lock before it writes segments for an interval. An exclusive lock is also preemptable except while the task is publishing segments. -Each task can have different lock priorities. The locks of higher-priority tasks can preempt the locks of lower-priority tasks. The lock preemption works based on _optimistic locking_. When a lock is preempted, it is not notified to the owner task immediately. Instead, it's notified when the owner task tries to acquire the same lock again or upgrade it. (Note that lock acquisition is idempotent unless the lock is preempted.) In general, tasks don't contend to acquire locks because they usually targets different dataSources or intervals. +Each task can have different lock priorities. The locks of higher-priority tasks can preempt the locks of lower-priority tasks. The lock preemption works based on _optimistic locking_. When a lock is preempted, it is not notified to the owner task immediately. Instead, it's notified when the owner task tries to acquire the same lock again. (Note that lock acquisition is idempotent unless the lock is preempted.) In general, tasks don't compete for acquiring locks because they usually targets different dataSources or intervals. -A task writing data into a dataSource must acquire exclusive locks for target intervals. Note that exclusive locks are still preemptable. As a result, the task must _upgrade_ its locks as non-preemptable when it executes a critical operation, _publishing segments_. Once the lock is upgraded, it can't be preempted by even higher-priority locks. After publishing segments, the task downgrades its locks as preemptable. +A task writing data into a dataSource must acquire exclusive locks for target intervals. Note that exclusive locks are still preemptable. That is, they also be able to be preempted by higher priority locks unless they are _publishing segments_ in a critical section. Once publishing segments is finished, those locks become preemptable again. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release -locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. +locks early if they desire. Task ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks. diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java index 836b1d076077..d83040ddf500 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java @@ -38,7 +38,7 @@ public class TaskLock private final Interval interval; private final String version; private final int priority; - private final TaskLockStatus lockStatus; + private final boolean revoked; @JsonCreator public TaskLock( @@ -48,7 +48,7 @@ public TaskLock( @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JsonProperty("priority") int priority, - @JsonProperty("lockStatus") @Nullable TaskLockStatus lockStatus // nullable for backward compatibility + @JsonProperty("revoked") boolean revoked ) { this.type = type == null ? TaskLockType.EXCLUSIVE : type; @@ -57,13 +57,7 @@ public TaskLock( this.interval = Preconditions.checkNotNull(interval, "interval"); this.version = Preconditions.checkNotNull(version, "version"); this.priority = priority; - this.lockStatus = lockStatus == null ? TaskLockStatus.NON_PREEMPTIBLE : lockStatus; - - Preconditions.checkArgument( - !this.type.equals(TaskLockType.SHARED) || this.lockStatus != TaskLockStatus.NON_PREEMPTIBLE, - "lock[%s] cannot be upgraded to non-preemptible", - this.type - ); + this.revoked = revoked; } public TaskLock( @@ -75,33 +69,7 @@ public TaskLock( int priority ) { - this(type, groupId, dataSource, interval, version, priority, TaskLockStatus.PREEMPTIBLE); - } - - public TaskLock upgrade() - { - return new TaskLock( - type, - groupId, - dataSource, - interval, - version, - priority, - lockStatus.transitTo(TaskLockStatus.NON_PREEMPTIBLE) - ); - } - - public TaskLock downgrade() - { - return new TaskLock( - type, - groupId, - dataSource, - interval, - version, - priority, - lockStatus.transitTo(TaskLockStatus.PREEMPTIBLE) - ); + this(type, groupId, dataSource, interval, version, priority, false); } public TaskLock revoke() @@ -113,7 +81,7 @@ public TaskLock revoke() interval, version, priority, - lockStatus.transitTo(TaskLockStatus.REVOKED) + true ); } @@ -154,19 +122,9 @@ public int getPriority() } @JsonProperty - public TaskLockStatus getLockStatus() - { - return lockStatus; - } - - public boolean isUpgraded() - { - return lockStatus == TaskLockStatus.NON_PREEMPTIBLE; - } - public boolean isRevoked() { - return lockStatus == TaskLockStatus.REVOKED; + return revoked; } @Override @@ -182,14 +140,14 @@ public boolean equals(Object o) this.interval.equals(that.interval) && this.version.equals(that.version) && this.priority == that.priority && - this.lockStatus == that.lockStatus; + this.revoked == that.revoked; } } @Override public int hashCode() { - return Objects.hashCode(type, groupId, dataSource, interval, version, priority, lockStatus); + return Objects.hashCode(type, groupId, dataSource, interval, version, priority, revoked); } @Override @@ -202,7 +160,7 @@ public String toString() .add("interval", interval) .add("version", version) .add("priority", priority) - .add("lockStatus", lockStatus) + .add("revoked", revoked) .toString(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java deleted file mode 100644 index f85d514bd8ec..000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLockStatus.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; -import io.druid.java.util.common.ISE; - -public enum TaskLockStatus -{ - PREEMPTIBLE, - NON_PREEMPTIBLE, - REVOKED; - - @JsonCreator - public static TaskLockStatus fromString(String status) - { - return TaskLockStatus.valueOf(status); - } - - @Override - @JsonValue - public String toString() - { - return this.name(); - } - - public TaskLockStatus transitTo(TaskLockStatus toStatus) - { - // The allowed transitions are - // PREEMPTIBLE -> NON_PREEMPTIBLE -> PREEMPTIBLE - // PREEMPTIBLE -> REVOKED - - if (this == toStatus) { - throw new ISE("Invalid transition from [%s] to [%s]", this, toStatus); - } - - if (this == NON_PREEMPTIBLE && toStatus == PREEMPTIBLE) { - return toStatus; - } else if (this == PREEMPTIBLE) { - return toStatus; - } else { - throw new ISE("Invalid transition from [%s] to [%s]", this, toStatus); - } - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 612281e78f2e..ef612f2d43e1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.LockResult; @@ -32,7 +33,7 @@ import javax.annotation.Nullable; -public class LockAcquireAction implements TaskAction +public class LockAcquireAction implements TaskAction { private final TaskLockType type; @@ -73,22 +74,21 @@ public long getTimeoutMs() } @Override - public TypeReference getReturnTypeReference() + public TypeReference getReturnTypeReference() { - return new TypeReference() + return new TypeReference() { }; } @Override - public LockResult perform(Task task, TaskActionToolbox toolbox) + public TaskLock perform(Task task, TaskActionToolbox toolbox) { try { - if (timeoutMs == 0) { - return toolbox.getTaskLockbox().lock(type, task, interval); - } else { - return toolbox.getTaskLockbox().lock(type, task, interval, timeoutMs); - } + final LockResult result = timeoutMs == 0 ? + toolbox.getTaskLockbox().lock(type, task, interval) : + toolbox.getTaskLockbox().lock(type, task, interval, timeoutMs); + return result.isOk() ? result.getTaskLock() : null; } catch (InterruptedException e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java index ebbedcff81b2..7f73660763ca 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; +import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.LockResult; @@ -30,7 +31,7 @@ import javax.annotation.Nullable; -public class LockTryAcquireAction implements TaskAction +public class LockTryAcquireAction implements TaskAction { @JsonIgnore private final TaskLockType type; @@ -61,17 +62,18 @@ public Interval getInterval() } @Override - public TypeReference getReturnTypeReference() + public TypeReference getReturnTypeReference() { - return new TypeReference() + return new TypeReference() { }; } @Override - public LockResult perform(Task task, TaskActionToolbox toolbox) + public TaskLock perform(Task task, TaskActionToolbox toolbox) { - return toolbox.getTaskLockbox().tryLock(type, task, interval); + final LockResult result = toolbox.getTaskLockbox().tryLock(type, task, interval); + return result.isOk() ? result.getTaskLock() : null; } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index 14ddd718b966..f76fc9770057 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -29,9 +29,12 @@ import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; +import org.joda.time.Interval; import java.io.IOException; +import java.util.List; import java.util.Set; +import java.util.stream.Collectors; public class SegmentMetadataUpdateAction implements TaskAction { @@ -65,17 +68,24 @@ public Void perform( { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - for (DataSegment segment : segments) { - if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { - // Acquired locks don't have to be released explicitly here because the fail result will make the index task - // calling this action failed, and the acquired locks should be released automatically when the task completes. - throw new ISE("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); - } + final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + + try { + toolbox.getTaskLockbox().doInCriticalSection( + task, + intervals, + () -> { + toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); + return null; + }, + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ); + } + catch (Exception e) { + throw new RuntimeException(e); } - - toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); - - segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index 41af97a773e0..5382d3bd1a34 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -29,9 +29,12 @@ import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; +import org.joda.time.Interval; import java.io.IOException; +import java.util.List; import java.util.Set; +import java.util.stream.Collectors; public class SegmentNukeAction implements TaskAction { @@ -65,17 +68,24 @@ public Void perform(Task task, TaskActionToolbox toolbox) throws IOException { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - for (DataSegment segment : segments) { - if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { - // Acquired locks don't have to be released explicitly here because the fail result will make the index task - // calling this action failed, and the acquired locks should be released automatically when the task completes. - throw new ISE("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); - } + final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + + try { + toolbox.getTaskLockbox().doInCriticalSection( + task, + intervals, + () -> { + toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); + return null; + }, + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ); + } + catch (Exception e) { + throw new RuntimeException(e); } - - toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); - - segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 4e214e949795..d6eafd94f0e8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -33,6 +33,7 @@ import java.io.IOException; import java.util.Set; +import java.util.stream.Collectors; /** * Insert segments into metadata storage. The segment versions must all be less than or equal to a lock held by @@ -104,38 +105,39 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - for (DataSegment segment : segments) { - if (!toolbox.getTaskLockbox().upgrade(task, segment.getInterval()).isOk()) { - LOG.info("Failed to upgrade the lock for task[%s] and interval[%s]", task.getId(), segment.getInterval()); - - // Acquired locks don't have to be released explicitly here because the fail result will make the index task - // calling this action failed, and the acquired locks should be released automatically when the task completes. - return SegmentPublishResult.fail(); - } + final SegmentPublishResult retVal; + try { + retVal = toolbox.getTaskLockbox().doInCriticalSection( + task, + segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + startMetadata, + endMetadata + ), + SegmentPublishResult::fail + ); + } + catch (Exception e) { + throw new RuntimeException(e); } - - final SegmentPublishResult retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( - segments, - startMetadata, - endMetadata - ); - - segments.forEach(segment -> toolbox.getTaskLockbox().downgrade(task, segment.getInterval())); - - // Emit metrics - final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) - .setDimension(DruidMetrics.TASK_TYPE, task.getType()); if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); - } else { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); - } + // Emit metrics + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) + .setDimension(DruidMetrics.TASK_TYPE, task.getType()); + + if (retVal.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + } else { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + } - for (DataSegment segment : retVal.getSegments()) { - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + for (DataSegment segment : retVal.getSegments()) { + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } } return retVal; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 467a4eff25a6..b9143f6eb9fd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -43,7 +43,6 @@ import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.hadoop.OverlordActionBasedUsedSegmentLister; -import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; @@ -207,11 +206,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception ); final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, the below line can incur http timeout error. - final LockResult lockResult = toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs) + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", interval ); - Tasks.checkLockResult(lockResult, interval); - version = lockResult.getTaskLock().getVersion(); + version = lock.getVersion(); } else { Iterable locks = getTaskLocks(toolbox.getTaskActionClient()); final TaskLock myLock = Iterables.getOnlyElement(locks); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 5e686c4b97e2..015ae5b79c3b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -34,13 +35,13 @@ import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; +import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockReleaseAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; @@ -229,10 +230,13 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception public void announceSegment(final DataSegment segment) throws IOException { // Side effect: Calling announceSegment causes a lock to be acquired - final LockResult lockResult = toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + segment.getInterval() ); - Tasks.checkLockResult(lockResult, segment.getInterval()); toolbox.getSegmentAnnouncer().announceSegment(segment); } @@ -252,10 +256,13 @@ public void announceSegments(Iterable segments) throws IOException { // Side effect: Calling announceSegments causes locks to be acquired for (DataSegment segment : segments) { - final LockResult lockResult = toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + segment.getInterval() ); - Tasks.checkLockResult(lockResult, segment.getInterval()); } toolbox.getSegmentAnnouncer().announceSegments(segments); } @@ -288,10 +295,13 @@ public String getVersion(final Interval interval) try { // Side effect: Calling getVersion causes a lock to be acquired final LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); - final LockResult lockResult = toolbox.getTaskActionClient().submit(action); + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit(action), + "Cannot acquire a lock for interval[%s]", + interval + ); - Tasks.checkLockResult(lockResult, interval); - return lockResult.getTaskLock().getVersion(); + return lock.getVersion(); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java index d5657d377c7f..6eb36405d7b5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.google.common.base.Preconditions; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.actions.LockTryAcquireAction; @@ -60,9 +61,11 @@ public static Map tryAcquireExclusiveLocks(TaskActionClient { final Map lockMap = new HashMap<>(); for (Interval interval : computeCompactIntervals(intervals)) { - final LockResult lockResult = client.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)); - checkLockResult(lockResult, interval); - lockMap.put(interval, lockResult.getTaskLock()); + final TaskLock lock = Preconditions.checkNotNull( + client.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)), + "Cannot acquire a lock for interval[%s]", interval + ); + lockMap.put(interval, lock); } return lockMap; } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java new file mode 100644 index 000000000000..26f08fe83436 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java @@ -0,0 +1,42 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.overlord; + +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.Task; +import org.joda.time.Interval; + +/** + * This class represents an action must be done while the task's lock is guaranteed to not be revoked in the middle of + * action. + * + * Implementations must not change the lock state by calling {@link TaskLockbox#lock(TaskLockType, Task, Interval)}, + * {@link TaskLockbox#lock(TaskLockType, Task, Interval, long)}, {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}, + * or {@link TaskLockbox#unlock(Task, Interval)}. + * + * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked + * until this action is finished. + * + * @see TaskLockbox#doInCriticalSection(Task, List, CriticalAction, CriticalAction) + */ +public interface CriticalAction +{ + T perform() throws Exception; +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 0114265111a5..e49bb312bd89 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -69,7 +69,7 @@ public class TaskLockbox // Datasource -> Interval -> list of (Tasks + TaskLock) // Multiple shared locks can be acquired for the same dataSource and interval. // Note that revoked locks are also maintained in this map to notify that those locks are revoked to the callers when - // they acquire the same locks again or request lock upgrade/downgrade. + // they acquire the same locks again. private final Map>> running = Maps.newHashMap(); private final TaskStorage taskStorage; private final ReentrantLock giant = new ReentrantLock(true); @@ -184,12 +184,14 @@ public int compare(Pair left, Pair right) } /** - * Acquires a lock on behalf of a task. Blocks until the lock is acquired. + * Acquires a lock on behalf of a task. Blocks until the lock is acquired. * * @param lockType lock type * @param task task to acquire lock for * @param interval interval to lock - * @return acquired TaskLock + * + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. * * @throws InterruptedException if the current thread is interrupted */ @@ -203,6 +205,9 @@ public LockResult lock( try { LockResult lockResult; while (!(lockResult = tryLock(lockType, task, interval)).isOk()) { + if (lockResult.isRevoked()) { + return lockResult; + } lockReleaseCondition.await(); } return lockResult; @@ -220,7 +225,8 @@ public LockResult lock( * @param interval interval to lock * @param timeoutMs maximum time to wait * - * @return acquired lock + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. * * @throws InterruptedException if the current thread is interrupted */ @@ -236,7 +242,7 @@ public LockResult lock( try { LockResult lockResult; while (!(lockResult = tryLock(lockType, task, interval)).isOk()) { - if (nanos <= 0) { + if (nanos <= 0 || lockResult.isRevoked()) { return lockResult; } nanos = lockReleaseCondition.awaitNanos(nanos); @@ -256,7 +262,9 @@ public LockResult lock( * @param task task that wants a lock * @param interval interval to lock * - * @return lock version if lock was acquired, absent otherwise + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. + * * @throws IllegalStateException if the task is not a valid active task */ public LockResult tryLock( @@ -308,6 +316,10 @@ public LockResult tryLock( } } + /** + * See {@link #createOrFindLockPosse(Task, Interval, String, TaskLockType)} + */ + @Nullable private TaskLockPosse createOrFindLockPosse( final Task task, final Interval interval, @@ -318,17 +330,19 @@ private TaskLockPosse createOrFindLockPosse( } /** - * Create a new {@link TaskLockPosse} or find an existing one for the given task and interval. + * Create a new {@link TaskLockPosse} or find an existing one for the given task and interval. Note that the returned + * {@link TaskLockPosse} can hold a revoked lock. * * @param task task acquiring a lock * @param interval interval to be locked * @param preferredVersion a preferred version string * @param lockType type of lock to be acquired * - * @return a lock posse + * @return a lock posse or null if any posse is found and a new poss cannot be created * * @see #createNewTaskLockPosse(TaskLockType, String, String, Interval, String, int) */ + @Nullable private TaskLockPosse createOrFindLockPosse( final Task task, final Interval interval, @@ -343,17 +357,18 @@ private TaskLockPosse createOrFindLockPosse( final int priority = task.getPriority(); final List foundPosses = findLockPossesOverlapsInterval(dataSource, interval); - // If we have some locks for dataSource and interval, check they can be reused. - // If they can't be reused, check lock priority and revoke existing locks if possible. if (foundPosses.size() > 0) { + // If we have some locks for dataSource and interval, check they can be reused. + // If they can't be reused, check lock priority and revoke existing locks if possible. + final List filteredPosses = foundPosses + .stream() + .filter(posse -> matchGroupIdAndContainInterval(posse.taskLock, task, interval)) + .collect(Collectors.toList()); - if (lockType.equals(TaskLockType.SHARED) && isAllSharedLocks(foundPosses)) { - final List possesForReusableLocks = foundPosses - .stream() - .filter(lockPosse -> matchGroupIdAndContainInterval(lockPosse.getTaskLock(), task, interval)) - .collect(Collectors.toList()); + if (filteredPosses.size() == 0) { + // case 1) this task doesn't have any lock, but others do - if (possesForReusableLocks.size() == 0) { + if (lockType.equals(TaskLockType.SHARED) && isAllSharedLocks(foundPosses)) { // Any number of shared locks can be acquired for the same dataSource and interval. return createNewTaskLockPosse( lockType, @@ -363,45 +378,44 @@ private TaskLockPosse createOrFindLockPosse( preferredVersion, priority ); - } else if (possesForReusableLocks.size() == 1) { - return possesForReusableLocks.get(0); } else { - throw new ISE( - "Task group[%s] has multiple locks for the same interval[%s]?", - task.getGroupId(), - interval - ); - } - } else { - if (foundPosses.size() == 1 && - matchGroupIdAndContainInterval(foundPosses.get(0).taskLock, task, interval)) { - final TaskLockPosse foundPosse = foundPosses.get(0); - if (lockType.equals(foundPosse.getTaskLock().getType())) { - return foundPosse; - } else { - throw new ISE( - "Task[%s] already acquired a lock for interval[%s] but different type[%s]", - task.getId(), + if (isAllRevocable(foundPosses, priority)) { + // Revoke all existing locks + foundPosses.forEach(this::revokeLock); + + return createNewTaskLockPosse( + lockType, + task.getGroupId(), + dataSource, interval, - foundPosse.getTaskLock().getType() + preferredVersion, + priority ); + } else { + log.info("Cannot create a new taskLockPosse because some locks of same or higher priorities exist"); + return null; } - } else if (isAllRevocable(foundPosses, priority)) { - // Revoke all existing locks - foundPosses.forEach(this::revokeLock); - - return createNewTaskLockPosse( - lockType, - task.getGroupId(), - dataSource, + } + } else if (filteredPosses.size() == 1) { + // case 2) we found a lock posse for the given task + final TaskLockPosse foundPosse = filteredPosses.get(0); + if (lockType.equals(foundPosse.getTaskLock().getType())) { + return foundPosse; + } else { + throw new ISE( + "Task[%s] already acquired a lock for interval[%s] but different type[%s]", + task.getId(), interval, - preferredVersion, - priority + foundPosse.getTaskLock().getType() ); - } else { - log.info("Cannot create a new taskLockPosse because some locks of higher priorities exist"); - return null; } + } else { + // case 3) we found multiple lock posses for the given task + throw new ISE( + "Task group[%s] has multiple locks for the same interval[%s]?", + task.getGroupId(), + interval + ); } } else { // We don't have any locks for dataSource and interval. @@ -479,95 +493,43 @@ private TaskLockPosse createNewTaskLockPosse( } /** - * Upgrade a lock for the given task and interval if possible. Shared locks and revoked locks cannot be upgraded. + * 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. * - * @return a succeeded {@link LockResult} if the lock is upgraded. If an upgrade for a revoked lock is requested, a - * failed {@link LockResult} is returned. + * The given action should be finished as soon as possible because all other methods in this class are blocked until + * this method is finished. * - * @throws IllegalStateException if an upgrade for a shared lock is requested + * @param task task performing a critical action + * @param intervals intervals + * @param actionOnValidLocks action to be performed when all locks are valid + * @param actionOnInvalidLocks action to be performed when some locks are invalid */ - public LockResult upgrade(Task task, Interval interval) + public T doInCriticalSection( + Task task, + List intervals, + CriticalAction actionOnValidLocks, + CriticalAction actionOnInvalidLocks + ) throws Exception { - giant.lock(); + giant.lockInterruptibly(); try { - final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosseContainingInterval(task, interval); - final TaskLock lock = taskLockPosseToUpdate.getTaskLock(); - - Preconditions.checkState( - lock.getType().equals(TaskLockType.EXCLUSIVE), - "Shared lock cannot be upgraded" - ); - - if (lock.isRevoked()) { - return LockResult.fail(true); - } else { - log.info("Upgrading taskLock[%s] for task[%s] and interval[%s]", lock, task.getId(), interval); - - if (lock.isUpgraded()) { - log.warn("TaskLock[%s] is already upgraded", lock); - return LockResult.ok(lock); - } else { - final TaskLock upgradedLock = lock.upgrade(); - taskStorage.replaceLock(task.getId(), lock, upgradedLock); - - // Possible exceptions are already checked above - final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); - - possesHolder.remove(taskLockPosseToUpdate); - possesHolder.add(taskLockPosseToUpdate.withTaskLock(upgradedLock)); - log.info("Upgraded taskLock[%s]"); - return LockResult.ok(upgradedLock); - } - } + return isTaskLocksValid(task, intervals) ? actionOnValidLocks.perform() : actionOnInvalidLocks.perform(); } finally { giant.unlock(); } } - /** - * Downgrade a lock for the given task and interval if possible. Shared locks cannot be downgraded. - * - * @return a downgraded lock if succeeded. - * - * @throws IllegalStateException if a downgrade for a shared lock is requested - */ - public TaskLock downgrade(Task task, Interval interval) + private boolean isTaskLocksValid(Task task, List intervals) { - giant.lock(); - - try { - final TaskLockPosse taskLockPosseToUpdate = getOnlyTaskLockPosseContainingInterval(task, interval); - final TaskLock lock = taskLockPosseToUpdate.taskLock; - - Preconditions.checkState( - lock.getType().equals(TaskLockType.EXCLUSIVE), - "Shared lock cannot be downgraded" - ); - - // Upgraded locks cannot be revoked, so ignore the check to verify the lock is revoked. - log.info("Downgrading taskLock[%s] for task[%s] and interval[%s]", lock, task.getId(), interval); - - if (!lock.isUpgraded()) { - log.warn("TaskLock[%s] is already downgraded", lock); - return lock; - } else { - final TaskLock downgradedLock = lock.downgrade(); - taskStorage.replaceLock(task.getId(), lock, downgradedLock); - - // Possible exceptions are already checked above - final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); - - possesHolder.remove(taskLockPosseToUpdate); - possesHolder.add(taskLockPosseToUpdate.withTaskLock(downgradedLock)); - log.info("Downgraded taskLock[%s]"); - return downgradedLock; - } - } - finally { - giant.unlock(); - } + return intervals + .stream() + .allMatch(interval -> { + final TaskLock lock = getOnlyTaskLockPosseContainingInterval(task, interval).getTaskLock(); + // Tasks cannot enter the critical section with a shared lock + return !lock.isRevoked() && lock.getType() != TaskLockType.SHARED; + }); } private void revokeLock(TaskLockPosse lockPosse) @@ -585,7 +547,7 @@ private void revokeLock(TaskLockPosse lockPosse) /** * Mark the lock as revoked. Note that revoked locks are NOT removed. Instead, they are maintained in {@link #running} * and {@link #taskStorage} as the normal locks do. This is to check locks are revoked when they are requested to be - * acquired or upgraded and notify to the callers if revoked. Revoked locks are removed by calling + * acquired and notify to the callers if revoked. Revoked locks are removed by calling * {@link #unlock(Task, Interval)}. * * @param taskId an id of the task holding the lock @@ -870,7 +832,7 @@ private static boolean isAllRevocable(List lockPosses, int tryLoc private static boolean isRevocable(TaskLockPosse lockPosse, int tryLockPriority) { final TaskLock existingLock = lockPosse.getTaskLock(); - return existingLock.getPriority() < tryLockPriority && !existingLock.isUpgraded(); + return existingLock.isRevoked() || existingLock.getPriority() < tryLockPriority; } private TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) @@ -889,27 +851,6 @@ private TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval } } - /** - * Find a {@link TaskLockPosse} for a task and an exact interval. - */ - private List findLockPosse(Task task, Interval interval) - { - final NavigableMap> dsRunning = running.get(task.getDataSource()); - - if (dsRunning == null || dsRunning.isEmpty()) { - throw new ISE("Cannot find lock for dataSource[%s]", task.getDataSource()); - } - - final List possesHolder = dsRunning.get(interval); - if (possesHolder == null || possesHolder.isEmpty()) { - throw new ISE("Not found lock for task[%s] and interval[%s]", task.getId(), interval); - } - - return possesHolder.stream() - .filter(posse -> posse.containsTask(task)) - .collect(Collectors.toList()); - } - @VisibleForTesting Set getActiveTasks() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/LockAcquireActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/LockAcquireActionTest.java new file mode 100644 index 000000000000..16638713d73f --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/LockAcquireActionTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Intervals; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +public class LockAcquireActionTest +{ + @Rule + public TaskActionTestKit actionTestKit = new TaskActionTestKit(); + + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testSerdeWithAllFields() throws IOException + { + final LockAcquireAction expected = new LockAcquireAction( + TaskLockType.SHARED, + Intervals.of("2017-01-01/2017-01-02"), + 1000 + ); + + final byte[] bytes = mapper.writeValueAsBytes(expected); + final LockAcquireAction actual = mapper.readValue(bytes, LockAcquireAction.class); + Assert.assertEquals(expected.getType(), actual.getType()); + Assert.assertEquals(expected.getInterval(), actual.getInterval()); + Assert.assertEquals(expected.getTimeoutMs(), actual.getTimeoutMs()); + } + + @Test + public void testSerdeFromJsonWithMissingFields() throws IOException + { + final String json = "{ \"type\": \"lockAcquire\", \"interval\" : \"2017-01-01/2017-01-02\" }"; + + final LockAcquireAction actual = mapper.readValue(json, LockAcquireAction.class); + final LockAcquireAction expected = new LockAcquireAction( + TaskLockType.EXCLUSIVE, + Intervals.of("2017-01-01/2017-01-02"), + 0 + ); + Assert.assertEquals(expected.getType(), actual.getType()); + Assert.assertEquals(expected.getInterval(), actual.getInterval()); + Assert.assertEquals(expected.getTimeoutMs(), actual.getTimeoutMs()); + } + + @Test(timeout = 5000L) + public void testWithLockType() + { + final Task task = NoopTask.create(); + final LockAcquireAction action = new LockAcquireAction( + TaskLockType.EXCLUSIVE, + Intervals.of("2017-01-01/2017-01-02"), + 1000 + ); + + actionTestKit.getTaskLockbox().add(task); + final TaskLock lock = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertNotNull(lock); + } + + @Test(timeout = 5000L) + public void testWithoutLockType() + { + final Task task = NoopTask.create(); + final LockAcquireAction action = new LockAcquireAction( + null, + Intervals.of("2017-01-01/2017-01-02"), + 1000 + ); + + actionTestKit.getTaskLockbox().add(task); + final TaskLock lock = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertNotNull(lock); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/LockTryAcquireActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/LockTryAcquireActionTest.java new file mode 100644 index 000000000000..29f06f521724 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/LockTryAcquireActionTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Intervals; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +public class LockTryAcquireActionTest +{ + @Rule + public TaskActionTestKit actionTestKit = new TaskActionTestKit(); + + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testSerdeWithAllFields() throws IOException + { + final LockTryAcquireAction expected = new LockTryAcquireAction( + TaskLockType.SHARED, + Intervals.of("2017-01-01/2017-01-02") + ); + + final byte[] bytes = mapper.writeValueAsBytes(expected); + final LockTryAcquireAction actual = mapper.readValue(bytes, LockTryAcquireAction.class); + Assert.assertEquals(expected.getType(), actual.getType()); + Assert.assertEquals(expected.getInterval(), actual.getInterval()); + } + + @Test + public void testSerdeFromJsonWithMissingFields() throws IOException + { + final String json = "{ \"type\": \"lockTryAcquire\", \"interval\" : \"2017-01-01/2017-01-02\" }"; + + final LockTryAcquireAction actual = mapper.readValue(json, LockTryAcquireAction.class); + final LockTryAcquireAction expected = new LockTryAcquireAction( + TaskLockType.EXCLUSIVE, + Intervals.of("2017-01-01/2017-01-02") + ); + Assert.assertEquals(expected.getType(), actual.getType()); + Assert.assertEquals(expected.getInterval(), actual.getInterval()); + } + + @Test(timeout = 5000L) + public void testWithLockType() + { + final Task task = NoopTask.create(); + final LockTryAcquireAction action = new LockTryAcquireAction( + TaskLockType.EXCLUSIVE, + Intervals.of("2017-01-01/2017-01-02") + ); + + actionTestKit.getTaskLockbox().add(task); + final TaskLock lock = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertNotNull(lock); + } + + @Test(timeout = 5000L) + public void testWithoutLockType() + { + final Task task = NoopTask.create(); + final LockTryAcquireAction action = new LockTryAcquireAction( + null, + Intervals.of("2017-01-01/2017-01-02") + ); + + actionTestKit.getTaskLockbox().add(task); + final TaskLock lock = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertNotNull(lock); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index 3198713bc709..0f304b659e89 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -35,6 +35,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; import java.util.Set; public class SegmentInsertActionTest @@ -93,8 +94,15 @@ public void testSimple() throws Exception final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); actionTestKit.getTaskLockbox().add(task); actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - actionTestKit.getTaskLockbox().upgrade(task, INTERVAL); - action.perform(task, actionTestKit.getTaskActionToolbox()); + actionTestKit.getTaskLockbox().doInCriticalSection( + task, + Collections.singletonList(INTERVAL), + () -> action.perform(task, actionTestKit.getTaskActionToolbox()), + () -> { + Assert.fail(); + return null; + } + ); Assert.assertEquals( ImmutableSet.of(SEGMENT1, SEGMENT2), @@ -112,11 +120,19 @@ public void testFailBadVersion() throws Exception final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - actionTestKit.getTaskLockbox().upgrade(task, INTERVAL); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); - final Set segments = action.perform(task, actionTestKit.getTaskActionToolbox()); + final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( + task, + Collections.singletonList(INTERVAL), + () -> action.perform(task, actionTestKit.getTaskActionToolbox()), + () -> { + Assert.fail(); + return null; + } + ); + Assert.assertEquals(ImmutableSet.of(SEGMENT3), segments); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index be12f0e1be41..465bb8c638bc 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -42,7 +42,6 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import io.druid.indexing.overlord.LockResult; import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; @@ -905,28 +904,24 @@ public RetType submit(TaskAction taskAction) throws IOExcepti } if (taskAction instanceof LockAcquireAction) { - return (RetType) LockResult.ok( - new TaskLock( - TaskLockType.EXCLUSIVE, - "groupId", - "test", - ((LockAcquireAction) taskAction).getInterval(), - DateTimes.nowUtc().toString(), - Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY - ) + return (RetType) new TaskLock( + TaskLockType.EXCLUSIVE, + "groupId", + "test", + ((LockAcquireAction) taskAction).getInterval(), + DateTimes.nowUtc().toString(), + Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY ); } if (taskAction instanceof LockTryAcquireAction) { - return (RetType) LockResult.ok( - new TaskLock( - TaskLockType.EXCLUSIVE, - "groupId", - "test", - ((LockTryAcquireAction) taskAction).getInterval(), - DateTimes.nowUtc().toString(), - Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY - ) + return (RetType) new TaskLock( + TaskLockType.EXCLUSIVE, + "groupId", + "test", + ((LockTryAcquireAction) taskAction).getInterval(), + DateTimes.nowUtc().toString(), + Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index 7edff629f186..2195de0b7f6b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -74,11 +74,10 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final LockResult lockResult1 = toolbox.getTaskActionClient().submit( + final TaskLock lock1 = toolbox.getTaskActionClient().submit( new LockAcquireAction(TaskLockType.EXCLUSIVE, interval1, 5000) ); - Assert.assertTrue(lockResult1.isOk() && !lockResult1.isRevoked()); - final TaskLock lock1 = lockResult1.getTaskLock(); + Assert.assertNotNull(lock1); final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) @@ -86,11 +85,10 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1); // Acquire lock for second interval - final LockResult lockResult2 = toolbox.getTaskActionClient().submit( + final TaskLock lock2 = toolbox.getTaskActionClient().submit( new LockAcquireAction(TaskLockType.EXCLUSIVE, interval2, 5000) ); - Assert.assertTrue(lockResult2.isOk() && !lockResult2.isRevoked()); - final TaskLock lock2 = lockResult2.getTaskLock(); + Assert.assertNotNull(lock2); final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 6fe55e8f5d14..593657e75c2b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -20,7 +20,8 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.indexing.common.TaskLock; +import com.google.common.collect.ImmutableList; +import io.druid.common.guava.SettableSupplier; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.config.TaskStorageConfig; @@ -38,6 +39,8 @@ import org.junit.Rule; import org.junit.Test; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -80,7 +83,8 @@ public void teardown() } @Test(timeout = 5000L) - public void testTryExclusiveLock() throws ExecutionException, InterruptedException, EntryExistsException + public void testDoInCriticalSectionWithDifferentTasks() + throws ExecutionException, InterruptedException, EntryExistsException { final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task lowPriorityTask = NoopTask.create(10); @@ -90,55 +94,116 @@ public void testTryExclusiveLock() throws ExecutionException, InterruptedExcepti taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); + final SettableSupplier intSupplier = new SettableSupplier<>(0); final CountDownLatch latch = new CountDownLatch(1); - final Future lowPriorityFuture = service.submit(() -> { - final LockResult lock = lockbox.tryLock( - TaskLockType.EXCLUSIVE, + + // lowPriorityTask acquires a lock first and increases the int of intSupplier in the critical section + final Future lowPriorityFuture = service.submit(() -> { + final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + Assert.assertTrue(result.isOk()); + Assert.assertFalse(result.isRevoked()); + + return lockbox.doInCriticalSection( lowPriorityTask, - interval + Collections.singletonList(interval), + () -> { + latch.countDown(); + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + }, + () -> { + Assert.fail(); + return null; + } ); - latch.countDown(); - return lock; }); - final Future highPriorityFuture = service.submit(() -> { + + // highPriorityTask awaits for the latch, acquires a lock, and increases the int of intSupplier in the critical + // section + final Future highPriorityFuture = service.submit(() -> { latch.await(); - return lockbox.tryLock( - TaskLockType.EXCLUSIVE, + final LockResult result = lockbox.lock(TaskLockType.EXCLUSIVE, highPriorityTask, interval); + Assert.assertTrue(result.isOk()); + Assert.assertFalse(result.isRevoked()); + + return lockbox.doInCriticalSection( highPriorityTask, - interval + Collections.singletonList(interval), + () -> { + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + }, + () -> { + Assert.fail(); + return null; + } ); }); - final TaskLock lowLock = lowPriorityFuture.get().getTaskLock(); - final TaskLock highLock = highPriorityFuture.get().getTaskLock(); + Assert.assertEquals(1, lowPriorityFuture.get().intValue()); + Assert.assertEquals(2, highPriorityFuture.get().intValue()); - Assert.assertNotNull(lowLock); - Assert.assertNotNull(highLock); + // the lock for lowPriorityTask must be revoked by the highPriorityTask after its work is done in critical section + final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + Assert.assertFalse(result.isOk()); + Assert.assertTrue(result.isRevoked()); + } - final Future lowUpgradeFuture = service.submit( - () -> lockbox.upgrade(lowPriorityTask, interval) - ); - final Future highUpgradeFuture = service.submit( - () -> lockbox.upgrade(highPriorityTask, interval) + @Test(timeout = 5000L) + public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception + { + final List intervals = ImmutableList.of( + Intervals.of("2017-01-01/2017-01-02"), + Intervals.of("2017-01-02/2017-01-03"), + Intervals.of("2017-01-03/2017-01-04") ); + final Task task = NoopTask.create(); + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); - final LockResult resultOfHighPriorityLock = highUpgradeFuture.get(); - Assert.assertTrue(resultOfHighPriorityLock.isOk()); - Assert.assertTrue(resultOfHighPriorityLock.getTaskLock().isUpgraded()); - assertEqualsExceptUpgraded(highLock, resultOfHighPriorityLock.getTaskLock()); + for (Interval interval : intervals) { + final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval); + Assert.assertTrue(result.isOk()); + } - final LockResult resultOfLowPriorityLock = lowUpgradeFuture.get(); - Assert.assertFalse(resultOfLowPriorityLock.isOk()); - Assert.assertTrue(resultOfLowPriorityLock.isRevoked()); - } + final SettableSupplier intSupplier = new SettableSupplier<>(0); + final CountDownLatch latch = new CountDownLatch(1); - private static void assertEqualsExceptUpgraded(TaskLock expected, TaskLock actual) - { - Assert.assertEquals(expected.getGroupId(), actual.getGroupId()); - Assert.assertEquals(expected.getDataSource(), actual.getDataSource()); - Assert.assertEquals(expected.getType(), actual.getType()); - Assert.assertEquals(expected.getInterval(), actual.getInterval()); - Assert.assertEquals(expected.getVersion(), actual.getVersion()); - Assert.assertEquals(expected.getPriority(), actual.getPriority()); + final Future future1 = service.submit(() -> lockbox.doInCriticalSection( + task, + ImmutableList.of(intervals.get(0), intervals.get(1)), + () -> { + latch.countDown(); + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + }, + () -> { + Assert.fail(); + return null; + } + )); + + final Future future2 = service.submit(() -> { + latch.await(); + return lockbox.doInCriticalSection( + task, + ImmutableList.of(intervals.get(1), intervals.get(2)), + () -> { + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + }, + () -> { + Assert.fail(); + return null; + } + ); + }); + + Assert.assertEquals(1, future1.get().intValue()); + Assert.assertEquals(2, future2.get().intValue()); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index 3ff9cd84df0f..0a4c5337164f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -45,6 +45,7 @@ import org.junit.rules.ExpectedException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -259,35 +260,20 @@ public void testSyncFromStorage() throws EntryExistsException } @Test - public void testUpgradeSharedLock() + public void testDoInCriticalSectionWithSharedLock() throws Exception { - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Shared lock cannot be upgraded"); - - final Interval interval = Intervals.of("2017-01-01/2017-01-02"); - final Task task = NoopTask.create(); - lockbox.add(task); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); - - lockbox.upgrade(task, interval); - } - - @Test - public void testDowngradeSharedLock() - { - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Shared lock cannot be downgraded"); - final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task task = NoopTask.create(); lockbox.add(task); Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); - lockbox.downgrade(task, interval); + Assert.assertFalse( + lockbox.doInCriticalSection(task, Collections.singletonList(interval), () -> true, () -> false) + ); } @Test - public void testUpgradeAndDowngradeExclusiveLock() + public void testDoInCriticalSectionWithExclusiveLock() throws Exception { final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task task = NoopTask.create(); @@ -295,39 +281,30 @@ public void testUpgradeAndDowngradeExclusiveLock() final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); - final LockResult upgradeResult = lockbox.upgrade(task, interval); - Assert.assertTrue(upgradeResult.isOk()); - Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); - - final TaskLock downgradedLock = lockbox.downgrade(task, interval); - Assert.assertNotNull(downgradedLock); - Assert.assertFalse(downgradedLock.isUpgraded()); + Assert.assertTrue( + lockbox.doInCriticalSection(task, Collections.singletonList(interval), () -> true, () -> false) + ); } @Test - public void testUpgradeDownGradeWithSmallerInterval() + public void testDoInCriticalSectionWithSmallerInterval() throws Exception { - final Interval interval = Intervals.of("2017-01-01/2017-02-01"); - final Interval smallInterval1 = Intervals.of("2017-01-01/2017-01-02"); - final Interval smallInterval2 = Intervals.of("2017-01-10/2017-01-11"); + final Interval interval = new Interval("2017-01-01/2017-02-01"); + final Interval smallInterval = new Interval("2017-01-10/2017-01-11"); final Task task = NoopTask.create(); lockbox.add(task); final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); - final LockResult upgradeResult = lockbox.upgrade(task, smallInterval1); - Assert.assertTrue(upgradeResult.isOk()); - Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); - - final TaskLock downgradedLock = lockbox.downgrade(task, smallInterval2); - Assert.assertNotNull(downgradedLock); - Assert.assertFalse(downgradedLock.isUpgraded()); + Assert.assertTrue( + lockbox.doInCriticalSection(task, Collections.singletonList(smallInterval), () -> true, () -> false) + ); } @Test - public void testPreemptAndUpgradeExclusiveLock() throws EntryExistsException + public void testPreemptionAndDoInCriticalSection() throws Exception { - final Interval interval = Intervals.of("2017-01-01/2017-01-02"); + final Interval interval = new Interval("2017-01-01/2017-01-02"); for (int i = 0; i < 5; i++) { final Task task = NoopTask.create(); lockbox.add(task); @@ -341,36 +318,13 @@ public void testPreemptAndUpgradeExclusiveLock() throws EntryExistsException final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).getTaskLock(); Assert.assertNotNull(lock); - final LockResult upgradeResult = lockbox.upgrade(highPriorityTask, interval); - Assert.assertTrue(upgradeResult.isOk()); - Assert.assertTrue(upgradeResult.getTaskLock().isUpgraded()); - } - - @Test - public void testPreemption() throws EntryExistsException - { - final Interval interval = Intervals.of("2017-01-01/2017-01-02"); - final Task lowPriorityTask = NoopTask.create(10); - final Task highPriorityTask = NoopTask.create(100); - lockbox.add(lowPriorityTask); - lockbox.add(highPriorityTask); - taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); - taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); - - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).isOk()); - Assert.assertNotNull(lockbox.upgrade(lowPriorityTask, interval)); - - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); - Assert.assertNotNull(lockbox.downgrade(lowPriorityTask, interval)); - - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); - - final List lowPriorityLocks = taskStorage.getLocks(lowPriorityTask.getId()); - Assert.assertTrue(lowPriorityLocks.stream().allMatch(TaskLock::isRevoked)); + Assert.assertTrue( + lockbox.doInCriticalSection(highPriorityTask, Collections.singletonList(interval), () -> true, () -> false) + ); } @Test - public void testUpgradeRevokedLock() throws EntryExistsException + public void testDoInCriticalSectionWithRevokedLock() throws Exception { final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task lowPriorityTask = NoopTask.create("task1", 0); @@ -385,13 +339,13 @@ public void testUpgradeRevokedLock() throws EntryExistsException Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); - final LockResult upgradeResult = lockbox.upgrade(lowPriorityTask, interval); - Assert.assertFalse(upgradeResult.isOk()); - Assert.assertTrue(upgradeResult.isRevoked()); + Assert.assertFalse( + lockbox.doInCriticalSection(lowPriorityTask, Collections.singletonList(interval), () -> true, () -> false) + ); } - @Test - public void testAcquireLockAfterRevoked() throws EntryExistsException + @Test(timeout = 5000L) + public void testAcquireLockAfterRevoked() throws EntryExistsException, InterruptedException { final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task lowPriorityTask = NoopTask.create("task1", 0); @@ -401,7 +355,7 @@ public void testAcquireLockAfterRevoked() throws EntryExistsException taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); - final TaskLock lowPriorityLock = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); + final TaskLock lowPriorityLock = lockbox.lock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); Assert.assertNotNull(lowPriorityLock); Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); @@ -409,7 +363,7 @@ public void testAcquireLockAfterRevoked() throws EntryExistsException lockbox.unlock(highPriorityTask, interval); // Acquire again - final LockResult lockResult = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + final LockResult lockResult = lockbox.lock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); Assert.assertFalse(lockResult.isOk()); Assert.assertTrue(lockResult.isRevoked()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); From 07ace37023fd0b7aa29b24e4c0b321096f761827 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 7 Sep 2017 21:36:47 +0900 Subject: [PATCH 15/20] fix build --- .../java/io/druid/indexing/overlord/TaskLockboxTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index 0a4c5337164f..fa36ba777b07 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -289,8 +289,8 @@ public void testDoInCriticalSectionWithExclusiveLock() throws Exception @Test public void testDoInCriticalSectionWithSmallerInterval() throws Exception { - final Interval interval = new Interval("2017-01-01/2017-02-01"); - final Interval smallInterval = new Interval("2017-01-10/2017-01-11"); + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + final Interval smallInterval = Intervals.of("2017-01-10/2017-01-11"); final Task task = NoopTask.create(); lockbox.add(task); final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); @@ -304,7 +304,7 @@ public void testDoInCriticalSectionWithSmallerInterval() throws Exception @Test public void testPreemptionAndDoInCriticalSection() throws Exception { - final Interval interval = new Interval("2017-01-01/2017-01-02"); + final Interval interval = Intervals.of("2017-01-01/2017-01-02"); for (int i = 0; i < 5; i++) { final Task task = NoopTask.create(); lockbox.add(task); From 42ab8f7111990130bd85e66e0e6d5ada43f502c4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Oct 2017 10:54:58 +0900 Subject: [PATCH 16/20] refactor CriticalAction --- .../io/druid/indexing/common/TaskLock.java | 2 +- .../actions/SegmentMetadataUpdateAction.java | 21 ++-- .../common/actions/SegmentNukeAction.java | 21 ++-- .../SegmentTransactionalInsertAction.java | 17 +-- .../indexing/overlord/CriticalAction.java | 54 +++++++++- .../druid/indexing/overlord/TaskLockbox.java | 7 +- .../actions/SegmentInsertActionTest.java | 29 +++-- .../overlord/TaskLockBoxConcurrencyTest.java | 100 +++++++++++------- .../indexing/overlord/TaskLockboxTest.java | 30 +++++- 9 files changed, 198 insertions(+), 83 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java index d83040ddf500..5c7ca14b30d1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskLock.java @@ -72,7 +72,7 @@ public TaskLock( this(type, groupId, dataSource, interval, version, priority, false); } - public TaskLock revoke() + public TaskLock revokedCopy() { return new TaskLock( type, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index f76fc9770057..fb65c3b787b3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.CriticalAction; import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -74,13 +75,19 @@ public Void perform( toolbox.getTaskLockbox().doInCriticalSection( task, intervals, - () -> { - toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); - return null; - }, - () -> { - throw new ISE("Some locks for task[%s] are already revoked", task.getId()); - } + CriticalAction.builder() + .onValidLocks( + () -> { + toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); + return null; + } + ) + .onInvalidLocks( + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ) + .build() ); } catch (Exception e) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index 5382d3bd1a34..b9bc03341252 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.CriticalAction; import io.druid.java.util.common.ISE; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -74,13 +75,19 @@ public Void perform(Task task, TaskActionToolbox toolbox) throws IOException toolbox.getTaskLockbox().doInCriticalSection( task, intervals, - () -> { - toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); - return null; - }, - () -> { - throw new ISE("Some locks for task[%s] are already revoked", task.getId()); - } + CriticalAction.builder() + .onValidLocks( + () -> { + toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); + return null; + } + ) + .onInvalidLocks( + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ) + .build() ); } catch (Exception e) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index d6eafd94f0e8..c22275def06a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.CriticalAction; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.java.util.common.logger.Logger; @@ -110,12 +111,16 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws retVal = toolbox.getTaskLockbox().doInCriticalSection( task, segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), - () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( - segments, - startMetadata, - endMetadata - ), - SegmentPublishResult::fail + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + startMetadata, + endMetadata + ) + ) + .onInvalidLocks(SegmentPublishResult::fail) + .build() ); } catch (Exception e) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java index 26f08fe83436..16e77d49c7f5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java @@ -19,11 +19,10 @@ package io.druid.indexing.overlord; -import io.druid.indexing.common.TaskLockType; -import io.druid.indexing.common.task.Task; -import org.joda.time.Interval; +import com.google.common.base.Preconditions; /** + * TODO: fix * This class represents an action must be done while the task's lock is guaranteed to not be revoked in the middle of * action. * @@ -36,7 +35,52 @@ * * @see TaskLockbox#doInCriticalSection(Task, List, CriticalAction, CriticalAction) */ -public interface CriticalAction +public class CriticalAction { - T perform() throws Exception; + private final Action actionOnValidLocks; + private final Action actionOnInvalidLocks; + + private CriticalAction(Action actionOnValidLocks, Action actionOnInvalidLocks) + { + this.actionOnValidLocks = Preconditions.checkNotNull(actionOnValidLocks, "actionOnValidLocks"); + this.actionOnInvalidLocks = Preconditions.checkNotNull(actionOnInvalidLocks, "actionOnInvalidLocks"); + } + + T perform(boolean isTaskLocksValid) throws Exception + { + return isTaskLocksValid ? actionOnValidLocks.perform() : actionOnInvalidLocks.perform(); + } + + public static Builder builder() + { + return new Builder<>(); + } + + public static class Builder + { + private Action actionOnInvalidLocks; + private Action actionOnValidLocks; + + public Builder onValidLocks(Action action) + { + this.actionOnValidLocks = action; + return this; + } + + public Builder onInvalidLocks(Action action) + { + this.actionOnInvalidLocks = action; + return this; + } + + public CriticalAction build() + { + return new CriticalAction<>(actionOnValidLocks, actionOnInvalidLocks); + } + } + + public interface Action + { + T perform() throws Exception; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 1844093e9d6d..f75edb21323e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -507,14 +507,13 @@ private TaskLockPosse createNewTaskLockPosse( public T doInCriticalSection( Task task, List intervals, - CriticalAction actionOnValidLocks, - CriticalAction actionOnInvalidLocks + CriticalAction actions ) throws Exception { giant.lockInterruptibly(); try { - return isTaskLocksValid(task, intervals) ? actionOnValidLocks.perform() : actionOnInvalidLocks.perform(); + return actions.perform(isTaskLocksValid(task, intervals)); } finally { giant.unlock(); @@ -572,7 +571,7 @@ private void revokeLock(String taskId, TaskLock lock) if (lock.isRevoked()) { log.warn("TaskLock[%s] is already revoked", lock); } else { - final TaskLock revokedLock = lock.revoke(); + final TaskLock revokedLock = lock.revokedCopy(); taskStorage.replaceLock(taskId, lock, revokedLock); final List possesHolder = running.get(task.getDataSource()).get(lock.getInterval()); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index 0f304b659e89..ce1f25cfb369 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -25,6 +25,7 @@ import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.CriticalAction; import io.druid.java.util.common.Intervals; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; @@ -97,11 +98,15 @@ public void testSimple() throws Exception actionTestKit.getTaskLockbox().doInCriticalSection( task, Collections.singletonList(INTERVAL), - () -> action.perform(task, actionTestKit.getTaskActionToolbox()), - () -> { - Assert.fail(); - return null; - } + CriticalAction.builder() + .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); Assert.assertEquals( @@ -126,11 +131,15 @@ public void testFailBadVersion() throws Exception final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( task, Collections.singletonList(INTERVAL), - () -> action.perform(task, actionTestKit.getTaskActionToolbox()), - () -> { - Assert.fail(); - return null; - } + CriticalAction.>builder() + .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); Assert.assertEquals(ImmutableSet.of(SEGMENT3), segments); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 593657e75c2b..4bc7e2a2a8d6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -106,16 +106,22 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( lowPriorityTask, Collections.singletonList(interval), - () -> { - latch.countDown(); - Thread.sleep(100); - intSupplier.set(intSupplier.get() + 1); - return intSupplier.get(); - }, - () -> { - Assert.fail(); - return null; - } + CriticalAction.builder() + .onValidLocks( + () -> { + latch.countDown(); + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + } + ) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); }); @@ -130,15 +136,21 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( highPriorityTask, Collections.singletonList(interval), - () -> { - Thread.sleep(100); - intSupplier.set(intSupplier.get() + 1); - return intSupplier.get(); - }, - () -> { - Assert.fail(); - return null; - } + CriticalAction.builder() + .onValidLocks( + () -> { + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + } + ) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); }); @@ -174,16 +186,22 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception final Future future1 = service.submit(() -> lockbox.doInCriticalSection( task, ImmutableList.of(intervals.get(0), intervals.get(1)), - () -> { - latch.countDown(); - Thread.sleep(100); - intSupplier.set(intSupplier.get() + 1); - return intSupplier.get(); - }, - () -> { - Assert.fail(); - return null; - } + CriticalAction.builder() + .onValidLocks( + () -> { + latch.countDown(); + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + } + ) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() )); final Future future2 = service.submit(() -> { @@ -191,15 +209,21 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception return lockbox.doInCriticalSection( task, ImmutableList.of(intervals.get(1), intervals.get(2)), - () -> { - Thread.sleep(100); - intSupplier.set(intSupplier.get() + 1); - return intSupplier.get(); - }, - () -> { - Assert.fail(); - return null; - } + CriticalAction.builder() + .onValidLocks( + () -> { + Thread.sleep(100); + intSupplier.set(intSupplier.get() + 1); + return intSupplier.get(); + } + ) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); }); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index fa36ba777b07..0271894f219c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -268,7 +268,11 @@ public void testDoInCriticalSectionWithSharedLock() throws Exception Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); Assert.assertFalse( - lockbox.doInCriticalSection(task, Collections.singletonList(interval), () -> true, () -> false) + lockbox.doInCriticalSection( + task, + Collections.singletonList(interval), + CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() + ) ); } @@ -282,7 +286,11 @@ public void testDoInCriticalSectionWithExclusiveLock() throws Exception Assert.assertNotNull(lock); Assert.assertTrue( - lockbox.doInCriticalSection(task, Collections.singletonList(interval), () -> true, () -> false) + lockbox.doInCriticalSection( + task, + Collections.singletonList(interval), + CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() + ) ); } @@ -297,7 +305,11 @@ public void testDoInCriticalSectionWithSmallerInterval() throws Exception Assert.assertNotNull(lock); Assert.assertTrue( - lockbox.doInCriticalSection(task, Collections.singletonList(smallInterval), () -> true, () -> false) + lockbox.doInCriticalSection( + task, + Collections.singletonList(smallInterval), + CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() + ) ); } @@ -319,7 +331,11 @@ public void testPreemptionAndDoInCriticalSection() throws Exception Assert.assertNotNull(lock); Assert.assertTrue( - lockbox.doInCriticalSection(highPriorityTask, Collections.singletonList(interval), () -> true, () -> false) + lockbox.doInCriticalSection( + highPriorityTask, + Collections.singletonList(interval), + CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() + ) ); } @@ -340,7 +356,11 @@ public void testDoInCriticalSectionWithRevokedLock() throws Exception Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); Assert.assertFalse( - lockbox.doInCriticalSection(lowPriorityTask, Collections.singletonList(interval), () -> true, () -> false) + lockbox.doInCriticalSection( + lowPriorityTask, + Collections.singletonList(interval), + CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() + ) ); } From 0ca6c89e616442a03a0f1ef2125627b1d3faa37a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Oct 2017 12:11:25 +0900 Subject: [PATCH 17/20] make replaceLock transactional --- .../MetadataStorageActionHandler.java | 43 ++++++++---- .../overlord/MetadataTaskStorage.java | 26 ++++--- .../SQLMetadataStorageActionHandler.java | 69 +++++++++++++++---- .../SQLMetadataStorageActionHandlerTest.java | 58 ++++++++++++++++ 4 files changed, 157 insertions(+), 39 deletions(-) diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java index eb3297d58cdd..fe4454ff1577 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java @@ -20,9 +20,7 @@ package io.druid.metadata; import com.google.common.base.Optional; - import io.druid.java.util.common.Pair; - import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -43,7 +41,7 @@ public interface MetadataStorageActionHandler getEntry(String entryId); + Optional getEntry(String entryId); /** * Retrieve the status for the entry with the given id. @@ -78,14 +76,14 @@ public void insert( * @param entryId entry id * @return optional status, absent if entry does not exist or status is not set */ - public Optional getStatus(String entryId); + Optional getStatus(String entryId); /** * Return all active entries with their respective status * * @return list of (entry, status) pairs */ - public List> getActiveEntriesWithStatus(); + List> getActiveEntriesWithStatus(); /** * Return all statuses for inactive entries created on or later than the given timestamp @@ -93,7 +91,7 @@ public void insert( * @param timestamp timestamp * @return list of statuses */ - public List getInactiveStatusesSince(DateTime timestamp); + List getInactiveStatusesSince(DateTime timestamp); /** * Add a lock to the given entry @@ -102,14 +100,25 @@ public void insert( * @param lock lock to add * @return true if the lock was added */ - public boolean addLock(String entryId, LockType lock); + boolean addLock(String entryId, LockType lock); + + /** + * Replace an existing lock with a new lock. + * + * @param entryId entry id + * @param oldLockId lock to be replaced + * @param newLock lock to be added + * + * @return true if the lock is replaced + */ + boolean replaceLock(String entryId, long oldLockId, LockType newLock); /** * Remove the lock with the given lock id. * * @param lockId lock id */ - public void removeLock(long lockId); + void removeLock(long lockId); /** * Add a log to the entry with the given id. @@ -118,7 +127,7 @@ public void insert( * @param log log to add * @return true if the log was added */ - public boolean addLog(String entryId, LogType log); + boolean addLog(String entryId, LogType log); /** * Returns the logs for the entry with the given id. @@ -126,7 +135,7 @@ public void insert( * @param entryId entry id * @return list of logs */ - public List getLogs(String entryId); + List getLogs(String entryId); /** * Returns the locks for the given entry @@ -134,5 +143,13 @@ public void insert( * @param entryId entry id * @return map of lockId to lock */ - public Map getLocks(String entryId); + Map getLocks(String entryId); + + /** + * Returns the lock id for the given entry and the lock. + * + * @return lock id if found. Otherwise null. + */ + @Nullable + Long getLockId(String entryId, LockType lock); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java index fd9b8e3dd755..2a3a3813bc42 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java @@ -261,10 +261,12 @@ public void replaceLock(String taskid, TaskLock oldLock, TaskLock newLock) taskid ); - // Even though these two operations are not atomically executed, the caller of replaceLock() is thread-safe and - // guarantees that two or more threads never call replaceLock() at the same time - removeLock(taskid, oldLock); - addLock(taskid, newLock); + final Long oldLockId = handler.getLockId(taskid, oldLock); + if (oldLockId == null) { + throw new ISE("Cannot find lock[%s]", oldLock); + } + + handler.replaceLock(taskid, oldLockId, newLock); } @Override @@ -273,16 +275,12 @@ public void removeLock(String taskid, TaskLock taskLockToRemove) Preconditions.checkNotNull(taskid, "taskid"); Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove"); - final Map taskLocks = getLocksWithIds(taskid); - - for (final Map.Entry taskLockWithId : taskLocks.entrySet()) { - final long id = taskLockWithId.getKey(); - final TaskLock taskLock = taskLockWithId.getValue(); - - if (taskLock.equals(taskLockToRemove)) { - log.info("Deleting TaskLock with id[%d]: %s", id, taskLock); - handler.removeLock(id); - } + final Long lockId = handler.getLockId(taskid, taskLockToRemove); + if (lockId == null) { + log.warn("Cannot find lock[%s]", taskLockToRemove); + } else { + log.info("Deleting TaskLock with id[%d]: %s", lockId, taskLockToRemove); + handler.removeLock(lockId); } } diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index c7edc2746121..c29b58809009 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -19,6 +19,7 @@ package io.druid.metadata; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; @@ -40,11 +41,13 @@ import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.util.ByteArrayMapper; +import javax.annotation.Nullable; import java.io.IOException; import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; import java.util.Map; +import java.util.Map.Entry; public class SQLMetadataStorageActionHandler implements MetadataStorageActionHandler @@ -315,20 +318,46 @@ public boolean addLock(final String entryId, final LockType lock) @Override public Boolean withHandle(Handle handle) throws Exception { - return handle.createStatement( - StringUtils.format( - "INSERT INTO %1$s (%2$s_id, lock_payload) VALUES (:entryId, :payload)", - lockTable, entryTypeName - ) - ) - .bind("entryId", entryId) - .bind("payload", jsonMapper.writeValueAsBytes(lock)) - .execute() == 1; + return addLock(handle, entryId, lock); } } ); } + private boolean addLock(Handle handle, String entryId, LockType lock) throws JsonProcessingException + { + final String statement = StringUtils.format( + "INSERT INTO %1$s (%2$s_id, lock_payload) VALUES (:entryId, :payload)", + lockTable, entryTypeName + ); + return handle.createStatement(statement) + .bind("entryId", entryId) + .bind("payload", jsonMapper.writeValueAsBytes(lock)) + .execute() == 1; + } + + @Override + public boolean replaceLock(final String entryId, final long oldLockId, final LockType newLock) + { + return connector.retryTransaction( + (handle, transactionStatus) -> { + int numDeletedRows = removeLock(handle, oldLockId); + + if (numDeletedRows != 1) { + transactionStatus.setRollbackOnly(); + final String message = numDeletedRows == 0 ? + StringUtils.format("Cannot find lock[%d]", oldLockId) : + StringUtils.format("Found multiple locks for lockId[%d]", oldLockId); + throw new RuntimeException(message); + } + + return addLock(handle, entryId, newLock); + }, + 3, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + } + @Override public void removeLock(final long lockId) { @@ -338,9 +367,7 @@ public void removeLock(final long lockId) @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement(StringUtils.format("DELETE FROM %s WHERE id = :id", lockTable)) - .bind("id", lockId) - .execute(); + removeLock(handle, lockId); return null; } @@ -348,6 +375,13 @@ public Void withHandle(Handle handle) throws Exception ); } + private int removeLock(Handle handle, long lockId) + { + return handle.createStatement(StringUtils.format("DELETE FROM %s WHERE id = :id", lockTable)) + .bind("id", lockId) + .execute(); + } + @Override public boolean addLog(final String entryId, final LogType log) { @@ -486,4 +520,15 @@ public Map fold( } ); } + + @Override + @Nullable + public Long getLockId(String entryId, LockType lock) + { + return getLocks(entryId).entrySet().stream() + .filter(entry -> entry.getValue().equals(lock)) + .map(Entry::getKey) + .findAny() + .orElse(null); + } } diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java index f052737e660b..e86fa2a77bae 100644 --- a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -268,4 +268,62 @@ public void testLocks() throws Exception ); Assert.assertEquals(updated.keySet(), locks.keySet()); } + + @Test + public void testReplaceLock() throws EntryExistsException + { + final String entryId = "ABC123"; + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); + + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + + Assert.assertEquals( + ImmutableMap.>of(), + handler.getLocks("non_exist_entry") + ); + + Assert.assertEquals( + ImmutableMap.>of(), + handler.getLocks(entryId) + ); + + final ImmutableMap lock1 = ImmutableMap.of("lock", 1); + final ImmutableMap lock2 = ImmutableMap.of("lock", 2); + + Assert.assertTrue(handler.addLock(entryId, lock1)); + + final Long lockId1 = handler.getLockId(entryId, lock1); + Assert.assertNotNull(lockId1); + + Assert.assertTrue(handler.replaceLock(entryId, lockId1, lock2)); + } + + @Test + public void testGetLockId() throws EntryExistsException + { + final String entryId = "ABC123"; + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); + + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + + Assert.assertEquals( + ImmutableMap.>of(), + handler.getLocks("non_exist_entry") + ); + + Assert.assertEquals( + ImmutableMap.>of(), + handler.getLocks(entryId) + ); + + final ImmutableMap lock1 = ImmutableMap.of("lock", 1); + final ImmutableMap lock2 = ImmutableMap.of("lock", 2); + + Assert.assertTrue(handler.addLock(entryId, lock1)); + + Assert.assertNotNull(handler.getLockId(entryId, lock1)); + Assert.assertNull(handler.getLockId(entryId, lock2)); + } } From 507c6bac2f9c19edddf84f0fdad29c3ddedc7832 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Oct 2017 12:29:10 +0900 Subject: [PATCH 18/20] fix formatting --- .../common/actions/SegmentAllocateAction.java | 3 --- .../actions/SegmentMetadataUpdateAction.java | 18 ++++++------- .../common/actions/SegmentNukeAction.java | 18 ++++++------- .../actions/SegmentInsertActionTest.java | 26 +++++++++---------- 4 files changed, 31 insertions(+), 34 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java index 036db8061664..d340e5b7df6b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java @@ -24,15 +24,12 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import io.druid.indexing.common.TaskLockType; -import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.LockResult; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.logger.Logger; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index fb65c3b787b3..2e33d5e15c09 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -77,16 +77,16 @@ public Void perform( intervals, CriticalAction.builder() .onValidLocks( - () -> { - toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); - return null; - } - ) + () -> { + toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments); + return null; + } + ) .onInvalidLocks( - () -> { - throw new ISE("Some locks for task[%s] are already revoked", task.getId()); - } - ) + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ) .build() ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index b9bc03341252..044cfebd00fa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -77,16 +77,16 @@ public Void perform(Task task, TaskActionToolbox toolbox) throws IOException intervals, CriticalAction.builder() .onValidLocks( - () -> { - toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); - return null; - } - ) + () -> { + toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments); + return null; + } + ) .onInvalidLocks( - () -> { - throw new ISE("Some locks for task[%s] are already revoked", task.getId()); - } - ) + () -> { + throw new ISE("Some locks for task[%s] are already revoked", task.getId()); + } + ) .build() ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index ce1f25cfb369..080630caa331 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -101,11 +101,11 @@ public void testSimple() throws Exception CriticalAction.builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( - () -> { - Assert.fail(); - return null; - } - ) + () -> { + Assert.fail(); + return null; + } + ) .build() ); @@ -132,14 +132,14 @@ public void testFailBadVersion() throws Exception task, Collections.singletonList(INTERVAL), CriticalAction.>builder() - .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) - .onInvalidLocks( - () -> { - Assert.fail(); - return null; - } - ) - .build() + .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) + .onInvalidLocks( + () -> { + Assert.fail(); + return null; + } + ) + .build() ); Assert.assertEquals(ImmutableSet.of(SEGMENT3), segments); From 5a29ee1e6478b0dcf282ca740924853e787c4888 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Oct 2017 15:33:56 +0900 Subject: [PATCH 19/20] fix javadoc --- .../io/druid/indexing/overlord/CriticalAction.java | 12 ++++++++---- .../java/io/druid/indexing/overlord/TaskLockbox.java | 11 +++++------ 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java index 16e77d49c7f5..df082e6cea99 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java @@ -20,11 +20,15 @@ package io.druid.indexing.overlord; import com.google.common.base.Preconditions; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.Task; +import org.joda.time.Interval; + +import java.util.List; /** - * TODO: fix - * This class represents an action must be done while the task's lock is guaranteed to not be revoked in the middle of - * action. + * This class represents a critical action must be done while the task's lock is guaranteed to not be revoked in the + * middle of the action. * * Implementations must not change the lock state by calling {@link TaskLockbox#lock(TaskLockType, Task, Interval)}, * {@link TaskLockbox#lock(TaskLockType, Task, Interval, long)}, {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}, @@ -33,7 +37,7 @@ * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked * until this action is finished. * - * @see TaskLockbox#doInCriticalSection(Task, List, CriticalAction, CriticalAction) + * @see TaskLockbox#doInCriticalSection(Task, List, CriticalAction) */ public class CriticalAction { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index f75edb21323e..ef478256d578 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -499,21 +499,20 @@ private TaskLockPosse createNewTaskLockPosse( * The given action should be finished as soon as possible because all other methods in this class are blocked until * this method is finished. * - * @param task task performing a critical action - * @param intervals intervals - * @param actionOnValidLocks action to be performed when all locks are valid - * @param actionOnInvalidLocks action to be performed when some locks are invalid + * @param task task performing a critical action + * @param intervals intervals + * @param action action to be performed inside of the critical section */ public T doInCriticalSection( Task task, List intervals, - CriticalAction actions + CriticalAction action ) throws Exception { giant.lockInterruptibly(); try { - return actions.perform(isTaskLocksValid(task, intervals)); + return action.perform(isTaskLocksValid(task, intervals)); } finally { giant.unlock(); From 4586f241a78707af09005a2ec833ac2b0b8c0b31 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Oct 2017 15:48:53 +0900 Subject: [PATCH 20/20] fix build --- .../main/java/io/druid/indexing/overlord/CriticalAction.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java index df082e6cea99..152a2a717b79 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/CriticalAction.java @@ -24,8 +24,6 @@ import io.druid.indexing.common.task.Task; import org.joda.time.Interval; -import java.util.List; - /** * This class represents a critical action must be done while the task's lock is guaranteed to not be revoked in the * middle of the action. @@ -37,7 +35,7 @@ * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked * until this action is finished. * - * @see TaskLockbox#doInCriticalSection(Task, List, CriticalAction) + * @see TaskLockbox#doInCriticalSection */ public class CriticalAction {