Reworked some of the shutdown logic to make the relationships clearer.

This commit is contained in:
Pfifer, Justin 2017-06-13 06:21:27 -07:00
parent 9686d7c3ca
commit 0829378957
5 changed files with 624 additions and 410 deletions

View file

@ -0,0 +1,19 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import lombok.Data;
import java.util.concurrent.CountDownLatch;
@Data
class GracefulShutdownContext {
private final CountDownLatch shutdownCompleteLatch;
private final CountDownLatch notificationCompleteLatch;
private final Worker worker;
static GracefulShutdownContext SHUTDOWN_ALREADY_COMPLETED = new GracefulShutdownContext(null, null, null);
boolean isShutdownAlreadyCompleted() {
return shutdownCompleteLatch == null && notificationCompleteLatch == null && worker == null;
}
}

View file

@ -1,7 +1,6 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.TimeUnit;
@ -9,9 +8,9 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
class RequestedShutdownCoordinator {
class GracefulShutdownCoordinator {
static Future<Boolean> startRequestedShutdown(Callable<Boolean> shutdownCallable) {
Future<Boolean> startGracefulShutdown(Callable<Boolean> shutdownCallable) {
FutureTask<Boolean> task = new FutureTask<>(shutdownCallable);
Thread shutdownThread = new Thread(task, "RequestedShutdownThread");
shutdownThread.start();
@ -19,45 +18,39 @@ class RequestedShutdownCoordinator {
}
static Callable<Boolean> createRequestedShutdownCallable(CountDownLatch shutdownCompleteLatch,
CountDownLatch notificationCompleteLatch, Worker worker) {
return new RequestedShutdownCallable(shutdownCompleteLatch, notificationCompleteLatch, worker);
Callable<Boolean> createGracefulShutdownCallable(Callable<GracefulShutdownContext> startWorkerShutdown) {
return new GracefulShutdownCallable(startWorkerShutdown);
}
static class RequestedShutdownCallable implements Callable<Boolean> {
static class GracefulShutdownCallable implements Callable<Boolean> {
private static final Log log = LogFactory.getLog(RequestedShutdownCallable.class);
private static final Log log = LogFactory.getLog(GracefulShutdownCallable.class);
private final CountDownLatch shutdownCompleteLatch;
private final CountDownLatch notificationCompleteLatch;
private final Worker worker;
private final Callable<GracefulShutdownContext> startWorkerShutdown;
RequestedShutdownCallable(CountDownLatch shutdownCompleteLatch, CountDownLatch notificationCompleteLatch,
Worker worker) {
this.shutdownCompleteLatch = shutdownCompleteLatch;
this.notificationCompleteLatch = notificationCompleteLatch;
this.worker = worker;
GracefulShutdownCallable(Callable<GracefulShutdownContext> startWorkerShutdown) {
this.startWorkerShutdown = startWorkerShutdown;
}
private boolean isWorkerShutdownComplete() {
return worker.isShutdownComplete() || worker.getShardInfoShardConsumerMap().isEmpty();
private boolean isWorkerShutdownComplete(GracefulShutdownContext context) {
return context.getWorker().isShutdownComplete() || context.getWorker().getShardInfoShardConsumerMap().isEmpty();
}
private String awaitingLogMessage() {
long awaitingNotification = notificationCompleteLatch.getCount();
long awaitingFinalShutdown = shutdownCompleteLatch.getCount();
private String awaitingLogMessage(GracefulShutdownContext context) {
long awaitingNotification = context.getNotificationCompleteLatch().getCount();
long awaitingFinalShutdown = context.getShutdownCompleteLatch().getCount();
return String.format(
"Waiting for %d record process to complete shutdown notification, and %d record processor to complete final shutdown ",
awaitingNotification, awaitingFinalShutdown);
}
private String awaitingFinalShutdownMessage() {
long outstanding = shutdownCompleteLatch.getCount();
private String awaitingFinalShutdownMessage(GracefulShutdownContext context) {
long outstanding = context.getShutdownCompleteLatch().getCount();
return String.format("Waiting for %d record processors to complete final shutdown", outstanding);
}
private boolean waitForRecordProcessors() {
private boolean waitForRecordProcessors(GracefulShutdownContext context) {
//
// Awaiting for all ShardConsumer/RecordProcessors to be notified that a shutdown has been requested.
@ -66,18 +59,18 @@ class RequestedShutdownCoordinator {
// ShardConsumer would start the lease loss shutdown, and may never call the notification methods.
//
try {
while (!notificationCompleteLatch.await(1, TimeUnit.SECONDS)) {
while (!context.getNotificationCompleteLatch().await(1, TimeUnit.SECONDS)) {
if (Thread.interrupted()) {
throw new InterruptedException();
}
log.info(awaitingLogMessage());
if (workerShutdownWithRemaining(shutdownCompleteLatch.getCount())) {
log.info(awaitingLogMessage(context));
if (workerShutdownWithRemaining(context.getShutdownCompleteLatch().getCount(), context)) {
return false;
}
}
} catch (InterruptedException ie) {
log.warn("Interrupted while waiting for notification complete, terminating shutdown. "
+ awaitingLogMessage());
+ awaitingLogMessage(context));
return false;
}
@ -90,7 +83,7 @@ class RequestedShutdownCoordinator {
// Once all record processors have been notified of the shutdown it is safe to allow the worker to
// start its shutdown behavior. Once shutdown starts it will stop renewer, and drop any remaining leases.
//
worker.shutdown();
context.getWorker().shutdown();
if (Thread.interrupted()) {
log.warn("Interrupted after worker shutdown, terminating shutdown");
@ -103,18 +96,18 @@ class RequestedShutdownCoordinator {
// ShardConsumer is terminated.
//
try {
while (!shutdownCompleteLatch.await(1, TimeUnit.SECONDS)) {
while (!context.getShutdownCompleteLatch().await(1, TimeUnit.SECONDS)) {
if (Thread.interrupted()) {
throw new InterruptedException();
}
log.info(awaitingFinalShutdownMessage());
if (workerShutdownWithRemaining(shutdownCompleteLatch.getCount())) {
log.info(awaitingFinalShutdownMessage(context));
if (workerShutdownWithRemaining(context.getShutdownCompleteLatch().getCount(), context)) {
return false;
}
}
} catch (InterruptedException ie) {
log.warn("Interrupted while waiting for shutdown completion, terminating shutdown. "
+ awaitingFinalShutdownMessage());
+ awaitingFinalShutdownMessage(context));
return false;
}
return true;
@ -128,13 +121,13 @@ class RequestedShutdownCoordinator {
* @param outstanding
* the number of record processor still awaiting shutdown.
*/
private boolean workerShutdownWithRemaining(long outstanding) {
if (isWorkerShutdownComplete()) {
private boolean workerShutdownWithRemaining(long outstanding, GracefulShutdownContext context) {
if (isWorkerShutdownComplete(context)) {
if (outstanding != 0) {
log.info("Shutdown completed, but shutdownCompleteLatch still had outstanding " + outstanding
+ " with a current value of " + shutdownCompleteLatch.getCount() + ". shutdownComplete: "
+ worker.isShutdownComplete() + " -- Consumer Map: "
+ worker.getShardInfoShardConsumerMap().size());
+ " with a current value of " + context.getShutdownCompleteLatch().getCount() + ". shutdownComplete: "
+ context.getWorker().isShutdownComplete() + " -- Consumer Map: "
+ context.getWorker().getShardInfoShardConsumerMap().size());
return true;
}
}
@ -143,7 +136,14 @@ class RequestedShutdownCoordinator {
@Override
public Boolean call() throws Exception {
return waitForRecordProcessors();
GracefulShutdownContext context;
try {
context = startWorkerShutdown.call();
} catch (Exception ex) {
log.warn("Caught exception while requesting initial worker shutdown.", ex);
throw ex;
}
return context.isShutdownAlreadyCompleted() || waitForRecordProcessors(context);
}
}
}

View file

@ -1,10 +1,12 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
import static org.junit.Assert.assertThat;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@ -18,13 +20,11 @@ import java.util.concurrent.TimeUnit;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.runners.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
import org.mockito.verification.VerificationMode;
@RunWith(MockitoJUnitRunner.class)
public class RequestedShutdownCoordinatorTest {
public class GracefulShutdownCoordinatorTest {
@Mock
private CountDownLatch shutdownCompleteLatch;
@ -33,6 +33,8 @@ public class RequestedShutdownCoordinatorTest {
@Mock
private Worker worker;
@Mock
private Callable<GracefulShutdownContext> contextCallable;
@Mock
private ConcurrentMap<ShardInfo, ShardConsumer> shardInfoConsumerMap;
@Test
@ -262,9 +264,18 @@ public class RequestedShutdownCoordinatorTest {
verify(worker).shutdown();
}
@Test(expected = IllegalStateException.class)
public void testWorkerShutdownCallableThrows() throws Exception {
Callable<Boolean> requestedShutdownCallable = new GracefulShutdownCoordinator().createGracefulShutdownCallable(contextCallable);
when(contextCallable.call()).thenThrow(new IllegalStateException("Bad Shutdown"));
requestedShutdownCallable.call();
}
private void verifyLatchAwait(CountDownLatch latch) throws Exception {
verifyLatchAwait(latch, times(1));
}
private void verifyLatchAwait(CountDownLatch latch, int times) throws Exception {
verifyLatchAwait(latch, times(times));
}
@ -277,9 +288,11 @@ public class RequestedShutdownCoordinatorTest {
when(latch.await(anyLong(), any(TimeUnit.class))).thenReturn(initial, remaining);
}
private Callable<Boolean> buildRequestedShutdownCallable() {
return RequestedShutdownCoordinator.createRequestedShutdownCallable(shutdownCompleteLatch,
private Callable<Boolean> buildRequestedShutdownCallable() throws Exception {
GracefulShutdownContext context = new GracefulShutdownContext(shutdownCompleteLatch,
notificationCompleteLatch, worker);
when(contextCallable.call()).thenReturn(context);
return new GracefulShutdownCoordinator().createGracefulShutdownCallable(contextCallable);
}
private void mockShardInfoConsumerMap(Integer initialItemCount, Integer... additionalItemCounts) {

View file

@ -762,7 +762,7 @@ public class WorkerTest {
verify(executorService, atLeastOnce()).submit(argThat(
both(isA(MetricsCollectingTaskDecorator.class)).and(TaskTypeMatcher.isOfType(TaskType.INITIALIZE))));
worker.requestShutdown();
worker.createWorkerShutdownCallable().call();
worker.runProcessLoop();
verify(executorService, atLeastOnce()).submit(argThat(both(isA(MetricsCollectingTaskDecorator.class))
@ -781,6 +781,146 @@ public class WorkerTest {
}
@Test(expected = IllegalStateException.class)
public void testShutdownCallableNotAllowedTwice() throws Exception {
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
StreamConfig streamConfig = mock(StreamConfig.class);
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
KinesisClientLeaseBuilder builder = new KinesisClientLeaseBuilder().withCheckpoint(checkpoint)
.withConcurrencyToken(UUID.randomUUID()).withLastCounterIncrementNanos(0L).withLeaseCounter(0L)
.withOwnerSwitchesSinceCheckpoint(0L).withLeaseOwner("Self");
final List<KinesisClientLease> leases = new ArrayList<>();
final List<ShardInfo> currentAssignments = new ArrayList<>();
KinesisClientLease lease = builder.withLeaseKey(String.format("shardId-%03d", 1)).build();
leases.add(lease);
currentAssignments.add(new ShardInfo(lease.getLeaseKey(), lease.getConcurrencyToken().toString(),
lease.getParentShardIds(), lease.getCheckpoint()));
when(leaseCoordinator.getAssignments()).thenAnswer(new Answer<List<KinesisClientLease>>() {
@Override
public List<KinesisClientLease> answer(InvocationOnMock invocation) throws Throwable {
return leases;
}
});
when(leaseCoordinator.getCurrentAssignments()).thenAnswer(new Answer<List<ShardInfo>>() {
@Override
public List<ShardInfo> answer(InvocationOnMock invocation) throws Throwable {
return currentAssignments;
}
});
IRecordProcessor processor = mock(IRecordProcessor.class);
when(recordProcessorFactory.createProcessor()).thenReturn(processor);
Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig,
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
@Override
void postConstruct() {
this.gracefuleShutdownStarted = true;
}
};
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
when(taskFuture.isDone()).thenReturn(true);
when(taskFuture.get()).thenReturn(taskResult);
worker.runProcessLoop();
verify(executorService, atLeastOnce()).submit(argThat(both(isA(MetricsCollectingTaskDecorator.class))
.and(TaskTypeMatcher.isOfType(TaskType.BLOCK_ON_PARENT_SHARDS))));
worker.runProcessLoop();
verify(executorService, atLeastOnce()).submit(argThat(
both(isA(MetricsCollectingTaskDecorator.class)).and(TaskTypeMatcher.isOfType(TaskType.INITIALIZE))));
assertThat(worker.hasGracefulShutdownStarted(), equalTo(true));
worker.createWorkerShutdownCallable().call();
}
@Test
public void testGracefulShutdownSingleFuture() throws Exception {
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
StreamConfig streamConfig = mock(StreamConfig.class);
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
KinesisClientLeaseBuilder builder = new KinesisClientLeaseBuilder().withCheckpoint(checkpoint)
.withConcurrencyToken(UUID.randomUUID()).withLastCounterIncrementNanos(0L).withLeaseCounter(0L)
.withOwnerSwitchesSinceCheckpoint(0L).withLeaseOwner("Self");
final List<KinesisClientLease> leases = new ArrayList<>();
final List<ShardInfo> currentAssignments = new ArrayList<>();
KinesisClientLease lease = builder.withLeaseKey(String.format("shardId-%03d", 1)).build();
leases.add(lease);
currentAssignments.add(new ShardInfo(lease.getLeaseKey(), lease.getConcurrencyToken().toString(),
lease.getParentShardIds(), lease.getCheckpoint()));
when(leaseCoordinator.getAssignments()).thenAnswer(new Answer<List<KinesisClientLease>>() {
@Override
public List<KinesisClientLease> answer(InvocationOnMock invocation) throws Throwable {
return leases;
}
});
when(leaseCoordinator.getCurrentAssignments()).thenAnswer(new Answer<List<ShardInfo>>() {
@Override
public List<ShardInfo> answer(InvocationOnMock invocation) throws Throwable {
return currentAssignments;
}
});
IRecordProcessor processor = mock(IRecordProcessor.class);
when(recordProcessorFactory.createProcessor()).thenReturn(processor);
GracefulShutdownCoordinator coordinator = mock(GracefulShutdownCoordinator.class);
when(coordinator.createGracefulShutdownCallable(any(Callable.class))).thenReturn(() -> true);
Future<Boolean> gracefulShutdownFuture = mock(Future.class);
when(coordinator.startGracefulShutdown(any(Callable.class))).thenReturn(gracefulShutdownFuture);
Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig,
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
@Override
void postConstruct() {
this.gracefulShutdownCoordinator = coordinator;
}
};
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
when(taskFuture.isDone()).thenReturn(true);
when(taskFuture.get()).thenReturn(taskResult);
worker.runProcessLoop();
verify(executorService, atLeastOnce()).submit(argThat(both(isA(MetricsCollectingTaskDecorator.class))
.and(TaskTypeMatcher.isOfType(TaskType.BLOCK_ON_PARENT_SHARDS))));
worker.runProcessLoop();
verify(executorService, atLeastOnce()).submit(argThat(
both(isA(MetricsCollectingTaskDecorator.class)).and(TaskTypeMatcher.isOfType(TaskType.INITIALIZE))));
Future<Boolean> firstFuture = worker.startGracefulShutdown();
Future<Boolean> secondFuture = worker.startGracefulShutdown();
assertThat(firstFuture, equalTo(secondFuture));
verify(coordinator).startGracefulShutdown(any(Callable.class));
}
@Test
public void testRequestShutdownNoLeases() throws Exception {
@ -830,7 +970,7 @@ public class WorkerTest {
verify(executorService, never()).submit(argThat(
both(isA(MetricsCollectingTaskDecorator.class)).and(TaskTypeMatcher.isOfType(TaskType.INITIALIZE))));
worker.requestShutdown();
worker.createWorkerShutdownCallable().call();
worker.runProcessLoop();
verify(executorService, never()).submit(argThat(both(isA(MetricsCollectingTaskDecorator.class))
@ -909,7 +1049,7 @@ public class WorkerTest {
.withField(InitializeTask.class, "shardInfo", equalTo(shardInfo2)))));
worker.getShardInfoShardConsumerMap().remove(shardInfo2);
worker.requestShutdown();
worker.createWorkerShutdownCallable().call();
leases.remove(1);
currentAssignments.remove(1);
worker.runProcessLoop();
@ -1194,6 +1334,24 @@ public class WorkerTest {
}
private abstract class InjectableWorker extends Worker {
InjectableWorker(String applicationName, IRecordProcessorFactory recordProcessorFactory,
StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream,
long parentShardPollIntervalMillis, long shardSyncIdleTimeMillis,
boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) {
super(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream,
parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion,
checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis,
failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization);
postConstruct();
}
abstract void postConstruct();
}
private KinesisClientLease makeLease(ExtendedSequenceNumber checkpoint, int shardId) {
return new KinesisClientLeaseBuilder().withCheckpoint(checkpoint).withConcurrencyToken(UUID.randomUUID())
.withLastCounterIncrementNanos(0L).withLeaseCounter(0L).withOwnerSwitchesSinceCheckpoint(0L)