Reworked some of the shutdown logic to make the relationships clearer.
This commit is contained in:
parent
9686d7c3ca
commit
0829378957
5 changed files with 624 additions and 410 deletions
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load diff
|
|
@ -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) {
|
||||
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Reference in a new issue