mirror of https://github.com/apache/druid.git
Fix indefinite WAITING batch task when lock is revoked (#11788)
* Fix indefinite WAITING batch task when lock is revoked * Use revoked property on TaskLock * Update TimeChunkLockAcquireAction to return TaskLock for revoked locks
This commit is contained in:
parent
9ca8f1ec97
commit
a96aed021e
|
@ -95,7 +95,7 @@ public class TimeChunkLockAcquireAction implements TaskAction<TaskLock>
|
|||
.lock(task, new TimeChunkLockRequest(type, task, interval, null))
|
||||
: toolbox.getTaskLockbox()
|
||||
.lock(task, new TimeChunkLockRequest(type, task, interval, null), timeoutMs);
|
||||
return result.isOk() ? result.getTaskLock() : null;
|
||||
return result.getTaskLock();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
|
|
|
@ -81,7 +81,8 @@ public class TimeChunkLockTryAcquireAction implements TaskAction<TaskLock>
|
|||
task,
|
||||
new TimeChunkLockRequest(type, task, interval, null)
|
||||
);
|
||||
return result.isOk() ? result.getTaskLock() : null;
|
||||
|
||||
return result.getTaskLock();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.druid.indexing.input.InputRowSchemas;
|
|||
import org.apache.druid.indexing.overlord.Segments;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.JodaUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.java.util.common.granularity.GranularityType;
|
||||
|
@ -404,6 +405,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
|||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", cur));
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
|||
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -78,7 +79,19 @@ public abstract class AbstractFixedIntervalTask extends AbstractTask
|
|||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
||||
{
|
||||
return taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
|
||||
final TaskLock lock = taskActionClient.submit(
|
||||
new TimeChunkLockTryAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
interval
|
||||
)
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
|
|||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
|
||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||
import org.apache.druid.indexing.common.LockGranularity;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
||||
import org.apache.druid.indexing.common.TaskReport;
|
||||
|
@ -299,13 +300,20 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
|
|||
)
|
||||
).isOk();
|
||||
} else {
|
||||
return toolbox.getTaskActionClient().submit(
|
||||
final TaskLock lock = toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
segmentId.getInterval(),
|
||||
1000L
|
||||
)
|
||||
) != null;
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", segmentId.getInterval()));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
|||
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentsRetriever;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.JodaUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
|
@ -198,7 +199,19 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
|
|||
Interval interval = JodaUtils.umbrellaInterval(
|
||||
JodaUtils.condenseIntervals(intervals)
|
||||
);
|
||||
return taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
|
||||
final TaskLock lock = taskActionClient.submit(
|
||||
new TimeChunkLockTryAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
interval
|
||||
)
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
|
@ -393,6 +406,9 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
|
|||
),
|
||||
"Cannot acquire a lock for interval[%s]", interval
|
||||
);
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
|
||||
}
|
||||
version = lock.getVersion();
|
||||
} else {
|
||||
Iterable<TaskLock> locks = getTaskLocks(toolbox.getTaskActionClient());
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
|||
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.query.NoopQueryRunner;
|
||||
|
@ -238,13 +239,16 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
public void announceSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
// Side effect: Calling announceSegment causes a lock to be acquired
|
||||
Preconditions.checkNotNull(
|
||||
final TaskLock lock = Preconditions.checkNotNull(
|
||||
toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs)
|
||||
),
|
||||
"Cannot acquire a lock for interval[%s]",
|
||||
segment.getInterval()
|
||||
);
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", segment.getInterval()));
|
||||
}
|
||||
toolbox.getSegmentAnnouncer().announceSegment(segment);
|
||||
}
|
||||
|
||||
|
@ -264,13 +268,16 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
{
|
||||
// Side effect: Calling announceSegments causes locks to be acquired
|
||||
for (DataSegment segment : segments) {
|
||||
Preconditions.checkNotNull(
|
||||
final TaskLock lock = Preconditions.checkNotNull(
|
||||
toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs)
|
||||
),
|
||||
"Cannot acquire a lock for interval[%s]",
|
||||
segment.getInterval()
|
||||
);
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", segment.getInterval()));
|
||||
}
|
||||
}
|
||||
toolbox.getSegmentAnnouncer().announceSegments(segments);
|
||||
}
|
||||
|
@ -312,7 +319,9 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
"Cannot acquire a lock for interval[%s]",
|
||||
interval
|
||||
);
|
||||
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
|
||||
}
|
||||
return lock.getVersion();
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskC
|
|||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.NonnullPair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
@ -350,6 +351,9 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner
|
|||
"Cannot acquire a lock for interval[%s]",
|
||||
interval
|
||||
);
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
|
||||
}
|
||||
version = lock.getVersion();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,9 +48,14 @@ public class LockResult
|
|||
return new LockResult(taskLock, newSegmentId, false);
|
||||
}
|
||||
|
||||
public static LockResult fail(boolean revoked)
|
||||
public static LockResult fail()
|
||||
{
|
||||
return new LockResult(null, null, revoked);
|
||||
return new LockResult(null, null, false);
|
||||
}
|
||||
|
||||
public static LockResult revoked(TaskLock taskLock)
|
||||
{
|
||||
return new LockResult(taskLock, null, true);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
|
@ -87,6 +92,6 @@ public class LockResult
|
|||
|
||||
public boolean isOk()
|
||||
{
|
||||
return taskLock != null;
|
||||
return taskLock != null && !revoked;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -351,7 +351,7 @@ public class TaskLockbox
|
|||
if (lockRequestForNewSegment.getGranularity() == LockGranularity.SEGMENT) {
|
||||
newSegmentId = allocateSegmentId(lockRequestForNewSegment, request.getVersion());
|
||||
if (newSegmentId == null) {
|
||||
return LockResult.fail(false);
|
||||
return LockResult.fail();
|
||||
}
|
||||
convertedRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, newSegmentId);
|
||||
} else {
|
||||
|
@ -400,7 +400,7 @@ public class TaskLockbox
|
|||
? ((SegmentLock) posseToUse.taskLock).getPartitionId()
|
||||
: null
|
||||
);
|
||||
return LockResult.fail(false);
|
||||
return LockResult.fail();
|
||||
}
|
||||
} else {
|
||||
log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
|
||||
|
@ -408,7 +408,10 @@ public class TaskLockbox
|
|||
}
|
||||
} else {
|
||||
final boolean lockRevoked = posseToUse != null && posseToUse.getTaskLock().isRevoked();
|
||||
return LockResult.fail(lockRevoked);
|
||||
if (lockRevoked) {
|
||||
return LockResult.revoked(posseToUse.getTaskLock());
|
||||
}
|
||||
return LockResult.fail();
|
||||
}
|
||||
}
|
||||
finally {
|
||||
|
@ -608,7 +611,8 @@ public class TaskLockbox
|
|||
* @param taskId an id of the task holding the lock
|
||||
* @param lock lock to be revoked
|
||||
*/
|
||||
private void revokeLock(String taskId, TaskLock lock)
|
||||
@VisibleForTesting
|
||||
protected void revokeLock(String taskId, TaskLock lock)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
|||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
|
||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||
import org.apache.druid.indexing.common.LockGranularity;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
||||
import org.apache.druid.indexing.common.TaskReport;
|
||||
|
@ -436,13 +437,20 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
)
|
||||
).isOk();
|
||||
} else {
|
||||
return toolbox.getTaskActionClient().submit(
|
||||
final TaskLock lock = toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
segmentId.getInterval(),
|
||||
1000L
|
||||
)
|
||||
) != null;
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
if (lock.isRevoked()) {
|
||||
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", segmentId.getInterval()));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -1426,6 +1426,61 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLockRevoked() throws Exception
|
||||
{
|
||||
final Task task = new AbstractFixedIntervalTask(
|
||||
"id1",
|
||||
"id1",
|
||||
new TaskResource("id1", 1),
|
||||
"ds",
|
||||
Intervals.of("2012-01-01/P1D"),
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final Interval interval = Intervals.of("2012-01-01/P1D");
|
||||
final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
interval
|
||||
);
|
||||
|
||||
final TaskLock lock = toolbox.getTaskActionClient().submit(action);
|
||||
if (lock == null) {
|
||||
throw new ISE("Failed to get a lock");
|
||||
}
|
||||
|
||||
final TaskLock lockBeforeRevoke = toolbox.getTaskActionClient().submit(action);
|
||||
Assert.assertFalse(lockBeforeRevoke.isRevoked());
|
||||
|
||||
taskLockbox.revokeLock(getId(), lock);
|
||||
|
||||
final TaskLock lockAfterRevoke = toolbox.getTaskActionClient().submit(action);
|
||||
Assert.assertTrue(lockAfterRevoke.isRevoked());
|
||||
return TaskStatus.failure(getId(), "lock revoked test");
|
||||
}
|
||||
};
|
||||
|
||||
final TaskStatus status = runTask(task);
|
||||
Assert.assertEquals(taskLocation, status.getLocation());
|
||||
Assert.assertEquals("statusCode", TaskState.FAILED, status.getStatusCode());
|
||||
Assert.assertEquals("segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
private TaskStatus runTask(final Task task) throws Exception
|
||||
{
|
||||
final Task dummyTask = new DefaultObjectMapper().readValue(
|
||||
|
|
Loading…
Reference in New Issue