KCL changes pertaining to distributed lease discovery, distributed lease cleanup, removing redundant shasrd sync and lease uptaste operastions. Refer release notes for more details.
This commit is contained in:
commit
710b784f73
111 changed files with 9234 additions and 1624 deletions
|
|
@ -112,6 +112,11 @@ public class StreamingShardRecordProcessorTest {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public PreparedCheckpointer prepareCheckpoint(Record record)
|
public PreparedCheckpointer prepareCheckpoint(Record record)
|
||||||
throws KinesisClientLibDependencyException,
|
throws KinesisClientLibDependencyException,
|
||||||
|
|
@ -119,6 +124,11 @@ public class StreamingShardRecordProcessorTest {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber)
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber)
|
||||||
throws KinesisClientLibDependencyException,
|
throws KinesisClientLibDependencyException,
|
||||||
|
|
@ -126,6 +136,11 @@ public class StreamingShardRecordProcessorTest {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber)
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber)
|
||||||
throws KinesisClientLibDependencyException,
|
throws KinesisClientLibDependencyException,
|
||||||
|
|
@ -133,6 +148,11 @@ public class StreamingShardRecordProcessorTest {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Checkpointer checkpointer() {
|
public Checkpointer checkpointer() {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
|
|
|
||||||
|
|
@ -171,6 +171,14 @@ public class MultiLangDaemonConfigurationTest {
|
||||||
utilsBean.setProperty(configuration, "retrievalMode", "invalid");
|
utilsBean.setProperty(configuration, "retrievalMode", "invalid");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// @Test
|
||||||
|
// TODO : Enable this test once https://github.com/awslabs/amazon-kinesis-client/issues/692 is resolved
|
||||||
|
public void testmetricsEnabledDimensions() {
|
||||||
|
MultiLangDaemonConfiguration configuration = baseConfiguration();
|
||||||
|
configuration.setMetricsEnabledDimensions(new String[]{"Operation"});
|
||||||
|
configuration.resolvedConfiguration(shardRecordProcessorFactory);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFanoutConfigSetConsumerName() {
|
public void testFanoutConfigSetConsumerName() {
|
||||||
String consumerArn = "test-consumer";
|
String consumerArn = "test-consumer";
|
||||||
|
|
|
||||||
|
|
@ -26,18 +26,26 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
public class Checkpoint {
|
public class Checkpoint {
|
||||||
private final ExtendedSequenceNumber checkpoint;
|
private final ExtendedSequenceNumber checkpoint;
|
||||||
private final ExtendedSequenceNumber pendingCheckpoint;
|
private final ExtendedSequenceNumber pendingCheckpoint;
|
||||||
|
private final byte[] pendingCheckpointState;
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) {
|
||||||
|
this(checkpoint, pendingCheckpoint, null);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor.
|
* Constructor.
|
||||||
*
|
*
|
||||||
* @param checkpoint the checkpoint sequence number - cannot be null or empty.
|
* @param checkpoint the checkpoint sequence number - cannot be null or empty.
|
||||||
* @param pendingCheckpoint the pending checkpoint sequence number - can be null.
|
* @param pendingCheckpoint the pending checkpoint sequence number - can be null.
|
||||||
|
* @param pendingCheckpointState the pending checkpoint state - can be null.
|
||||||
*/
|
*/
|
||||||
public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) {
|
public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState) {
|
||||||
if (checkpoint == null || checkpoint.sequenceNumber().isEmpty()) {
|
if (checkpoint == null || checkpoint.sequenceNumber().isEmpty()) {
|
||||||
throw new IllegalArgumentException("Checkpoint cannot be null or empty");
|
throw new IllegalArgumentException("Checkpoint cannot be null or empty");
|
||||||
}
|
}
|
||||||
this.checkpoint = checkpoint;
|
this.checkpoint = checkpoint;
|
||||||
this.pendingCheckpoint = pendingCheckpoint;
|
this.pendingCheckpoint = pendingCheckpoint;
|
||||||
|
this.pendingCheckpointState = pendingCheckpointState;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -60,7 +60,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
public synchronized void checkpoint()
|
public synchronized void checkpoint()
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Checkpointing {}, token {} at largest permitted value {}", shardInfo.shardId(),
|
log.debug("Checkpointing {}, token {} at largest permitted value {}", ShardInfo.getLeaseKey(shardInfo),
|
||||||
shardInfo.concurrencyToken(), this.largestPermittedCheckpointValue);
|
shardInfo.concurrencyToken(), this.largestPermittedCheckpointValue);
|
||||||
}
|
}
|
||||||
advancePosition(this.largestPermittedCheckpointValue);
|
advancePosition(this.largestPermittedCheckpointValue);
|
||||||
|
|
@ -116,7 +116,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
&& newCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) {
|
&& newCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) {
|
||||||
|
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Checkpointing {}, token {} at specific extended sequence number {}", shardInfo.shardId(),
|
log.debug("Checkpointing {}, token {} at specific extended sequence number {}", ShardInfo.getLeaseKey(shardInfo),
|
||||||
shardInfo.concurrencyToken(), newCheckpoint);
|
shardInfo.concurrencyToken(), newCheckpoint);
|
||||||
}
|
}
|
||||||
this.advancePosition(newCheckpoint);
|
this.advancePosition(newCheckpoint);
|
||||||
|
|
@ -144,8 +144,15 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public synchronized PreparedCheckpointer prepareCheckpoint(Record record)
|
public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
//
|
//
|
||||||
// TODO: UserRecord Deprecation
|
// TODO: UserRecord Deprecation
|
||||||
//
|
//
|
||||||
|
|
@ -154,10 +161,19 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
} /*else if (record instanceof UserRecord) {
|
} /*else if (record instanceof UserRecord) {
|
||||||
return prepareCheckpoint(record.sequenceNumber(), ((UserRecord) record).subSequenceNumber());
|
return prepareCheckpoint(record.sequenceNumber(), ((UserRecord) record).subSequenceNumber());
|
||||||
} */ else {
|
} */ else {
|
||||||
return prepareCheckpoint(record.sequenceNumber(), 0);
|
return prepareCheckpoint(record.sequenceNumber(), 0, applicationState);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public synchronized PreparedCheckpointer prepareCheckpoint(Record record)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
|
return prepareCheckpoint(record, null);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
|
|
@ -167,13 +183,30 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
return prepareCheckpoint(sequenceNumber, 0);
|
return prepareCheckpoint(sequenceNumber, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException {
|
||||||
|
return prepareCheckpoint(sequenceNumber, 0, applicationState);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public synchronized PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber)
|
public synchronized PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber)
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
|
return prepareCheckpoint(sequenceNumber, subSequenceNumber, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException {
|
||||||
if (subSequenceNumber < 0) {
|
if (subSequenceNumber < 0) {
|
||||||
throw new IllegalArgumentException("Could not checkpoint at invalid, negative subsequence number "
|
throw new IllegalArgumentException("Could not checkpoint at invalid, negative subsequence number "
|
||||||
+ subSequenceNumber);
|
+ subSequenceNumber);
|
||||||
|
|
@ -189,9 +222,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
|
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Preparing checkpoint {}, token {} at specific extended sequence number {}",
|
log.debug("Preparing checkpoint {}, token {} at specific extended sequence number {}",
|
||||||
shardInfo.shardId(), shardInfo.concurrencyToken(), pendingCheckpoint);
|
ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), pendingCheckpoint);
|
||||||
}
|
}
|
||||||
return doPrepareCheckpoint(pendingCheckpoint);
|
return doPrepareCheckpoint(pendingCheckpoint, applicationState);
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException(String.format(
|
throw new IllegalArgumentException(String.format(
|
||||||
"Could not prepare checkpoint at extended sequence number %s as it did not fall into acceptable "
|
"Could not prepare checkpoint at extended sequence number %s as it did not fall into acceptable "
|
||||||
|
|
@ -252,10 +285,10 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) {
|
if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) {
|
||||||
try {
|
try {
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Setting {}, token {} checkpoint to {}", shardInfo.shardId(),
|
log.debug("Setting {}, token {} checkpoint to {}", ShardInfo.getLeaseKey(shardInfo),
|
||||||
shardInfo.concurrencyToken(), checkpointToRecord);
|
shardInfo.concurrencyToken(), checkpointToRecord);
|
||||||
}
|
}
|
||||||
checkpointer.setCheckpoint(shardInfo.shardId(), checkpointToRecord, shardInfo.concurrencyToken());
|
checkpointer.setCheckpoint(ShardInfo.getLeaseKey(shardInfo), checkpointToRecord, shardInfo.concurrencyToken());
|
||||||
lastCheckpointValue = checkpointToRecord;
|
lastCheckpointValue = checkpointToRecord;
|
||||||
} catch (ThrottlingException | ShutdownException | InvalidStateException
|
} catch (ThrottlingException | ShutdownException | InvalidStateException
|
||||||
| KinesisClientLibDependencyException e) {
|
| KinesisClientLibDependencyException e) {
|
||||||
|
|
@ -290,7 +323,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
* @throws ThrottlingException
|
* @throws ThrottlingException
|
||||||
* @throws ShutdownException
|
* @throws ShutdownException
|
||||||
*/
|
*/
|
||||||
private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber)
|
private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber, byte[] applicationState)
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||||
|
|
||||||
ExtendedSequenceNumber newPrepareCheckpoint = extendedSequenceNumber;
|
ExtendedSequenceNumber newPrepareCheckpoint = extendedSequenceNumber;
|
||||||
|
|
@ -308,7 +341,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
checkpointer.prepareCheckpoint(shardInfo.shardId(), newPrepareCheckpoint, shardInfo.concurrencyToken());
|
checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken(), applicationState);
|
||||||
} catch (ThrottlingException | ShutdownException | InvalidStateException
|
} catch (ThrottlingException | ShutdownException | InvalidStateException
|
||||||
| KinesisClientLibDependencyException e) {
|
| KinesisClientLibDependencyException e) {
|
||||||
throw e;
|
throw e;
|
||||||
|
|
|
||||||
|
|
@ -54,54 +54,59 @@ public class DynamoDBCheckpointer implements Checkpointer {
|
||||||
private String operation;
|
private String operation;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setCheckpoint(final String shardId, final ExtendedSequenceNumber checkpointValue,
|
public void setCheckpoint(final String leaseKey, final ExtendedSequenceNumber checkpointValue,
|
||||||
final String concurrencyToken) throws KinesisClientLibException {
|
final String concurrencyToken) throws KinesisClientLibException {
|
||||||
try {
|
try {
|
||||||
boolean wasSuccessful = setCheckpoint(shardId, checkpointValue, UUID.fromString(concurrencyToken));
|
boolean wasSuccessful = setCheckpoint(leaseKey, checkpointValue, UUID.fromString(concurrencyToken));
|
||||||
if (!wasSuccessful) {
|
if (!wasSuccessful) {
|
||||||
throw new ShutdownException("Can't update checkpoint - instance doesn't hold the lease for this shard");
|
throw new ShutdownException("Can't update checkpoint - instance doesn't hold the lease for this shard");
|
||||||
}
|
}
|
||||||
} catch (ProvisionedThroughputException e) {
|
} catch (ProvisionedThroughputException e) {
|
||||||
throw new ThrottlingException("Got throttled while updating checkpoint.", e);
|
throw new ThrottlingException("Got throttled while updating checkpoint.", e);
|
||||||
} catch (InvalidStateException e) {
|
} catch (InvalidStateException e) {
|
||||||
String message = "Unable to save checkpoint for shardId " + shardId;
|
String message = "Unable to save checkpoint for shardId " + leaseKey;
|
||||||
log.error(message, e);
|
log.error(message, e);
|
||||||
throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e);
|
throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e);
|
||||||
} catch (DependencyException e) {
|
} catch (DependencyException e) {
|
||||||
throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + shardId, e);
|
throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + leaseKey, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ExtendedSequenceNumber getCheckpoint(final String shardId) throws KinesisClientLibException {
|
public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws KinesisClientLibException {
|
||||||
try {
|
try {
|
||||||
return leaseRefresher.getLease(shardId).checkpoint();
|
return leaseRefresher.getLease(leaseKey).checkpoint();
|
||||||
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
||||||
String message = "Unable to fetch checkpoint for shardId " + shardId;
|
String message = "Unable to fetch checkpoint for shardId " + leaseKey;
|
||||||
log.error(message, e);
|
log.error(message, e);
|
||||||
throw new KinesisClientLibIOException(message, e);
|
throw new KinesisClientLibIOException(message, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Checkpoint getCheckpointObject(final String shardId) throws KinesisClientLibException {
|
public Checkpoint getCheckpointObject(final String leaseKey) throws KinesisClientLibException {
|
||||||
try {
|
try {
|
||||||
Lease lease = leaseRefresher.getLease(shardId);
|
Lease lease = leaseRefresher.getLease(leaseKey);
|
||||||
log.debug("[{}] Retrieved lease => {}", shardId, lease);
|
log.debug("[{}] Retrieved lease => {}", leaseKey, lease);
|
||||||
return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint());
|
return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint(), lease.pendingCheckpointState());
|
||||||
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
||||||
String message = "Unable to fetch checkpoint for shardId " + shardId;
|
String message = "Unable to fetch checkpoint for shardId " + leaseKey;
|
||||||
log.error(message, e);
|
log.error(message, e);
|
||||||
throw new KinesisClientLibIOException(message, e);
|
throw new KinesisClientLibIOException(message, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void prepareCheckpoint(final String shardId, final ExtendedSequenceNumber pendingCheckpoint,
|
public void prepareCheckpoint(final String leaseKey, final ExtendedSequenceNumber pendingCheckpoint,
|
||||||
final String concurrencyToken) throws KinesisClientLibException {
|
final String concurrencyToken) throws KinesisClientLibException {
|
||||||
|
prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException {
|
||||||
try {
|
try {
|
||||||
boolean wasSuccessful =
|
boolean wasSuccessful =
|
||||||
prepareCheckpoint(shardId, pendingCheckpoint, UUID.fromString(concurrencyToken));
|
prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken), pendingCheckpointState);
|
||||||
if (!wasSuccessful) {
|
if (!wasSuccessful) {
|
||||||
throw new ShutdownException(
|
throw new ShutdownException(
|
||||||
"Can't prepare checkpoint - instance doesn't hold the lease for this shard");
|
"Can't prepare checkpoint - instance doesn't hold the lease for this shard");
|
||||||
|
|
@ -109,42 +114,44 @@ public class DynamoDBCheckpointer implements Checkpointer {
|
||||||
} catch (ProvisionedThroughputException e) {
|
} catch (ProvisionedThroughputException e) {
|
||||||
throw new ThrottlingException("Got throttled while preparing checkpoint.", e);
|
throw new ThrottlingException("Got throttled while preparing checkpoint.", e);
|
||||||
} catch (InvalidStateException e) {
|
} catch (InvalidStateException e) {
|
||||||
String message = "Unable to prepare checkpoint for shardId " + shardId;
|
String message = "Unable to prepare checkpoint for shardId " + leaseKey;
|
||||||
log.error(message, e);
|
log.error(message, e);
|
||||||
throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e);
|
throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e);
|
||||||
} catch (DependencyException e) {
|
} catch (DependencyException e) {
|
||||||
throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + shardId, e);
|
throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + leaseKey, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public boolean setCheckpoint(String shardId, ExtendedSequenceNumber checkpoint, UUID concurrencyToken)
|
public boolean setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpoint, UUID concurrencyToken)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId);
|
Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey);
|
||||||
if (lease == null) {
|
if (lease == null) {
|
||||||
log.info("Worker {} could not update checkpoint for shard {} because it does not hold the lease",
|
log.info("Worker {} could not update checkpoint for shard {} because it does not hold the lease",
|
||||||
leaseCoordinator.workerIdentifier(), shardId);
|
leaseCoordinator.workerIdentifier(), leaseKey);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
lease.checkpoint(checkpoint);
|
lease.checkpoint(checkpoint);
|
||||||
lease.pendingCheckpoint(null);
|
lease.pendingCheckpoint(null);
|
||||||
|
lease.pendingCheckpointState(null);
|
||||||
lease.ownerSwitchesSinceCheckpoint(0L);
|
lease.ownerSwitchesSinceCheckpoint(0L);
|
||||||
|
|
||||||
return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId);
|
return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken)
|
boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken, byte[] pendingCheckpointState)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId);
|
Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey);
|
||||||
if (lease == null) {
|
if (lease == null) {
|
||||||
log.info("Worker {} could not prepare checkpoint for shard {} because it does not hold the lease",
|
log.info("Worker {} could not prepare checkpoint for shard {} because it does not hold the lease",
|
||||||
leaseCoordinator.workerIdentifier(), shardId);
|
leaseCoordinator.workerIdentifier(), leaseKey);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
lease.pendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null"));
|
lease.pendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null"));
|
||||||
return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId);
|
lease.pendingCheckpointState(pendingCheckpointState);
|
||||||
|
return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -15,14 +15,18 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.common;
|
package software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.Setter;
|
||||||
|
import lombok.ToString;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
|
||||||
import lombok.Data;
|
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
|
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
|
||||||
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
|
import software.amazon.awssdk.utils.Either;
|
||||||
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
||||||
import software.amazon.kinesis.coordinator.CoordinatorConfig;
|
import software.amazon.kinesis.coordinator.CoordinatorConfig;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
||||||
|
|
@ -30,19 +34,21 @@ import software.amazon.kinesis.lifecycle.LifecycleConfig;
|
||||||
import software.amazon.kinesis.metrics.MetricsConfig;
|
import software.amazon.kinesis.metrics.MetricsConfig;
|
||||||
import software.amazon.kinesis.processor.ProcessorConfig;
|
import software.amazon.kinesis.processor.ProcessorConfig;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
||||||
|
import software.amazon.kinesis.processor.MultiStreamTracker;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalConfig;
|
import software.amazon.kinesis.retrieval.RetrievalConfig;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This Builder is useful to create all configurations for the KCL with default values.
|
* This Builder is useful to create all configurations for the KCL with default values.
|
||||||
*/
|
*/
|
||||||
@Data
|
@Getter @Setter @ToString @EqualsAndHashCode
|
||||||
@Accessors(fluent = true)
|
@Accessors(fluent = true)
|
||||||
public class ConfigsBuilder {
|
public class ConfigsBuilder {
|
||||||
/**
|
/**
|
||||||
* Name of the stream to consume records from
|
* Either the name of the stream to consume records from
|
||||||
|
* Or MultiStreamTracker for all the streams to consume records from
|
||||||
*/
|
*/
|
||||||
@NonNull
|
private Either<MultiStreamTracker, String> appStreamTracker;
|
||||||
private final String streamName;
|
|
||||||
/**
|
/**
|
||||||
* Application name for the KCL Worker
|
* Application name for the KCL Worker
|
||||||
*/
|
*/
|
||||||
|
|
@ -108,6 +114,52 @@ public class ConfigsBuilder {
|
||||||
return namespace;
|
return namespace;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor to initialize ConfigsBuilder with StreamName
|
||||||
|
* @param streamName
|
||||||
|
* @param applicationName
|
||||||
|
* @param kinesisClient
|
||||||
|
* @param dynamoDBClient
|
||||||
|
* @param cloudWatchClient
|
||||||
|
* @param workerIdentifier
|
||||||
|
* @param shardRecordProcessorFactory
|
||||||
|
*/
|
||||||
|
public ConfigsBuilder(@NonNull String streamName, @NonNull String applicationName,
|
||||||
|
@NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient,
|
||||||
|
@NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier,
|
||||||
|
@NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) {
|
||||||
|
this.appStreamTracker = Either.right(streamName);
|
||||||
|
this.applicationName = applicationName;
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.dynamoDBClient = dynamoDBClient;
|
||||||
|
this.cloudWatchClient = cloudWatchClient;
|
||||||
|
this.workerIdentifier = workerIdentifier;
|
||||||
|
this.shardRecordProcessorFactory = shardRecordProcessorFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor to initialize ConfigsBuilder with MultiStreamTracker
|
||||||
|
* @param multiStreamTracker
|
||||||
|
* @param applicationName
|
||||||
|
* @param kinesisClient
|
||||||
|
* @param dynamoDBClient
|
||||||
|
* @param cloudWatchClient
|
||||||
|
* @param workerIdentifier
|
||||||
|
* @param shardRecordProcessorFactory
|
||||||
|
*/
|
||||||
|
public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName,
|
||||||
|
@NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient,
|
||||||
|
@NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier,
|
||||||
|
@NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) {
|
||||||
|
this.appStreamTracker = Either.left(multiStreamTracker);
|
||||||
|
this.applicationName = applicationName;
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.dynamoDBClient = dynamoDBClient;
|
||||||
|
this.cloudWatchClient = cloudWatchClient;
|
||||||
|
this.workerIdentifier = workerIdentifier;
|
||||||
|
this.shardRecordProcessorFactory = shardRecordProcessorFactory;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new instance of CheckpointConfig
|
* Creates a new instance of CheckpointConfig
|
||||||
*
|
*
|
||||||
|
|
@ -132,8 +184,7 @@ public class ConfigsBuilder {
|
||||||
* @return LeaseManagementConfig
|
* @return LeaseManagementConfig
|
||||||
*/
|
*/
|
||||||
public LeaseManagementConfig leaseManagementConfig() {
|
public LeaseManagementConfig leaseManagementConfig() {
|
||||||
return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), streamName(),
|
return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), workerIdentifier());
|
||||||
workerIdentifier());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -170,6 +221,10 @@ public class ConfigsBuilder {
|
||||||
* @return RetrievalConfig
|
* @return RetrievalConfig
|
||||||
*/
|
*/
|
||||||
public RetrievalConfig retrievalConfig() {
|
public RetrievalConfig retrievalConfig() {
|
||||||
return new RetrievalConfig(kinesisClient(), streamName(), applicationName());
|
final RetrievalConfig retrievalConfig =
|
||||||
|
appStreamTracker.map(
|
||||||
|
multiStreamTracker -> new RetrievalConfig(kinesisClient(), multiStreamTracker, applicationName()),
|
||||||
|
streamName -> new RetrievalConfig(kinesisClient(), streamName, applicationName()));
|
||||||
|
return retrievalConfig;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -28,11 +28,11 @@ public class DiagnosticUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Util for RecordPublisher to measure the event delivery latency of the executor service and take appropriate action.
|
* Util for RecordPublisher to measure the event delivery latency of the executor service and take appropriate action.
|
||||||
* @param shardId of the shard that is having delayed delivery
|
* @param resourceIdentifier of the shard that is having delayed delivery
|
||||||
* @param enqueueTimestamp of the event submitted to the executor service
|
* @param enqueueTimestamp of the event submitted to the executor service
|
||||||
* @param log Slf4j Logger from RecordPublisher to log the events
|
* @param log Slf4j Logger from RecordPublisher to log the events
|
||||||
*/
|
*/
|
||||||
public static void takeDelayedDeliveryActionIfRequired(String shardId, Instant enqueueTimestamp, Logger log) {
|
public static void takeDelayedDeliveryActionIfRequired(String resourceIdentifier, Instant enqueueTimestamp, Logger log) {
|
||||||
final long durationBetweenEnqueueAndAckInMillis = Duration
|
final long durationBetweenEnqueueAndAckInMillis = Duration
|
||||||
.between(enqueueTimestamp, Instant.now()).toMillis();
|
.between(enqueueTimestamp, Instant.now()).toMillis();
|
||||||
if (durationBetweenEnqueueAndAckInMillis > MAX_TIME_BETWEEN_REQUEST_RESPONSE / 3) {
|
if (durationBetweenEnqueueAndAckInMillis > MAX_TIME_BETWEEN_REQUEST_RESPONSE / 3) {
|
||||||
|
|
@ -41,9 +41,9 @@ public class DiagnosticUtils {
|
||||||
"{}: Record delivery time to shard consumer is high at {} millis. Check the ExecutorStateEvent logs"
|
"{}: Record delivery time to shard consumer is high at {} millis. Check the ExecutorStateEvent logs"
|
||||||
+ " to see the state of the executor service. Also check if the RecordProcessor's processing "
|
+ " to see the state of the executor service. Also check if the RecordProcessor's processing "
|
||||||
+ "time is high. ",
|
+ "time is high. ",
|
||||||
shardId, durationBetweenEnqueueAndAckInMillis);
|
resourceIdentifier, durationBetweenEnqueueAndAckInMillis);
|
||||||
} else if (log.isDebugEnabled()) {
|
} else if (log.isDebugEnabled()) {
|
||||||
log.debug("{}: Record delivery time to shard consumer is {} millis", shardId,
|
log.debug("{}: Record delivery time to shard consumer is {} millis", resourceIdentifier,
|
||||||
durationBetweenEnqueueAndAckInMillis);
|
durationBetweenEnqueueAndAckInMillis);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,84 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import org.apache.commons.lang3.Validate;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.HashKeyRange;
|
||||||
|
|
||||||
|
import java.math.BigInteger;
|
||||||
|
|
||||||
|
@Value @Accessors(fluent = true)
|
||||||
|
/**
|
||||||
|
* Lease POJO to hold the starting hashkey range and ending hashkey range of kinesis shards.
|
||||||
|
*/
|
||||||
|
public class HashKeyRangeForLease {
|
||||||
|
|
||||||
|
private final BigInteger startingHashKey;
|
||||||
|
private final BigInteger endingHashKey;
|
||||||
|
|
||||||
|
public HashKeyRangeForLease(BigInteger startingHashKey, BigInteger endingHashKey) {
|
||||||
|
Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0,
|
||||||
|
"StartingHashKey %s must be less than EndingHashKey %s ", startingHashKey, endingHashKey);
|
||||||
|
this.startingHashKey = startingHashKey;
|
||||||
|
this.endingHashKey = endingHashKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Serialize the startingHashKey for persisting in external storage
|
||||||
|
*
|
||||||
|
* @return Serialized startingHashKey
|
||||||
|
*/
|
||||||
|
public String serializedStartingHashKey() {
|
||||||
|
return startingHashKey.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Serialize the endingHashKey for persisting in external storage
|
||||||
|
*
|
||||||
|
* @return Serialized endingHashKey
|
||||||
|
*/
|
||||||
|
public String serializedEndingHashKey() {
|
||||||
|
return endingHashKey.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deserialize from serialized hashKeyRange string from external storage.
|
||||||
|
*
|
||||||
|
* @param startingHashKeyStr
|
||||||
|
* @param endingHashKeyStr
|
||||||
|
* @return HashKeyRangeForLease
|
||||||
|
*/
|
||||||
|
public static HashKeyRangeForLease deserialize(@NonNull String startingHashKeyStr, @NonNull String endingHashKeyStr) {
|
||||||
|
final BigInteger startingHashKey = new BigInteger(startingHashKeyStr);
|
||||||
|
final BigInteger endingHashKey = new BigInteger(endingHashKeyStr);
|
||||||
|
Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0,
|
||||||
|
"StartingHashKey %s must be less than EndingHashKey %s ", startingHashKeyStr, endingHashKeyStr);
|
||||||
|
return new HashKeyRangeForLease(startingHashKey, endingHashKey);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Construct HashKeyRangeForLease from Kinesis HashKeyRange
|
||||||
|
*
|
||||||
|
* @param hashKeyRange
|
||||||
|
* @return HashKeyRangeForLease
|
||||||
|
*/
|
||||||
|
public static HashKeyRangeForLease fromHashKeyRange(HashKeyRange hashKeyRange) {
|
||||||
|
return deserialize(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -14,6 +14,7 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.common;
|
package software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
import lombok.ToString;
|
import lombok.ToString;
|
||||||
|
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
|
@ -22,7 +23,7 @@ import java.util.Date;
|
||||||
* Class that houses the entities needed to specify the position in the stream from where a new application should
|
* Class that houses the entities needed to specify the position in the stream from where a new application should
|
||||||
* start.
|
* start.
|
||||||
*/
|
*/
|
||||||
@ToString
|
@ToString @EqualsAndHashCode
|
||||||
public class InitialPositionInStreamExtended {
|
public class InitialPositionInStreamExtended {
|
||||||
|
|
||||||
private final InitialPositionInStream position;
|
private final InitialPositionInStream position;
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,41 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import lombok.Builder;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configuration for lease cleanup.
|
||||||
|
*/
|
||||||
|
@Builder
|
||||||
|
@Getter
|
||||||
|
@Accessors(fluent=true)
|
||||||
|
public class LeaseCleanupConfig {
|
||||||
|
/**
|
||||||
|
* Interval at which to run lease cleanup thread.
|
||||||
|
*/
|
||||||
|
private final long leaseCleanupIntervalMillis;
|
||||||
|
/**
|
||||||
|
* Interval at which to check if a lease is completed or not.
|
||||||
|
*/
|
||||||
|
private final long completedLeaseCleanupIntervalMillis;
|
||||||
|
/**
|
||||||
|
* Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not.
|
||||||
|
*/
|
||||||
|
private final long garbageLeaseCleanupIntervalMillis;
|
||||||
|
}
|
||||||
|
|
@ -1,3 +1,18 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.common;
|
package software.amazon.kinesis.common;
|
||||||
|
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,29 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import lombok.Data;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
|
||||||
|
@Data
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
public class StreamConfig {
|
||||||
|
private final StreamIdentifier streamIdentifier;
|
||||||
|
private final InitialPositionInStreamExtended initialPositionInStreamExtended;
|
||||||
|
private String consumerArn;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.common;
|
||||||
|
|
||||||
|
import com.google.common.base.Joiner;
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import software.amazon.awssdk.utils.Validate;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
@EqualsAndHashCode @Getter @Accessors(fluent = true)
|
||||||
|
public class StreamIdentifier {
|
||||||
|
private final Optional<String> accountIdOptional;
|
||||||
|
private final String streamName;
|
||||||
|
private final Optional<Long> streamCreationEpochOptional;
|
||||||
|
|
||||||
|
private static final String DELIMITER = ":";
|
||||||
|
private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*");
|
||||||
|
|
||||||
|
private StreamIdentifier(@NonNull String accountId, @NonNull String streamName, @NonNull Long streamCreationEpoch) {
|
||||||
|
this.accountIdOptional = Optional.of(accountId);
|
||||||
|
this.streamName = streamName;
|
||||||
|
this.streamCreationEpochOptional = Optional.of(streamCreationEpoch);
|
||||||
|
}
|
||||||
|
|
||||||
|
private StreamIdentifier(@NonNull String streamName) {
|
||||||
|
this.accountIdOptional = Optional.empty();
|
||||||
|
this.streamName = streamName;
|
||||||
|
this.streamCreationEpochOptional = Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Serialize the current StreamIdentifier instance.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public String serialize() {
|
||||||
|
return accountIdOptional.isPresent() ?
|
||||||
|
Joiner.on(DELIMITER).join(accountIdOptional.get(), streamName, streamCreationEpochOptional.get()) :
|
||||||
|
streamName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return serialize();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a multi stream instance for StreamIdentifier from serialized stream identifier.
|
||||||
|
* The serialized stream identifier should be of the format account:stream:creationepoch
|
||||||
|
* @param streamIdentifierSer
|
||||||
|
* @return StreamIdentifier
|
||||||
|
*/
|
||||||
|
public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) {
|
||||||
|
if (PATTERN.matcher(streamIdentifierSer).matches()) {
|
||||||
|
final String[] split = streamIdentifierSer.split(DELIMITER);
|
||||||
|
return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2]));
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a single stream instance for StreamIdentifier from stream name.
|
||||||
|
* @param streamName
|
||||||
|
* @return StreamIdentifier
|
||||||
|
*/
|
||||||
|
public static StreamIdentifier singleStreamInstance(String streamName) {
|
||||||
|
Validate.notEmpty(streamName, "StreamName should not be empty");
|
||||||
|
return new StreamIdentifier(streamName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,152 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
|
||||||
|
import java.time.Instant;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
import java.util.function.BooleanSupplier;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An implementation of the {@code LeaderDecider} to elect leader(s) based on workerId.
|
||||||
|
* Leases are shuffled using a predetermined constant seed so that lease ordering is
|
||||||
|
* preserved across workers.
|
||||||
|
* This reduces the probability of choosing the leader workers co-located on the same
|
||||||
|
* host in case workerId starts with a common string (e.g. IP Address).
|
||||||
|
* Hence if a host has 3 workers, IPADDRESS_Worker1, IPADDRESS_Worker2, and IPADDRESS_Worker3,
|
||||||
|
* we don't end up choosing all 3 for shard sync as a result of natural ordering of Strings.
|
||||||
|
* This ensures redundancy for shard-sync during host failures.
|
||||||
|
*/
|
||||||
|
@Slf4j
|
||||||
|
class DeterministicShuffleShardSyncLeaderDecider
|
||||||
|
implements LeaderDecider {
|
||||||
|
// Fixed seed so that the shuffle order is preserved across workers
|
||||||
|
static final int DETERMINISTIC_SHUFFLE_SEED = 1947;
|
||||||
|
|
||||||
|
private static final long ELECTION_INITIAL_DELAY_MILLIS = 60 * 1000;
|
||||||
|
private static final long ELECTION_SCHEDULING_INTERVAL_MILLIS = 5 * 60 * 1000;
|
||||||
|
private static final int AWAIT_TERMINATION_MILLIS = 5000;
|
||||||
|
|
||||||
|
private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
|
||||||
|
|
||||||
|
private final LeaseRefresher leaseRefresher;
|
||||||
|
private final int numPeriodicShardSyncWorkers;
|
||||||
|
private final ScheduledExecutorService leaderElectionThreadPool;
|
||||||
|
|
||||||
|
private volatile Set<String> leaders;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param leaseRefresher LeaseManager instance used to fetch leases.
|
||||||
|
* @param leaderElectionThreadPool Thread-pool to be used for leaderElection.
|
||||||
|
* @param numPeriodicShardSyncWorkers Number of leaders that will be elected to perform periodic shard syncs.
|
||||||
|
*/
|
||||||
|
DeterministicShuffleShardSyncLeaderDecider(LeaseRefresher leaseRefresher, ScheduledExecutorService leaderElectionThreadPool,
|
||||||
|
int numPeriodicShardSyncWorkers) {
|
||||||
|
this.leaseRefresher = leaseRefresher;
|
||||||
|
this.leaderElectionThreadPool = leaderElectionThreadPool;
|
||||||
|
this.numPeriodicShardSyncWorkers = numPeriodicShardSyncWorkers;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Shuffles the leases deterministically and elects numPeriodicShardSyncWorkers number of workers
|
||||||
|
* as leaders (workers that will perform shard sync).
|
||||||
|
*/
|
||||||
|
private void electLeaders() {
|
||||||
|
try {
|
||||||
|
log.debug("Started leader election at: " + Instant.now());
|
||||||
|
List<Lease> leases = leaseRefresher.listLeases();
|
||||||
|
List<String> uniqueHosts = leases.stream().map(Lease::leaseOwner)
|
||||||
|
.filter(owner -> owner != null).distinct().sorted().collect(Collectors.toList());
|
||||||
|
|
||||||
|
Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED));
|
||||||
|
int numShardSyncWorkers = Math.min(uniqueHosts.size(), numPeriodicShardSyncWorkers);
|
||||||
|
// In case value is currently being read, we wait for reading to complete before updating the variable.
|
||||||
|
// This is to prevent any ConcurrentModificationException exceptions.
|
||||||
|
readWriteLock.writeLock().lock();
|
||||||
|
leaders = new HashSet<>(uniqueHosts.subList(0, numShardSyncWorkers));
|
||||||
|
log.info("Elected leaders: " + String.join(", ", leaders));
|
||||||
|
log.debug("Completed leader election at: " + Instant.now());
|
||||||
|
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
||||||
|
log.error("Exception occurred while trying to fetch all leases for leader election", e);
|
||||||
|
} catch (Throwable t) {
|
||||||
|
log.error("Unknown exception during leader election.", t);
|
||||||
|
} finally {
|
||||||
|
readWriteLock.writeLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isWorkerLeaderForShardSync(String workerId) {
|
||||||
|
return CollectionUtils.isNullOrEmpty(leaders) || leaders.contains(workerId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized Boolean isLeader(String workerId) {
|
||||||
|
// if no leaders yet, synchronously get leaders. This will happen at first Shard Sync.
|
||||||
|
if (executeConditionCheckWithReadLock(() -> CollectionUtils.isNullOrEmpty(leaders))) {
|
||||||
|
electLeaders();
|
||||||
|
// start a scheduled executor that will periodically update leaders.
|
||||||
|
// The first run will be after a minute.
|
||||||
|
// We don't need jitter since it is scheduled with a fixed delay and time taken to scan leases
|
||||||
|
// will be different at different times and on different hosts/workers.
|
||||||
|
leaderElectionThreadPool.scheduleWithFixedDelay(this::electLeaders, ELECTION_INITIAL_DELAY_MILLIS,
|
||||||
|
ELECTION_SCHEDULING_INTERVAL_MILLIS, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
return executeConditionCheckWithReadLock(() -> isWorkerLeaderForShardSync(workerId));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void shutdown() {
|
||||||
|
try {
|
||||||
|
leaderElectionThreadPool.shutdown();
|
||||||
|
if (leaderElectionThreadPool.awaitTermination(AWAIT_TERMINATION_MILLIS, TimeUnit.MILLISECONDS)) {
|
||||||
|
log.info("Successfully stopped leader election on the worker");
|
||||||
|
} else {
|
||||||
|
leaderElectionThreadPool.shutdownNow();
|
||||||
|
log.info(String.format("Stopped leader election thread after awaiting termination for %d milliseconds",
|
||||||
|
AWAIT_TERMINATION_MILLIS));
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
log.debug("Encountered InterruptedException while awaiting leader election threadPool termination");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Execute condition checks using shared variables under a read-write lock.
|
||||||
|
private boolean executeConditionCheckWithReadLock(BooleanSupplier action) {
|
||||||
|
try {
|
||||||
|
readWriteLock.readLock().lock();
|
||||||
|
return action.getAsBoolean();
|
||||||
|
} finally {
|
||||||
|
readWriteLock.readLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -28,7 +28,7 @@ import java.util.concurrent.ThreadPoolExecutor;
|
||||||
@ToString(exclude = "isThreadPoolExecutor")
|
@ToString(exclude = "isThreadPoolExecutor")
|
||||||
@Slf4j
|
@Slf4j
|
||||||
@KinesisClientInternalApi
|
@KinesisClientInternalApi
|
||||||
class ExecutorStateEvent implements DiagnosticEvent {
|
public class ExecutorStateEvent implements DiagnosticEvent {
|
||||||
private static final String MESSAGE = "Current thread pool executor state: ";
|
private static final String MESSAGE = "Current thread pool executor state: ";
|
||||||
|
|
||||||
private boolean isThreadPoolExecutor;
|
private boolean isThreadPoolExecutor;
|
||||||
|
|
@ -41,6 +41,11 @@ class ExecutorStateEvent implements DiagnosticEvent {
|
||||||
private int maximumPoolSize;
|
private int maximumPoolSize;
|
||||||
|
|
||||||
ExecutorStateEvent(ExecutorService executor, LeaseCoordinator leaseCoordinator) {
|
ExecutorStateEvent(ExecutorService executor, LeaseCoordinator leaseCoordinator) {
|
||||||
|
this(executor);
|
||||||
|
this.leasesOwned = leaseCoordinator.getAssignments().size();
|
||||||
|
}
|
||||||
|
|
||||||
|
public ExecutorStateEvent(ExecutorService executor) {
|
||||||
if (executor instanceof ThreadPoolExecutor) {
|
if (executor instanceof ThreadPoolExecutor) {
|
||||||
this.isThreadPoolExecutor = true;
|
this.isThreadPoolExecutor = true;
|
||||||
|
|
||||||
|
|
@ -52,8 +57,6 @@ class ExecutorStateEvent implements DiagnosticEvent {
|
||||||
this.largestPoolSize = ex.getLargestPoolSize();
|
this.largestPoolSize = ex.getLargestPoolSize();
|
||||||
this.maximumPoolSize = ex.getMaximumPoolSize();
|
this.maximumPoolSize = ex.getMaximumPoolSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
this.leasesOwned = leaseCoordinator.getAssignments().size();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,39 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used in conjunction with periodic shard sync.
|
||||||
|
* Implement this interface to allow KCL to decide if the current worker should execute shard sync.
|
||||||
|
* When periodic shard sync is enabled, PeriodicShardSyncManager periodically checks if the current
|
||||||
|
* worker is one of the leaders designated to execute shard-sync and then acts accordingly.
|
||||||
|
*/
|
||||||
|
public interface LeaderDecider {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Method invoked to check the given workerId corresponds to one of the workers
|
||||||
|
* designated to execute shard-syncs periodically.
|
||||||
|
*
|
||||||
|
* @param workerId ID of the worker
|
||||||
|
* @return True if the worker with ID workerId can execute shard-sync. False otherwise.
|
||||||
|
*/
|
||||||
|
Boolean isLeader(String workerId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Can be invoked, if needed, to shutdown any clients/thread-pools
|
||||||
|
* being used in the LeaderDecider implementation.
|
||||||
|
*/
|
||||||
|
void shutdown();
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,437 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.collect.ComparisonChain;
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import org.apache.commons.lang3.Validate;
|
||||||
|
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
|
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
||||||
|
import software.amazon.kinesis.leases.UpdateField;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
import software.amazon.kinesis.lifecycle.TaskResult;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.math.BigInteger;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The top level orchestrator for coordinating the periodic shard sync related
|
||||||
|
* activities.
|
||||||
|
*/
|
||||||
|
@Getter
|
||||||
|
@EqualsAndHashCode
|
||||||
|
@Slf4j
|
||||||
|
class PeriodicShardSyncManager {
|
||||||
|
private static final long INITIAL_DELAY = 60 * 1000L;
|
||||||
|
@VisibleForTesting
|
||||||
|
static final BigInteger MIN_HASH_KEY = BigInteger.ZERO;
|
||||||
|
@VisibleForTesting
|
||||||
|
static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE);
|
||||||
|
static final String PERIODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager";
|
||||||
|
private Map<StreamIdentifier, HashRangeHoleTracker> hashRangeHoleTrackerMap = new HashMap<>();
|
||||||
|
|
||||||
|
private final String workerId;
|
||||||
|
private final LeaderDecider leaderDecider;
|
||||||
|
private final LeaseRefresher leaseRefresher;
|
||||||
|
private final Map<StreamIdentifier, StreamConfig> currentStreamConfigMap;
|
||||||
|
private final Function<StreamConfig, ShardSyncTaskManager> shardSyncTaskManagerProvider;
|
||||||
|
private final ScheduledExecutorService shardSyncThreadPool;
|
||||||
|
private final boolean isMultiStreamingMode;
|
||||||
|
private final MetricsFactory metricsFactory;
|
||||||
|
private final long leasesRecoveryAuditorExecutionFrequencyMillis;
|
||||||
|
private final int leasesRecoveryAuditorInconsistencyConfidenceThreshold;
|
||||||
|
private boolean isRunning;
|
||||||
|
|
||||||
|
PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher,
|
||||||
|
Map<StreamIdentifier, StreamConfig> currentStreamConfigMap,
|
||||||
|
Function<StreamConfig, ShardSyncTaskManager> shardSyncTaskManagerProvider, boolean isMultiStreamingMode,
|
||||||
|
MetricsFactory metricsFactory,
|
||||||
|
long leasesRecoveryAuditorExecutionFrequencyMillis,
|
||||||
|
int leasesRecoveryAuditorInconsistencyConfidenceThreshold) {
|
||||||
|
this(workerId, leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider,
|
||||||
|
Executors.newSingleThreadScheduledExecutor(), isMultiStreamingMode, metricsFactory,
|
||||||
|
leasesRecoveryAuditorExecutionFrequencyMillis, leasesRecoveryAuditorInconsistencyConfidenceThreshold);
|
||||||
|
}
|
||||||
|
|
||||||
|
PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher,
|
||||||
|
Map<StreamIdentifier, StreamConfig> currentStreamConfigMap,
|
||||||
|
Function<StreamConfig, ShardSyncTaskManager> shardSyncTaskManagerProvider,
|
||||||
|
ScheduledExecutorService shardSyncThreadPool, boolean isMultiStreamingMode,
|
||||||
|
MetricsFactory metricsFactory,
|
||||||
|
long leasesRecoveryAuditorExecutionFrequencyMillis,
|
||||||
|
int leasesRecoveryAuditorInconsistencyConfidenceThreshold) {
|
||||||
|
Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager.");
|
||||||
|
Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager.");
|
||||||
|
this.workerId = workerId;
|
||||||
|
this.leaderDecider = leaderDecider;
|
||||||
|
this.leaseRefresher = leaseRefresher;
|
||||||
|
this.currentStreamConfigMap = currentStreamConfigMap;
|
||||||
|
this.shardSyncTaskManagerProvider = shardSyncTaskManagerProvider;
|
||||||
|
this.shardSyncThreadPool = shardSyncThreadPool;
|
||||||
|
this.isMultiStreamingMode = isMultiStreamingMode;
|
||||||
|
this.metricsFactory = metricsFactory;
|
||||||
|
this.leasesRecoveryAuditorExecutionFrequencyMillis = leasesRecoveryAuditorExecutionFrequencyMillis;
|
||||||
|
this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized TaskResult start() {
|
||||||
|
if (!isRunning) {
|
||||||
|
final Runnable periodicShardSyncer = () -> {
|
||||||
|
try {
|
||||||
|
runShardSync();
|
||||||
|
} catch (Throwable t) {
|
||||||
|
log.error("Error during runShardSync.", t);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
shardSyncThreadPool.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, leasesRecoveryAuditorExecutionFrequencyMillis,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
isRunning = true;
|
||||||
|
|
||||||
|
}
|
||||||
|
return new TaskResult(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runs shardSync once
|
||||||
|
* Does not schedule periodic shardSync
|
||||||
|
* @return the result of the task
|
||||||
|
*/
|
||||||
|
public synchronized void syncShardsOnce() throws Exception {
|
||||||
|
// TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing
|
||||||
|
// TODO: for already synced streams
|
||||||
|
for(Map.Entry<StreamIdentifier, StreamConfig> streamConfigEntry : currentStreamConfigMap.entrySet()) {
|
||||||
|
final StreamIdentifier streamIdentifier = streamConfigEntry.getKey();
|
||||||
|
log.info("Syncing Kinesis shard info for " + streamIdentifier);
|
||||||
|
final StreamConfig streamConfig = streamConfigEntry.getValue();
|
||||||
|
final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfig);
|
||||||
|
final TaskResult taskResult = shardSyncTaskManager.callShardSyncTask();
|
||||||
|
if (taskResult.getException() != null) {
|
||||||
|
throw taskResult.getException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void stop() {
|
||||||
|
if (isRunning) {
|
||||||
|
log.info(String.format("Shutting down leader decider on worker %s", workerId));
|
||||||
|
leaderDecider.shutdown();
|
||||||
|
log.info(String.format("Shutting down periodic shard sync task scheduler on worker %s", workerId));
|
||||||
|
shardSyncThreadPool.shutdown();
|
||||||
|
isRunning = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runShardSync() {
|
||||||
|
if (leaderDecider.isLeader(workerId)) {
|
||||||
|
log.info(String.format("WorkerId %s is leader, running the periodic shard sync task", workerId));
|
||||||
|
|
||||||
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory,
|
||||||
|
PERIODIC_SHARD_SYNC_MANAGER);
|
||||||
|
int numStreamsWithPartialLeases = 0;
|
||||||
|
int numStreamsToSync = 0;
|
||||||
|
boolean isRunSuccess = false;
|
||||||
|
final long runStartMillis = System.currentTimeMillis();
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Construct the stream to leases map to be used in the lease sync
|
||||||
|
final Map<StreamIdentifier, List<Lease>> streamToLeasesMap = getStreamToLeasesMap(
|
||||||
|
currentStreamConfigMap.keySet());
|
||||||
|
|
||||||
|
// For each of the stream, check if shard sync needs to be done based on the leases state.
|
||||||
|
for (Map.Entry<StreamIdentifier, StreamConfig> streamConfigEntry : currentStreamConfigMap.entrySet()) {
|
||||||
|
final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(),
|
||||||
|
streamToLeasesMap.get(streamConfigEntry.getKey()));
|
||||||
|
|
||||||
|
numStreamsWithPartialLeases += shardSyncResponse.isHoleDetected() ? 1 : 0;
|
||||||
|
numStreamsToSync += shardSyncResponse.shouldDoShardSync ? 1 : 0;
|
||||||
|
|
||||||
|
if (shardSyncResponse.shouldDoShardSync()) {
|
||||||
|
log.info("Periodic shard syncer initiating shard sync for {} due to the reason - {} ",
|
||||||
|
streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision());
|
||||||
|
final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider
|
||||||
|
.apply(streamConfigEntry.getValue());
|
||||||
|
if (!shardSyncTaskManager.submitShardSyncTask()) {
|
||||||
|
log.warn(
|
||||||
|
"Failed to submit shard sync task for stream {}. This could be due to the previous pending shard sync task.",
|
||||||
|
shardSyncTaskManager.shardDetector().streamIdentifier().streamName());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
log.info("Skipping shard sync for {} due to the reason - {}", streamConfigEntry.getKey(),
|
||||||
|
shardSyncResponse.reasonForDecision());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
isRunSuccess = true;
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.error("Caught exception while running periodic shard syncer.", e);
|
||||||
|
} finally {
|
||||||
|
scope.addData("NumStreamsWithPartialLeases", numStreamsWithPartialLeases, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
scope.addData("NumStreamsToSync", numStreamsToSync, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
MetricsUtil.addSuccessAndLatency(scope, isRunSuccess, runStartMillis, MetricsLevel.SUMMARY);
|
||||||
|
scope.end();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
log.debug("WorkerId {} is not a leader, not running the shard sync task", workerId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<StreamIdentifier, List<Lease>> getStreamToLeasesMap(
|
||||||
|
final Set<StreamIdentifier> streamIdentifiersToFilter)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
final List<Lease> leases = leaseRefresher.listLeases();
|
||||||
|
if (!isMultiStreamingMode) {
|
||||||
|
Validate.isTrue(streamIdentifiersToFilter.size() == 1);
|
||||||
|
return Collections.singletonMap(streamIdentifiersToFilter.iterator().next(), leases);
|
||||||
|
} else {
|
||||||
|
final Map<StreamIdentifier, List<Lease>> streamToLeasesMap = new HashMap<>();
|
||||||
|
for (Lease lease : leases) {
|
||||||
|
StreamIdentifier streamIdentifier = StreamIdentifier
|
||||||
|
.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier());
|
||||||
|
if (streamIdentifiersToFilter.contains(streamIdentifier)) {
|
||||||
|
streamToLeasesMap.computeIfAbsent(streamIdentifier, s -> new ArrayList<>()).add(lease);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return streamToLeasesMap;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
ShardSyncResponse checkForShardSync(StreamIdentifier streamIdentifier, List<Lease> leases) {
|
||||||
|
if (CollectionUtils.isNullOrEmpty(leases)) {
|
||||||
|
// If the leases is null or empty then we need to do shard sync
|
||||||
|
log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier);
|
||||||
|
return new ShardSyncResponse(true, false, "No leases found for " + streamIdentifier);
|
||||||
|
}
|
||||||
|
// Check if there are any holes in the leases and return the first hole if present.
|
||||||
|
Optional<HashRangeHole> hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases);
|
||||||
|
if (hashRangeHoleOpt.isPresent()) {
|
||||||
|
// If hole is present, check if the hole is detected consecutively in previous occurrences.
|
||||||
|
// If hole is determined with high confidence return true; return false otherwise
|
||||||
|
// We are using the high confidence factor to avoid shard sync on any holes during resharding and
|
||||||
|
// lease cleanups or any intermittent issues.
|
||||||
|
final HashRangeHoleTracker hashRangeHoleTracker = hashRangeHoleTrackerMap
|
||||||
|
.computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker());
|
||||||
|
final boolean hasHoleWithHighConfidence = hashRangeHoleTracker
|
||||||
|
.hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get());
|
||||||
|
return new ShardSyncResponse(hasHoleWithHighConfidence, true,
|
||||||
|
"Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles()
|
||||||
|
+ " times. Shard sync will be initiated when threshold reaches "
|
||||||
|
+ leasesRecoveryAuditorInconsistencyConfidenceThreshold);
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// If hole is not present, clear any previous tracking for this stream and return false;
|
||||||
|
hashRangeHoleTrackerMap.remove(streamIdentifier);
|
||||||
|
return new ShardSyncResponse(false, false, "Hash Ranges are complete for " + streamIdentifier);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Value
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
@VisibleForTesting
|
||||||
|
static class ShardSyncResponse {
|
||||||
|
private final boolean shouldDoShardSync;
|
||||||
|
private final boolean isHoleDetected;
|
||||||
|
private final String reasonForDecision;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Optional<HashRangeHole> hasHoleInLeases(StreamIdentifier streamIdentifier, List<Lease> leases) {
|
||||||
|
// Filter the leases with any checkpoint other than shard end.
|
||||||
|
List<Lease> activeLeases = leases.stream()
|
||||||
|
.filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()).collect(Collectors.toList());
|
||||||
|
List<Lease> activeLeasesWithHashRanges = fillWithHashRangesIfRequired(streamIdentifier, activeLeases);
|
||||||
|
return checkForHoleInHashKeyRanges(streamIdentifier, activeLeasesWithHashRanges);
|
||||||
|
}
|
||||||
|
|
||||||
|
// If leases are missing hashranges information, update the leases in-memory as well as in the lease storage
|
||||||
|
// by learning from kinesis shards.
|
||||||
|
private List<Lease> fillWithHashRangesIfRequired(StreamIdentifier streamIdentifier, List<Lease> activeLeases) {
|
||||||
|
List<Lease> activeLeasesWithNoHashRanges = activeLeases.stream()
|
||||||
|
.filter(lease -> lease.hashKeyRangeForLease() == null).collect(Collectors.toList());
|
||||||
|
Optional<Lease> minLeaseOpt = activeLeasesWithNoHashRanges.stream().min(Comparator.comparing(Lease::leaseKey));
|
||||||
|
if (minLeaseOpt.isPresent()) {
|
||||||
|
// TODO : use minLease for new ListShards with startingShardId
|
||||||
|
final Lease minLease = minLeaseOpt.get();
|
||||||
|
final ShardDetector shardDetector = shardSyncTaskManagerProvider
|
||||||
|
.apply(currentStreamConfigMap.get(streamIdentifier)).shardDetector();
|
||||||
|
final Map<String, Shard> kinesisShards = shardDetector.listShards().stream()
|
||||||
|
.collect(Collectors.toMap(Shard::shardId, shard -> shard));
|
||||||
|
return activeLeases.stream().map(lease -> {
|
||||||
|
if (lease.hashKeyRangeForLease() == null) {
|
||||||
|
final String shardId = lease instanceof MultiStreamLease ?
|
||||||
|
((MultiStreamLease) lease).shardId() :
|
||||||
|
lease.leaseKey();
|
||||||
|
final Shard shard = kinesisShards.get(shardId);
|
||||||
|
if(shard == null) {
|
||||||
|
return lease;
|
||||||
|
}
|
||||||
|
lease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange()));
|
||||||
|
try {
|
||||||
|
leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE);
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.warn(
|
||||||
|
"Unable to update hash range key information for lease {} of stream {}. This may result in explicit lease sync.",
|
||||||
|
lease.leaseKey(), streamIdentifier);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return lease;
|
||||||
|
}).filter(lease -> lease.hashKeyRangeForLease() != null).collect(Collectors.toList());
|
||||||
|
} else {
|
||||||
|
return activeLeases;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
static Optional<HashRangeHole> checkForHoleInHashKeyRanges(StreamIdentifier streamIdentifier,
|
||||||
|
List<Lease> leasesWithHashKeyRanges) {
|
||||||
|
// Sort the hash ranges by starting hash key.
|
||||||
|
List<Lease> sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges);
|
||||||
|
if(sortedLeasesWithHashKeyRanges.isEmpty()) {
|
||||||
|
log.error("No leases with valid hashranges found for stream {}", streamIdentifier);
|
||||||
|
return Optional.of(new HashRangeHole());
|
||||||
|
}
|
||||||
|
// Validate for hashranges bounds.
|
||||||
|
if (!sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease().startingHashKey().equals(MIN_HASH_KEY) || !sortedLeasesWithHashKeyRanges
|
||||||
|
.get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease().endingHashKey().equals(MAX_HASH_KEY)) {
|
||||||
|
log.error("Incomplete hash range found for stream {} between {} and {}.", streamIdentifier,
|
||||||
|
sortedLeasesWithHashKeyRanges.get(0),
|
||||||
|
sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1));
|
||||||
|
return Optional.of(new HashRangeHole(sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease(),
|
||||||
|
sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease()));
|
||||||
|
}
|
||||||
|
// Check for any holes in the sorted hashrange intervals.
|
||||||
|
if (sortedLeasesWithHashKeyRanges.size() > 1) {
|
||||||
|
Lease leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(0);
|
||||||
|
HashKeyRangeForLease leftLeaseHashRange = leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease();
|
||||||
|
for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) {
|
||||||
|
final HashKeyRangeForLease rightLeaseHashRange = sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease();
|
||||||
|
final BigInteger rangeDiff = rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey());
|
||||||
|
// Case of overlapping leases when the rangediff is 0 or negative.
|
||||||
|
// signum() will be -1 for negative and 0 if value is 0.
|
||||||
|
// Merge the range for further tracking.
|
||||||
|
if (rangeDiff.signum() <= 0) {
|
||||||
|
leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(),
|
||||||
|
leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey()));
|
||||||
|
} else {
|
||||||
|
// Case of non overlapping leases when rangediff is positive. signum() will be 1 for positive.
|
||||||
|
// If rangeDiff is 1, then it is a case of continuous hashrange. If not, it is a hole.
|
||||||
|
if (!rangeDiff.equals(BigInteger.ONE)) {
|
||||||
|
log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier,
|
||||||
|
leftMostLeaseToReportInCaseOfHole, sortedLeasesWithHashKeyRanges.get(i));
|
||||||
|
return Optional.of(new HashRangeHole(leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease(),
|
||||||
|
sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease()));
|
||||||
|
}
|
||||||
|
leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(i);
|
||||||
|
leftLeaseHashRange = rightLeaseHashRange;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
static List<Lease> sortLeasesByHashRange(List<Lease> leasesWithHashKeyRanges) {
|
||||||
|
if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1)
|
||||||
|
return leasesWithHashKeyRanges;
|
||||||
|
Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator());
|
||||||
|
return leasesWithHashKeyRanges;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Value
|
||||||
|
private static class HashRangeHole {
|
||||||
|
HashRangeHole() {
|
||||||
|
hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole, HashKeyRangeForLease hashRangeAtEndOfPossibleHole) {
|
||||||
|
this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole;
|
||||||
|
this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole;
|
||||||
|
}
|
||||||
|
|
||||||
|
private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole;
|
||||||
|
private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole;
|
||||||
|
}
|
||||||
|
|
||||||
|
private class HashRangeHoleTracker {
|
||||||
|
private HashRangeHole hashRangeHole;
|
||||||
|
@Getter
|
||||||
|
private Integer numConsecutiveHoles;
|
||||||
|
|
||||||
|
public boolean hasHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) {
|
||||||
|
if (hashRangeHole.equals(this.hashRangeHole)) {
|
||||||
|
++this.numConsecutiveHoles;
|
||||||
|
} else {
|
||||||
|
this.hashRangeHole = hashRangeHole;
|
||||||
|
this.numConsecutiveHoles = 1;
|
||||||
|
}
|
||||||
|
return numConsecutiveHoles >= leasesRecoveryAuditorInconsistencyConfidenceThreshold;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class to compare leases based on their hash range.
|
||||||
|
*/
|
||||||
|
private static class HashKeyRangeComparator implements Comparator<Lease>, Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(Lease lease, Lease otherLease) {
|
||||||
|
Validate.notNull(lease);
|
||||||
|
Validate.notNull(otherLease);
|
||||||
|
Validate.notNull(lease.hashKeyRangeForLease());
|
||||||
|
Validate.notNull(otherLease.hashKeyRangeForLease());
|
||||||
|
return ComparisonChain.start()
|
||||||
|
.compare(lease.hashKeyRangeForLease().startingHashKey(), otherLease.hashKeyRangeForLease().startingHashKey())
|
||||||
|
.compare(lease.hashKeyRangeForLease().endingHashKey(), otherLease.hashKeyRangeForLease().endingHashKey())
|
||||||
|
.result();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -15,42 +15,66 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.coordinator;
|
package software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.base.Stopwatch;
|
||||||
|
|
||||||
|
import io.reactivex.plugins.RxJavaPlugins;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.time.Instant;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import io.reactivex.plugins.RxJavaPlugins;
|
|
||||||
import lombok.AccessLevel;
|
import lombok.AccessLevel;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.NoArgsConstructor;
|
import lombok.NoArgsConstructor;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.awssdk.utils.Validate;
|
||||||
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
||||||
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
||||||
|
import software.amazon.kinesis.common.InitialPositionInStream;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseCleanupManager;
|
||||||
import software.amazon.kinesis.leases.LeaseCoordinator;
|
import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
|
import software.amazon.kinesis.leases.LeaseSerializer;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
import software.amazon.kinesis.leases.ShardDetector;
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.leases.ShardPrioritization;
|
import software.amazon.kinesis.leases.ShardPrioritization;
|
||||||
import software.amazon.kinesis.leases.ShardSyncTask;
|
|
||||||
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
||||||
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
|
||||||
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator;
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator;
|
||||||
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer;
|
||||||
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import software.amazon.kinesis.lifecycle.LifecycleConfig;
|
import software.amazon.kinesis.lifecycle.LifecycleConfig;
|
||||||
import software.amazon.kinesis.lifecycle.ShardConsumer;
|
import software.amazon.kinesis.lifecycle.ShardConsumer;
|
||||||
import software.amazon.kinesis.lifecycle.ShardConsumerArgument;
|
import software.amazon.kinesis.lifecycle.ShardConsumerArgument;
|
||||||
|
|
@ -59,10 +83,14 @@ import software.amazon.kinesis.lifecycle.ShutdownNotification;
|
||||||
import software.amazon.kinesis.lifecycle.ShutdownReason;
|
import software.amazon.kinesis.lifecycle.ShutdownReason;
|
||||||
import software.amazon.kinesis.lifecycle.TaskResult;
|
import software.amazon.kinesis.lifecycle.TaskResult;
|
||||||
import software.amazon.kinesis.metrics.CloudWatchMetricsFactory;
|
import software.amazon.kinesis.metrics.CloudWatchMetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator;
|
|
||||||
import software.amazon.kinesis.metrics.MetricsConfig;
|
import software.amazon.kinesis.metrics.MetricsConfig;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
import software.amazon.kinesis.processor.Checkpointer;
|
import software.amazon.kinesis.processor.Checkpointer;
|
||||||
|
import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy;
|
||||||
|
import software.amazon.kinesis.processor.MultiStreamTracker;
|
||||||
import software.amazon.kinesis.processor.ProcessorConfig;
|
import software.amazon.kinesis.processor.ProcessorConfig;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
||||||
import software.amazon.kinesis.processor.ShutdownNotificationAware;
|
import software.amazon.kinesis.processor.ShutdownNotificationAware;
|
||||||
|
|
@ -70,6 +98,8 @@ import software.amazon.kinesis.retrieval.AggregatorUtil;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalConfig;
|
import software.amazon.kinesis.retrieval.RetrievalConfig;
|
||||||
|
|
||||||
|
import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.StreamsLeasesDeletionType;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
|
@ -78,6 +108,17 @@ import software.amazon.kinesis.retrieval.RetrievalConfig;
|
||||||
@Slf4j
|
@Slf4j
|
||||||
public class Scheduler implements Runnable {
|
public class Scheduler implements Runnable {
|
||||||
|
|
||||||
|
private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1;
|
||||||
|
private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L;
|
||||||
|
private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L;
|
||||||
|
private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L;
|
||||||
|
private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L;
|
||||||
|
private static final boolean SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS = false;
|
||||||
|
private static final String MULTI_STREAM_TRACKER = "MultiStreamTracker";
|
||||||
|
private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count";
|
||||||
|
private static final String PENDING_STREAMS_DELETION_COUNT = "StreamsPendingDeletion.Count";
|
||||||
|
private static final String DELETED_STREAMS_COUNT = "DeletedStreams.Count";
|
||||||
|
|
||||||
private SchedulerLog slog = new SchedulerLog();
|
private SchedulerLog slog = new SchedulerLog();
|
||||||
|
|
||||||
private final CheckpointConfig checkpointConfig;
|
private final CheckpointConfig checkpointConfig;
|
||||||
|
|
@ -100,25 +141,32 @@ public class Scheduler implements Runnable {
|
||||||
private final DiagnosticEventHandler diagnosticEventHandler;
|
private final DiagnosticEventHandler diagnosticEventHandler;
|
||||||
// private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
// private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||||
private final LeaseCoordinator leaseCoordinator;
|
private final LeaseCoordinator leaseCoordinator;
|
||||||
private final ShardSyncTaskManager shardSyncTaskManager;
|
private final Function<StreamConfig, ShardSyncTaskManager> shardSyncTaskManagerProvider;
|
||||||
|
private final Map<StreamConfig, ShardSyncTaskManager> streamToShardSyncTaskManagerMap = new HashMap<>();
|
||||||
|
private final PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager;
|
||||||
private final ShardPrioritization shardPrioritization;
|
private final ShardPrioritization shardPrioritization;
|
||||||
private final boolean cleanupLeasesUponShardCompletion;
|
private final boolean cleanupLeasesUponShardCompletion;
|
||||||
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
|
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||||
private final GracefulShutdownCoordinator gracefulShutdownCoordinator;
|
private final GracefulShutdownCoordinator gracefulShutdownCoordinator;
|
||||||
private final WorkerStateChangeListener workerStateChangeListener;
|
private final WorkerStateChangeListener workerStateChangeListener;
|
||||||
private final InitialPositionInStreamExtended initialPosition;
|
|
||||||
private final MetricsFactory metricsFactory;
|
private final MetricsFactory metricsFactory;
|
||||||
private final long failoverTimeMillis;
|
private final long failoverTimeMillis;
|
||||||
private final long taskBackoffTimeMillis;
|
private final long taskBackoffTimeMillis;
|
||||||
private final String streamName;
|
private final boolean isMultiStreamMode;
|
||||||
|
private final Map<StreamIdentifier, StreamConfig> currentStreamConfigMap;
|
||||||
|
private MultiStreamTracker multiStreamTracker;
|
||||||
|
private FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy;
|
||||||
private final long listShardsBackoffTimeMillis;
|
private final long listShardsBackoffTimeMillis;
|
||||||
private final int maxListShardsRetryAttempts;
|
private final int maxListShardsRetryAttempts;
|
||||||
private final LeaseRefresher leaseRefresher;
|
private final LeaseRefresher leaseRefresher;
|
||||||
private final ShardDetector shardDetector;
|
private final Function<StreamConfig, ShardDetector> shardDetectorProvider;
|
||||||
private final boolean ignoreUnexpetedChildShards;
|
private final boolean ignoreUnexpetedChildShards;
|
||||||
private final AggregatorUtil aggregatorUtil;
|
private final AggregatorUtil aggregatorUtil;
|
||||||
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
private final Function<StreamConfig, HierarchicalShardSyncer> hierarchicalShardSyncerProvider;
|
||||||
private final long schedulerInitializationBackoffTimeMillis;
|
private final long schedulerInitializationBackoffTimeMillis;
|
||||||
|
private final LeaderDecider leaderDecider;
|
||||||
|
private final Map<StreamIdentifier, Instant> staleStreamDeletionMap = new HashMap<>();
|
||||||
|
private final LeaseCleanupManager leaseCleanupManager;
|
||||||
|
|
||||||
// Holds consumers for shards the worker is currently tracking. Key is shard
|
// Holds consumers for shards the worker is currently tracking. Key is shard
|
||||||
// info, value is ShardConsumer.
|
// info, value is ShardConsumer.
|
||||||
|
|
@ -130,6 +178,9 @@ public class Scheduler implements Runnable {
|
||||||
|
|
||||||
private final Object lock = new Object();
|
private final Object lock = new Object();
|
||||||
|
|
||||||
|
private Stopwatch streamSyncWatch = Stopwatch.createUnstarted();
|
||||||
|
private boolean leasesSyncedOnAppInit = false;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used to ensure that only one requestedShutdown is in progress at a time.
|
* Used to ensure that only one requestedShutdown is in progress at a time.
|
||||||
*/
|
*/
|
||||||
|
|
@ -170,9 +221,25 @@ public class Scheduler implements Runnable {
|
||||||
this.retrievalConfig = retrievalConfig;
|
this.retrievalConfig = retrievalConfig;
|
||||||
|
|
||||||
this.applicationName = this.coordinatorConfig.applicationName();
|
this.applicationName = this.coordinatorConfig.applicationName();
|
||||||
|
this.isMultiStreamMode = this.retrievalConfig.appStreamTracker().map(
|
||||||
|
multiStreamTracker -> true, streamConfig -> false);
|
||||||
|
this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map(
|
||||||
|
multiStreamTracker -> {
|
||||||
|
this.multiStreamTracker = multiStreamTracker;
|
||||||
|
this.formerStreamsLeasesDeletionStrategy = multiStreamTracker.formerStreamsLeasesDeletionStrategy();
|
||||||
|
return multiStreamTracker.streamConfigList().stream()
|
||||||
|
.collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc));
|
||||||
|
},
|
||||||
|
streamConfig ->
|
||||||
|
Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig));
|
||||||
this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts();
|
this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts();
|
||||||
this.metricsFactory = this.metricsConfig.metricsFactory();
|
this.metricsFactory = this.metricsConfig.metricsFactory();
|
||||||
this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory()
|
// Determine leaseSerializer based on availability of MultiStreamTracker.
|
||||||
|
final LeaseSerializer leaseSerializer = isMultiStreamMode ?
|
||||||
|
new DynamoDBMultiStreamLeaseSerializer() :
|
||||||
|
new DynamoDBLeaseSerializer();
|
||||||
|
this.leaseCoordinator = this.leaseManagementConfig
|
||||||
|
.leaseManagementFactory(leaseSerializer, isMultiStreamMode)
|
||||||
.createLeaseCoordinator(this.metricsFactory);
|
.createLeaseCoordinator(this.metricsFactory);
|
||||||
this.leaseRefresher = this.leaseCoordinator.leaseRefresher();
|
this.leaseRefresher = this.leaseCoordinator.leaseRefresher();
|
||||||
|
|
||||||
|
|
@ -190,9 +257,9 @@ public class Scheduler implements Runnable {
|
||||||
this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService();
|
this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService();
|
||||||
this.diagnosticEventFactory = diagnosticEventFactory;
|
this.diagnosticEventFactory = diagnosticEventFactory;
|
||||||
this.diagnosticEventHandler = new DiagnosticEventLogger();
|
this.diagnosticEventHandler = new DiagnosticEventLogger();
|
||||||
|
this.shardSyncTaskManagerProvider = streamConfig -> this.leaseManagementConfig
|
||||||
this.shardSyncTaskManager = this.leaseManagementConfig.leaseManagementFactory()
|
.leaseManagementFactory(leaseSerializer, isMultiStreamMode)
|
||||||
.createShardSyncTaskManager(this.metricsFactory);
|
.createShardSyncTaskManager(this.metricsFactory, streamConfig);
|
||||||
this.shardPrioritization = this.coordinatorConfig.shardPrioritization();
|
this.shardPrioritization = this.coordinatorConfig.shardPrioritization();
|
||||||
this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion();
|
this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion();
|
||||||
this.skipShardSyncAtWorkerInitializationIfLeasesExist =
|
this.skipShardSyncAtWorkerInitializationIfLeasesExist =
|
||||||
|
|
@ -209,19 +276,26 @@ public class Scheduler implements Runnable {
|
||||||
this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory()
|
this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory()
|
||||||
.createWorkerStateChangeListener();
|
.createWorkerStateChangeListener();
|
||||||
}
|
}
|
||||||
this.initialPosition = retrievalConfig.initialPositionInStreamExtended();
|
this.leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher,
|
||||||
|
Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT);
|
||||||
this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis();
|
this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis();
|
||||||
this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis();
|
this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis();
|
||||||
// this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds();
|
// this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds();
|
||||||
// this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool();
|
// this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool();
|
||||||
this.streamName = this.retrievalConfig.streamName();
|
|
||||||
this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis();
|
this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis();
|
||||||
this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts();
|
this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts();
|
||||||
this.shardDetector = this.shardSyncTaskManager.shardDetector();
|
this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector();
|
||||||
this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards();
|
this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards();
|
||||||
this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil();
|
this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil();
|
||||||
this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer();
|
this.hierarchicalShardSyncerProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).hierarchicalShardSyncer();
|
||||||
this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis();
|
this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis();
|
||||||
|
this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager(
|
||||||
|
leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap,
|
||||||
|
shardSyncTaskManagerProvider, isMultiStreamMode, metricsFactory,
|
||||||
|
leaseManagementConfig.leasesRecoveryAuditorExecutionFrequencyMillis(),
|
||||||
|
leaseManagementConfig.leasesRecoveryAuditorInconsistencyConfidenceThreshold());
|
||||||
|
this.leaseCleanupManager = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer, isMultiStreamMode)
|
||||||
|
.createLeaseCleanupManager(metricsFactory);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -241,7 +315,6 @@ public class Scheduler implements Runnable {
|
||||||
workerStateChangeListener.onAllInitializationAttemptsFailed(e);
|
workerStateChangeListener.onAllInitializationAttemptsFailed(e);
|
||||||
shutdown();
|
shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
while (!shouldShutdown()) {
|
while (!shouldShutdown()) {
|
||||||
runProcessLoop();
|
runProcessLoop();
|
||||||
}
|
}
|
||||||
|
|
@ -264,28 +337,31 @@ public class Scheduler implements Runnable {
|
||||||
log.info("Initializing LeaseCoordinator");
|
log.info("Initializing LeaseCoordinator");
|
||||||
leaseCoordinator.initialize();
|
leaseCoordinator.initialize();
|
||||||
|
|
||||||
TaskResult result = null;
|
TaskResult result;
|
||||||
if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) {
|
if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) {
|
||||||
log.info("Syncing Kinesis shard info");
|
if (shouldInitiateLeaseSync()) {
|
||||||
ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition,
|
log.info("Worker {} is initiating the lease sync.", leaseManagementConfig.workerIdentifier());
|
||||||
cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer,
|
leaderElectedPeriodicShardSyncManager.syncShardsOnce();
|
||||||
metricsFactory);
|
|
||||||
result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call();
|
}
|
||||||
} else {
|
} else {
|
||||||
log.info("Skipping shard sync per configuration setting (and lease table is not empty)");
|
log.info("Skipping shard sync per configuration setting (and lease table is not empty)");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (result == null || result.getException() == null) {
|
leaseCleanupManager.start();
|
||||||
|
|
||||||
|
// If we reach this point, then we either skipped the lease sync or did not have any exception
|
||||||
|
// for any of the shard sync in the previous attempt.
|
||||||
if (!leaseCoordinator.isRunning()) {
|
if (!leaseCoordinator.isRunning()) {
|
||||||
log.info("Starting LeaseCoordinator");
|
log.info("Starting LeaseCoordinator");
|
||||||
leaseCoordinator.start();
|
leaseCoordinator.start();
|
||||||
} else {
|
} else {
|
||||||
log.info("LeaseCoordinator is already running. No need to start it.");
|
log.info("LeaseCoordinator is already running. No need to start it.");
|
||||||
}
|
}
|
||||||
|
log.info("Scheduling periodicShardSync");
|
||||||
|
leaderElectedPeriodicShardSyncManager.start();
|
||||||
|
streamSyncWatch.start();
|
||||||
isDone = true;
|
isDone = true;
|
||||||
} else {
|
|
||||||
lastException = result.getException();
|
|
||||||
}
|
|
||||||
} catch (LeasingException e) {
|
} catch (LeasingException e) {
|
||||||
log.error("Caught exception when initializing LeaseCoordinator", e);
|
log.error("Caught exception when initializing LeaseCoordinator", e);
|
||||||
lastException = e;
|
lastException = e;
|
||||||
|
|
@ -296,6 +372,7 @@ public class Scheduler implements Runnable {
|
||||||
if (!isDone) {
|
if (!isDone) {
|
||||||
try {
|
try {
|
||||||
Thread.sleep(schedulerInitializationBackoffTimeMillis);
|
Thread.sleep(schedulerInitializationBackoffTimeMillis);
|
||||||
|
leaderElectedPeriodicShardSyncManager.stop();
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
log.debug("Sleep interrupted while initializing worker.");
|
log.debug("Sleep interrupted while initializing worker.");
|
||||||
}
|
}
|
||||||
|
|
@ -309,30 +386,42 @@ public class Scheduler implements Runnable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
boolean shouldInitiateLeaseSync() throws InterruptedException,
|
||||||
|
DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS);
|
||||||
|
long waitUntil = System.currentTimeMillis() + waitTime;
|
||||||
|
|
||||||
|
boolean shouldInitiateLeaseSync = true;
|
||||||
|
while (System.currentTimeMillis() < waitUntil && (shouldInitiateLeaseSync = leaseRefresher.isLeaseTableEmpty())) {
|
||||||
|
// check every 3 seconds if lease table is still empty,
|
||||||
|
// to minimize contention between all workers bootstrapping at the same time
|
||||||
|
log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS);
|
||||||
|
Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS);
|
||||||
|
}
|
||||||
|
return shouldInitiateLeaseSync;
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
void runProcessLoop() {
|
void runProcessLoop() {
|
||||||
try {
|
try {
|
||||||
boolean foundCompletedShard = false;
|
|
||||||
Set<ShardInfo> assignedShards = new HashSet<>();
|
Set<ShardInfo> assignedShards = new HashSet<>();
|
||||||
for (ShardInfo shardInfo : getShardInfoForAssignments()) {
|
for (ShardInfo shardInfo : getShardInfoForAssignments()) {
|
||||||
ShardConsumer shardConsumer = createOrGetShardConsumer(shardInfo,
|
ShardConsumer shardConsumer = createOrGetShardConsumer(shardInfo,
|
||||||
processorConfig.shardRecordProcessorFactory());
|
processorConfig.shardRecordProcessorFactory(), leaseCleanupManager);
|
||||||
|
|
||||||
if (shardConsumer.isShutdown() && shardConsumer.shutdownReason().equals(ShutdownReason.SHARD_END)) {
|
|
||||||
foundCompletedShard = true;
|
|
||||||
} else {
|
|
||||||
shardConsumer.executeLifecycle();
|
shardConsumer.executeLifecycle();
|
||||||
}
|
|
||||||
assignedShards.add(shardInfo);
|
assignedShards.add(shardInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (foundCompletedShard) {
|
|
||||||
shardSyncTaskManager.syncShardAndLeaseInfo();
|
|
||||||
}
|
|
||||||
|
|
||||||
// clean up shard consumers for unassigned shards
|
// clean up shard consumers for unassigned shards
|
||||||
cleanupShardConsumers(assignedShards);
|
cleanupShardConsumers(assignedShards);
|
||||||
|
|
||||||
|
// check for new streams and sync with the scheduler state
|
||||||
|
if (isLeader()) {
|
||||||
|
checkAndSyncStreamShardsAndLeases();
|
||||||
|
}
|
||||||
|
|
||||||
logExecutorState();
|
logExecutorState();
|
||||||
slog.info("Sleeping ...");
|
slog.info("Sleeping ...");
|
||||||
Thread.sleep(shardConsumerDispatchPollIntervalMillis);
|
Thread.sleep(shardConsumerDispatchPollIntervalMillis);
|
||||||
|
|
@ -348,6 +437,218 @@ public class Scheduler implements Runnable {
|
||||||
slog.resetInfoLogging();
|
slog.resetInfoLogging();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private boolean isLeader() {
|
||||||
|
return leaderDecider.isLeader(leaseManagementConfig.workerIdentifier());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note: This method has package level access solely for testing purposes.
|
||||||
|
* Sync all streams method.
|
||||||
|
* @return streams that are being synced by this worker
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
Set<StreamIdentifier> checkAndSyncStreamShardsAndLeases()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
final Set<StreamIdentifier> streamsSynced = new HashSet<>();
|
||||||
|
|
||||||
|
if (shouldSyncStreamsNow()) {
|
||||||
|
final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER);
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
final Map<StreamIdentifier, StreamConfig> newStreamConfigMap = new HashMap<>();
|
||||||
|
final Duration waitPeriodToDeleteOldStreams = formerStreamsLeasesDeletionStrategy.waitPeriodToDeleteFormerStreams();
|
||||||
|
// Making an immutable copy
|
||||||
|
newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream()
|
||||||
|
.collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)));
|
||||||
|
|
||||||
|
List<MultiStreamLease> leases;
|
||||||
|
|
||||||
|
// This is done to ensure that we clean up the stale streams lingering in the lease table.
|
||||||
|
if (!leasesSyncedOnAppInit && isMultiStreamMode) {
|
||||||
|
leases = fetchMultiStreamLeases();
|
||||||
|
syncStreamsFromLeaseTableOnAppInit(leases);
|
||||||
|
leasesSyncedOnAppInit = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// For new streams discovered, do a shard sync and update the currentStreamConfigMap
|
||||||
|
for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) {
|
||||||
|
if (!currentStreamConfigMap.containsKey(streamIdentifier)) {
|
||||||
|
log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream.");
|
||||||
|
ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier));
|
||||||
|
shardSyncTaskManager.submitShardSyncTask();
|
||||||
|
currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier));
|
||||||
|
streamsSynced.add(streamIdentifier);
|
||||||
|
} else {
|
||||||
|
if (log.isDebugEnabled()) {
|
||||||
|
log.debug(streamIdentifier + " is already being processed - skipping shard sync.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final Consumer<StreamIdentifier> enqueueStreamLeaseDeletionOperation = streamIdentifier -> {
|
||||||
|
if (!newStreamConfigMap.containsKey(streamIdentifier)) {
|
||||||
|
staleStreamDeletionMap.putIfAbsent(streamIdentifier, Instant.now());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) {
|
||||||
|
// Now, we are identifying the stale/old streams and enqueuing it for deferred deletion.
|
||||||
|
// It is assumed that all the workers will always have the latest and consistent snapshot of streams
|
||||||
|
// from the multiStreamTracker.
|
||||||
|
//
|
||||||
|
// The following streams transition state among two workers are NOT considered safe, where Worker 2, on
|
||||||
|
// initialization learn about D from lease table and delete the leases for D, as it is not available
|
||||||
|
// in its latest MultiStreamTracker.
|
||||||
|
// Worker 1 : A,B,C -> A,B,C,D (latest)
|
||||||
|
// Worker 2 : BOOTS_UP -> A,B,C (stale)
|
||||||
|
//
|
||||||
|
// The following streams transition state among two workers are NOT considered safe, where Worker 2 might
|
||||||
|
// end up deleting the leases for A and D and loose progress made so far.
|
||||||
|
// Worker 1 : A,B,C -> A,B,C,D (latest)
|
||||||
|
// Worker 2 : A,B,C -> B,C (stale/partial)
|
||||||
|
//
|
||||||
|
// In order to give workers with stale stream info, sufficient time to learn about the new streams
|
||||||
|
// before attempting to delete it, we will be deferring the leases deletion based on the
|
||||||
|
// defer time period.
|
||||||
|
|
||||||
|
currentStreamConfigMap.keySet().stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier));
|
||||||
|
|
||||||
|
} else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) {
|
||||||
|
Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiersForLeaseCleanup()).ifPresent(
|
||||||
|
streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier)));
|
||||||
|
} else {
|
||||||
|
// Remove the old/stale streams identified through the new and existing streams list, without
|
||||||
|
// cleaning up their leases. Disabling deprecated shard sync + lease cleanup through a flag.
|
||||||
|
Iterator<StreamIdentifier> currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator();
|
||||||
|
while (currentSetOfStreamsIter.hasNext()) {
|
||||||
|
StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next();
|
||||||
|
if (!newStreamConfigMap.containsKey(streamIdentifier)) {
|
||||||
|
if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) {
|
||||||
|
log.info(
|
||||||
|
"Found old/deleted stream : {}. Triggering shard sync. Removing from tracked active streams.", streamIdentifier);
|
||||||
|
ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(
|
||||||
|
currentStreamConfigMap.get(streamIdentifier));
|
||||||
|
shardSyncTaskManager.submitShardSyncTask();
|
||||||
|
} else {
|
||||||
|
log.info(
|
||||||
|
"Found old/deleted stream : {}. Removing from tracked active streams, but not cleaning up leases,"
|
||||||
|
+ " as part of this workflow", streamIdentifier);
|
||||||
|
}
|
||||||
|
currentSetOfStreamsIter.remove();
|
||||||
|
streamsSynced.add(streamIdentifier);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Now let's scan the streamIdentifiersForLeaseCleanup eligible for deferred deletion and delete them.
|
||||||
|
// StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and
|
||||||
|
// the streamIdentifiersForLeaseCleanup are not present in the latest snapshot.
|
||||||
|
final Map<Boolean, Set<StreamIdentifier>> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors
|
||||||
|
.partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), Collectors.toSet()));
|
||||||
|
final Set<StreamIdentifier> staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier ->
|
||||||
|
Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet());
|
||||||
|
final Set<StreamIdentifier> deletedStreamsLeases = deleteMultiStreamLeases(staleStreamIdsToBeDeleted);
|
||||||
|
streamsSynced.addAll(deletedStreamsLeases);
|
||||||
|
|
||||||
|
// Purge the active streams from stale streams list.
|
||||||
|
final Set<StreamIdentifier> staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true);
|
||||||
|
removeStreamsFromStaleStreamsList(staleStreamIdsToBeRevived);
|
||||||
|
|
||||||
|
log.warn(
|
||||||
|
"Streams enqueued for deletion for lease table cleanup along with their scheduled time for deletion: {} ",
|
||||||
|
staleStreamDeletionMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey,
|
||||||
|
entry -> entry.getValue().plus(waitPeriodToDeleteOldStreams))));
|
||||||
|
|
||||||
|
streamSyncWatch.reset().start();
|
||||||
|
|
||||||
|
MetricsUtil.addCount(metricsScope, ACTIVE_STREAMS_COUNT, newStreamConfigMap.size(), MetricsLevel.SUMMARY);
|
||||||
|
MetricsUtil.addCount(metricsScope, PENDING_STREAMS_DELETION_COUNT, staleStreamDeletionMap.size(),
|
||||||
|
MetricsLevel.SUMMARY);
|
||||||
|
MetricsUtil.addCount(metricsScope, DELETED_STREAMS_COUNT, deletedStreamsLeases.size(), MetricsLevel.SUMMARY);
|
||||||
|
} finally {
|
||||||
|
MetricsUtil.endScope(metricsScope);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return streamsSynced;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting boolean shouldSyncStreamsNow() {
|
||||||
|
return isMultiStreamMode &&
|
||||||
|
(streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void syncStreamsFromLeaseTableOnAppInit(List<MultiStreamLease> leases) {
|
||||||
|
final Set<StreamIdentifier> streamIdentifiers = leases.stream()
|
||||||
|
.map(lease -> StreamIdentifier.multiStreamInstance(lease.streamIdentifier()))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
for (StreamIdentifier streamIdentifier : streamIdentifiers) {
|
||||||
|
if (!currentStreamConfigMap.containsKey(streamIdentifier)) {
|
||||||
|
currentStreamConfigMap.put(streamIdentifier, getDefaultStreamConfig(streamIdentifier));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<MultiStreamLease> fetchMultiStreamLeases()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
return (List<MultiStreamLease>) ((List) leaseCoordinator.leaseRefresher().listLeases());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void removeStreamsFromStaleStreamsList(Set<StreamIdentifier> streamIdentifiers) {
|
||||||
|
for(StreamIdentifier streamIdentifier : streamIdentifiers) {
|
||||||
|
staleStreamDeletionMap.remove(streamIdentifier);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Set<StreamIdentifier> deleteMultiStreamLeases(Set<StreamIdentifier> streamIdentifiers)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
final Set<StreamIdentifier> streamsSynced = new HashSet<>();
|
||||||
|
List<MultiStreamLease> leases = null;
|
||||||
|
Map<String, List<MultiStreamLease>> streamIdToShardsMap = null;
|
||||||
|
for(StreamIdentifier streamIdentifier : streamIdentifiers) {
|
||||||
|
if (leases == null) {
|
||||||
|
// Lazy Load once and use many times for this iteration.
|
||||||
|
leases = fetchMultiStreamLeases();
|
||||||
|
}
|
||||||
|
if (streamIdToShardsMap == null) {
|
||||||
|
// Lazy load once and use many times for this iteration.
|
||||||
|
streamIdToShardsMap = leases.stream().collect(Collectors
|
||||||
|
.groupingBy(MultiStreamLease::streamIdentifier,
|
||||||
|
Collectors.toCollection(ArrayList::new)));
|
||||||
|
}
|
||||||
|
log.warn("Found old/deleted stream: " + streamIdentifier + ". Directly deleting leases of this stream.");
|
||||||
|
// Deleting leases will cause the workers to shutdown the record processors for these shards.
|
||||||
|
if (deleteMultiStreamLeases(streamIdToShardsMap.get(streamIdentifier.serialize()))) {
|
||||||
|
currentStreamConfigMap.remove(streamIdentifier);
|
||||||
|
staleStreamDeletionMap.remove(streamIdentifier);
|
||||||
|
streamsSynced.add(streamIdentifier);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return streamsSynced;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean deleteMultiStreamLeases(List<MultiStreamLease> leases) {
|
||||||
|
if (leases != null) {
|
||||||
|
for (MultiStreamLease lease : leases) {
|
||||||
|
try {
|
||||||
|
leaseRefresher.deleteLease(lease);
|
||||||
|
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
||||||
|
log.error(
|
||||||
|
"Unable to delete stale stream lease {}. Skipping further deletions for this stream. Will retry later.",
|
||||||
|
lease.leaseKey(), e);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// When a stream is no longer needed to be tracked, return a default StreamConfig with LATEST for faster shard end.
|
||||||
|
private StreamConfig getDefaultStreamConfig(StreamIdentifier streamIdentifier) {
|
||||||
|
return new StreamConfig(streamIdentifier, InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns whether worker can shutdown immediately. Note that this method is called from Worker's {{@link #run()}
|
* Returns whether worker can shutdown immediately. Note that this method is called from Worker's {{@link #run()}
|
||||||
* method before every loop run, so method must do minimum amount of work to not impact shard processing timings.
|
* method before every loop run, so method must do minimum amount of work to not impact shard processing timings.
|
||||||
|
|
@ -516,6 +817,7 @@ public class Scheduler implements Runnable {
|
||||||
// Lost leases will force Worker to begin shutdown process for all shard consumers in
|
// Lost leases will force Worker to begin shutdown process for all shard consumers in
|
||||||
// Worker.run().
|
// Worker.run().
|
||||||
leaseCoordinator.stop();
|
leaseCoordinator.stop();
|
||||||
|
leaderElectedPeriodicShardSyncManager.stop();
|
||||||
workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.SHUT_DOWN);
|
workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.SHUT_DOWN);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -549,7 +851,7 @@ public class Scheduler implements Runnable {
|
||||||
if (!firstItem) {
|
if (!firstItem) {
|
||||||
builder.append(", ");
|
builder.append(", ");
|
||||||
}
|
}
|
||||||
builder.append(shardInfo.shardId());
|
builder.append(ShardInfo.getLeaseKey(shardInfo));
|
||||||
firstItem = false;
|
firstItem = false;
|
||||||
}
|
}
|
||||||
slog.info("Current stream shard assignments: " + builder.toString());
|
slog.info("Current stream shard assignments: " + builder.toString());
|
||||||
|
|
@ -569,7 +871,8 @@ public class Scheduler implements Runnable {
|
||||||
* @return ShardConsumer for the shard
|
* @return ShardConsumer for the shard
|
||||||
*/
|
*/
|
||||||
ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo,
|
ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo,
|
||||||
@NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) {
|
@NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory,
|
||||||
|
@NonNull final LeaseCleanupManager leaseCleanupManager) {
|
||||||
ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo);
|
ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo);
|
||||||
// Instantiate a new consumer if we don't have one, or the one we
|
// Instantiate a new consumer if we don't have one, or the one we
|
||||||
// had was from an earlier
|
// had was from an earlier
|
||||||
|
|
@ -578,24 +881,38 @@ public class Scheduler implements Runnable {
|
||||||
// completely processed (shutdown reason terminate).
|
// completely processed (shutdown reason terminate).
|
||||||
if ((consumer == null)
|
if ((consumer == null)
|
||||||
|| (consumer.isShutdown() && consumer.shutdownReason().equals(ShutdownReason.LEASE_LOST))) {
|
|| (consumer.isShutdown() && consumer.shutdownReason().equals(ShutdownReason.LEASE_LOST))) {
|
||||||
consumer = buildConsumer(shardInfo, shardRecordProcessorFactory);
|
consumer = buildConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
shardInfoShardConsumerMap.put(shardInfo, consumer);
|
shardInfoShardConsumerMap.put(shardInfo, consumer);
|
||||||
slog.infoForce("Created new shardConsumer for : " + shardInfo);
|
slog.infoForce("Created new shardConsumer for : " + shardInfo);
|
||||||
}
|
}
|
||||||
return consumer;
|
return consumer;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private ShardSyncTaskManager createOrGetShardSyncTaskManager(StreamConfig streamConfig) {
|
||||||
|
return streamToShardSyncTaskManagerMap.computeIfAbsent(streamConfig, s -> shardSyncTaskManagerProvider.apply(s));
|
||||||
|
}
|
||||||
|
|
||||||
protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo,
|
protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo,
|
||||||
@NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) {
|
@NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory,
|
||||||
RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory);
|
@NonNull final LeaseCleanupManager leaseCleanupManager) {
|
||||||
ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo,
|
ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo,
|
||||||
checkpoint);
|
checkpoint);
|
||||||
|
// The only case where streamName is not available will be when multistreamtracker not set. In this case,
|
||||||
|
// get the default stream name for the single stream application.
|
||||||
|
final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt());
|
||||||
|
|
||||||
|
// Irrespective of single stream app or multi stream app, streamConfig should always be available.
|
||||||
|
// If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config
|
||||||
|
// to gracefully complete the reading.
|
||||||
|
final StreamConfig streamConfig = currentStreamConfigMap.getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier));
|
||||||
|
Validate.notNull(streamConfig, "StreamConfig should not be null");
|
||||||
|
RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory);
|
||||||
ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo,
|
ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo,
|
||||||
streamName,
|
streamConfig.streamIdentifier(),
|
||||||
leaseCoordinator,
|
leaseCoordinator,
|
||||||
executorService,
|
executorService,
|
||||||
cache,
|
cache,
|
||||||
shardRecordProcessorFactory.shardRecordProcessor(),
|
shardRecordProcessorFactory.shardRecordProcessor(streamIdentifier),
|
||||||
checkpoint,
|
checkpoint,
|
||||||
checkpointer,
|
checkpointer,
|
||||||
parentShardPollIntervalMillis,
|
parentShardPollIntervalMillis,
|
||||||
|
|
@ -605,13 +922,14 @@ public class Scheduler implements Runnable {
|
||||||
maxListShardsRetryAttempts,
|
maxListShardsRetryAttempts,
|
||||||
processorConfig.callProcessRecordsEvenForEmptyRecordList(),
|
processorConfig.callProcessRecordsEvenForEmptyRecordList(),
|
||||||
shardConsumerDispatchPollIntervalMillis,
|
shardConsumerDispatchPollIntervalMillis,
|
||||||
initialPosition,
|
streamConfig.initialPositionInStreamExtended(),
|
||||||
cleanupLeasesUponShardCompletion,
|
cleanupLeasesUponShardCompletion,
|
||||||
ignoreUnexpetedChildShards,
|
ignoreUnexpetedChildShards,
|
||||||
shardDetector,
|
shardDetectorProvider.apply(streamConfig),
|
||||||
aggregatorUtil,
|
aggregatorUtil,
|
||||||
hierarchicalShardSyncer,
|
hierarchicalShardSyncerProvider.apply(streamConfig),
|
||||||
metricsFactory);
|
metricsFactory,
|
||||||
|
leaseCleanupManager);
|
||||||
return new ShardConsumer(cache, executorService, shardInfo, lifecycleConfig.logWarningForTaskAfterMillis(),
|
return new ShardConsumer(cache, executorService, shardInfo, lifecycleConfig.logWarningForTaskAfterMillis(),
|
||||||
argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning());
|
argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning());
|
||||||
}
|
}
|
||||||
|
|
@ -633,7 +951,7 @@ public class Scheduler implements Runnable {
|
||||||
ShardConsumer consumer = shardInfoShardConsumerMap.get(shard);
|
ShardConsumer consumer = shardInfoShardConsumerMap.get(shard);
|
||||||
if (consumer.leaseLost()) {
|
if (consumer.leaseLost()) {
|
||||||
shardInfoShardConsumerMap.remove(shard);
|
shardInfoShardConsumerMap.remove(shard);
|
||||||
log.debug("Removed consumer for {} as lease has been lost", shard.shardId());
|
log.debug("Removed consumer for {} as lease has been lost", ShardInfo.getLeaseKey(shard));
|
||||||
} else {
|
} else {
|
||||||
consumer.executeLifecycle();
|
consumer.executeLifecycle();
|
||||||
}
|
}
|
||||||
|
|
@ -660,6 +978,18 @@ public class Scheduler implements Runnable {
|
||||||
executorStateEvent.accept(diagnosticEventHandler);
|
executorStateEvent.accept(diagnosticEventHandler);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private StreamIdentifier getStreamIdentifier(Optional<String> streamIdentifierString) {
|
||||||
|
final StreamIdentifier streamIdentifier;
|
||||||
|
if (streamIdentifierString.isPresent()) {
|
||||||
|
streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierString.get());
|
||||||
|
} else {
|
||||||
|
Validate.isTrue(!isMultiStreamMode, "Should not be in MultiStream Mode");
|
||||||
|
streamIdentifier = this.currentStreamConfigMap.values().iterator().next().streamIdentifier();
|
||||||
|
}
|
||||||
|
Validate.notNull(streamIdentifier, "Stream identifier should not be empty");
|
||||||
|
return streamIdentifier;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Logger for suppressing too much INFO logging. To avoid too much logging information Worker will output logging at
|
* Logger for suppressing too much INFO logging. To avoid too much logging information Worker will output logging at
|
||||||
* INFO level for a single pass through the main loop every minute. At DEBUG level it will output all INFO logs on
|
* INFO level for a single pass through the main loop every minute. At DEBUG level it will output all INFO logs on
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import software.amazon.awssdk.core.SdkBytes;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
|
||||||
|
|
@ -36,6 +37,14 @@ public class DynamoUtils {
|
||||||
return AttributeValue.builder().ss(collectionValue).build();
|
return AttributeValue.builder().ss(collectionValue).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static AttributeValue createAttributeValue(byte[] byteBufferValue) {
|
||||||
|
if (byteBufferValue == null) {
|
||||||
|
throw new IllegalArgumentException("Byte buffer attributeValues cannot be null or empty.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return AttributeValue.builder().b(SdkBytes.fromByteArray(byteBufferValue)).build();
|
||||||
|
}
|
||||||
|
|
||||||
public static AttributeValue createAttributeValue(String stringValue) {
|
public static AttributeValue createAttributeValue(String stringValue) {
|
||||||
if (stringValue == null || stringValue.isEmpty()) {
|
if (stringValue == null || stringValue.isEmpty()) {
|
||||||
throw new IllegalArgumentException("String attributeValues cannot be null or empty.");
|
throw new IllegalArgumentException("String attributeValues cannot be null or empty.");
|
||||||
|
|
@ -52,6 +61,15 @@ public class DynamoUtils {
|
||||||
return AttributeValue.builder().n(longValue.toString()).build();
|
return AttributeValue.builder().n(longValue.toString()).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static byte[] safeGetByteArray(Map<String, AttributeValue> dynamoRecord, String key) {
|
||||||
|
AttributeValue av = dynamoRecord.get(key);
|
||||||
|
if (av == null) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
return av.b().asByteArray();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static Long safeGetLong(Map<String, AttributeValue> dynamoRecord, String key) {
|
public static Long safeGetLong(Map<String, AttributeValue> dynamoRecord, String key) {
|
||||||
AttributeValue av = dynamoRecord.get(key);
|
AttributeValue av = dynamoRecord.get(key);
|
||||||
if (av == null) {
|
if (av == null) {
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -19,34 +19,41 @@ import java.time.Duration;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.time.temporal.ChronoUnit;
|
import java.time.temporal.ChronoUnit;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
|
||||||
|
|
||||||
import lombok.AccessLevel;
|
import lombok.AccessLevel;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.Synchronized;
|
import lombok.Synchronized;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse;
|
||||||
import software.amazon.awssdk.services.kinesis.model.KinesisException;
|
import software.amazon.awssdk.services.kinesis.model.KinesisException;
|
||||||
import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
|
import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
|
import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
|
import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ResourceInUseException;
|
import software.amazon.awssdk.services.kinesis.model.ResourceInUseException;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
import software.amazon.awssdk.services.kinesis.model.Shard;
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ShardFilter;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
|
||||||
import software.amazon.awssdk.utils.CollectionUtils;
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.FutureUtils;
|
import software.amazon.kinesis.common.FutureUtils;
|
||||||
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -59,8 +66,8 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
@NonNull
|
@NonNull @Getter
|
||||||
private final String streamName;
|
private final StreamIdentifier streamIdentifier;
|
||||||
private final long listShardsBackoffTimeInMillis;
|
private final long listShardsBackoffTimeInMillis;
|
||||||
private final int maxListShardsRetryAttempts;
|
private final int maxListShardsRetryAttempts;
|
||||||
private final long listShardsCacheAllowedAgeInSeconds;
|
private final long listShardsCacheAllowedAgeInSeconds;
|
||||||
|
|
@ -77,16 +84,16 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis,
|
public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis,
|
||||||
int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload,
|
int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload,
|
||||||
int cacheMissWarningModulus) {
|
int cacheMissWarningModulus) {
|
||||||
this(kinesisClient, streamName, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts,
|
this(kinesisClient, StreamIdentifier.singleStreamInstance(streamName), listShardsBackoffTimeInMillis, maxListShardsRetryAttempts,
|
||||||
listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus,
|
listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus,
|
||||||
LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT);
|
LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT);
|
||||||
}
|
}
|
||||||
|
|
||||||
public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis,
|
public KinesisShardDetector(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, long listShardsBackoffTimeInMillis,
|
||||||
int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload,
|
int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload,
|
||||||
int cacheMissWarningModulus, Duration kinesisRequestTimeout) {
|
int cacheMissWarningModulus, Duration kinesisRequestTimeout) {
|
||||||
this.kinesisClient = kinesisClient;
|
this.kinesisClient = kinesisClient;
|
||||||
this.streamName = streamName;
|
this.streamIdentifier = streamIdentifier;
|
||||||
this.listShardsBackoffTimeInMillis = listShardsBackoffTimeInMillis;
|
this.listShardsBackoffTimeInMillis = listShardsBackoffTimeInMillis;
|
||||||
this.maxListShardsRetryAttempts = maxListShardsRetryAttempts;
|
this.maxListShardsRetryAttempts = maxListShardsRetryAttempts;
|
||||||
this.listShardsCacheAllowedAgeInSeconds = listShardsCacheAllowedAgeInSeconds;
|
this.listShardsCacheAllowedAgeInSeconds = listShardsCacheAllowedAgeInSeconds;
|
||||||
|
|
@ -149,12 +156,18 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
@Override
|
@Override
|
||||||
@Synchronized
|
@Synchronized
|
||||||
public List<Shard> listShards() {
|
public List<Shard> listShards() {
|
||||||
|
return listShardsWithFilter(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Synchronized
|
||||||
|
public List<Shard> listShardsWithFilter(ShardFilter shardFilter) {
|
||||||
final List<Shard> shards = new ArrayList<>();
|
final List<Shard> shards = new ArrayList<>();
|
||||||
ListShardsResponse result;
|
ListShardsResponse result;
|
||||||
String nextToken = null;
|
String nextToken = null;
|
||||||
|
|
||||||
do {
|
do {
|
||||||
result = listShards(nextToken);
|
result = listShards(shardFilter, nextToken);
|
||||||
|
|
||||||
if (result == null) {
|
if (result == null) {
|
||||||
/*
|
/*
|
||||||
|
|
@ -172,15 +185,16 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
return shards;
|
return shards;
|
||||||
}
|
}
|
||||||
|
|
||||||
private ListShardsResponse listShards(final String nextToken) {
|
private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken) {
|
||||||
final AWSExceptionManager exceptionManager = new AWSExceptionManager();
|
final AWSExceptionManager exceptionManager = new AWSExceptionManager();
|
||||||
|
exceptionManager.add(ResourceNotFoundException.class, t -> t);
|
||||||
exceptionManager.add(LimitExceededException.class, t -> t);
|
exceptionManager.add(LimitExceededException.class, t -> t);
|
||||||
exceptionManager.add(ResourceInUseException.class, t -> t);
|
exceptionManager.add(ResourceInUseException.class, t -> t);
|
||||||
exceptionManager.add(KinesisException.class, t -> t);
|
exceptionManager.add(KinesisException.class, t -> t);
|
||||||
|
|
||||||
ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder();
|
ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder().shardFilter(shardFilter);
|
||||||
if (StringUtils.isEmpty(nextToken)) {
|
if (StringUtils.isEmpty(nextToken)) {
|
||||||
request = request.streamName(streamName);
|
request = request.streamName(streamIdentifier.streamName());
|
||||||
} else {
|
} else {
|
||||||
request = request.nextToken(nextToken);
|
request = request.nextToken(nextToken);
|
||||||
}
|
}
|
||||||
|
|
@ -189,10 +203,9 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
int remainingRetries = maxListShardsRetryAttempts;
|
int remainingRetries = maxListShardsRetryAttempts;
|
||||||
|
|
||||||
while (result == null) {
|
while (result == null) {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
try {
|
try {
|
||||||
result = FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request.build()), kinesisRequestTimeout);
|
result = getListShardsResponse(request.build());
|
||||||
} catch (ExecutionException e) {
|
} catch (ExecutionException e) {
|
||||||
throw exceptionManager.apply(e.getCause());
|
throw exceptionManager.apply(e.getCause());
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
|
|
@ -205,14 +218,20 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
+ " Active or Updating)");
|
+ " Active or Updating)");
|
||||||
return null;
|
return null;
|
||||||
} catch (LimitExceededException e) {
|
} catch (LimitExceededException e) {
|
||||||
log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamName,
|
log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamIdentifier,
|
||||||
listShardsBackoffTimeInMillis);
|
listShardsBackoffTimeInMillis);
|
||||||
try {
|
try {
|
||||||
Thread.sleep(listShardsBackoffTimeInMillis);
|
Thread.sleep(listShardsBackoffTimeInMillis);
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
log.debug("Stream {} : Sleep was interrupted ", streamName, ie);
|
log.debug("Stream {} : Sleep was interrupted ", streamIdentifier, ie);
|
||||||
}
|
}
|
||||||
lastException = e;
|
lastException = e;
|
||||||
|
} catch (ResourceNotFoundException e) {
|
||||||
|
log.warn("Got ResourceNotFoundException when fetching shard list for {}. Stream no longer exists.",
|
||||||
|
streamIdentifier.streamName());
|
||||||
|
return ListShardsResponse.builder().shards(Collections.emptyList())
|
||||||
|
.nextToken(null)
|
||||||
|
.build();
|
||||||
} catch (TimeoutException te) {
|
} catch (TimeoutException te) {
|
||||||
throw new RuntimeException(te);
|
throw new RuntimeException(te);
|
||||||
}
|
}
|
||||||
|
|
@ -243,4 +262,31 @@ public class KinesisShardDetector implements ShardDetector {
|
||||||
log.debug("{}. Age doesn't exceed limit of {} seconds.", message, listShardsCacheAllowedAgeInSeconds);
|
log.debug("{}. Age doesn't exceed limit of {} seconds.", message, listShardsCacheAllowedAgeInSeconds);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ListShardsResponse getListShardsResponse(ListShardsRequest request) throws
|
||||||
|
ExecutionException, TimeoutException, InterruptedException {
|
||||||
|
return FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request), kinesisRequestTimeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ChildShard> getChildShards(final String shardId) throws InterruptedException, ExecutionException, TimeoutException {
|
||||||
|
final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder()
|
||||||
|
.streamName(streamIdentifier.streamName())
|
||||||
|
.shardIteratorType(ShardIteratorType.LATEST)
|
||||||
|
.shardId(shardId)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
final GetShardIteratorResponse getShardIteratorResponse =
|
||||||
|
FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), kinesisRequestTimeout);
|
||||||
|
|
||||||
|
final GetRecordsRequest getRecordsRequest = KinesisRequestsBuilder.getRecordsRequestBuilder()
|
||||||
|
.shardIterator(getShardIteratorResponse.shardIterator())
|
||||||
|
.build();
|
||||||
|
|
||||||
|
final GetRecordsResponse getRecordsResponse =
|
||||||
|
FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(getRecordsRequest), kinesisRequestTimeout);
|
||||||
|
|
||||||
|
return getRecordsResponse.childShards();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -14,22 +14,21 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import com.google.common.collect.Collections2;
|
|
||||||
import lombok.EqualsAndHashCode;
|
import lombok.EqualsAndHashCode;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.NoArgsConstructor;
|
import lombok.NoArgsConstructor;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.ToString;
|
import lombok.ToString;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class contains data pertaining to a Lease. Distributed systems may use leases to partition work across a
|
* This class contains data pertaining to a Lease. Distributed systems may use leases to partition work across a
|
||||||
* fleet of workers. Each unit of work (identified by a leaseKey) has a corresponding Lease. Every worker will contend
|
* fleet of workers. Each unit of work (identified by a leaseKey) has a corresponding Lease. Every worker will contend
|
||||||
|
|
@ -40,7 +39,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
@NoArgsConstructor
|
@NoArgsConstructor
|
||||||
@Getter
|
@Getter
|
||||||
@Accessors(fluent = true)
|
@Accessors(fluent = true)
|
||||||
@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos"})
|
@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds", "pendingCheckpointState"})
|
||||||
@ToString
|
@ToString
|
||||||
public class Lease {
|
public class Lease {
|
||||||
/*
|
/*
|
||||||
|
|
@ -84,11 +83,21 @@ public class Lease {
|
||||||
* @return pending checkpoint, possibly null.
|
* @return pending checkpoint, possibly null.
|
||||||
*/
|
*/
|
||||||
private ExtendedSequenceNumber pendingCheckpoint;
|
private ExtendedSequenceNumber pendingCheckpoint;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Last pending application state. Deliberately excluded from hashCode and equals.
|
||||||
|
*
|
||||||
|
* @return pending checkpoint state, possibly null.
|
||||||
|
*/
|
||||||
|
private byte[] pendingCheckpointState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return count of distinct lease holders between checkpoints.
|
* @return count of distinct lease holders between checkpoints.
|
||||||
*/
|
*/
|
||||||
private Long ownerSwitchesSinceCheckpoint = 0L;
|
private Long ownerSwitchesSinceCheckpoint = 0L;
|
||||||
private Set<String> parentShardIds = new HashSet<>();
|
private Set<String> parentShardIds = new HashSet<>();
|
||||||
|
private Set<String> childShardIds = new HashSet<>();
|
||||||
|
private HashKeyRangeForLease hashKeyRangeForLease;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Copy constructor, used by clone().
|
* Copy constructor, used by clone().
|
||||||
|
|
@ -98,13 +107,24 @@ public class Lease {
|
||||||
protected Lease(Lease lease) {
|
protected Lease(Lease lease) {
|
||||||
this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(),
|
this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(),
|
||||||
lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(),
|
lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(),
|
||||||
lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds());
|
lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.childShardIds(),
|
||||||
|
lease.pendingCheckpointState(), lease.hashKeyRangeForLease());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter,
|
||||||
|
final UUID concurrencyToken, final Long lastCounterIncrementNanos,
|
||||||
|
final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint,
|
||||||
|
final Long ownerSwitchesSinceCheckpoint, final Set<String> parentShardIds) {
|
||||||
|
this(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, pendingCheckpoint,
|
||||||
|
ownerSwitchesSinceCheckpoint, parentShardIds, new HashSet<>(), null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter,
|
public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter,
|
||||||
final UUID concurrencyToken, final Long lastCounterIncrementNanos,
|
final UUID concurrencyToken, final Long lastCounterIncrementNanos,
|
||||||
final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint,
|
final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint,
|
||||||
final Long ownerSwitchesSinceCheckpoint, final Set<String> parentShardIds) {
|
final Long ownerSwitchesSinceCheckpoint, final Set<String> parentShardIds, final Set<String> childShardIds,
|
||||||
|
final byte[] pendingCheckpointState, final HashKeyRangeForLease hashKeyRangeForLease) {
|
||||||
this.leaseKey = leaseKey;
|
this.leaseKey = leaseKey;
|
||||||
this.leaseOwner = leaseOwner;
|
this.leaseOwner = leaseOwner;
|
||||||
this.leaseCounter = leaseCounter;
|
this.leaseCounter = leaseCounter;
|
||||||
|
|
@ -116,6 +136,11 @@ public class Lease {
|
||||||
if (parentShardIds != null) {
|
if (parentShardIds != null) {
|
||||||
this.parentShardIds.addAll(parentShardIds);
|
this.parentShardIds.addAll(parentShardIds);
|
||||||
}
|
}
|
||||||
|
if (childShardIds != null) {
|
||||||
|
this.childShardIds.addAll(childShardIds);
|
||||||
|
}
|
||||||
|
this.hashKeyRangeForLease = hashKeyRangeForLease;
|
||||||
|
this.pendingCheckpointState = pendingCheckpointState;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -135,7 +160,9 @@ public class Lease {
|
||||||
ownerSwitchesSinceCheckpoint(lease.ownerSwitchesSinceCheckpoint());
|
ownerSwitchesSinceCheckpoint(lease.ownerSwitchesSinceCheckpoint());
|
||||||
checkpoint(lease.checkpoint);
|
checkpoint(lease.checkpoint);
|
||||||
pendingCheckpoint(lease.pendingCheckpoint);
|
pendingCheckpoint(lease.pendingCheckpoint);
|
||||||
|
pendingCheckpointState(lease.pendingCheckpointState);
|
||||||
parentShardIds(lease.parentShardIds);
|
parentShardIds(lease.parentShardIds);
|
||||||
|
childShardIds(lease.childShardIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -214,6 +241,15 @@ public class Lease {
|
||||||
this.pendingCheckpoint = pendingCheckpoint;
|
this.pendingCheckpoint = pendingCheckpoint;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets pending checkpoint state.
|
||||||
|
*
|
||||||
|
* @param pendingCheckpointState can be null
|
||||||
|
*/
|
||||||
|
public void pendingCheckpointState(byte[] pendingCheckpointState) {
|
||||||
|
this.pendingCheckpointState = pendingCheckpointState;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets ownerSwitchesSinceCheckpoint.
|
* Sets ownerSwitchesSinceCheckpoint.
|
||||||
*
|
*
|
||||||
|
|
@ -233,6 +269,27 @@ public class Lease {
|
||||||
this.parentShardIds.addAll(parentShardIds);
|
this.parentShardIds.addAll(parentShardIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets childShardIds.
|
||||||
|
*
|
||||||
|
* @param childShardIds may not be null
|
||||||
|
*/
|
||||||
|
public void childShardIds(@NonNull final Collection<String> childShardIds) {
|
||||||
|
this.childShardIds.addAll(childShardIds);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the hash range key for this shard.
|
||||||
|
* @param hashKeyRangeForLease
|
||||||
|
*/
|
||||||
|
public void hashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) {
|
||||||
|
if (this.hashKeyRangeForLease == null) {
|
||||||
|
this.hashKeyRangeForLease = hashKeyRangeForLease;
|
||||||
|
} else if (!this.hashKeyRangeForLease.equals(hashKeyRangeForLease)) {
|
||||||
|
throw new IllegalArgumentException("hashKeyRange is immutable");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets leaseOwner.
|
* Sets leaseOwner.
|
||||||
*
|
*
|
||||||
|
|
@ -250,4 +307,6 @@ public class Lease {
|
||||||
public Lease copy() {
|
public Lease copy() {
|
||||||
return new Lease(this);
|
return new Lease(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,348 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.base.Stopwatch;
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.RequiredArgsConstructor;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class to cleanup of any expired/closed shard leases. It will cleanup leases periodically as defined by
|
||||||
|
* {@link LeaseManagementConfig#leaseCleanupConfig()} asynchronously.
|
||||||
|
*/
|
||||||
|
@Accessors(fluent=true)
|
||||||
|
@Slf4j
|
||||||
|
@RequiredArgsConstructor
|
||||||
|
@EqualsAndHashCode
|
||||||
|
public class LeaseCleanupManager {
|
||||||
|
@NonNull
|
||||||
|
private final LeaseCoordinator leaseCoordinator;
|
||||||
|
@NonNull
|
||||||
|
private final MetricsFactory metricsFactory;
|
||||||
|
@NonNull
|
||||||
|
private final ScheduledExecutorService deletionThreadPool;
|
||||||
|
private final boolean cleanupLeasesUponShardCompletion;
|
||||||
|
private final long leaseCleanupIntervalMillis;
|
||||||
|
private final long completedLeaseCleanupIntervalMillis;
|
||||||
|
private final long garbageLeaseCleanupIntervalMillis;
|
||||||
|
private final Stopwatch completedLeaseStopwatch = Stopwatch.createUnstarted();
|
||||||
|
private final Stopwatch garbageLeaseStopwatch = Stopwatch.createUnstarted();
|
||||||
|
|
||||||
|
private final Queue<LeasePendingDeletion> deletionQueue = new ConcurrentLinkedQueue<>();
|
||||||
|
|
||||||
|
private static final long INITIAL_DELAY = 0L;
|
||||||
|
|
||||||
|
@Getter
|
||||||
|
private volatile boolean isRunning = false;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Starts the lease cleanup thread, which is scheduled periodically as specified by
|
||||||
|
* {@link LeaseCleanupManager#leaseCleanupIntervalMillis}
|
||||||
|
*/
|
||||||
|
public void start() {
|
||||||
|
if (!isRunning) {
|
||||||
|
log.info("Starting lease cleanup thread.");
|
||||||
|
completedLeaseStopwatch.reset().start();
|
||||||
|
garbageLeaseStopwatch.reset().start();
|
||||||
|
deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
isRunning = true;
|
||||||
|
} else {
|
||||||
|
log.info("Lease cleanup thread already running, no need to start.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Enqueues a lease for deletion without check for duplicate entry. Use {@link #isEnqueuedForDeletion}
|
||||||
|
* for checking the duplicate entries.
|
||||||
|
* @param leasePendingDeletion
|
||||||
|
*/
|
||||||
|
public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) {
|
||||||
|
final Lease lease = leasePendingDeletion.lease();
|
||||||
|
if (lease == null) {
|
||||||
|
log.warn("Cannot enqueue lease {} for deferred deletion - instance doesn't hold the lease for that shard.",
|
||||||
|
lease.leaseKey());
|
||||||
|
} else {
|
||||||
|
log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey());
|
||||||
|
if (!deletionQueue.add(leasePendingDeletion)) {
|
||||||
|
log.warn("Unable to enqueue lease {} for deletion.", lease.leaseKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if lease was already enqueued for deletion.
|
||||||
|
* //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597.
|
||||||
|
* @param leasePendingDeletion
|
||||||
|
* @return true if enqueued for deletion; false otherwise.
|
||||||
|
*/
|
||||||
|
public boolean isEnqueuedForDeletion(LeasePendingDeletion leasePendingDeletion) {
|
||||||
|
return deletionQueue.contains(leasePendingDeletion);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns how many leases are currently waiting in the queue pending deletion.
|
||||||
|
* @return number of leases pending deletion.
|
||||||
|
*/
|
||||||
|
private int leasesPendingDeletion() {
|
||||||
|
return deletionQueue.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean timeToCheckForCompletedShard() {
|
||||||
|
return completedLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= completedLeaseCleanupIntervalMillis;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean timeToCheckForGarbageShard() {
|
||||||
|
return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis;
|
||||||
|
}
|
||||||
|
|
||||||
|
public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion,
|
||||||
|
boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException,
|
||||||
|
InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
final Lease lease = leasePendingDeletion.lease();
|
||||||
|
final ShardInfo shardInfo = leasePendingDeletion.shardInfo();
|
||||||
|
final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier();
|
||||||
|
|
||||||
|
final AWSExceptionManager exceptionManager = createExceptionManager();
|
||||||
|
|
||||||
|
boolean cleanedUpCompletedLease = false;
|
||||||
|
boolean cleanedUpGarbageLease = false;
|
||||||
|
boolean alreadyCheckedForGarbageCollection = false;
|
||||||
|
boolean wereChildShardsPresent = false;
|
||||||
|
boolean wasResourceNotFound = false;
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) {
|
||||||
|
final Lease leaseFromDDB = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey());
|
||||||
|
if(leaseFromDDB != null) {
|
||||||
|
Set<String> childShardKeys = leaseFromDDB.childShardIds();
|
||||||
|
if (CollectionUtils.isNullOrEmpty(childShardKeys)) {
|
||||||
|
try {
|
||||||
|
childShardKeys = leasePendingDeletion.getChildShardsFromService();
|
||||||
|
|
||||||
|
if (CollectionUtils.isNullOrEmpty(childShardKeys)) {
|
||||||
|
log.error(
|
||||||
|
"No child shards returned from service for shard {} for {} while cleaning up lease.",
|
||||||
|
shardInfo.shardId(), streamIdentifier.streamName());
|
||||||
|
} else {
|
||||||
|
wereChildShardsPresent = true;
|
||||||
|
updateLeaseWithChildShards(leasePendingDeletion, childShardKeys);
|
||||||
|
}
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
throw exceptionManager.apply(e.getCause());
|
||||||
|
} finally {
|
||||||
|
alreadyCheckedForGarbageCollection = true;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
wereChildShardsPresent = true;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys);
|
||||||
|
} catch (Exception e) {
|
||||||
|
// Suppressing the exception here, so that we can attempt for garbage cleanup.
|
||||||
|
log.warn("Unable to cleanup lease for shard {} in {}", shardInfo.shardId(), streamIdentifier.streamName(), e);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
log.info("Lease not present in lease table while cleaning the shard {} of {}", shardInfo.shardId(), streamIdentifier.streamName());
|
||||||
|
cleanedUpCompletedLease = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) {
|
||||||
|
try {
|
||||||
|
wereChildShardsPresent = !CollectionUtils
|
||||||
|
.isNullOrEmpty(leasePendingDeletion.getChildShardsFromService());
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
throw exceptionManager.apply(e.getCause());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (ResourceNotFoundException e) {
|
||||||
|
wasResourceNotFound = true;
|
||||||
|
cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease, e);
|
||||||
|
}
|
||||||
|
|
||||||
|
return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent,
|
||||||
|
wasResourceNotFound);
|
||||||
|
}
|
||||||
|
|
||||||
|
// A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the
|
||||||
|
// stream (known explicitly from ResourceNotFound being thrown when processing this shard),
|
||||||
|
private boolean cleanupLeaseForGarbageShard(Lease lease, Throwable e) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
log.warn("Deleting lease {} as it is not present in the stream.", lease, e);
|
||||||
|
leaseCoordinator.leaseRefresher().deleteLease(lease);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
for (String parentShard : lease.parentShardIds()) {
|
||||||
|
final Lease parentLease = leaseCoordinator.leaseRefresher().getLease(ShardInfo.getLeaseKey(shardInfo, parentShard));
|
||||||
|
|
||||||
|
if (parentLease != null) {
|
||||||
|
log.warn("Lease {} has a parent lease {} which is still present in the lease table, skipping deletion " +
|
||||||
|
"for this lease.", lease, parentLease);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// We should only be deleting the current shard's lease if
|
||||||
|
// 1. All of its children are currently being processed, i.e their checkpoint is not TRIM_HORIZON or AT_TIMESTAMP.
|
||||||
|
// 2. Its parent shard lease(s) have already been deleted.
|
||||||
|
private boolean cleanupLeaseForCompletedShard(Lease lease, ShardInfo shardInfo, Set<String> childShardKeys)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException {
|
||||||
|
final Set<String> processedChildShardLeaseKeys = new HashSet<>();
|
||||||
|
final Set<String> childShardLeaseKeys = childShardKeys.stream().map(ck -> ShardInfo.getLeaseKey(shardInfo, ck))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
for (String childShardLeaseKey : childShardLeaseKeys) {
|
||||||
|
final Lease childShardLease = Optional.ofNullable(
|
||||||
|
leaseCoordinator.leaseRefresher().getLease(childShardLeaseKey))
|
||||||
|
.orElseThrow(() -> new IllegalStateException(
|
||||||
|
"Child lease " + childShardLeaseKey + " for completed shard not found in "
|
||||||
|
+ "lease table - not cleaning up lease " + lease));
|
||||||
|
|
||||||
|
if (!childShardLease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) && !childShardLease
|
||||||
|
.checkpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) {
|
||||||
|
processedChildShardLeaseKeys.add(childShardLease.leaseKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!allParentShardLeasesDeleted(lease, shardInfo) || !Objects.equals(childShardLeaseKeys, processedChildShardLeaseKeys)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
log.info("Deleting lease {} as it has been completely processed and processing of child shard(s) has begun.",
|
||||||
|
lease);
|
||||||
|
leaseCoordinator.leaseRefresher().deleteLease(lease);
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateLeaseWithChildShards(LeasePendingDeletion leasePendingDeletion, Set<String> childShardKeys)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
final Lease updatedLease = leasePendingDeletion.lease();
|
||||||
|
updatedLease.childShardIds(childShardKeys);
|
||||||
|
|
||||||
|
leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
private AWSExceptionManager createExceptionManager() {
|
||||||
|
final AWSExceptionManager exceptionManager = new AWSExceptionManager();
|
||||||
|
exceptionManager.add(ResourceNotFoundException.class, t -> t);
|
||||||
|
|
||||||
|
return exceptionManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void cleanupLeases() {
|
||||||
|
log.info("Number of pending leases to clean before the scan : {}", leasesPendingDeletion());
|
||||||
|
if (deletionQueue.isEmpty()) {
|
||||||
|
log.debug("No leases pending deletion.");
|
||||||
|
} else if (timeToCheckForCompletedShard() | timeToCheckForGarbageShard()) {
|
||||||
|
final Queue<LeasePendingDeletion> failedDeletions = new ConcurrentLinkedQueue<>();
|
||||||
|
boolean completedLeaseCleanedUp = false;
|
||||||
|
boolean garbageLeaseCleanedUp = false;
|
||||||
|
|
||||||
|
log.debug("Attempting to clean up {} lease(s).", deletionQueue.size());
|
||||||
|
|
||||||
|
while (!deletionQueue.isEmpty()) {
|
||||||
|
final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll();
|
||||||
|
final String leaseKey = leasePendingDeletion.lease().leaseKey();
|
||||||
|
final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier();
|
||||||
|
boolean deletionSucceeded = false;
|
||||||
|
try {
|
||||||
|
final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion,
|
||||||
|
timeToCheckForCompletedShard(), timeToCheckForGarbageShard());
|
||||||
|
completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease();
|
||||||
|
garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease();
|
||||||
|
|
||||||
|
if (leaseCleanupResult.leaseCleanedUp()) {
|
||||||
|
log.info("Successfully cleaned up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult);
|
||||||
|
deletionSucceeded = true;
|
||||||
|
} else {
|
||||||
|
log.warn("Unable to clean up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult);
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " +
|
||||||
|
"scheduled execution.", leaseKey, streamIdentifier, e);
|
||||||
|
}
|
||||||
|
if (!deletionSucceeded) {
|
||||||
|
log.debug("Did not cleanup lease {} for {}. Re-enqueueing for deletion.", leaseKey, streamIdentifier);
|
||||||
|
failedDeletions.add(leasePendingDeletion);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (completedLeaseCleanedUp) {
|
||||||
|
log.debug("At least one completed lease was cleaned up - restarting interval");
|
||||||
|
completedLeaseStopwatch.reset().start();
|
||||||
|
}
|
||||||
|
if (garbageLeaseCleanedUp) {
|
||||||
|
log.debug("At least one garbage lease was cleaned up - restarting interval");
|
||||||
|
garbageLeaseStopwatch.reset().start();
|
||||||
|
}
|
||||||
|
deletionQueue.addAll(failedDeletions);
|
||||||
|
|
||||||
|
log.info("Number of pending leases to clean after the scan : {}", leasesPendingDeletion());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class LeaseCleanupThread implements Runnable {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
cleanupLeases();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Value
|
||||||
|
public static class LeaseCleanupResult {
|
||||||
|
boolean cleanedUpCompletedLease;
|
||||||
|
boolean cleanedUpGarbageLease;
|
||||||
|
boolean wereChildShardsPresent;
|
||||||
|
boolean wasResourceNotFound;
|
||||||
|
|
||||||
|
public boolean leaseCleanedUp() {
|
||||||
|
return cleanedUpCompletedLease | cleanedUpGarbageLease;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -92,6 +92,9 @@ public interface LeaseCoordinator {
|
||||||
*
|
*
|
||||||
* @param lease lease object containing updated values
|
* @param lease lease object containing updated values
|
||||||
* @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease
|
* @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease
|
||||||
|
* @param operation that performs updateLease
|
||||||
|
* @param singleStreamShardId for metrics emission in single stream mode. MultiStream mode will get the
|
||||||
|
* shardId from the lease object
|
||||||
*
|
*
|
||||||
* @return true if update succeeded, false otherwise
|
* @return true if update succeeded, false otherwise
|
||||||
*
|
*
|
||||||
|
|
@ -99,7 +102,7 @@ public interface LeaseCoordinator {
|
||||||
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
||||||
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
||||||
*/
|
*/
|
||||||
boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId)
|
boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -15,23 +15,26 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.SynchronousQueue;
|
import java.util.concurrent.SynchronousQueue;
|
||||||
import java.util.concurrent.ThreadFactory;
|
import java.util.concurrent.ThreadFactory;
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Function;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
|
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
|
import org.apache.commons.lang3.Validate;
|
||||||
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.BillingMode;
|
import software.amazon.awssdk.services.dynamodb.model.BillingMode;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStream;
|
import software.amazon.kinesis.common.InitialPositionInStream;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.LeaseCleanupConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory;
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory;
|
||||||
import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback;
|
import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
@ -46,6 +49,19 @@ public class LeaseManagementConfig {
|
||||||
|
|
||||||
public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofMinutes(1);
|
public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofMinutes(1);
|
||||||
|
|
||||||
|
public static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis();
|
||||||
|
public static final long DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(5).toMillis();
|
||||||
|
public static final long DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(30).toMillis();
|
||||||
|
public static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L;
|
||||||
|
public static final int DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY = 3;
|
||||||
|
|
||||||
|
|
||||||
|
public static final LeaseCleanupConfig DEFAULT_LEASE_CLEANUP_CONFIG = LeaseCleanupConfig.builder()
|
||||||
|
.leaseCleanupIntervalMillis(DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS)
|
||||||
|
.completedLeaseCleanupIntervalMillis(DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS)
|
||||||
|
.garbageLeaseCleanupIntervalMillis(DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS)
|
||||||
|
.build();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Name of the table to use in DynamoDB
|
* Name of the table to use in DynamoDB
|
||||||
*
|
*
|
||||||
|
|
@ -71,7 +87,7 @@ public class LeaseManagementConfig {
|
||||||
* Name of the Kinesis Data Stream to read records from.
|
* Name of the Kinesis Data Stream to read records from.
|
||||||
*/
|
*/
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String streamName;
|
private String streamName;
|
||||||
/**
|
/**
|
||||||
* Used to distinguish different workers/processes of a KCL application.
|
* Used to distinguish different workers/processes of a KCL application.
|
||||||
*
|
*
|
||||||
|
|
@ -106,6 +122,15 @@ public class LeaseManagementConfig {
|
||||||
*/
|
*/
|
||||||
private boolean cleanupLeasesUponShardCompletion = true;
|
private boolean cleanupLeasesUponShardCompletion = true;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configuration for lease cleanup in {@link LeaseCleanupManager}.
|
||||||
|
*
|
||||||
|
* <p>Default lease cleanup interval value: 1 minute.</p>
|
||||||
|
* <p>Default completed lease cleanup threshold: 5 minute.</p>
|
||||||
|
* <p>Default garbage lease cleanup threshold: 30 minute.</p>
|
||||||
|
*/
|
||||||
|
private final LeaseCleanupConfig leaseCleanupConfig = DEFAULT_LEASE_CLEANUP_CONFIG;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The max number of leases (shards) this worker should process.
|
* The max number of leases (shards) this worker should process.
|
||||||
* This can be useful to avoid overloading (and thrashing) a worker when a host has resource constraints
|
* This can be useful to avoid overloading (and thrashing) a worker when a host has resource constraints
|
||||||
|
|
@ -116,7 +141,7 @@ public class LeaseManagementConfig {
|
||||||
*
|
*
|
||||||
* <p>Default value: {@link Integer#MAX_VALUE}</p>
|
* <p>Default value: {@link Integer#MAX_VALUE}</p>
|
||||||
*/
|
*/
|
||||||
private int maxLeasesForWorker = Integer.MAX_VALUE;;
|
private int maxLeasesForWorker = Integer.MAX_VALUE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Max leases to steal from another worker at one time (for load balancing).
|
* Max leases to steal from another worker at one time (for load balancing).
|
||||||
|
|
@ -141,6 +166,11 @@ public class LeaseManagementConfig {
|
||||||
*/
|
*/
|
||||||
private int initialLeaseTableWriteCapacity = 10;
|
private int initialLeaseTableWriteCapacity = 10;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configurable functional interface to override the existing shardDetector.
|
||||||
|
*/
|
||||||
|
private Function<StreamConfig, ShardDetector> customShardDetectorProvider;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The size of the thread pool to create for the lease renewer to use.
|
* The size of the thread pool to create for the lease renewer to use.
|
||||||
*
|
*
|
||||||
|
|
@ -168,6 +198,20 @@ public class LeaseManagementConfig {
|
||||||
|
|
||||||
private BillingMode billingMode = BillingMode.PROVISIONED;
|
private BillingMode billingMode = BillingMode.PROVISIONED;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Frequency (in millis) of the auditor job to scan for partial leases in the lease table.
|
||||||
|
* If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on
|
||||||
|
* {@link #leasesRecoveryAuditorInconsistencyConfidenceThreshold}
|
||||||
|
*/
|
||||||
|
private long leasesRecoveryAuditorExecutionFrequencyMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table
|
||||||
|
* is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times,
|
||||||
|
* then it would trigger a shard sync.
|
||||||
|
*/
|
||||||
|
private int leasesRecoveryAuditorInconsistencyConfidenceThreshold = DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The initial position for getting records from Kinesis streams.
|
* The initial position for getting records from Kinesis streams.
|
||||||
*
|
*
|
||||||
|
|
@ -182,6 +226,24 @@ public class LeaseManagementConfig {
|
||||||
|
|
||||||
private MetricsFactory metricsFactory = new NullMetricsFactory();
|
private MetricsFactory metricsFactory = new NullMetricsFactory();
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient,
|
||||||
|
String streamName, String workerIdentifier) {
|
||||||
|
this.tableName = tableName;
|
||||||
|
this.dynamoDBClient = dynamoDBClient;
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.streamName = streamName;
|
||||||
|
this.workerIdentifier = workerIdentifier;
|
||||||
|
}
|
||||||
|
|
||||||
|
public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient,
|
||||||
|
String workerIdentifier) {
|
||||||
|
this.tableName = tableName;
|
||||||
|
this.dynamoDBClient = dynamoDBClient;
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.workerIdentifier = workerIdentifier;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the metrics factory.
|
* Returns the metrics factory.
|
||||||
*
|
*
|
||||||
|
|
@ -240,12 +302,21 @@ public class LeaseManagementConfig {
|
||||||
*/
|
*/
|
||||||
private TableCreatorCallback tableCreatorCallback = TableCreatorCallback.NOOP_TABLE_CREATOR_CALLBACK;
|
private TableCreatorCallback tableCreatorCallback = TableCreatorCallback.NOOP_TABLE_CREATOR_CALLBACK;
|
||||||
|
|
||||||
private HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer();
|
private HierarchicalShardSyncer hierarchicalShardSyncer;
|
||||||
|
|
||||||
private LeaseManagementFactory leaseManagementFactory;
|
private LeaseManagementFactory leaseManagementFactory;
|
||||||
|
|
||||||
|
private HierarchicalShardSyncer hierarchicalShardSyncer() {
|
||||||
|
if(hierarchicalShardSyncer == null) {
|
||||||
|
hierarchicalShardSyncer = new HierarchicalShardSyncer();
|
||||||
|
}
|
||||||
|
return hierarchicalShardSyncer;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
public LeaseManagementFactory leaseManagementFactory() {
|
public LeaseManagementFactory leaseManagementFactory() {
|
||||||
if (leaseManagementFactory == null) {
|
if (leaseManagementFactory == null) {
|
||||||
|
Validate.notEmpty(streamName(), "Stream name is empty");
|
||||||
leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(),
|
leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(),
|
||||||
streamName(),
|
streamName(),
|
||||||
dynamoDBClient(),
|
dynamoDBClient(),
|
||||||
|
|
@ -275,4 +346,54 @@ public class LeaseManagementConfig {
|
||||||
return leaseManagementFactory;
|
return leaseManagementFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Vends LeaseManagementFactory that performs serde based on leaseSerializer and shard sync based on isMultiStreamingMode
|
||||||
|
* @param leaseSerializer
|
||||||
|
* @param isMultiStreamingMode
|
||||||
|
* @return LeaseManagementFactory
|
||||||
|
*/
|
||||||
|
public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) {
|
||||||
|
if(leaseManagementFactory == null) {
|
||||||
|
leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(),
|
||||||
|
dynamoDBClient(),
|
||||||
|
tableName(),
|
||||||
|
workerIdentifier(),
|
||||||
|
executorService(),
|
||||||
|
failoverTimeMillis(),
|
||||||
|
epsilonMillis(),
|
||||||
|
maxLeasesForWorker(),
|
||||||
|
maxLeasesToStealAtOneTime(),
|
||||||
|
maxLeaseRenewalThreads(),
|
||||||
|
cleanupLeasesUponShardCompletion(),
|
||||||
|
ignoreUnexpectedChildShards(),
|
||||||
|
shardSyncIntervalMillis(),
|
||||||
|
consistentReads(),
|
||||||
|
listShardsBackoffTimeInMillis(),
|
||||||
|
maxListShardsRetryAttempts(),
|
||||||
|
maxCacheMissesBeforeReload(),
|
||||||
|
listShardsCacheAllowedAgeInSeconds(),
|
||||||
|
cacheMissWarningModulus(),
|
||||||
|
initialLeaseTableReadCapacity(),
|
||||||
|
initialLeaseTableWriteCapacity(),
|
||||||
|
hierarchicalShardSyncer(),
|
||||||
|
tableCreatorCallback(),
|
||||||
|
dynamoDbRequestTimeout(),
|
||||||
|
billingMode(),
|
||||||
|
leaseSerializer,
|
||||||
|
customShardDetectorProvider(),
|
||||||
|
isMultiStreamingMode,
|
||||||
|
leaseCleanupConfig());
|
||||||
|
}
|
||||||
|
return leaseManagementFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set leaseManagementFactory and return the current LeaseManagementConfig instance.
|
||||||
|
* @param leaseManagementFactory
|
||||||
|
* @return LeaseManagementConfig
|
||||||
|
*/
|
||||||
|
public LeaseManagementConfig leaseManagementFactory(final LeaseManagementFactory leaseManagementFactory) {
|
||||||
|
this.leaseManagementFactory = leaseManagementFactory;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher;
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
||||||
|
|
@ -26,7 +27,18 @@ public interface LeaseManagementFactory {
|
||||||
|
|
||||||
ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory);
|
ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory);
|
||||||
|
|
||||||
|
default ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
DynamoDBLeaseRefresher createLeaseRefresher();
|
DynamoDBLeaseRefresher createLeaseRefresher();
|
||||||
|
|
||||||
ShardDetector createShardDetector();
|
ShardDetector createShardDetector();
|
||||||
|
|
||||||
|
default ShardDetector createShardDetector(StreamConfig streamConfig) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
|
@ -60,6 +61,18 @@ public interface LeaseRefresher {
|
||||||
*/
|
*/
|
||||||
boolean waitUntilLeaseTableExists(long secondsBetweenPolls, long timeoutSeconds) throws DependencyException;
|
boolean waitUntilLeaseTableExists(long secondsBetweenPolls, long timeoutSeconds) throws DependencyException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* List all leases for a given stream synchronously.
|
||||||
|
*
|
||||||
|
* @throws DependencyException if DynamoDB scan fails in an unexpected way
|
||||||
|
* @throws InvalidStateException if lease table does not exist
|
||||||
|
* @throws ProvisionedThroughputException if DynamoDB scan fails due to lack of capacity
|
||||||
|
*
|
||||||
|
* @return list of leases
|
||||||
|
*/
|
||||||
|
List<Lease> listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException,
|
||||||
|
ProvisionedThroughputException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List all objects in table synchronously.
|
* List all objects in table synchronously.
|
||||||
*
|
*
|
||||||
|
|
@ -86,15 +99,15 @@ public interface LeaseRefresher {
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param shardId Get the lease for this shardId
|
* @param leaseKey Get the lease for this leasekey
|
||||||
*
|
*
|
||||||
* @throws InvalidStateException if lease table does not exist
|
* @throws InvalidStateException if lease table does not exist
|
||||||
* @throws ProvisionedThroughputException if DynamoDB get fails due to lack of capacity
|
* @throws ProvisionedThroughputException if DynamoDB get fails due to lack of capacity
|
||||||
* @throws DependencyException if DynamoDB get fails in an unexpected way
|
* @throws DependencyException if DynamoDB get fails in an unexpected way
|
||||||
*
|
*
|
||||||
* @return lease for the specified shardId, or null if one doesn't exist
|
* @return lease for the specified leaseKey, or null if one doesn't exist
|
||||||
*/
|
*/
|
||||||
Lease getLease(String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Renew a lease by incrementing the lease counter. Conditional on the leaseCounter in DynamoDB matching the leaseCounter
|
* Renew a lease by incrementing the lease counter. Conditional on the leaseCounter in DynamoDB matching the leaseCounter
|
||||||
|
|
@ -178,6 +191,21 @@ public interface LeaseRefresher {
|
||||||
boolean updateLease(Lease lease)
|
boolean updateLease(Lease lease)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update application-specific fields of the given lease in DynamoDB. Does not update fields managed by the leasing
|
||||||
|
* library such as leaseCounter, leaseOwner, or leaseKey.
|
||||||
|
*
|
||||||
|
* @return true if update succeeded, false otherwise
|
||||||
|
*
|
||||||
|
* @throws InvalidStateException if lease table does not exist
|
||||||
|
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
||||||
|
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
||||||
|
*/
|
||||||
|
default void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
throw new UnsupportedOperationException("updateLeaseWithNoExpectation is not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check (synchronously) if there are any leases in the lease table.
|
* Check (synchronously) if there are any leases in the lease table.
|
||||||
*
|
*
|
||||||
|
|
@ -193,13 +221,13 @@ public interface LeaseRefresher {
|
||||||
* Gets the current checkpoint of the shard. This is useful in the resharding use case
|
* Gets the current checkpoint of the shard. This is useful in the resharding use case
|
||||||
* where we will wait for the parent shard to complete before starting on the records from a child shard.
|
* where we will wait for the parent shard to complete before starting on the records from a child shard.
|
||||||
*
|
*
|
||||||
* @param shardId Checkpoint of this shard will be returned
|
* @param leaseKey Checkpoint of this shard will be returned
|
||||||
* @return Checkpoint of this shard, or null if the shard record doesn't exist.
|
* @return Checkpoint of this shard, or null if the shard record doesn't exist.
|
||||||
*
|
*
|
||||||
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
||||||
* @throws InvalidStateException if lease table does not exist
|
* @throws InvalidStateException if lease table does not exist
|
||||||
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
||||||
*/
|
*/
|
||||||
ExtendedSequenceNumber getCheckpoint(String shardId)
|
ExtendedSequenceNumber getCheckpoint(String leaseKey)
|
||||||
throws ProvisionedThroughputException, InvalidStateException, DependencyException;
|
throws ProvisionedThroughputException, InvalidStateException, DependencyException;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -86,6 +86,9 @@ public interface LeaseRenewer {
|
||||||
*
|
*
|
||||||
* @param lease lease object containing updated data
|
* @param lease lease object containing updated data
|
||||||
* @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease
|
* @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease
|
||||||
|
* @param operation that performs updateLease
|
||||||
|
* @param singleStreamShardId shardId for metrics emission in single stream mode. MultiStream mode will get the
|
||||||
|
* shardId from the lease object
|
||||||
*
|
*
|
||||||
* @return true if update succeeds, false otherwise
|
* @return true if update succeeds, false otherwise
|
||||||
*
|
*
|
||||||
|
|
@ -93,7 +96,7 @@ public interface LeaseRenewer {
|
||||||
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
||||||
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
||||||
*/
|
*/
|
||||||
boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId)
|
boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -16,14 +16,11 @@ package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
|
||||||
import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
|
import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue;
|
import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
|
import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utility class that manages the mapping of Lease objects/operations to records in DynamoDB.
|
* Utility class that manages the mapping of Lease objects/operations to records in DynamoDB.
|
||||||
|
|
@ -46,6 +43,11 @@ public interface LeaseSerializer {
|
||||||
*/
|
*/
|
||||||
Lease fromDynamoRecord(Map<String, AttributeValue> dynamoRecord);
|
Lease fromDynamoRecord(Map<String, AttributeValue> dynamoRecord);
|
||||||
|
|
||||||
|
|
||||||
|
default Lease fromDynamoRecord(Map<String, AttributeValue> dynamoRecord, Lease leaseToUpdate) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param lease
|
* @param lease
|
||||||
* @return the attribute value map representing a Lease's hash key given a Lease object.
|
* @return the attribute value map representing a Lease's hash key given a Lease object.
|
||||||
|
|
@ -77,6 +79,14 @@ public interface LeaseSerializer {
|
||||||
*/
|
*/
|
||||||
Map<String, ExpectedAttributeValue> getDynamoNonexistantExpectation();
|
Map<String, ExpectedAttributeValue> getDynamoNonexistantExpectation();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param leaseKey
|
||||||
|
* @return the attribute value map asserting that a lease does exist.
|
||||||
|
*/
|
||||||
|
default Map<String, ExpectedAttributeValue> getDynamoExistentExpectation(String leaseKey) {
|
||||||
|
throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param lease
|
* @param lease
|
||||||
* @return the attribute value map that increments a lease counter
|
* @return the attribute value map that increments a lease counter
|
||||||
|
|
@ -103,6 +113,15 @@ public interface LeaseSerializer {
|
||||||
*/
|
*/
|
||||||
Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease);
|
Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param lease
|
||||||
|
* @param updateField
|
||||||
|
* @return the attribute value map that updates application-specific data for a lease
|
||||||
|
*/
|
||||||
|
default Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the key schema for creating a DynamoDB table to store leases
|
* @return the key schema for creating a DynamoDB table to store leases
|
||||||
*/
|
*/
|
||||||
|
|
@ -112,4 +131,5 @@ public interface LeaseSerializer {
|
||||||
* @return attribute definitions for creating a DynamoDB table to store leases
|
* @return attribute definitions for creating a DynamoDB table to store leases
|
||||||
*/
|
*/
|
||||||
Collection<AttributeDefinition> getAttributeDefinitions();
|
Collection<AttributeDefinition> getAttributeDefinitions();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,80 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
|
import lombok.NoArgsConstructor;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.Setter;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import org.apache.commons.lang3.Validate;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static com.google.common.base.Verify.verifyNotNull;
|
||||||
|
|
||||||
|
@Setter
|
||||||
|
@NoArgsConstructor
|
||||||
|
@Getter
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
@EqualsAndHashCode(callSuper = true)
|
||||||
|
public class MultiStreamLease extends Lease {
|
||||||
|
|
||||||
|
@NonNull private String streamIdentifier;
|
||||||
|
@NonNull private String shardId;
|
||||||
|
|
||||||
|
public MultiStreamLease(MultiStreamLease other) {
|
||||||
|
super(other);
|
||||||
|
streamIdentifier(other.streamIdentifier);
|
||||||
|
shardId(other.shardId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void update(Lease other) {
|
||||||
|
MultiStreamLease casted = validateAndCast(other);
|
||||||
|
super.update(casted);
|
||||||
|
streamIdentifier(casted.streamIdentifier);
|
||||||
|
shardId(casted.shardId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static String getLeaseKey(String streamIdentifier, String shardId) {
|
||||||
|
verifyNotNull(streamIdentifier, "streamIdentifier should not be null");
|
||||||
|
verifyNotNull(shardId, "shardId should not be null");
|
||||||
|
return streamIdentifier + ":" + shardId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a deep copy of this object. Type-unsafe - there aren't good mechanisms for copy-constructing generics.
|
||||||
|
*
|
||||||
|
* @return A deep copy of this object.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public MultiStreamLease copy() {
|
||||||
|
return new MultiStreamLease(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Validate and cast the lease to MultiStream lease
|
||||||
|
* @param lease
|
||||||
|
* @return MultiStreamLease
|
||||||
|
*/
|
||||||
|
public static MultiStreamLease validateAndCast(Lease lease) {
|
||||||
|
Validate.isInstanceOf(MultiStreamLease.class, lease);
|
||||||
|
return (MultiStreamLease) lease;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -15,16 +15,75 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
import software.amazon.awssdk.services.kinesis.model.Shard;
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ShardFilter;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public interface ShardDetector {
|
public interface ShardDetector {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets shard based on shardId.
|
||||||
|
*
|
||||||
|
* @param shardId
|
||||||
|
* @return Shard
|
||||||
|
*/
|
||||||
Shard shard(String shardId);
|
Shard shard(String shardId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* List shards.
|
||||||
|
*
|
||||||
|
* @return Shards
|
||||||
|
*/
|
||||||
List<Shard> listShards();
|
List<Shard> listShards();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* List shards with shard filter.
|
||||||
|
*
|
||||||
|
* @param ShardFilter
|
||||||
|
* @return Shards
|
||||||
|
*/
|
||||||
|
default List<Shard> listShardsWithFilter(ShardFilter shardFilter) {
|
||||||
|
throw new UnsupportedOperationException("listShardsWithFilter not available.");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets stream identifier.
|
||||||
|
*
|
||||||
|
* @return StreamIdentifier
|
||||||
|
*/
|
||||||
|
default StreamIdentifier streamIdentifier() {
|
||||||
|
throw new UnsupportedOperationException("StreamName not available");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets a list shards response based on the request.
|
||||||
|
*
|
||||||
|
* @param request list shards request
|
||||||
|
* @return ListShardsResponse which contains list shards response
|
||||||
|
*/
|
||||||
|
default ListShardsResponse getListShardsResponse(ListShardsRequest request) throws Exception {
|
||||||
|
throw new UnsupportedOperationException("getListShardsResponse not available.");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the children shards of a shard.
|
||||||
|
* @param shardId
|
||||||
|
* @return
|
||||||
|
* @throws InterruptedException
|
||||||
|
* @throws ExecutionException
|
||||||
|
* @throws TimeoutException
|
||||||
|
*/
|
||||||
|
default List<ChildShard> getChildShards(String shardId) throws InterruptedException, ExecutionException, TimeoutException {
|
||||||
|
throw new UnsupportedOperationException("getChildShards not available.");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,6 +18,7 @@ import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
|
|
@ -36,6 +37,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
@ToString
|
@ToString
|
||||||
public class ShardInfo {
|
public class ShardInfo {
|
||||||
|
|
||||||
|
private final Optional<String> streamIdentifierSerOpt;
|
||||||
private final String shardId;
|
private final String shardId;
|
||||||
private final String concurrencyToken;
|
private final String concurrencyToken;
|
||||||
// Sorted list of parent shardIds.
|
// Sorted list of parent shardIds.
|
||||||
|
|
@ -54,11 +56,27 @@ public class ShardInfo {
|
||||||
* @param checkpoint
|
* @param checkpoint
|
||||||
* the latest checkpoint from lease
|
* the latest checkpoint from lease
|
||||||
*/
|
*/
|
||||||
// TODO: check what values can be null
|
|
||||||
public ShardInfo(@NonNull final String shardId,
|
public ShardInfo(@NonNull final String shardId,
|
||||||
final String concurrencyToken,
|
final String concurrencyToken,
|
||||||
final Collection<String> parentShardIds,
|
final Collection<String> parentShardIds,
|
||||||
final ExtendedSequenceNumber checkpoint) {
|
final ExtendedSequenceNumber checkpoint) {
|
||||||
|
this(shardId, concurrencyToken, parentShardIds, checkpoint, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new ShardInfo object that has an option to pass a serialized streamIdentifier.
|
||||||
|
* The checkpoint is not part of the equality, but is used for debugging output.
|
||||||
|
* @param shardId
|
||||||
|
* @param concurrencyToken
|
||||||
|
* @param parentShardIds
|
||||||
|
* @param checkpoint
|
||||||
|
* @param streamIdentifierSer
|
||||||
|
*/
|
||||||
|
public ShardInfo(@NonNull final String shardId,
|
||||||
|
final String concurrencyToken,
|
||||||
|
final Collection<String> parentShardIds,
|
||||||
|
final ExtendedSequenceNumber checkpoint,
|
||||||
|
final String streamIdentifierSer) {
|
||||||
this.shardId = shardId;
|
this.shardId = shardId;
|
||||||
this.concurrencyToken = concurrencyToken;
|
this.concurrencyToken = concurrencyToken;
|
||||||
this.parentShardIds = new LinkedList<>();
|
this.parentShardIds = new LinkedList<>();
|
||||||
|
|
@ -69,6 +87,7 @@ public class ShardInfo {
|
||||||
// This makes it easy to check for equality in ShardInfo.equals method.
|
// This makes it easy to check for equality in ShardInfo.equals method.
|
||||||
Collections.sort(this.parentShardIds);
|
Collections.sort(this.parentShardIds);
|
||||||
this.checkpoint = checkpoint;
|
this.checkpoint = checkpoint;
|
||||||
|
this.streamIdentifierSerOpt = Optional.ofNullable(streamIdentifierSer);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -94,7 +113,8 @@ public class ShardInfo {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return new HashCodeBuilder().append(concurrencyToken).append(parentShardIds).append(shardId).toHashCode();
|
return new HashCodeBuilder()
|
||||||
|
.append(concurrencyToken).append(parentShardIds).append(shardId).append(streamIdentifierSerOpt.orElse("")).toHashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -118,8 +138,30 @@ public class ShardInfo {
|
||||||
}
|
}
|
||||||
ShardInfo other = (ShardInfo) obj;
|
ShardInfo other = (ShardInfo) obj;
|
||||||
return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken)
|
return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken)
|
||||||
.append(parentShardIds, other.parentShardIds).append(shardId, other.shardId).isEquals();
|
.append(parentShardIds, other.parentShardIds).append(shardId, other.shardId)
|
||||||
|
.append(streamIdentifierSerOpt.orElse(""), other.streamIdentifierSerOpt.orElse("")).isEquals();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to derive lease key from ShardInfo.
|
||||||
|
* @param shardInfo
|
||||||
|
* @return lease key
|
||||||
|
*/
|
||||||
|
public static String getLeaseKey(ShardInfo shardInfo) {
|
||||||
|
return getLeaseKey(shardInfo, shardInfo.shardId());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to derive lease key from ShardInfo and shardId to override.
|
||||||
|
* @param shardInfo
|
||||||
|
* @param shardIdOverride
|
||||||
|
* @return lease key
|
||||||
|
*/
|
||||||
|
public static String getLeaseKey(ShardInfo shardInfo, String shardIdOverride) {
|
||||||
|
return shardInfo.streamIdentifierSerOpt().isPresent() ?
|
||||||
|
MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardIdOverride) :
|
||||||
|
shardIdOverride;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import software.amazon.kinesis.lifecycle.ConsumerTask;
|
||||||
import software.amazon.kinesis.lifecycle.TaskResult;
|
import software.amazon.kinesis.lifecycle.TaskResult;
|
||||||
import software.amazon.kinesis.lifecycle.TaskType;
|
import software.amazon.kinesis.lifecycle.TaskType;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
import software.amazon.kinesis.metrics.MetricsScope;
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
import software.amazon.kinesis.metrics.MetricsUtil;
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
|
|
||||||
|
|
@ -45,6 +46,7 @@ public class ShardSyncTask implements ConsumerTask {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final InitialPositionInStreamExtended initialPosition;
|
private final InitialPositionInStreamExtended initialPosition;
|
||||||
private final boolean cleanupLeasesUponShardCompletion;
|
private final boolean cleanupLeasesUponShardCompletion;
|
||||||
|
private final boolean garbageCollectLeases;
|
||||||
private final boolean ignoreUnexpectedChildShards;
|
private final boolean ignoreUnexpectedChildShards;
|
||||||
private final long shardSyncTaskIdleTimeMillis;
|
private final long shardSyncTaskIdleTimeMillis;
|
||||||
@NonNull
|
@NonNull
|
||||||
|
|
@ -62,17 +64,25 @@ public class ShardSyncTask implements ConsumerTask {
|
||||||
public TaskResult call() {
|
public TaskResult call() {
|
||||||
Exception exception = null;
|
Exception exception = null;
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHARD_SYNC_TASK_OPERATION);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHARD_SYNC_TASK_OPERATION);
|
||||||
|
boolean shardSyncSuccess = true;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition,
|
boolean didPerformShardSync = hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher,
|
||||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, scope);
|
initialPosition, scope, ignoreUnexpectedChildShards,
|
||||||
if (shardSyncTaskIdleTimeMillis > 0) {
|
leaseRefresher.isLeaseTableEmpty());
|
||||||
|
|
||||||
|
if (didPerformShardSync && shardSyncTaskIdleTimeMillis > 0) {
|
||||||
Thread.sleep(shardSyncTaskIdleTimeMillis);
|
Thread.sleep(shardSyncTaskIdleTimeMillis);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.error("Caught exception while sync'ing Kinesis shards and leases", e);
|
log.error("Caught exception while sync'ing Kinesis shards and leases", e);
|
||||||
exception = e;
|
exception = e;
|
||||||
|
shardSyncSuccess = false;
|
||||||
} finally {
|
} finally {
|
||||||
|
// NOTE: This metric is reflecting if a shard sync task succeeds. Customer can use this metric to monitor if
|
||||||
|
// their application encounter any shard sync failures. This metric can help to detect potential shard stuck issues
|
||||||
|
// that are due to shard sync failures.
|
||||||
|
MetricsUtil.addSuccess(scope, "SyncShards", shardSyncSuccess, MetricsLevel.DETAILED);
|
||||||
MetricsUtil.endScope(scope);
|
MetricsUtil.endScope(scope);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,9 +14,11 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
|
||||||
|
|
@ -24,6 +26,7 @@ import lombok.Data;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.kinesis.coordinator.ExecutorStateEvent;
|
||||||
import software.amazon.kinesis.lifecycle.ConsumerTask;
|
import software.amazon.kinesis.lifecycle.ConsumerTask;
|
||||||
import software.amazon.kinesis.lifecycle.TaskResult;
|
import software.amazon.kinesis.lifecycle.TaskResult;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
@ -45,6 +48,7 @@ public class ShardSyncTaskManager {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final InitialPositionInStreamExtended initialPositionInStream;
|
private final InitialPositionInStreamExtended initialPositionInStream;
|
||||||
private final boolean cleanupLeasesUponShardCompletion;
|
private final boolean cleanupLeasesUponShardCompletion;
|
||||||
|
private final boolean garbageCollectLeases;
|
||||||
private final boolean ignoreUnexpectedChildShards;
|
private final boolean ignoreUnexpectedChildShards;
|
||||||
private final long shardSyncIdleTimeMillis;
|
private final long shardSyncIdleTimeMillis;
|
||||||
@NonNull
|
@NonNull
|
||||||
|
|
@ -53,6 +57,10 @@ public class ShardSyncTaskManager {
|
||||||
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final MetricsFactory metricsFactory;
|
private final MetricsFactory metricsFactory;
|
||||||
|
private ConsumerTask currentTask;
|
||||||
|
private CompletableFuture<TaskResult> future;
|
||||||
|
private AtomicBoolean shardSyncRequestPending;
|
||||||
|
private final ReentrantLock lock;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor.
|
* Constructor.
|
||||||
|
|
@ -77,11 +85,14 @@ public class ShardSyncTaskManager {
|
||||||
this.leaseRefresher = leaseRefresher;
|
this.leaseRefresher = leaseRefresher;
|
||||||
this.initialPositionInStream = initialPositionInStream;
|
this.initialPositionInStream = initialPositionInStream;
|
||||||
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
||||||
|
this.garbageCollectLeases = true;
|
||||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||||
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
|
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
|
||||||
this.executorService = executorService;
|
this.executorService = executorService;
|
||||||
this.hierarchicalShardSyncer = new HierarchicalShardSyncer();
|
this.hierarchicalShardSyncer = new HierarchicalShardSyncer();
|
||||||
this.metricsFactory = metricsFactory;
|
this.metricsFactory = metricsFactory;
|
||||||
|
this.shardSyncRequestPending = new AtomicBoolean(false);
|
||||||
|
this.lock = new ReentrantLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -105,21 +116,48 @@ public class ShardSyncTaskManager {
|
||||||
this.leaseRefresher = leaseRefresher;
|
this.leaseRefresher = leaseRefresher;
|
||||||
this.initialPositionInStream = initialPositionInStream;
|
this.initialPositionInStream = initialPositionInStream;
|
||||||
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
||||||
|
this.garbageCollectLeases = true;
|
||||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||||
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
|
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
|
||||||
this.executorService = executorService;
|
this.executorService = executorService;
|
||||||
this.hierarchicalShardSyncer = hierarchicalShardSyncer;
|
this.hierarchicalShardSyncer = hierarchicalShardSyncer;
|
||||||
this.metricsFactory = metricsFactory;
|
this.metricsFactory = metricsFactory;
|
||||||
|
this.shardSyncRequestPending = new AtomicBoolean(false);
|
||||||
|
this.lock = new ReentrantLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
private ConsumerTask currentTask;
|
/**
|
||||||
private Future<TaskResult> future;
|
* Call a ShardSyncTask and return the Task Result.
|
||||||
|
* @return the Task Result.
|
||||||
|
*/
|
||||||
|
public TaskResult callShardSyncTask() {
|
||||||
|
final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector,
|
||||||
|
leaseRefresher,
|
||||||
|
initialPositionInStream,
|
||||||
|
cleanupLeasesUponShardCompletion,
|
||||||
|
garbageCollectLeases,
|
||||||
|
ignoreUnexpectedChildShards,
|
||||||
|
shardSyncIdleTimeMillis,
|
||||||
|
hierarchicalShardSyncer,
|
||||||
|
metricsFactory);
|
||||||
|
final ConsumerTask metricCollectingTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory);
|
||||||
|
return metricCollectingTask.call();
|
||||||
|
}
|
||||||
|
|
||||||
public synchronized boolean syncShardAndLeaseInfo() {
|
/**
|
||||||
|
* Submit a ShardSyncTask and return if the submission is successful.
|
||||||
|
* @return if the casting is successful.
|
||||||
|
*/
|
||||||
|
public boolean submitShardSyncTask() {
|
||||||
|
try {
|
||||||
|
lock.lock();
|
||||||
return checkAndSubmitNextTask();
|
return checkAndSubmitNextTask();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized boolean checkAndSubmitNextTask() {
|
private boolean checkAndSubmitNextTask() {
|
||||||
boolean submittedNewTask = false;
|
boolean submittedNewTask = false;
|
||||||
if ((future == null) || future.isCancelled() || future.isDone()) {
|
if ((future == null) || future.isCancelled() || future.isDone()) {
|
||||||
if ((future != null) && future.isDone()) {
|
if ((future != null) && future.isDone()) {
|
||||||
|
|
@ -140,23 +178,54 @@ public class ShardSyncTaskManager {
|
||||||
leaseRefresher,
|
leaseRefresher,
|
||||||
initialPositionInStream,
|
initialPositionInStream,
|
||||||
cleanupLeasesUponShardCompletion,
|
cleanupLeasesUponShardCompletion,
|
||||||
|
garbageCollectLeases,
|
||||||
ignoreUnexpectedChildShards,
|
ignoreUnexpectedChildShards,
|
||||||
shardSyncIdleTimeMillis,
|
shardSyncIdleTimeMillis,
|
||||||
hierarchicalShardSyncer,
|
hierarchicalShardSyncer,
|
||||||
metricsFactory),
|
metricsFactory),
|
||||||
metricsFactory);
|
metricsFactory);
|
||||||
future = executorService.submit(currentTask);
|
future = CompletableFuture.supplyAsync(() -> currentTask.call(), executorService)
|
||||||
|
.whenComplete((taskResult, exception) -> handlePendingShardSyncs(exception, taskResult));
|
||||||
|
|
||||||
|
log.info(new ExecutorStateEvent(executorService).message());
|
||||||
|
|
||||||
submittedNewTask = true;
|
submittedNewTask = true;
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Submitted new {} task.", currentTask.taskType());
|
log.debug("Submitted new {} task.", currentTask.taskType());
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("Previous {} task still pending. Not submitting new task.", currentTask.taskType());
|
log.debug("Previous {} task still pending. Not submitting new task. "
|
||||||
|
+ "Enqueued a request that will be executed when the current request completes.", currentTask.taskType());
|
||||||
}
|
}
|
||||||
|
shardSyncRequestPending.compareAndSet(false /*expected*/, true /*update*/);
|
||||||
}
|
}
|
||||||
|
|
||||||
return submittedNewTask;
|
return submittedNewTask;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void handlePendingShardSyncs(Throwable exception, TaskResult taskResult) {
|
||||||
|
if (exception != null || taskResult.getException() != null) {
|
||||||
|
log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException());
|
||||||
|
}
|
||||||
|
// Acquire lock here. If shardSyncRequestPending is false in this completionStage and
|
||||||
|
// submitShardSyncTask is invoked, before completion stage exits (future completes)
|
||||||
|
// but right after the value of shardSyncRequestPending is checked, it will result in
|
||||||
|
// shardSyncRequestPending being set to true, but no pending futures to trigger the next
|
||||||
|
// ShardSyncTask. By executing this stage in a Reentrant lock, we ensure that if the
|
||||||
|
// previous task is in this completion stage, checkAndSubmitNextTask is not invoked
|
||||||
|
// until this completionStage exits.
|
||||||
|
try {
|
||||||
|
lock.lock();
|
||||||
|
if (shardSyncRequestPending.get()) {
|
||||||
|
shardSyncRequestPending.set(false);
|
||||||
|
// reset future to null, so next call creates a new one
|
||||||
|
// without trying to get results from the old future.
|
||||||
|
future = null;
|
||||||
|
checkAndSubmitNextTask();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,26 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* These are the special fields that will be updated only once during the lifetime of the lease.
|
||||||
|
* Since these are meta information that will not affect lease ownership or data durability, we allow
|
||||||
|
* any elected leader or worker to set these fields directly without any conditional checks.
|
||||||
|
* Note that though HASH_KEY_RANGE will be available during lease initialization in newer versions, we keep this
|
||||||
|
* for backfilling while rolling forward to newer versions.
|
||||||
|
*/
|
||||||
|
public enum UpdateField {
|
||||||
|
CHILD_SHARDS, HASH_KEY_RANGE
|
||||||
|
}
|
||||||
|
|
@ -38,6 +38,7 @@ import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
import software.amazon.kinesis.leases.LeaseRenewer;
|
import software.amazon.kinesis.leases.LeaseRenewer;
|
||||||
import software.amazon.kinesis.leases.LeaseTaker;
|
import software.amazon.kinesis.leases.LeaseTaker;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
|
@ -348,8 +349,8 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean updateLease(final Lease lease, final UUID concurrencyToken, final String operation,
|
public boolean updateLease(final Lease lease, final UUID concurrencyToken, final String operation,
|
||||||
final String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
final String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
return leaseRenewer.updateLease(lease, concurrencyToken, operation, shardId);
|
return leaseRenewer.updateLease(lease, concurrencyToken, operation, singleStreamShardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -377,10 +378,20 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator {
|
||||||
return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList());
|
return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to convert the basic lease or multistream lease to ShardInfo
|
||||||
|
* @param lease
|
||||||
|
* @return ShardInfo
|
||||||
|
*/
|
||||||
public static ShardInfo convertLeaseToAssignment(final Lease lease) {
|
public static ShardInfo convertLeaseToAssignment(final Lease lease) {
|
||||||
|
if (lease instanceof MultiStreamLease) {
|
||||||
|
return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(),
|
||||||
|
lease.checkpoint(), ((MultiStreamLease) lease).streamIdentifier());
|
||||||
|
} else {
|
||||||
return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(),
|
return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(),
|
||||||
lease.checkpoint());
|
lease.checkpoint());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,8 @@ package software.amazon.kinesis.leases.dynamodb;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.function.Function;
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
||||||
|
|
@ -25,11 +26,16 @@ import software.amazon.awssdk.services.dynamodb.model.BillingMode;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.LeaseCleanupConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
||||||
import software.amazon.kinesis.leases.KinesisShardDetector;
|
import software.amazon.kinesis.leases.KinesisShardDetector;
|
||||||
|
import software.amazon.kinesis.leases.LeaseCleanupManager;
|
||||||
import software.amazon.kinesis.leases.LeaseCoordinator;
|
import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementFactory;
|
import software.amazon.kinesis.leases.LeaseManagementFactory;
|
||||||
|
import software.amazon.kinesis.leases.LeaseSerializer;
|
||||||
import software.amazon.kinesis.leases.ShardDetector;
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
@ -44,8 +50,6 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String streamName;
|
|
||||||
@NonNull
|
|
||||||
private final DynamoDbAsyncClient dynamoDBClient;
|
private final DynamoDbAsyncClient dynamoDBClient;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String tableName;
|
private final String tableName;
|
||||||
|
|
@ -54,9 +58,13 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final ExecutorService executorService;
|
private final ExecutorService executorService;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final InitialPositionInStreamExtended initialPositionInStream;
|
private final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
private final LeaseSerializer leaseSerializer;
|
||||||
|
@NonNull
|
||||||
|
private StreamConfig streamConfig;
|
||||||
|
|
||||||
|
private Function<StreamConfig, ShardDetector> customShardDetectorProvider;
|
||||||
|
|
||||||
private final long failoverTimeMillis;
|
private final long failoverTimeMillis;
|
||||||
private final long epsilonMillis;
|
private final long epsilonMillis;
|
||||||
|
|
@ -77,6 +85,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
private final TableCreatorCallback tableCreatorCallback;
|
private final TableCreatorCallback tableCreatorCallback;
|
||||||
private final Duration dynamoDbRequestTimeout;
|
private final Duration dynamoDbRequestTimeout;
|
||||||
private final BillingMode billingMode;
|
private final BillingMode billingMode;
|
||||||
|
private final boolean isMultiStreamMode;
|
||||||
|
private final LeaseCleanupConfig leaseCleanupConfig;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor.
|
* Constructor.
|
||||||
|
|
@ -309,6 +319,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
* @param dynamoDbRequestTimeout
|
* @param dynamoDbRequestTimeout
|
||||||
* @param billingMode
|
* @param billingMode
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName,
|
public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName,
|
||||||
final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier,
|
final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier,
|
||||||
final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream,
|
final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream,
|
||||||
|
|
@ -321,13 +332,118 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity,
|
final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity,
|
||||||
final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback,
|
final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback,
|
||||||
Duration dynamoDbRequestTimeout, BillingMode billingMode) {
|
Duration dynamoDbRequestTimeout, BillingMode billingMode) {
|
||||||
|
|
||||||
|
this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName,
|
||||||
|
workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker,
|
||||||
|
maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion,
|
||||||
|
ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis,
|
||||||
|
maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds,
|
||||||
|
cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity,
|
||||||
|
hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor.
|
||||||
|
*
|
||||||
|
* @param kinesisClient
|
||||||
|
* @param streamConfig
|
||||||
|
* @param dynamoDBClient
|
||||||
|
* @param tableName
|
||||||
|
* @param workerIdentifier
|
||||||
|
* @param executorService
|
||||||
|
* @param failoverTimeMillis
|
||||||
|
* @param epsilonMillis
|
||||||
|
* @param maxLeasesForWorker
|
||||||
|
* @param maxLeasesToStealAtOneTime
|
||||||
|
* @param maxLeaseRenewalThreads
|
||||||
|
* @param cleanupLeasesUponShardCompletion
|
||||||
|
* @param ignoreUnexpectedChildShards
|
||||||
|
* @param shardSyncIntervalMillis
|
||||||
|
* @param consistentReads
|
||||||
|
* @param listShardsBackoffTimeMillis
|
||||||
|
* @param maxListShardsRetryAttempts
|
||||||
|
* @param maxCacheMissesBeforeReload
|
||||||
|
* @param listShardsCacheAllowedAgeInSeconds
|
||||||
|
* @param cacheMissWarningModulus
|
||||||
|
* @param initialLeaseTableReadCapacity
|
||||||
|
* @param initialLeaseTableWriteCapacity
|
||||||
|
* @param deprecatedHierarchicalShardSyncer
|
||||||
|
* @param tableCreatorCallback
|
||||||
|
* @param dynamoDbRequestTimeout
|
||||||
|
* @param billingMode
|
||||||
|
*/
|
||||||
|
private DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig,
|
||||||
|
final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier,
|
||||||
|
final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis,
|
||||||
|
final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads,
|
||||||
|
final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards,
|
||||||
|
final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis,
|
||||||
|
final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload,
|
||||||
|
final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus,
|
||||||
|
final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity,
|
||||||
|
final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback,
|
||||||
|
Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) {
|
||||||
|
this(kinesisClient, dynamoDBClient, tableName,
|
||||||
|
workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker,
|
||||||
|
maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion,
|
||||||
|
ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis,
|
||||||
|
maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds,
|
||||||
|
cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity,
|
||||||
|
deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer,
|
||||||
|
null, false, LeaseManagementConfig.DEFAULT_LEASE_CLEANUP_CONFIG);
|
||||||
|
this.streamConfig = streamConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor.
|
||||||
|
* @param kinesisClient
|
||||||
|
* @param dynamoDBClient
|
||||||
|
* @param tableName
|
||||||
|
* @param workerIdentifier
|
||||||
|
* @param executorService
|
||||||
|
* @param failoverTimeMillis
|
||||||
|
* @param epsilonMillis
|
||||||
|
* @param maxLeasesForWorker
|
||||||
|
* @param maxLeasesToStealAtOneTime
|
||||||
|
* @param maxLeaseRenewalThreads
|
||||||
|
* @param cleanupLeasesUponShardCompletion
|
||||||
|
* @param ignoreUnexpectedChildShards
|
||||||
|
* @param shardSyncIntervalMillis
|
||||||
|
* @param consistentReads
|
||||||
|
* @param listShardsBackoffTimeMillis
|
||||||
|
* @param maxListShardsRetryAttempts
|
||||||
|
* @param maxCacheMissesBeforeReload
|
||||||
|
* @param listShardsCacheAllowedAgeInSeconds
|
||||||
|
* @param cacheMissWarningModulus
|
||||||
|
* @param initialLeaseTableReadCapacity
|
||||||
|
* @param initialLeaseTableWriteCapacity
|
||||||
|
* @param deprecatedHierarchicalShardSyncer
|
||||||
|
* @param tableCreatorCallback
|
||||||
|
* @param dynamoDbRequestTimeout
|
||||||
|
* @param billingMode
|
||||||
|
* @param leaseSerializer
|
||||||
|
* @param customShardDetectorProvider
|
||||||
|
* @param isMultiStreamMode
|
||||||
|
* @param leaseCleanupConfig
|
||||||
|
*/
|
||||||
|
public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient,
|
||||||
|
final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier,
|
||||||
|
final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis,
|
||||||
|
final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads,
|
||||||
|
final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards,
|
||||||
|
final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis,
|
||||||
|
final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload,
|
||||||
|
final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus,
|
||||||
|
final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity,
|
||||||
|
final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback,
|
||||||
|
Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer,
|
||||||
|
Function<StreamConfig, ShardDetector> customShardDetectorProvider, boolean isMultiStreamMode,
|
||||||
|
LeaseCleanupConfig leaseCleanupConfig) {
|
||||||
this.kinesisClient = kinesisClient;
|
this.kinesisClient = kinesisClient;
|
||||||
this.streamName = streamName;
|
|
||||||
this.dynamoDBClient = dynamoDBClient;
|
this.dynamoDBClient = dynamoDBClient;
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.workerIdentifier = workerIdentifier;
|
this.workerIdentifier = workerIdentifier;
|
||||||
this.executorService = executorService;
|
this.executorService = executorService;
|
||||||
this.initialPositionInStream = initialPositionInStream;
|
|
||||||
this.failoverTimeMillis = failoverTimeMillis;
|
this.failoverTimeMillis = failoverTimeMillis;
|
||||||
this.epsilonMillis = epsilonMillis;
|
this.epsilonMillis = epsilonMillis;
|
||||||
this.maxLeasesForWorker = maxLeasesForWorker;
|
this.maxLeasesForWorker = maxLeasesForWorker;
|
||||||
|
|
@ -344,10 +460,14 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
this.cacheMissWarningModulus = cacheMissWarningModulus;
|
this.cacheMissWarningModulus = cacheMissWarningModulus;
|
||||||
this.initialLeaseTableReadCapacity = initialLeaseTableReadCapacity;
|
this.initialLeaseTableReadCapacity = initialLeaseTableReadCapacity;
|
||||||
this.initialLeaseTableWriteCapacity = initialLeaseTableWriteCapacity;
|
this.initialLeaseTableWriteCapacity = initialLeaseTableWriteCapacity;
|
||||||
this.hierarchicalShardSyncer = hierarchicalShardSyncer;
|
this.deprecatedHierarchicalShardSyncer = deprecatedHierarchicalShardSyncer;
|
||||||
this.tableCreatorCallback = tableCreatorCallback;
|
this.tableCreatorCallback = tableCreatorCallback;
|
||||||
this.dynamoDbRequestTimeout = dynamoDbRequestTimeout;
|
this.dynamoDbRequestTimeout = dynamoDbRequestTimeout;
|
||||||
this.billingMode = billingMode;
|
this.billingMode = billingMode;
|
||||||
|
this.leaseSerializer = leaseSerializer;
|
||||||
|
this.customShardDetectorProvider = customShardDetectorProvider;
|
||||||
|
this.isMultiStreamMode = isMultiStreamMode;
|
||||||
|
this.leaseCleanupConfig = leaseCleanupConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -364,29 +484,77 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
metricsFactory);
|
metricsFactory);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override @Deprecated
|
||||||
public ShardSyncTaskManager createShardSyncTaskManager(@NonNull final MetricsFactory metricsFactory) {
|
public ShardSyncTaskManager createShardSyncTaskManager(@NonNull final MetricsFactory metricsFactory) {
|
||||||
return new ShardSyncTaskManager(this.createShardDetector(),
|
return new ShardSyncTaskManager(this.createShardDetector(),
|
||||||
this.createLeaseRefresher(),
|
this.createLeaseRefresher(),
|
||||||
initialPositionInStream,
|
streamConfig.initialPositionInStreamExtended(),
|
||||||
|
cleanupLeasesUponShardCompletion,
|
||||||
|
ignoreUnexpectedChildShards,
|
||||||
|
shardSyncIntervalMillis,
|
||||||
|
executorService, deprecatedHierarchicalShardSyncer,
|
||||||
|
metricsFactory);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create ShardSyncTaskManager from the streamConfig passed
|
||||||
|
* @param metricsFactory
|
||||||
|
* @param streamConfig
|
||||||
|
* @return ShardSyncTaskManager
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) {
|
||||||
|
return new ShardSyncTaskManager(this.createShardDetector(streamConfig),
|
||||||
|
this.createLeaseRefresher(),
|
||||||
|
streamConfig.initialPositionInStreamExtended(),
|
||||||
cleanupLeasesUponShardCompletion,
|
cleanupLeasesUponShardCompletion,
|
||||||
ignoreUnexpectedChildShards,
|
ignoreUnexpectedChildShards,
|
||||||
shardSyncIntervalMillis,
|
shardSyncIntervalMillis,
|
||||||
executorService,
|
executorService,
|
||||||
hierarchicalShardSyncer,
|
new HierarchicalShardSyncer(isMultiStreamMode, streamConfig.streamIdentifier().toString()),
|
||||||
metricsFactory);
|
metricsFactory);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DynamoDBLeaseRefresher createLeaseRefresher() {
|
public DynamoDBLeaseRefresher createLeaseRefresher() {
|
||||||
return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, new DynamoDBLeaseSerializer(), consistentReads,
|
return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, leaseSerializer, consistentReads,
|
||||||
tableCreatorCallback, dynamoDbRequestTimeout, billingMode);
|
tableCreatorCallback, dynamoDbRequestTimeout, billingMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@Deprecated
|
||||||
public ShardDetector createShardDetector() {
|
public ShardDetector createShardDetector() {
|
||||||
return new KinesisShardDetector(kinesisClient, streamName, listShardsBackoffTimeMillis,
|
return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(),
|
||||||
|
listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds,
|
||||||
|
maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* KinesisShardDetector supports reading from service only using streamName. Support for accountId and
|
||||||
|
* stream creation epoch is yet to be provided.
|
||||||
|
* @param streamConfig
|
||||||
|
* @return ShardDetector
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public ShardDetector createShardDetector(StreamConfig streamConfig) {
|
||||||
|
return customShardDetectorProvider != null ? customShardDetectorProvider.apply(streamConfig) :
|
||||||
|
new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis,
|
||||||
maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload,
|
maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload,
|
||||||
cacheMissWarningModulus, dynamoDbRequestTimeout);
|
cacheMissWarningModulus, dynamoDbRequestTimeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* LeaseCleanupManager cleans up leases in the lease table for shards which have either expired past the
|
||||||
|
* stream's retention period or have been completely processed.
|
||||||
|
* @param metricsFactory
|
||||||
|
* @return LeaseCleanupManager
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) {
|
||||||
|
return new LeaseCleanupManager(createLeaseCoordinator(metricsFactory),
|
||||||
|
metricsFactory, Executors.newSingleThreadScheduledExecutor(),
|
||||||
|
cleanupLeasesUponShardCompletion, leaseCleanupConfig.leaseCleanupIntervalMillis(),
|
||||||
|
leaseCleanupConfig.completedLeaseCleanupIntervalMillis(),
|
||||||
|
leaseCleanupConfig.garbageLeaseCleanupIntervalMillis());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,8 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.leases.dynamodb;
|
package software.amazon.kinesis.leases.dynamodb;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
@ -21,24 +23,44 @@ import java.util.Map;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.*;
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.BillingMode;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.DeleteItemRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.DynamoDbException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.GetItemRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.GetItemResponse;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.LimitExceededException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputExceededException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ResourceInUseException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ScanRequest;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.ScanResponse;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.TableStatus;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest;
|
||||||
import software.amazon.awssdk.utils.CollectionUtils;
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.FutureUtils;
|
import software.amazon.kinesis.common.FutureUtils;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
import software.amazon.kinesis.leases.LeaseSerializer;
|
import software.amazon.kinesis.leases.LeaseSerializer;
|
||||||
|
import software.amazon.kinesis.leases.UpdateField;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.BillingMode;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An implementation of {@link LeaseRefresher} that uses DynamoDB.
|
* An implementation of {@link LeaseRefresher} that uses DynamoDB.
|
||||||
|
|
@ -58,6 +80,9 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
|
||||||
|
|
||||||
private boolean newTableCreated = false;
|
private boolean newTableCreated = false;
|
||||||
|
|
||||||
|
private static final String STREAM_NAME = "streamName";
|
||||||
|
private static final String DDB_STREAM_NAME = ":streamName";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor.
|
* Constructor.
|
||||||
*
|
*
|
||||||
|
|
@ -264,12 +289,21 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
|
||||||
return System.currentTimeMillis() - startTime;
|
return System.currentTimeMillis() - startTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public List<Lease> listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException,
|
||||||
|
InvalidStateException, ProvisionedThroughputException {
|
||||||
|
return list( null, streamIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<Lease> listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
public List<Lease> listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
return list(null);
|
return list(null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -278,22 +312,50 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
|
||||||
@Override
|
@Override
|
||||||
public boolean isLeaseTableEmpty()
|
public boolean isLeaseTableEmpty()
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
return list(1).isEmpty();
|
return list(1, 1, null).isEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List with the given page size. Package access for integration testing.
|
* List with the given page size. Package access for integration testing.
|
||||||
*
|
*
|
||||||
* @param limit number of items to consider at a time - used by integration tests to force paging.
|
* @param limit number of items to consider at a time - used by integration tests to force paging.
|
||||||
|
* @param streamIdentifier streamIdentifier for multi-stream mode. Can be null.
|
||||||
* @return list of leases
|
* @return list of leases
|
||||||
* @throws InvalidStateException if table does not exist
|
* @throws InvalidStateException if table does not exist
|
||||||
* @throws DependencyException if DynamoDB scan fail in an unexpected way
|
* @throws DependencyException if DynamoDB scan fail in an unexpected way
|
||||||
* @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity
|
* @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity
|
||||||
*/
|
*/
|
||||||
List<Lease> list(Integer limit) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
List<Lease> list(Integer limit, StreamIdentifier streamIdentifier)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
return list(limit, Integer.MAX_VALUE, streamIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* List with the given page size. Package access for integration testing.
|
||||||
|
*
|
||||||
|
* @param limit number of items to consider at a time - used by integration tests to force paging.
|
||||||
|
* @param maxPages mad paginated scan calls
|
||||||
|
* @param streamIdentifier streamIdentifier for multi-stream mode. Can be null.
|
||||||
|
* @return list of leases
|
||||||
|
* @throws InvalidStateException if table does not exist
|
||||||
|
* @throws DependencyException if DynamoDB scan fail in an unexpected way
|
||||||
|
* @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity
|
||||||
|
*/
|
||||||
|
private List<Lease> list(Integer limit, Integer maxPages, StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException,
|
||||||
|
ProvisionedThroughputException {
|
||||||
|
|
||||||
log.debug("Listing leases from table {}", table);
|
log.debug("Listing leases from table {}", table);
|
||||||
|
|
||||||
ScanRequest.Builder scanRequestBuilder = ScanRequest.builder().tableName(table);
|
ScanRequest.Builder scanRequestBuilder = ScanRequest.builder().tableName(table);
|
||||||
|
|
||||||
|
if (streamIdentifier != null) {
|
||||||
|
final Map<String, AttributeValue> expressionAttributeValues = ImmutableMap.of(
|
||||||
|
DDB_STREAM_NAME, AttributeValue.builder().s(streamIdentifier.serialize()).build()
|
||||||
|
);
|
||||||
|
scanRequestBuilder = scanRequestBuilder.filterExpression(STREAM_NAME + " = " + DDB_STREAM_NAME)
|
||||||
|
.expressionAttributeValues(expressionAttributeValues);
|
||||||
|
}
|
||||||
|
|
||||||
if (limit != null) {
|
if (limit != null) {
|
||||||
scanRequestBuilder = scanRequestBuilder.limit(limit);
|
scanRequestBuilder = scanRequestBuilder.limit(limit);
|
||||||
}
|
}
|
||||||
|
|
@ -315,7 +377,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<String, AttributeValue> lastEvaluatedKey = scanResult.lastEvaluatedKey();
|
Map<String, AttributeValue> lastEvaluatedKey = scanResult.lastEvaluatedKey();
|
||||||
if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey)) {
|
if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey) || --maxPages <= 0) {
|
||||||
// Signify that we're done.
|
// Signify that we're done.
|
||||||
scanResult = null;
|
scanResult = null;
|
||||||
log.debug("lastEvaluatedKey was null - scan finished.");
|
log.debug("lastEvaluatedKey was null - scan finished.");
|
||||||
|
|
@ -634,14 +696,40 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
log.debug("Updating lease without expectation {}", lease);
|
||||||
|
final AWSExceptionManager exceptionManager = createExceptionManager();
|
||||||
|
exceptionManager.add(ConditionalCheckFailedException.class, t -> t);
|
||||||
|
Map<String, AttributeValueUpdate> updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField);
|
||||||
|
UpdateItemRequest request = UpdateItemRequest.builder().tableName(table).key(serializer.getDynamoHashKey(lease))
|
||||||
|
.expected(serializer.getDynamoExistentExpectation(lease.leaseKey()))
|
||||||
|
.attributeUpdates(updates).build();
|
||||||
|
try {
|
||||||
|
try {
|
||||||
|
FutureUtils.resolveOrCancelFuture(dynamoDBClient.updateItem(request), dynamoDbRequestTimeout);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
throw exceptionManager.apply(e.getCause());
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new DependencyException(e);
|
||||||
|
}
|
||||||
|
} catch (ConditionalCheckFailedException e) {
|
||||||
|
log.warn("Lease update failed for lease with key {} because the lease did not exist at the time of the update",
|
||||||
|
lease.leaseKey(), e);
|
||||||
|
} catch (DynamoDbException | TimeoutException e) {
|
||||||
|
throw convertAndRethrowExceptions("update", lease.leaseKey(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ExtendedSequenceNumber getCheckpoint(String shardId)
|
public ExtendedSequenceNumber getCheckpoint(String leaseKey)
|
||||||
throws ProvisionedThroughputException, InvalidStateException, DependencyException {
|
throws ProvisionedThroughputException, InvalidStateException, DependencyException {
|
||||||
ExtendedSequenceNumber checkpoint = null;
|
ExtendedSequenceNumber checkpoint = null;
|
||||||
Lease lease = getLease(shardId);
|
Lease lease = getLease(leaseKey);
|
||||||
if (lease != null) {
|
if (lease != null) {
|
||||||
checkpoint = lease.checkpoint();
|
checkpoint = lease.checkpoint();
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -36,9 +36,11 @@ import lombok.RequiredArgsConstructor;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
import software.amazon.kinesis.leases.LeaseRenewer;
|
import software.amazon.kinesis.leases.LeaseRenewer;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
|
@ -269,7 +271,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String shardId)
|
public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String singleStreamShardId)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
verifyNotNull(lease, "lease cannot be null");
|
verifyNotNull(lease, "lease cannot be null");
|
||||||
verifyNotNull(lease.leaseKey(), "leaseKey cannot be null");
|
verifyNotNull(lease.leaseKey(), "leaseKey cannot be null");
|
||||||
|
|
@ -296,8 +298,12 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer {
|
||||||
}
|
}
|
||||||
|
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation);
|
||||||
if (StringUtils.isNotEmpty(shardId)) {
|
if (lease instanceof MultiStreamLease) {
|
||||||
MetricsUtil.addShardId(scope, shardId);
|
MetricsUtil.addStreamId(scope,
|
||||||
|
StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier()));
|
||||||
|
MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId());
|
||||||
|
} else if (StringUtils.isNotEmpty(singleStreamShardId)) {
|
||||||
|
MetricsUtil.addShardId(scope, singleStreamShardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
|
|
|
||||||
|
|
@ -30,10 +30,13 @@ import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
|
import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.KeyType;
|
import software.amazon.awssdk.services.dynamodb.model.KeyType;
|
||||||
import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
|
import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
import software.amazon.kinesis.leases.DynamoUtils;
|
import software.amazon.kinesis.leases.DynamoUtils;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
import software.amazon.kinesis.leases.LeaseSerializer;
|
import software.amazon.kinesis.leases.LeaseSerializer;
|
||||||
|
import software.amazon.kinesis.leases.UpdateField;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -50,7 +53,11 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
|
||||||
private static final String CHECKPOINT_SUBSEQUENCE_NUMBER_KEY = "checkpointSubSequenceNumber";
|
private static final String CHECKPOINT_SUBSEQUENCE_NUMBER_KEY = "checkpointSubSequenceNumber";
|
||||||
private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint";
|
private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint";
|
||||||
private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber";
|
private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber";
|
||||||
|
private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState";
|
||||||
private static final String PARENT_SHARD_ID_KEY = "parentShardId";
|
private static final String PARENT_SHARD_ID_KEY = "parentShardId";
|
||||||
|
private static final String CHILD_SHARD_IDS_KEY = "childShardIds";
|
||||||
|
private static final String STARTING_HASH_KEY = "startingHashKey";
|
||||||
|
private static final String ENDING_HASH_KEY = "endingHashKey";
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, AttributeValue> toDynamoRecord(final Lease lease) {
|
public Map<String, AttributeValue> toDynamoRecord(final Lease lease) {
|
||||||
|
|
@ -69,39 +76,65 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
|
||||||
if (lease.parentShardIds() != null && !lease.parentShardIds().isEmpty()) {
|
if (lease.parentShardIds() != null && !lease.parentShardIds().isEmpty()) {
|
||||||
result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.parentShardIds()));
|
result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.parentShardIds()));
|
||||||
}
|
}
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) {
|
||||||
|
result.put(CHILD_SHARD_IDS_KEY, DynamoUtils.createAttributeValue(lease.childShardIds()));
|
||||||
|
}
|
||||||
|
|
||||||
if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) {
|
if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) {
|
||||||
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().sequenceNumber()));
|
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().sequenceNumber()));
|
||||||
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().subSequenceNumber()));
|
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().subSequenceNumber()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (lease.pendingCheckpointState() != null) {
|
||||||
|
result.put(PENDING_CHECKPOINT_STATE_KEY, DynamoUtils.createAttributeValue(lease.checkpoint().subSequenceNumber()));
|
||||||
|
}
|
||||||
|
|
||||||
|
if(lease.hashKeyRangeForLease() != null) {
|
||||||
|
result.put(STARTING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()));
|
||||||
|
result.put(ENDING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()));
|
||||||
|
}
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Lease fromDynamoRecord(final Map<String, AttributeValue> dynamoRecord) {
|
public Lease fromDynamoRecord(final Map<String, AttributeValue> dynamoRecord) {
|
||||||
Lease result = new Lease();
|
final Lease result = new Lease();
|
||||||
result.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY));
|
return fromDynamoRecord(dynamoRecord, result);
|
||||||
result.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY));
|
}
|
||||||
result.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY));
|
|
||||||
|
|
||||||
result.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY));
|
@Override
|
||||||
result.checkpoint(
|
public Lease fromDynamoRecord(Map<String, AttributeValue> dynamoRecord, Lease leaseToUpdate) {
|
||||||
|
leaseToUpdate.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY));
|
||||||
|
leaseToUpdate.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY));
|
||||||
|
leaseToUpdate.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY));
|
||||||
|
|
||||||
|
leaseToUpdate.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY));
|
||||||
|
leaseToUpdate.checkpoint(
|
||||||
new ExtendedSequenceNumber(
|
new ExtendedSequenceNumber(
|
||||||
DynamoUtils.safeGetString(dynamoRecord, CHECKPOINT_SEQUENCE_NUMBER_KEY),
|
DynamoUtils.safeGetString(dynamoRecord, CHECKPOINT_SEQUENCE_NUMBER_KEY),
|
||||||
DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY))
|
DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY))
|
||||||
);
|
);
|
||||||
result.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY));
|
leaseToUpdate.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY));
|
||||||
|
leaseToUpdate.childShardIds(DynamoUtils.safeGetSS(dynamoRecord, CHILD_SHARD_IDS_KEY));
|
||||||
|
|
||||||
if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) {
|
if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) {
|
||||||
result.pendingCheckpoint(
|
leaseToUpdate.pendingCheckpoint(
|
||||||
new ExtendedSequenceNumber(
|
new ExtendedSequenceNumber(
|
||||||
DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY),
|
DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY),
|
||||||
DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY))
|
DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY))
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
leaseToUpdate.pendingCheckpointState(DynamoUtils.safeGetByteArray(dynamoRecord, PENDING_CHECKPOINT_STATE_KEY));
|
||||||
|
|
||||||
|
final String startingHashKey, endingHashKey;
|
||||||
|
if (!Strings.isNullOrEmpty(startingHashKey = DynamoUtils.safeGetString(dynamoRecord, STARTING_HASH_KEY))
|
||||||
|
&& !Strings.isNullOrEmpty(endingHashKey = DynamoUtils.safeGetString(dynamoRecord, ENDING_HASH_KEY))) {
|
||||||
|
leaseToUpdate.hashKeyRange(HashKeyRangeForLease.deserialize(startingHashKey, endingHashKey));
|
||||||
|
}
|
||||||
|
|
||||||
|
return leaseToUpdate;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -159,6 +192,19 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, ExpectedAttributeValue> getDynamoExistentExpectation(String leaseKey) {
|
||||||
|
Map<String, ExpectedAttributeValue> result = new HashMap<>();
|
||||||
|
|
||||||
|
ExpectedAttributeValue expectedAV = ExpectedAttributeValue.builder()
|
||||||
|
.exists(true)
|
||||||
|
.value(DynamoUtils.createAttributeValue(leaseKey))
|
||||||
|
.build();
|
||||||
|
result.put(LEASE_KEY_KEY, expectedAV);
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, AttributeValueUpdate> getDynamoLeaseCounterUpdate(final Lease lease) {
|
public Map<String, AttributeValueUpdate> getDynamoLeaseCounterUpdate(final Lease lease) {
|
||||||
return getDynamoLeaseCounterUpdate(lease.leaseCounter());
|
return getDynamoLeaseCounterUpdate(lease.leaseCounter());
|
||||||
|
|
@ -198,7 +244,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
private AttributeValueUpdate putUpdate(AttributeValue attributeValue) {
|
protected AttributeValueUpdate putUpdate(AttributeValue attributeValue) {
|
||||||
return AttributeValueUpdate.builder().value(attributeValue).action(AttributeAction.PUT).build();
|
return AttributeValueUpdate.builder().value(attributeValue).action(AttributeAction.PUT).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -216,6 +262,45 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
|
||||||
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
|
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
|
||||||
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
|
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (lease.pendingCheckpointState() != null) {
|
||||||
|
result.put(PENDING_CHECKPOINT_STATE_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.pendingCheckpointState())));
|
||||||
|
} else {
|
||||||
|
result.put(PENDING_CHECKPOINT_STATE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) {
|
||||||
|
result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds())));
|
||||||
|
}
|
||||||
|
|
||||||
|
if(lease.hashKeyRangeForLease() != null) {
|
||||||
|
result.put(STARTING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey())));
|
||||||
|
result.put(ENDING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey())));
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease,
|
||||||
|
UpdateField updateField) {
|
||||||
|
Map<String, AttributeValueUpdate> result = new HashMap<>();
|
||||||
|
switch (updateField) {
|
||||||
|
case CHILD_SHARDS:
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) {
|
||||||
|
result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds())));
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
case HASH_KEY_RANGE:
|
||||||
|
if (lease.hashKeyRangeForLease() != null) {
|
||||||
|
result.put(STARTING_HASH_KEY, putUpdate(
|
||||||
|
DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey())));
|
||||||
|
result.put(ENDING_HASH_KEY, putUpdate(
|
||||||
|
DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey())));
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -25,7 +25,7 @@ import java.util.Map.Entry;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
|
@ -36,8 +36,8 @@ import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsScope;
|
|
||||||
import software.amazon.kinesis.metrics.MetricsLevel;
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
import software.amazon.kinesis.metrics.MetricsUtil;
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -48,6 +48,7 @@ import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
public class DynamoDBLeaseTaker implements LeaseTaker {
|
public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
private static final int TAKE_RETRIES = 3;
|
private static final int TAKE_RETRIES = 3;
|
||||||
private static final int SCAN_RETRIES = 1;
|
private static final int SCAN_RETRIES = 1;
|
||||||
|
private long veryOldLeaseDurationNanosMultiplier = 3;
|
||||||
|
|
||||||
// See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable.
|
// See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable.
|
||||||
private static final Callable<Long> SYSTEM_CLOCK_CALLABLE = System::nanoTime;
|
private static final Callable<Long> SYSTEM_CLOCK_CALLABLE = System::nanoTime;
|
||||||
|
|
@ -95,6 +96,18 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Overrides the default very old lease duration nanos multiplier to increase the threshold for taking very old leases.
|
||||||
|
* Setting this to a higher value than 3 will increase the threshold for very old lease taking.
|
||||||
|
*
|
||||||
|
* @param veryOldLeaseDurationNanosMultipler Very old lease duration multiplier for adjusting very old lease taking.
|
||||||
|
* @return LeaseTaker
|
||||||
|
*/
|
||||||
|
public DynamoDBLeaseTaker withVeryOldLeaseDurationNanosMultipler(long veryOldLeaseDurationNanosMultipler) {
|
||||||
|
this.veryOldLeaseDurationNanosMultiplier = veryOldLeaseDurationNanosMultipler;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Max leases to steal from a more loaded Worker at one time (for load balancing).
|
* Max leases to steal from a more loaded Worker at one time (for load balancing).
|
||||||
* Setting this to a higher number can allow for faster load convergence (e.g. during deployments, cold starts),
|
* Setting this to a higher number can allow for faster load convergence (e.g. during deployments, cold starts),
|
||||||
|
|
@ -329,16 +342,24 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
Set<Lease> leasesToTake = new HashSet<>();
|
Set<Lease> leasesToTake = new HashSet<>();
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION);
|
||||||
MetricsUtil.addWorkerIdentifier(scope, workerIdentifier);
|
MetricsUtil.addWorkerIdentifier(scope, workerIdentifier);
|
||||||
|
List<Lease> veryOldLeases = new ArrayList<>();
|
||||||
|
|
||||||
|
int numLeases = 0;
|
||||||
|
int numWorkers = 0;
|
||||||
|
int numLeasesToReachTarget = 0;
|
||||||
|
int leaseSpillover = 0;
|
||||||
|
int veryOldLeaseCount = 0;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
int numLeases = allLeases.size();
|
numLeases = allLeases.size();
|
||||||
int numWorkers = leaseCounts.size();
|
numWorkers = leaseCounts.size();
|
||||||
|
|
||||||
if (numLeases == 0) {
|
if (numLeases == 0) {
|
||||||
// If there are no leases, I shouldn't try to take any.
|
// If there are no leases, I shouldn't try to take any.
|
||||||
return leasesToTake;
|
return leasesToTake;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
int target;
|
int target;
|
||||||
if (numWorkers >= numLeases) {
|
if (numWorkers >= numLeases) {
|
||||||
// If we have n leases and n or more workers, each worker can have up to 1 lease, including myself.
|
// If we have n leases and n or more workers, each worker can have up to 1 lease, including myself.
|
||||||
|
|
@ -353,7 +374,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
|
|
||||||
// Spill over is the number of leases this worker should have claimed, but did not because it would
|
// Spill over is the number of leases this worker should have claimed, but did not because it would
|
||||||
// exceed the max allowed for this worker.
|
// exceed the max allowed for this worker.
|
||||||
int leaseSpillover = Math.max(0, target - maxLeasesForWorker);
|
leaseSpillover = Math.max(0, target - maxLeasesForWorker);
|
||||||
if (target > maxLeasesForWorker) {
|
if (target > maxLeasesForWorker) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"Worker {} target is {} leases and maxLeasesForWorker is {}. Resetting target to {},"
|
"Worker {} target is {} leases and maxLeasesForWorker is {}. Resetting target to {},"
|
||||||
|
|
@ -362,11 +383,29 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
workerIdentifier, target, maxLeasesForWorker, maxLeasesForWorker, leaseSpillover);
|
workerIdentifier, target, maxLeasesForWorker, maxLeasesForWorker, leaseSpillover);
|
||||||
target = maxLeasesForWorker;
|
target = maxLeasesForWorker;
|
||||||
}
|
}
|
||||||
scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
int myCount = leaseCounts.get(workerIdentifier);
|
int myCount = leaseCounts.get(workerIdentifier);
|
||||||
int numLeasesToReachTarget = target - myCount;
|
numLeasesToReachTarget = target - myCount;
|
||||||
|
|
||||||
|
int currentLeaseCount = leaseCounts.get(workerIdentifier);
|
||||||
|
// If there are leases that have been expired for an extended period of
|
||||||
|
// time, take them with priority, disregarding the target (computed
|
||||||
|
// later) but obeying the maximum limit per worker.
|
||||||
|
veryOldLeases = allLeases.values().stream()
|
||||||
|
.filter(lease -> System.nanoTime() - lease.lastCounterIncrementNanos()
|
||||||
|
> veryOldLeaseDurationNanosMultiplier * leaseDurationNanos)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
if (!veryOldLeases.isEmpty()) {
|
||||||
|
Collections.shuffle(veryOldLeases);
|
||||||
|
veryOldLeaseCount = Math.max(0, Math.min(maxLeasesForWorker - currentLeaseCount, veryOldLeases.size()));
|
||||||
|
HashSet<Lease> result = new HashSet<>(veryOldLeases.subList(0, veryOldLeaseCount));
|
||||||
|
if (veryOldLeaseCount > 0) {
|
||||||
|
log.info("Taking leases that have been expired for a long time: {}", result);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
if (numLeasesToReachTarget <= 0) {
|
if (numLeasesToReachTarget <= 0) {
|
||||||
// If we don't need anything, return the empty set.
|
// If we don't need anything, return the empty set.
|
||||||
|
|
@ -376,7 +415,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
// Shuffle expiredLeases so workers don't all try to contend for the same leases.
|
// Shuffle expiredLeases so workers don't all try to contend for the same leases.
|
||||||
Collections.shuffle(expiredLeases);
|
Collections.shuffle(expiredLeases);
|
||||||
|
|
||||||
int originalExpiredLeasesSize = expiredLeases.size();
|
|
||||||
if (expiredLeases.size() > 0) {
|
if (expiredLeases.size() > 0) {
|
||||||
// If we have expired leases, get up to <needed> leases from expiredLeases
|
// If we have expired leases, get up to <needed> leases from expiredLeases
|
||||||
for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) {
|
for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) {
|
||||||
|
|
@ -397,16 +435,19 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
log.info(
|
log.info(
|
||||||
"Worker {} saw {} total leases, {} available leases, {} "
|
"Worker {} saw {} total leases, {} available leases, {} "
|
||||||
+ "workers. Target is {} leases, I have {} leases, I will take {} leases",
|
+ "workers. Target is {} leases, I have {} leases, I will take {} leases",
|
||||||
workerIdentifier, numLeases, originalExpiredLeasesSize, numWorkers, target, myCount,
|
workerIdentifier, numLeases, expiredLeases.size(), numWorkers, target, myCount,
|
||||||
leasesToTake.size());
|
leasesToTake.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED);
|
|
||||||
scope.addData("ExpiredLeases", originalExpiredLeasesSize, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
|
||||||
scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
|
||||||
scope.addData("NeededLeases", numLeasesToReachTarget, StandardUnit.COUNT, MetricsLevel.DETAILED);
|
|
||||||
scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED);
|
|
||||||
} finally {
|
} finally {
|
||||||
|
scope.addData("ExpiredLeases", expiredLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED);
|
||||||
|
scope.addData("NeededLeases", Math.max(numLeasesToReachTarget, 0), StandardUnit.COUNT, MetricsLevel.DETAILED);
|
||||||
|
scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED);
|
||||||
|
scope.addData("VeryOldLeases", veryOldLeaseCount, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
|
||||||
MetricsUtil.endScope(scope);
|
MetricsUtil.endScope(scope);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -499,10 +540,13 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
|
||||||
*/
|
*/
|
||||||
private Map<String, Integer> computeLeaseCounts(List<Lease> expiredLeases) {
|
private Map<String, Integer> computeLeaseCounts(List<Lease> expiredLeases) {
|
||||||
Map<String, Integer> leaseCounts = new HashMap<>();
|
Map<String, Integer> leaseCounts = new HashMap<>();
|
||||||
|
// The set will give much faster lookup than the original list, an
|
||||||
|
// important optimization when the list is large
|
||||||
|
Set<Lease> expiredLeasesSet = new HashSet<>(expiredLeases);
|
||||||
|
|
||||||
// Compute the number of leases per worker by looking through allLeases and ignoring leases that have expired.
|
// Compute the number of leases per worker by looking through allLeases and ignoring leases that have expired.
|
||||||
for (Lease lease : allLeases.values()) {
|
for (Lease lease : allLeases.values()) {
|
||||||
if (!expiredLeases.contains(lease)) {
|
if (!expiredLeasesSet.contains(lease)) {
|
||||||
String leaseOwner = lease.leaseOwner();
|
String leaseOwner = lease.leaseOwner();
|
||||||
Integer oldCount = leaseCounts.get(leaseOwner);
|
Integer oldCount = leaseCounts.get(leaseOwner);
|
||||||
if (oldCount == null) {
|
if (oldCount == null) {
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,63 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases.dynamodb;
|
||||||
|
|
||||||
|
import lombok.NoArgsConstructor;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
|
||||||
|
import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
|
||||||
|
import software.amazon.kinesis.leases.DynamoUtils;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static software.amazon.kinesis.leases.MultiStreamLease.validateAndCast;
|
||||||
|
|
||||||
|
@NoArgsConstructor
|
||||||
|
public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer {
|
||||||
|
|
||||||
|
// Keeping the stream id as "streamName" for legacy reasons.
|
||||||
|
private static final String STREAM_ID_KEY = "streamName";
|
||||||
|
private static final String SHARD_ID_KEY = "shardId";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, AttributeValue> toDynamoRecord(Lease lease) {
|
||||||
|
final MultiStreamLease multiStreamLease = validateAndCast(lease);
|
||||||
|
final Map<String, AttributeValue> result = super.toDynamoRecord(multiStreamLease);
|
||||||
|
result.put(STREAM_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier()));
|
||||||
|
result.put(SHARD_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.shardId()));
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public MultiStreamLease fromDynamoRecord(Map<String, AttributeValue> dynamoRecord) {
|
||||||
|
final MultiStreamLease multiStreamLease = (MultiStreamLease) super
|
||||||
|
.fromDynamoRecord(dynamoRecord, new MultiStreamLease());
|
||||||
|
multiStreamLease.streamIdentifier(DynamoUtils.safeGetString(dynamoRecord, STREAM_ID_KEY));
|
||||||
|
multiStreamLease.shardId(DynamoUtils.safeGetString(dynamoRecord, SHARD_ID_KEY));
|
||||||
|
return multiStreamLease;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease) {
|
||||||
|
final MultiStreamLease multiStreamLease = validateAndCast(lease);
|
||||||
|
final Map<String, AttributeValueUpdate> result = super.getDynamoUpdateLeaseUpdate(multiStreamLease);
|
||||||
|
result.put(STREAM_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier())));
|
||||||
|
result.put(SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.shardId())));
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,27 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases.exceptions;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exception type for all exceptions thrown by the customer implemented code.
|
||||||
|
*/
|
||||||
|
public class CustomerApplicationException extends Exception {
|
||||||
|
|
||||||
|
public CustomerApplicationException(Throwable e) { super(e);}
|
||||||
|
|
||||||
|
public CustomerApplicationException(String message, Throwable e) { super(message, e);}
|
||||||
|
|
||||||
|
public CustomerApplicationException(String message) { super(message);}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases.exceptions;
|
||||||
|
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class for cleaning up leases.
|
||||||
|
*/
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
@Value
|
||||||
|
@EqualsAndHashCode(exclude = {"queueEntryTime"})
|
||||||
|
public class LeasePendingDeletion {
|
||||||
|
private final StreamIdentifier streamIdentifier;
|
||||||
|
private final Lease lease;
|
||||||
|
private final ShardInfo shardInfo;
|
||||||
|
private final ShardDetector shardDetector;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Discovers the child shards for this lease.
|
||||||
|
* @return
|
||||||
|
* @throws InterruptedException
|
||||||
|
* @throws ExecutionException
|
||||||
|
* @throws TimeoutException
|
||||||
|
*/
|
||||||
|
public Set<String> getChildShardsFromService() throws InterruptedException, ExecutionException, TimeoutException {
|
||||||
|
return shardDetector.getChildShards(shardInfo.shardId()).stream().map(c -> c.shardId()).collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -19,6 +19,7 @@ import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public class ShardSyncer {
|
public class ShardSyncer {
|
||||||
private static final HierarchicalShardSyncer HIERARCHICAL_SHARD_SYNCER = new HierarchicalShardSyncer();
|
private static final HierarchicalShardSyncer HIERARCHICAL_SHARD_SYNCER = new HierarchicalShardSyncer();
|
||||||
|
private static final boolean garbageCollectLeases = true;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>NOTE: This method is deprecated and will be removed in a future release.</p>
|
* <p>NOTE: This method is deprecated and will be removed in a future release.</p>
|
||||||
|
|
@ -26,7 +27,6 @@ public class ShardSyncer {
|
||||||
* @param shardDetector
|
* @param shardDetector
|
||||||
* @param leaseRefresher
|
* @param leaseRefresher
|
||||||
* @param initialPosition
|
* @param initialPosition
|
||||||
* @param cleanupLeasesOfCompletedShards
|
|
||||||
* @param ignoreUnexpectedChildShards
|
* @param ignoreUnexpectedChildShards
|
||||||
* @param scope
|
* @param scope
|
||||||
* @throws DependencyException
|
* @throws DependencyException
|
||||||
|
|
@ -37,10 +37,9 @@ public class ShardSyncer {
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public static synchronized void checkAndCreateLeasesForNewShards(@NonNull final ShardDetector shardDetector,
|
public static synchronized void checkAndCreateLeasesForNewShards(@NonNull final ShardDetector shardDetector,
|
||||||
final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition,
|
final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition,
|
||||||
final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards,
|
final boolean ignoreUnexpectedChildShards, final MetricsScope scope)
|
||||||
final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException,
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException {
|
||||||
KinesisClientLibIOException {
|
|
||||||
HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition,
|
HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition,
|
||||||
cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope);
|
scope, ignoreUnexpectedChildShards, leaseRefresher.isLeaseTableEmpty());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -54,28 +54,29 @@ public class BlockOnParentShardTask implements ConsumerTask {
|
||||||
@Override
|
@Override
|
||||||
public TaskResult call() {
|
public TaskResult call() {
|
||||||
Exception exception = null;
|
Exception exception = null;
|
||||||
|
final String shardInfoId = ShardInfo.getLeaseKey(shardInfo);
|
||||||
try {
|
try {
|
||||||
boolean blockedOnParentShard = false;
|
boolean blockedOnParentShard = false;
|
||||||
for (String shardId : shardInfo.parentShardIds()) {
|
for (String shardId : shardInfo.parentShardIds()) {
|
||||||
Lease lease = leaseRefresher.getLease(shardId);
|
final String leaseKey = ShardInfo.getLeaseKey(shardInfo, shardId);
|
||||||
|
final Lease lease = leaseRefresher.getLease(leaseKey);
|
||||||
if (lease != null) {
|
if (lease != null) {
|
||||||
ExtendedSequenceNumber checkpoint = lease.checkpoint();
|
ExtendedSequenceNumber checkpoint = lease.checkpoint();
|
||||||
if ((checkpoint == null) || (!checkpoint.equals(ExtendedSequenceNumber.SHARD_END))) {
|
if ((checkpoint == null) || (!checkpoint.equals(ExtendedSequenceNumber.SHARD_END))) {
|
||||||
log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardId, checkpoint);
|
log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardInfoId, checkpoint);
|
||||||
blockedOnParentShard = true;
|
blockedOnParentShard = true;
|
||||||
exception = new BlockedOnParentShardException("Parent shard not yet done");
|
exception = new BlockedOnParentShardException("Parent shard not yet done");
|
||||||
break;
|
break;
|
||||||
} else {
|
} else {
|
||||||
log.debug("Shard {} has been completely processed.", shardId);
|
log.debug("Shard {} has been completely processed.", shardInfoId);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardId);
|
log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardInfoId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!blockedOnParentShard) {
|
if (!blockedOnParentShard) {
|
||||||
log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfo.shardId());
|
log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfoId);
|
||||||
return new TaskResult(null);
|
return new TaskResult(null);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|
@ -85,7 +86,7 @@ public class BlockOnParentShardTask implements ConsumerTask {
|
||||||
try {
|
try {
|
||||||
Thread.sleep(parentShardPollIntervalMillis);
|
Thread.sleep(parentShardPollIntervalMillis);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfo.shardId(), e);
|
log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfoId, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
return new TaskResult(exception);
|
return new TaskResult(exception);
|
||||||
|
|
|
||||||
|
|
@ -146,7 +146,7 @@ class ConsumerStates {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ConsumerState shutdownTransition(ShutdownReason shutdownReason) {
|
public ConsumerState shutdownTransition(ShutdownReason shutdownReason) {
|
||||||
return ShardConsumerState.SHUTDOWN_COMPLETE.consumerState();
|
return ShardConsumerState.SHUTTING_DOWN.consumerState();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -496,7 +496,10 @@ class ConsumerStates {
|
||||||
argument.taskBackoffTimeMillis(),
|
argument.taskBackoffTimeMillis(),
|
||||||
argument.recordsPublisher(),
|
argument.recordsPublisher(),
|
||||||
argument.hierarchicalShardSyncer(),
|
argument.hierarchicalShardSyncer(),
|
||||||
argument.metricsFactory());
|
argument.metricsFactory(),
|
||||||
|
input == null ? null : input.childShards(),
|
||||||
|
argument.streamIdentifier(),
|
||||||
|
argument.leaseCleanupManager());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.checkpoint.Checkpoint;
|
import software.amazon.kinesis.checkpoint.Checkpoint;
|
||||||
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.lifecycle.events.InitializationInput;
|
import software.amazon.kinesis.lifecycle.events.InitializationInput;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
@ -75,9 +76,10 @@ public class InitializeTask implements ConsumerTask {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
log.debug("Initializing ShardId {}", shardInfo);
|
log.debug("Initializing ShardId {}", shardInfo);
|
||||||
Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(shardInfo.shardId());
|
final String leaseKey = ShardInfo.getLeaseKey(shardInfo);
|
||||||
|
Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(leaseKey);
|
||||||
ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.checkpoint();
|
ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.checkpoint();
|
||||||
log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", shardInfo.shardId(), initialCheckpoint,
|
log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", leaseKey, initialCheckpoint,
|
||||||
initialPositionInStream);
|
initialPositionInStream);
|
||||||
|
|
||||||
cache.start(initialCheckpoint, initialPositionInStream);
|
cache.start(initialCheckpoint, initialPositionInStream);
|
||||||
|
|
@ -90,6 +92,7 @@ public class InitializeTask implements ConsumerTask {
|
||||||
.shardId(shardInfo.shardId())
|
.shardId(shardInfo.shardId())
|
||||||
.extendedSequenceNumber(initialCheckpoint)
|
.extendedSequenceNumber(initialCheckpoint)
|
||||||
.pendingCheckpointSequenceNumber(initialCheckpointObject.pendingCheckpoint())
|
.pendingCheckpointSequenceNumber(initialCheckpointObject.pendingCheckpoint())
|
||||||
|
.pendingCheckpointState(initialCheckpointObject.pendingCheckpointState())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory,
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory,
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
import software.amazon.awssdk.services.kinesis.model.Shard;
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.ShardDetector;
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
||||||
|
|
@ -60,6 +61,7 @@ public class ProcessTask implements ConsumerTask {
|
||||||
private final ProcessRecordsInput processRecordsInput;
|
private final ProcessRecordsInput processRecordsInput;
|
||||||
private final MetricsFactory metricsFactory;
|
private final MetricsFactory metricsFactory;
|
||||||
private final AggregatorUtil aggregatorUtil;
|
private final AggregatorUtil aggregatorUtil;
|
||||||
|
private final String shardInfoId;
|
||||||
|
|
||||||
public ProcessTask(@NonNull ShardInfo shardInfo,
|
public ProcessTask(@NonNull ShardInfo shardInfo,
|
||||||
@NonNull ShardRecordProcessor shardRecordProcessor,
|
@NonNull ShardRecordProcessor shardRecordProcessor,
|
||||||
|
|
@ -74,6 +76,7 @@ public class ProcessTask implements ConsumerTask {
|
||||||
@NonNull AggregatorUtil aggregatorUtil,
|
@NonNull AggregatorUtil aggregatorUtil,
|
||||||
@NonNull MetricsFactory metricsFactory) {
|
@NonNull MetricsFactory metricsFactory) {
|
||||||
this.shardInfo = shardInfo;
|
this.shardInfo = shardInfo;
|
||||||
|
this.shardInfoId = ShardInfo.getLeaseKey(shardInfo);
|
||||||
this.shardRecordProcessor = shardRecordProcessor;
|
this.shardRecordProcessor = shardRecordProcessor;
|
||||||
this.recordProcessorCheckpointer = recordProcessorCheckpointer;
|
this.recordProcessorCheckpointer = recordProcessorCheckpointer;
|
||||||
this.backoffTimeMillis = backoffTimeMillis;
|
this.backoffTimeMillis = backoffTimeMillis;
|
||||||
|
|
@ -106,6 +109,8 @@ public class ProcessTask implements ConsumerTask {
|
||||||
@Override
|
@Override
|
||||||
public TaskResult call() {
|
public TaskResult call() {
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION);
|
||||||
|
shardInfo.streamIdentifierSerOpt()
|
||||||
|
.ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId)));
|
||||||
MetricsUtil.addShardId(scope, shardInfo.shardId());
|
MetricsUtil.addShardId(scope, shardInfo.shardId());
|
||||||
long startTimeMillis = System.currentTimeMillis();
|
long startTimeMillis = System.currentTimeMillis();
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
|
|
@ -121,7 +126,7 @@ public class ProcessTask implements ConsumerTask {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (processRecordsInput.isAtShardEnd() && processRecordsInput.records().isEmpty()) {
|
if (processRecordsInput.isAtShardEnd() && processRecordsInput.records().isEmpty()) {
|
||||||
log.info("Reached end of shard {} and have no records to process", shardInfo.shardId());
|
log.info("Reached end of shard {} and have no records to process", shardInfoId);
|
||||||
return new TaskResult(null, true);
|
return new TaskResult(null, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -142,13 +147,13 @@ public class ProcessTask implements ConsumerTask {
|
||||||
}
|
}
|
||||||
success = true;
|
success = true;
|
||||||
} catch (RuntimeException e) {
|
} catch (RuntimeException e) {
|
||||||
log.error("ShardId {}: Caught exception: ", shardInfo.shardId(), e);
|
log.error("ShardId {}: Caught exception: ", shardInfoId, e);
|
||||||
exception = e;
|
exception = e;
|
||||||
backoff();
|
backoff();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (processRecordsInput.isAtShardEnd()) {
|
if (processRecordsInput.isAtShardEnd()) {
|
||||||
log.info("Reached end of shard {}, and processed {} records", shardInfo.shardId(), processRecordsInput.records().size());
|
log.info("Reached end of shard {}, and processed {} records", shardInfoId, processRecordsInput.records().size());
|
||||||
return new TaskResult(null, true);
|
return new TaskResult(null, true);
|
||||||
}
|
}
|
||||||
return new TaskResult(exception);
|
return new TaskResult(exception);
|
||||||
|
|
@ -174,7 +179,7 @@ public class ProcessTask implements ConsumerTask {
|
||||||
try {
|
try {
|
||||||
Thread.sleep(this.backoffTimeMillis);
|
Thread.sleep(this.backoffTimeMillis);
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
log.debug("{}: Sleep was interrupted", shardInfo.shardId(), ie);
|
log.debug("{}: Sleep was interrupted", shardInfoId, ie);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -188,20 +193,22 @@ public class ProcessTask implements ConsumerTask {
|
||||||
*/
|
*/
|
||||||
private void callProcessRecords(ProcessRecordsInput input, List<KinesisClientRecord> records) {
|
private void callProcessRecords(ProcessRecordsInput input, List<KinesisClientRecord> records) {
|
||||||
log.debug("Calling application processRecords() with {} records from {}", records.size(),
|
log.debug("Calling application processRecords() with {} records from {}", records.size(),
|
||||||
shardInfo.shardId());
|
shardInfoId);
|
||||||
|
|
||||||
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime())
|
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime())
|
||||||
.checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build();
|
.checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build();
|
||||||
|
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION);
|
||||||
|
shardInfo.streamIdentifierSerOpt()
|
||||||
|
.ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId)));
|
||||||
MetricsUtil.addShardId(scope, shardInfo.shardId());
|
MetricsUtil.addShardId(scope, shardInfo.shardId());
|
||||||
final long startTime = System.currentTimeMillis();
|
final long startTime = System.currentTimeMillis();
|
||||||
try {
|
try {
|
||||||
shardRecordProcessor.processRecords(processRecordsInput);
|
shardRecordProcessor.processRecords(processRecordsInput);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.error("ShardId {}: Application processRecords() threw an exception when processing shard ",
|
log.error("ShardId {}: Application processRecords() threw an exception when processing shard ",
|
||||||
shardInfo.shardId(), e);
|
shardInfoId, e);
|
||||||
log.error("ShardId {}: Skipping over the following data records: {}", shardInfo.shardId(), records);
|
log.error("ShardId {}: Skipping over the following data records: {}", shardInfoId, records);
|
||||||
} finally {
|
} finally {
|
||||||
MetricsUtil.addLatency(scope, RECORD_PROCESSOR_PROCESS_RECORDS_METRIC, startTime, MetricsLevel.SUMMARY);
|
MetricsUtil.addLatency(scope, RECORD_PROCESSOR_PROCESS_RECORDS_METRIC, startTime, MetricsLevel.SUMMARY);
|
||||||
MetricsUtil.endScope(scope);
|
MetricsUtil.endScope(scope);
|
||||||
|
|
@ -226,17 +233,17 @@ public class ProcessTask implements ConsumerTask {
|
||||||
* the time when the task started
|
* the time when the task started
|
||||||
*/
|
*/
|
||||||
private void handleNoRecords(long startTimeMillis) {
|
private void handleNoRecords(long startTimeMillis) {
|
||||||
log.debug("Kinesis didn't return any records for shard {}", shardInfo.shardId());
|
log.debug("Kinesis didn't return any records for shard {}", shardInfoId);
|
||||||
|
|
||||||
long sleepTimeMillis = idleTimeInMilliseconds - (System.currentTimeMillis() - startTimeMillis);
|
long sleepTimeMillis = idleTimeInMilliseconds - (System.currentTimeMillis() - startTimeMillis);
|
||||||
if (sleepTimeMillis > 0) {
|
if (sleepTimeMillis > 0) {
|
||||||
sleepTimeMillis = Math.max(sleepTimeMillis, idleTimeInMilliseconds);
|
sleepTimeMillis = Math.max(sleepTimeMillis, idleTimeInMilliseconds);
|
||||||
try {
|
try {
|
||||||
log.debug("Sleeping for {} ms since there were no new records in shard {}", sleepTimeMillis,
|
log.debug("Sleeping for {} ms since there were no new records in shard {}", sleepTimeMillis,
|
||||||
shardInfo.shardId());
|
shardInfoId);
|
||||||
Thread.sleep(sleepTimeMillis);
|
Thread.sleep(sleepTimeMillis);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
log.debug("ShardId {}: Sleep was interrupted", shardInfo.shardId());
|
log.debug("ShardId {}: Sleep was interrupted", shardInfoId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -273,8 +280,8 @@ public class ProcessTask implements ConsumerTask {
|
||||||
|
|
||||||
if (extendedSequenceNumber.compareTo(lastCheckpointValue) <= 0) {
|
if (extendedSequenceNumber.compareTo(lastCheckpointValue) <= 0) {
|
||||||
recordIterator.remove();
|
recordIterator.remove();
|
||||||
log.debug("removing record with ESN {} because the ESN is <= checkpoint ({})", extendedSequenceNumber,
|
log.debug("{} : removing record with ESN {} because the ESN is <= checkpoint ({})", shardInfoId,
|
||||||
lastCheckpointValue);
|
extendedSequenceNumber, lastCheckpointValue);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ package software.amazon.kinesis.lifecycle;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
@ -32,6 +33,7 @@ import lombok.Getter;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException;
|
import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
|
|
@ -62,6 +64,7 @@ public class ShardConsumer {
|
||||||
private final Function<ConsumerTask, ConsumerTask> taskMetricsDecorator;
|
private final Function<ConsumerTask, ConsumerTask> taskMetricsDecorator;
|
||||||
private final int bufferSize;
|
private final int bufferSize;
|
||||||
private final TaskExecutionListener taskExecutionListener;
|
private final TaskExecutionListener taskExecutionListener;
|
||||||
|
private final String streamIdentifier;
|
||||||
|
|
||||||
private ConsumerTask currentTask;
|
private ConsumerTask currentTask;
|
||||||
private TaskOutcome taskOutcome;
|
private TaskOutcome taskOutcome;
|
||||||
|
|
@ -85,6 +88,8 @@ public class ShardConsumer {
|
||||||
|
|
||||||
private final ShardConsumerSubscriber subscriber;
|
private final ShardConsumerSubscriber subscriber;
|
||||||
|
|
||||||
|
private ProcessRecordsInput shardEndProcessRecordsInput;
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public ShardConsumer(RecordsPublisher recordsPublisher, ExecutorService executorService, ShardInfo shardInfo,
|
public ShardConsumer(RecordsPublisher recordsPublisher, ExecutorService executorService, ShardInfo shardInfo,
|
||||||
Optional<Long> logWarningForTaskAfterMillis, ShardConsumerArgument shardConsumerArgument,
|
Optional<Long> logWarningForTaskAfterMillis, ShardConsumerArgument shardConsumerArgument,
|
||||||
|
|
@ -124,6 +129,7 @@ public class ShardConsumer {
|
||||||
this.recordsPublisher = recordsPublisher;
|
this.recordsPublisher = recordsPublisher;
|
||||||
this.executorService = executorService;
|
this.executorService = executorService;
|
||||||
this.shardInfo = shardInfo;
|
this.shardInfo = shardInfo;
|
||||||
|
this.streamIdentifier = shardInfo.streamIdentifierSerOpt().orElse("single_stream_mode");
|
||||||
this.shardConsumerArgument = shardConsumerArgument;
|
this.shardConsumerArgument = shardConsumerArgument;
|
||||||
this.logWarningForTaskAfterMillis = logWarningForTaskAfterMillis;
|
this.logWarningForTaskAfterMillis = logWarningForTaskAfterMillis;
|
||||||
this.taskExecutionListener = taskExecutionListener;
|
this.taskExecutionListener = taskExecutionListener;
|
||||||
|
|
@ -146,6 +152,7 @@ public class ShardConsumer {
|
||||||
processData(input);
|
processData(input);
|
||||||
if (taskOutcome == TaskOutcome.END_OF_SHARD) {
|
if (taskOutcome == TaskOutcome.END_OF_SHARD) {
|
||||||
markForShutdown(ShutdownReason.SHARD_END);
|
markForShutdown(ShutdownReason.SHARD_END);
|
||||||
|
shardEndProcessRecordsInput = input;
|
||||||
subscription.cancel();
|
subscription.cancel();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -208,8 +215,8 @@ public class ShardConsumer {
|
||||||
}
|
}
|
||||||
Throwable dispatchFailure = subscriber.getAndResetDispatchFailure();
|
Throwable dispatchFailure = subscriber.getAndResetDispatchFailure();
|
||||||
if (dispatchFailure != null) {
|
if (dispatchFailure != null) {
|
||||||
log.warn("Exception occurred while dispatching incoming data. The incoming data has been skipped",
|
log.warn("{} : Exception occurred while dispatching incoming data. The incoming data has been skipped",
|
||||||
dispatchFailure);
|
streamIdentifier, dispatchFailure);
|
||||||
return dispatchFailure;
|
return dispatchFailure;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -238,7 +245,7 @@ public class ShardConsumer {
|
||||||
Instant now = Instant.now();
|
Instant now = Instant.now();
|
||||||
Duration timeSince = Duration.between(subscriber.lastDataArrival(), now);
|
Duration timeSince = Duration.between(subscriber.lastDataArrival(), now);
|
||||||
if (timeSince.toMillis() > value) {
|
if (timeSince.toMillis() > value) {
|
||||||
log.warn("Last time data arrived: {} ({})", lastDataArrival, timeSince);
|
log.warn("{} : Last time data arrived: {} ({})", streamIdentifier, lastDataArrival, timeSince);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
@ -250,11 +257,11 @@ public class ShardConsumer {
|
||||||
if (taken != null) {
|
if (taken != null) {
|
||||||
String message = longRunningTaskMessage(taken);
|
String message = longRunningTaskMessage(taken);
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("{} Not submitting new task.", message);
|
log.debug("{} : {} Not submitting new task.", streamIdentifier, message);
|
||||||
}
|
}
|
||||||
logWarningForTaskAfterMillis.ifPresent(value -> {
|
logWarningForTaskAfterMillis.ifPresent(value -> {
|
||||||
if (taken.toMillis() > value) {
|
if (taken.toMillis() > value) {
|
||||||
log.warn(message);
|
log.warn("{} : {}", streamIdentifier, message);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
@ -303,7 +310,7 @@ public class ShardConsumer {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
executeTask(null);
|
executeTask(shardEndProcessRecordsInput);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}, executorService);
|
}, executorService);
|
||||||
|
|
@ -358,7 +365,7 @@ public class ShardConsumer {
|
||||||
nextState = currentState.failureTransition();
|
nextState = currentState.failureTransition();
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
log.error("No handler for outcome of {}", outcome.name());
|
log.error("{} : No handler for outcome of {}", streamIdentifier, outcome.name());
|
||||||
nextState = currentState.failureTransition();
|
nextState = currentState.failureTransition();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
@ -382,9 +389,9 @@ public class ShardConsumer {
|
||||||
Exception taskException = taskResult.getException();
|
Exception taskException = taskResult.getException();
|
||||||
if (taskException instanceof BlockedOnParentShardException) {
|
if (taskException instanceof BlockedOnParentShardException) {
|
||||||
// No need to log the stack trace for this exception (it is very specific).
|
// No need to log the stack trace for this exception (it is very specific).
|
||||||
log.debug("Shard {} is blocked on completion of parent shard.", shardInfo.shardId());
|
log.debug("{} : Shard {} is blocked on completion of parent shard.", streamIdentifier, shardInfo.shardId());
|
||||||
} else {
|
} else {
|
||||||
log.debug("Caught exception running {} task: ", currentTask.taskType(), taskResult.getException());
|
log.debug("{} : Caught exception running {} task: ", streamIdentifier, currentTask.taskType(), taskResult.getException());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -411,10 +418,10 @@ public class ShardConsumer {
|
||||||
* @return true if shutdown is complete (false if shutdown is still in progress)
|
* @return true if shutdown is complete (false if shutdown is still in progress)
|
||||||
*/
|
*/
|
||||||
public boolean leaseLost() {
|
public boolean leaseLost() {
|
||||||
log.debug("Shutdown({}): Lease lost triggered.", shardInfo.shardId());
|
log.debug("{} : Shutdown({}): Lease lost triggered.", streamIdentifier, shardInfo.shardId());
|
||||||
if (subscriber != null) {
|
if (subscriber != null) {
|
||||||
subscriber.cancel();
|
subscriber.cancel();
|
||||||
log.debug("Shutdown({}): Subscriber cancelled.", shardInfo.shardId());
|
log.debug("{} : Shutdown({}): Subscriber cancelled.", streamIdentifier, shardInfo.shardId());
|
||||||
}
|
}
|
||||||
markForShutdown(ShutdownReason.LEASE_LOST);
|
markForShutdown(ShutdownReason.LEASE_LOST);
|
||||||
return isShutdown();
|
return isShutdown();
|
||||||
|
|
|
||||||
|
|
@ -21,8 +21,9 @@ import lombok.experimental.Accessors;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.LeaseCleanupManager;
|
||||||
import software.amazon.kinesis.leases.LeaseCoordinator;
|
import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
|
||||||
import software.amazon.kinesis.leases.ShardDetector;
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
||||||
|
|
@ -41,7 +42,7 @@ public class ShardConsumerArgument {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final ShardInfo shardInfo;
|
private final ShardInfo shardInfo;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String streamName;
|
private final StreamIdentifier streamIdentifier;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final LeaseCoordinator leaseCoordinator;
|
private final LeaseCoordinator leaseCoordinator;
|
||||||
@NonNull
|
@NonNull
|
||||||
|
|
@ -71,4 +72,5 @@ public class ShardConsumerArgument {
|
||||||
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
private final HierarchicalShardSyncer hierarchicalShardSyncer;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final MetricsFactory metricsFactory;
|
private final MetricsFactory metricsFactory;
|
||||||
|
private final LeaseCleanupManager leaseCleanupManager;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -24,6 +24,7 @@ import lombok.experimental.Accessors;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import org.reactivestreams.Subscriber;
|
import org.reactivestreams.Subscriber;
|
||||||
import org.reactivestreams.Subscription;
|
import org.reactivestreams.Subscription;
|
||||||
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RecordsRetrieved;
|
import software.amazon.kinesis.retrieval.RecordsRetrieved;
|
||||||
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
||||||
|
|
@ -40,8 +41,8 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
private final int bufferSize;
|
private final int bufferSize;
|
||||||
private final ShardConsumer shardConsumer;
|
private final ShardConsumer shardConsumer;
|
||||||
private final int readTimeoutsToIgnoreBeforeWarning;
|
private final int readTimeoutsToIgnoreBeforeWarning;
|
||||||
|
private final String shardInfoId;
|
||||||
private volatile int readTimeoutSinceLastRead = 0;
|
private volatile int readTimeoutSinceLastRead = 0;
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
final Object lockObject = new Object();
|
final Object lockObject = new Object();
|
||||||
// This holds the last time an attempt of request to upstream service was made including the first try to
|
// This holds the last time an attempt of request to upstream service was made including the first try to
|
||||||
|
|
@ -70,6 +71,7 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
this.bufferSize = bufferSize;
|
this.bufferSize = bufferSize;
|
||||||
this.shardConsumer = shardConsumer;
|
this.shardConsumer = shardConsumer;
|
||||||
this.readTimeoutsToIgnoreBeforeWarning = readTimeoutsToIgnoreBeforeWarning;
|
this.readTimeoutsToIgnoreBeforeWarning = readTimeoutsToIgnoreBeforeWarning;
|
||||||
|
this.shardInfoId = ShardInfo.getLeaseKey(shardConsumer.shardInfo());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -107,7 +109,7 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
if (retrievalFailure != null) {
|
if (retrievalFailure != null) {
|
||||||
synchronized (lockObject) {
|
synchronized (lockObject) {
|
||||||
String logMessage = String.format("%s: Failure occurred in retrieval. Restarting data requests",
|
String logMessage = String.format("%s: Failure occurred in retrieval. Restarting data requests",
|
||||||
shardConsumer.shardInfo().shardId());
|
shardInfoId);
|
||||||
if (retrievalFailure instanceof RetryableRetrievalException) {
|
if (retrievalFailure instanceof RetryableRetrievalException) {
|
||||||
log.debug(logMessage, retrievalFailure.getCause());
|
log.debug(logMessage, retrievalFailure.getCause());
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -130,7 +132,7 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
if (timeSinceLastResponse.toMillis() > maxTimeBetweenRequests) {
|
if (timeSinceLastResponse.toMillis() > maxTimeBetweenRequests) {
|
||||||
log.error(
|
log.error(
|
||||||
"{}: Last request was dispatched at {}, but no response as of {} ({}). Cancelling subscription, and restarting. Last successful request details -- {}",
|
"{}: Last request was dispatched at {}, but no response as of {} ({}). Cancelling subscription, and restarting. Last successful request details -- {}",
|
||||||
shardConsumer.shardInfo().shardId(), lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails());
|
shardInfoId, lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails());
|
||||||
cancel();
|
cancel();
|
||||||
|
|
||||||
// Start the subscription again which will update the lastRequestTime as well.
|
// Start the subscription again which will update the lastRequestTime as well.
|
||||||
|
|
@ -157,7 +159,7 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
subscription);
|
subscription);
|
||||||
|
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
log.warn("{}: Caught exception from handleInput", shardConsumer.shardInfo().shardId(), t);
|
log.warn("{}: Caught exception from handleInput", shardInfoId, t);
|
||||||
synchronized (lockObject) {
|
synchronized (lockObject) {
|
||||||
dispatchFailure = t;
|
dispatchFailure = t;
|
||||||
}
|
}
|
||||||
|
|
@ -192,8 +194,8 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
protected void logOnErrorWarning(Throwable t) {
|
protected void logOnErrorWarning(Throwable t) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: onError(). Cancelling subscription, and marking self as failed. KCL will "
|
"{}: onError(). Cancelling subscription, and marking self as failed. KCL will "
|
||||||
+ "recreate the subscription as neccessary to continue processing. Last successful request details -- {}",
|
+ "recreate the subscription as necessary to continue processing. Last successful request details -- {}",
|
||||||
shardConsumer.shardInfo().shardId(), recordsPublisher.getLastSuccessfulRequestDetails(), t);
|
shardInfoId, recordsPublisher.getLastSuccessfulRequestDetails(), t);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void logOnErrorReadTimeoutWarning(Throwable t) {
|
protected void logOnErrorReadTimeoutWarning(Throwable t) {
|
||||||
|
|
@ -202,14 +204,14 @@ class ShardConsumerSubscriber implements Subscriber<RecordsRetrieved> {
|
||||||
+ "are seeing this warning frequently consider increasing the SDK timeouts "
|
+ "are seeing this warning frequently consider increasing the SDK timeouts "
|
||||||
+ "by providing an OverrideConfiguration to the kinesis client. Alternatively you"
|
+ "by providing an OverrideConfiguration to the kinesis client. Alternatively you"
|
||||||
+ "can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress"
|
+ "can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress"
|
||||||
+ "intermittant ReadTimeout warnings. Last successful request details -- {}",
|
+ "intermittent ReadTimeout warnings. Last successful request details -- {}",
|
||||||
shardConsumer.shardInfo().shardId(), recordsPublisher.getLastSuccessfulRequestDetails(), t);
|
shardInfoId, recordsPublisher.getLastSuccessfulRequestDetails(), t);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onComplete() {
|
public void onComplete() {
|
||||||
log.debug("{}: onComplete(): Received onComplete. Activity should be triggered externally",
|
log.debug("{}: onComplete(): Received onComplete. Activity should be triggered externally",
|
||||||
shardConsumer.shardInfo().shardId());
|
shardInfoId);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void cancel() {
|
public void cancel() {
|
||||||
|
|
|
||||||
|
|
@ -16,35 +16,46 @@ package software.amazon.kinesis.lifecycle;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
import com.sun.org.apache.bcel.internal.generic.LUSHR;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.RequiredArgsConstructor;
|
import lombok.RequiredArgsConstructor;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
import software.amazon.awssdk.services.kinesis.model.Shard;
|
|
||||||
import software.amazon.awssdk.utils.CollectionUtils;
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException;
|
||||||
|
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseCleanupManager;
|
||||||
import software.amazon.kinesis.leases.LeaseCoordinator;
|
import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
|
||||||
import software.amazon.kinesis.leases.ShardDetector;
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
import software.amazon.kinesis.leases.UpdateField;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.CustomerApplicationException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
|
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
|
||||||
import software.amazon.kinesis.lifecycle.events.ShardEndedInput;
|
import software.amazon.kinesis.lifecycle.events.ShardEndedInput;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsScope;
|
|
||||||
import software.amazon.kinesis.metrics.MetricsLevel;
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
import software.amazon.kinesis.metrics.MetricsUtil;
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.Random;
|
||||||
import java.util.Collection;
|
import java.util.Set;
|
||||||
import java.util.List;
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Task for invoking the ShardRecordProcessor shutdown() callback.
|
* Task for invoking the ShardRecordProcessor shutdown() callback.
|
||||||
|
|
@ -55,6 +66,8 @@ import java.util.List;
|
||||||
public class ShutdownTask implements ConsumerTask {
|
public class ShutdownTask implements ConsumerTask {
|
||||||
private static final String SHUTDOWN_TASK_OPERATION = "ShutdownTask";
|
private static final String SHUTDOWN_TASK_OPERATION = "ShutdownTask";
|
||||||
private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown";
|
private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown";
|
||||||
|
@VisibleForTesting
|
||||||
|
static final int RETRY_RANDOM_MAX_RANGE = 30;
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
private final ShardInfo shardInfo;
|
private final ShardInfo shardInfo;
|
||||||
|
|
@ -82,6 +95,14 @@ public class ShutdownTask implements ConsumerTask {
|
||||||
|
|
||||||
private final TaskType taskType = TaskType.SHUTDOWN;
|
private final TaskType taskType = TaskType.SHUTDOWN;
|
||||||
|
|
||||||
|
private final List<ChildShard> childShards;
|
||||||
|
@NonNull
|
||||||
|
private final StreamIdentifier streamIdentifier;
|
||||||
|
@NonNull
|
||||||
|
private final LeaseCleanupManager leaseCleanupManager;
|
||||||
|
|
||||||
|
private static final Function<ShardInfo, String> leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Invokes ShardRecordProcessor shutdown() API.
|
* Invokes ShardRecordProcessor shutdown() API.
|
||||||
* (non-Javadoc)
|
* (non-Javadoc)
|
||||||
|
|
@ -94,84 +115,48 @@ public class ShutdownTask implements ConsumerTask {
|
||||||
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHUTDOWN_TASK_OPERATION);
|
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHUTDOWN_TASK_OPERATION);
|
||||||
|
|
||||||
Exception exception;
|
Exception exception;
|
||||||
boolean applicationException = false;
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
try {
|
try {
|
||||||
ShutdownReason localReason = reason;
|
log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}",
|
||||||
List<Shard> latestShards = null;
|
leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason);
|
||||||
/*
|
|
||||||
* Revalidate if the current shard is closed before shutting down the shard consumer with reason SHARD_END
|
|
||||||
* If current shard is not closed, shut down the shard consumer with reason LEASE_LOST that allows
|
|
||||||
* active workers to contend for the lease of this shard.
|
|
||||||
*/
|
|
||||||
if (localReason == ShutdownReason.SHARD_END) {
|
|
||||||
latestShards = shardDetector.listShards();
|
|
||||||
|
|
||||||
//If latestShards is empty, should also shutdown the ShardConsumer without checkpoint with SHARD_END
|
|
||||||
if (CollectionUtils.isNullOrEmpty(latestShards) || !isShardInContextParentOfAny(latestShards)) {
|
|
||||||
localReason = ShutdownReason.LEASE_LOST;
|
|
||||||
dropLease();
|
|
||||||
log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.shardId());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If we reached end of the shard, set sequence number to SHARD_END.
|
|
||||||
if (localReason == ShutdownReason.SHARD_END) {
|
|
||||||
recordProcessorCheckpointer
|
|
||||||
.sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue());
|
|
||||||
recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END);
|
|
||||||
}
|
|
||||||
|
|
||||||
log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}",
|
|
||||||
shardInfo.shardId(), shardInfo.concurrencyToken(), localReason);
|
|
||||||
final ShutdownInput shutdownInput = ShutdownInput.builder().shutdownReason(localReason)
|
|
||||||
.checkpointer(recordProcessorCheckpointer).build();
|
|
||||||
final long startTime = System.currentTimeMillis();
|
final long startTime = System.currentTimeMillis();
|
||||||
|
final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo));
|
||||||
|
final Runnable leaseLostAction = () -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build());
|
||||||
|
|
||||||
|
if (reason == ShutdownReason.SHARD_END) {
|
||||||
try {
|
try {
|
||||||
if (localReason == ShutdownReason.SHARD_END) {
|
takeShardEndAction(currentShardLease, scope, startTime);
|
||||||
shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
|
} catch (InvalidStateException e) {
|
||||||
ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue();
|
// If InvalidStateException happens, it indicates we have a non recoverable error in short term.
|
||||||
if (lastCheckpointValue == null
|
// In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry shutting down.
|
||||||
|| !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) {
|
log.warn("Lease {}: Invalid state encountered while shutting down shardConsumer with SHARD_END reason. " +
|
||||||
throw new IllegalArgumentException("Application didn't checkpoint at end of shard "
|
"Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", leaseKeyProvider.apply(shardInfo), e);
|
||||||
+ shardInfo.shardId() + ". Application must checkpoint upon shard end. " +
|
dropLease(currentShardLease);
|
||||||
"See ShardRecordProcessor.shardEnded javadocs for more information.");
|
throwOnApplicationException(leaseLostAction, scope, startTime);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
shardRecordProcessor.leaseLost(LeaseLostInput.builder().build());
|
throwOnApplicationException(leaseLostAction, scope, startTime);
|
||||||
}
|
|
||||||
log.debug("Shutting down retrieval strategy.");
|
|
||||||
recordsPublisher.shutdown();
|
|
||||||
log.debug("Record processor completed shutdown() for shard {}", shardInfo.shardId());
|
|
||||||
} catch (Exception e) {
|
|
||||||
applicationException = true;
|
|
||||||
throw e;
|
|
||||||
} finally {
|
|
||||||
MetricsUtil.addLatency(scope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (localReason == ShutdownReason.SHARD_END) {
|
log.debug("Shutting down retrieval strategy for shard {}.", leaseKeyProvider.apply(shardInfo));
|
||||||
log.debug("Looking for child shards of shard {}", shardInfo.shardId());
|
recordsPublisher.shutdown();
|
||||||
// create leases for the child shards
|
log.debug("Record processor completed shutdown() for shard {}", leaseKeyProvider.apply(shardInfo));
|
||||||
hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(),
|
|
||||||
initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards);
|
|
||||||
log.debug("Finished checking for child shards of shard {}", shardInfo.shardId());
|
|
||||||
}
|
|
||||||
|
|
||||||
return new TaskResult(null);
|
return new TaskResult(null);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
if (applicationException) {
|
if (e instanceof CustomerApplicationException) {
|
||||||
log.error("Application exception. ", e);
|
log.error("Shard {}: Application exception. ", leaseKeyProvider.apply(shardInfo), e);
|
||||||
} else {
|
} else {
|
||||||
log.error("Caught exception: ", e);
|
log.error("Shard {}: Caught exception: ", leaseKeyProvider.apply(shardInfo), e);
|
||||||
}
|
}
|
||||||
exception = e;
|
exception = e;
|
||||||
// backoff if we encounter an exception.
|
// backoff if we encounter an exception.
|
||||||
try {
|
try {
|
||||||
Thread.sleep(this.backoffTimeMillis);
|
Thread.sleep(this.backoffTimeMillis);
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
log.debug("Interrupted sleep", ie);
|
log.debug("Shard {}: Interrupted sleep", leaseKeyProvider.apply(shardInfo), ie);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
|
@ -179,7 +164,137 @@ public class ShutdownTask implements ConsumerTask {
|
||||||
}
|
}
|
||||||
|
|
||||||
return new TaskResult(exception);
|
return new TaskResult(exception);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup.
|
||||||
|
private void takeShardEndAction(Lease currentShardLease,
|
||||||
|
MetricsScope scope, long startTime)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException,
|
||||||
|
CustomerApplicationException {
|
||||||
|
// Create new lease for the child shards if they don't exist.
|
||||||
|
// We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards.
|
||||||
|
// This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception.
|
||||||
|
// In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards.
|
||||||
|
// This scenario could happen when customer deletes the stream while leaving the KCL application running.
|
||||||
|
if (currentShardLease == null) {
|
||||||
|
throw new InvalidStateException(leaseKeyProvider.apply(shardInfo)
|
||||||
|
+ " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.");
|
||||||
|
}
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(childShards)) {
|
||||||
|
createLeasesForChildShardsIfNotExist();
|
||||||
|
updateLeaseWithChildShards(currentShardLease);
|
||||||
|
}
|
||||||
|
final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease,
|
||||||
|
shardInfo, shardDetector);
|
||||||
|
if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) {
|
||||||
|
boolean isSuccess = false;
|
||||||
|
try {
|
||||||
|
isSuccess = attemptShardEndCheckpointing(scope, startTime);
|
||||||
|
} finally {
|
||||||
|
// Check if either the shard end ddb persist is successful or
|
||||||
|
// if childshards is empty. When child shards is empty then either it is due to
|
||||||
|
// completed shard being reprocessed or we got RNF from service.
|
||||||
|
// For these cases enqueue the lease for deletion.
|
||||||
|
if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) {
|
||||||
|
leaseCleanupManager.enqueueForDeletion(leasePendingDeletion);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException,
|
||||||
|
CustomerApplicationException {
|
||||||
|
final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo)))
|
||||||
|
.orElseThrow(() -> new InvalidStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist."));
|
||||||
|
if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) {
|
||||||
|
// Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number.
|
||||||
|
// The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded.
|
||||||
|
throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime);
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void applicationCheckpointAndVerification() {
|
||||||
|
recordProcessorCheckpointer
|
||||||
|
.sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue());
|
||||||
|
recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
|
||||||
|
final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue();
|
||||||
|
if (lastCheckpointValue == null
|
||||||
|
|| !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) {
|
||||||
|
throw new IllegalArgumentException("Application didn't checkpoint at end of shard "
|
||||||
|
+ leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " +
|
||||||
|
"See ShardRecordProcessor.shardEnded javadocs for more information.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void throwOnApplicationException(Runnable action, MetricsScope metricsScope, final long startTime) throws CustomerApplicationException {
|
||||||
|
try {
|
||||||
|
action.run();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new CustomerApplicationException("Customer application throws exception for shard " + leaseKeyProvider.apply(shardInfo) +": ", e);
|
||||||
|
} finally {
|
||||||
|
MetricsUtil.addLatency(metricsScope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createLeasesForChildShardsIfNotExist()
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
// For child shard resulted from merge of two parent shards, verify if both the parents are either present or
|
||||||
|
// not present in the lease table before creating the lease entry.
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(childShards) && childShards.size() == 1) {
|
||||||
|
final ChildShard childShard = childShards.get(0);
|
||||||
|
final List<String> parentLeaseKeys = childShard.parentShards().stream()
|
||||||
|
.map(parentShardId -> ShardInfo.getLeaseKey(shardInfo, parentShardId)).collect(Collectors.toList());
|
||||||
|
if (parentLeaseKeys.size() != 2) {
|
||||||
|
throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard
|
||||||
|
+ " does not contain other parent information.");
|
||||||
|
} else {
|
||||||
|
boolean isValidLeaseTableState =
|
||||||
|
Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects
|
||||||
|
.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1)));
|
||||||
|
if (!isValidLeaseTableState) {
|
||||||
|
if (!isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) {
|
||||||
|
throw new BlockedOnParentShardException(
|
||||||
|
"Shard " + shardInfo.shardId() + "'s only child shard " + childShard
|
||||||
|
+ " has partial parent information in lease table. Hence deferring lease creation of child shard.");
|
||||||
|
} else {
|
||||||
|
throw new InvalidStateException(
|
||||||
|
"Shard " + shardInfo.shardId() + "'s only child shard " + childShard
|
||||||
|
+ " has partial parent information in lease table. Hence deferring lease creation of child shard.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Attempt create leases for child shards.
|
||||||
|
for(ChildShard childShard : childShards) {
|
||||||
|
final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId());
|
||||||
|
if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) {
|
||||||
|
final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier());
|
||||||
|
leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate);
|
||||||
|
log.info("Shard {}: Created child shard lease: {}", shardInfo.shardId(), leaseToCreate.leaseKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true for 1 in N probability.
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
boolean isOneInNProbability(int n) {
|
||||||
|
Random r = new Random();
|
||||||
|
return 1 == r.nextInt((n - 1) + 1) + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateLeaseWithChildShards(Lease currentLease)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
Set<String> childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet());
|
||||||
|
|
||||||
|
final Lease updatedLease = currentLease.copy();
|
||||||
|
updatedLease.childShardIds(childShardIds);
|
||||||
|
leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS);
|
||||||
|
log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
@ -197,26 +312,13 @@ public class ShutdownTask implements ConsumerTask {
|
||||||
return reason;
|
return reason;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isShardInContextParentOfAny(List<Shard> shards) {
|
private void dropLease(Lease currentLease) {
|
||||||
for(Shard shard : shards) {
|
if (currentLease == null) {
|
||||||
if (isChildShardOfShardInContext(shard)) {
|
log.warn("Shard {}: Unable to find the lease for shard. Will shutdown the shardConsumer directly.", leaseKeyProvider.apply(shardInfo));
|
||||||
return true;
|
return;
|
||||||
}
|
} else {
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean isChildShardOfShardInContext(Shard shard) {
|
|
||||||
return (StringUtils.equals(shard.parentShardId(), shardInfo.shardId())
|
|
||||||
|| StringUtils.equals(shard.adjacentParentShardId(), shardInfo.shardId()));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void dropLease() {
|
|
||||||
Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId());
|
|
||||||
leaseCoordinator.dropLease(currentLease);
|
leaseCoordinator.dropLease(currentLease);
|
||||||
if(currentLease != null) {
|
log.info("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey());
|
||||||
log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -47,4 +47,12 @@ public class InitializationInput {
|
||||||
* completing the checkpoint.
|
* completing the checkpoint.
|
||||||
*/
|
*/
|
||||||
private final ExtendedSequenceNumber pendingCheckpointSequenceNumber;
|
private final ExtendedSequenceNumber pendingCheckpointSequenceNumber;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The last pending application state of the previous record processor. May be null.
|
||||||
|
*
|
||||||
|
* This will only be set if the previous record processor had prepared a checkpoint, but lost its lease before
|
||||||
|
* completing the checkpoint.
|
||||||
|
*/
|
||||||
|
private final byte[] pendingCheckpointState;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import lombok.EqualsAndHashCode;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.ToString;
|
import lombok.ToString;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
||||||
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
|
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
|
||||||
import software.amazon.kinesis.retrieval.KinesisClientRecord;
|
import software.amazon.kinesis.retrieval.KinesisClientRecord;
|
||||||
|
|
@ -56,6 +57,7 @@ public class ProcessRecordsInput {
|
||||||
* The records received from Kinesis. These records may have been de-aggregated if they were published by the KPL.
|
* The records received from Kinesis. These records may have been de-aggregated if they were published by the KPL.
|
||||||
*/
|
*/
|
||||||
private List<KinesisClientRecord> records;
|
private List<KinesisClientRecord> records;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A checkpointer that the {@link ShardRecordProcessor} can use to checkpoint its progress.
|
* A checkpointer that the {@link ShardRecordProcessor} can use to checkpoint its progress.
|
||||||
*/
|
*/
|
||||||
|
|
@ -66,6 +68,11 @@ public class ProcessRecordsInput {
|
||||||
* This value does not include the {@link #timeSpentInCache()}.
|
* This value does not include the {@link #timeSpentInCache()}.
|
||||||
*/
|
*/
|
||||||
private Long millisBehindLatest;
|
private Long millisBehindLatest;
|
||||||
|
/**
|
||||||
|
* A list of child shards if the current GetRecords request reached the shard end.
|
||||||
|
* If not at the shard end, this should be an empty list.
|
||||||
|
*/
|
||||||
|
private List<ChildShard> childShards;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* How long the records spent waiting to be dispatched to the {@link ShardRecordProcessor}
|
* How long the records spent waiting to be dispatched to the {@link ShardRecordProcessor}
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,7 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
|
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
|
@ -26,6 +27,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
public class MetricsUtil {
|
public class MetricsUtil {
|
||||||
public static final String OPERATION_DIMENSION_NAME = "Operation";
|
public static final String OPERATION_DIMENSION_NAME = "Operation";
|
||||||
public static final String SHARD_ID_DIMENSION_NAME = "ShardId";
|
public static final String SHARD_ID_DIMENSION_NAME = "ShardId";
|
||||||
|
public static final String STREAM_IDENTIFIER = "StreamId";
|
||||||
private static final String WORKER_IDENTIFIER_DIMENSION = "WorkerIdentifier";
|
private static final String WORKER_IDENTIFIER_DIMENSION = "WorkerIdentifier";
|
||||||
private static final String TIME_METRIC = "Time";
|
private static final String TIME_METRIC = "Time";
|
||||||
private static final String SUCCESS_METRIC = "Success";
|
private static final String SUCCESS_METRIC = "Success";
|
||||||
|
|
@ -51,6 +53,11 @@ public class MetricsUtil {
|
||||||
addOperation(metricsScope, SHARD_ID_DIMENSION_NAME, shardId);
|
addOperation(metricsScope, SHARD_ID_DIMENSION_NAME, shardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void addStreamId(@NonNull final MetricsScope metricsScope, @NonNull final StreamIdentifier streamId) {
|
||||||
|
streamId.accountIdOptional()
|
||||||
|
.ifPresent(acc -> addOperation(metricsScope, STREAM_IDENTIFIER, streamId.serialize()));
|
||||||
|
}
|
||||||
|
|
||||||
public static void addWorkerIdentifier(@NonNull final MetricsScope metricsScope,
|
public static void addWorkerIdentifier(@NonNull final MetricsScope metricsScope,
|
||||||
@NonNull final String workerIdentifier) {
|
@NonNull final String workerIdentifier) {
|
||||||
addOperation(metricsScope, WORKER_IDENTIFIER_DIMENSION, workerIdentifier);
|
addOperation(metricsScope, WORKER_IDENTIFIER_DIMENSION, workerIdentifier);
|
||||||
|
|
@ -87,6 +94,11 @@ public class MetricsUtil {
|
||||||
metricsScope.addData(metricName, success ? 1 : 0, StandardUnit.COUNT, metricsLevel);
|
metricsScope.addData(metricName, success ? 1 : 0, StandardUnit.COUNT, metricsLevel);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void addCount(@NonNull final MetricsScope metricsScope, final String dimension,
|
||||||
|
final long count, @NonNull final MetricsLevel metricsLevel) {
|
||||||
|
metricsScope.addData(dimension, count, StandardUnit.COUNT, metricsLevel);
|
||||||
|
}
|
||||||
|
|
||||||
public static void endScope(@NonNull final MetricsScope metricsScope) {
|
public static void endScope(@NonNull final MetricsScope metricsScope) {
|
||||||
metricsScope.end();
|
metricsScope.end();
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -27,50 +27,66 @@ public interface Checkpointer {
|
||||||
* Record a checkpoint for a shard (e.g. sequence and subsequence numbers of last record processed
|
* Record a checkpoint for a shard (e.g. sequence and subsequence numbers of last record processed
|
||||||
* by application). Upon failover, record processing is resumed from this point.
|
* by application). Upon failover, record processing is resumed from this point.
|
||||||
*
|
*
|
||||||
* @param shardId Checkpoint is specified for this shard.
|
* @param leaseKey Checkpoint is specified for this shard.
|
||||||
* @param checkpointValue Value of the checkpoint (e.g. Kinesis sequence number and subsequence number)
|
* @param checkpointValue Value of the checkpoint (e.g. Kinesis sequence number and subsequence number)
|
||||||
* @param concurrencyToken Used with conditional writes to prevent stale updates
|
* @param concurrencyToken Used with conditional writes to prevent stale updates
|
||||||
* (e.g. if there was a fail over to a different record processor, we don't want to
|
* (e.g. if there was a fail over to a different record processor, we don't want to
|
||||||
* overwrite it's checkpoint)
|
* overwrite it's checkpoint)
|
||||||
* @throws KinesisClientLibException Thrown if we were unable to save the checkpoint
|
* @throws KinesisClientLibException Thrown if we were unable to save the checkpoint
|
||||||
*/
|
*/
|
||||||
void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken)
|
void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken)
|
||||||
throws KinesisClientLibException;
|
throws KinesisClientLibException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the current checkpoint stored for the specified shard. Useful for checking that the parent shard
|
* Get the current checkpoint stored for the specified shard. Useful for checking that the parent shard
|
||||||
* has been completely processed before we start processing the child shard.
|
* has been completely processed before we start processing the child shard.
|
||||||
*
|
*
|
||||||
* @param shardId Current checkpoint for this shard is fetched
|
* @param leaseKey Current checkpoint for this shard is fetched
|
||||||
* @return Current checkpoint for this shard, null if there is no record for this shard.
|
* @return Current checkpoint for this shard, null if there is no record for this shard.
|
||||||
* @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint
|
* @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint
|
||||||
*/
|
*/
|
||||||
ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException;
|
ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the current checkpoint stored for the specified shard, which holds the sequence numbers for the checkpoint
|
* Get the current checkpoint stored for the specified shard, which holds the sequence numbers for the checkpoint
|
||||||
* and pending checkpoint. Useful for checking that the parent shard has been completely processed before we start
|
* and pending checkpoint. Useful for checking that the parent shard has been completely processed before we start
|
||||||
* processing the child shard.
|
* processing the child shard.
|
||||||
*
|
*
|
||||||
* @param shardId Current checkpoint for this shard is fetched
|
* @param leaseKey Current checkpoint for this shard is fetched
|
||||||
* @return Current checkpoint object for this shard, null if there is no record for this shard.
|
* @return Current checkpoint object for this shard, null if there is no record for this shard.
|
||||||
* @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint
|
* @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint
|
||||||
*/
|
*/
|
||||||
Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException;
|
Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Record intent to checkpoint for a shard. Upon failover, the pendingCheckpointValue will be passed to the new
|
* Record intent to checkpoint for a shard. Upon failover, the pendingCheckpointValue will be passed to the new
|
||||||
* ShardRecordProcessor's initialize() method.
|
* ShardRecordProcessor's initialize() method.
|
||||||
*
|
*
|
||||||
* @param shardId Checkpoint is specified for this shard.
|
* @param leaseKey Checkpoint is specified for this shard.
|
||||||
* @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number)
|
* @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number)
|
||||||
* @param concurrencyToken Used with conditional writes to prevent stale updates
|
* @param concurrencyToken Used with conditional writes to prevent stale updates
|
||||||
* (e.g. if there was a fail over to a different record processor, we don't want to
|
* (e.g. if there was a fail over to a different record processor, we don't want to
|
||||||
* overwrite it's checkpoint)
|
* overwrite it's checkpoint)
|
||||||
* @throws KinesisClientLibException Thrown if we were unable to save the checkpoint
|
* @throws KinesisClientLibException Thrown if we were unable to save the checkpoint
|
||||||
*/
|
*/
|
||||||
void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken)
|
void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken)
|
||||||
|
throws KinesisClientLibException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record intent to checkpoint for a shard. Upon failover, the pendingCheckpoint and pendingCheckpointState will be
|
||||||
|
* passed to the new ShardRecordProcessor's initialize() method.
|
||||||
|
*
|
||||||
|
* @param leaseKey Checkpoint is specified for this shard.
|
||||||
|
* @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number)
|
||||||
|
* @param concurrencyToken Used with conditional writes to prevent stale updates
|
||||||
|
* (e.g. if there was a fail over to a different record processor, we don't want to
|
||||||
|
* overwrite it's checkpoint)
|
||||||
|
* @param pendingCheckpointState Serialized application state at the pending checkpoint.
|
||||||
|
*
|
||||||
|
* @throws KinesisClientLibException Thrown if we were unable to save the checkpoint
|
||||||
|
*/
|
||||||
|
void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState)
|
||||||
throws KinesisClientLibException;
|
throws KinesisClientLibException;
|
||||||
|
|
||||||
void operation(String operation);
|
void operation(String operation);
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,109 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.processor;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy for cleaning up the leases for former streams.
|
||||||
|
*/
|
||||||
|
public interface FormerStreamsLeasesDeletionStrategy {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* StreamIdentifiers for which leases needs to be cleaned up in the lease table.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
List<StreamIdentifier> streamIdentifiersForLeaseCleanup();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Duration to wait before deleting the leases for this stream.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
Duration waitPeriodToDeleteFormerStreams();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy type for deleting the leases of former active streams.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
StreamsLeasesDeletionType leaseDeletionType();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* StreamsLeasesDeletionType identifying the different lease cleanup strategies.
|
||||||
|
*/
|
||||||
|
enum StreamsLeasesDeletionType {
|
||||||
|
NO_STREAMS_LEASES_DELETION,
|
||||||
|
FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION,
|
||||||
|
PROVIDED_STREAMS_DEFERRED_DELETION
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy for not cleaning up leases for former streams.
|
||||||
|
*/
|
||||||
|
final class NoLeaseDeletionStrategy implements FormerStreamsLeasesDeletionStrategy {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
throw new UnsupportedOperationException("StreamIdentifiers not required");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final StreamsLeasesDeletionType leaseDeletionType() {
|
||||||
|
return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy for auto detection the old of former streams based on the {@link MultiStreamTracker#streamConfigList()}
|
||||||
|
* and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()}
|
||||||
|
*/
|
||||||
|
abstract class AutoDetectionAndDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
throw new UnsupportedOperationException("StreamIdentifiers not required");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final StreamsLeasesDeletionType leaseDeletionType() {
|
||||||
|
return StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy to detect the streams for deletion through {@link #streamIdentifiersForLeaseCleanup()} provided by customer at runtime
|
||||||
|
* and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()}
|
||||||
|
*/
|
||||||
|
abstract class ProvidedStreamsDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final StreamsLeasesDeletionType leaseDeletionType() {
|
||||||
|
return StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -0,0 +1,45 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.processor;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Interface for stream trackers. This is useful for KCL Workers that need
|
||||||
|
* to consume data from multiple streams.
|
||||||
|
* KCL will periodically probe this interface to learn about the new and old streams.
|
||||||
|
*/
|
||||||
|
public interface MultiStreamTracker {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the list of stream config, to be processed by the current application.
|
||||||
|
* <b>Note that the streams list CAN be changed during the application runtime.</b>
|
||||||
|
* This method will be called periodically by the KCL to learn about the change in streams to process.
|
||||||
|
*
|
||||||
|
* @return List of StreamConfig
|
||||||
|
*/
|
||||||
|
List<StreamConfig> streamConfigList();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy to delete leases of old streams in the lease table.
|
||||||
|
* <b>Note that the strategy CANNOT be changed during the application runtime.</b>
|
||||||
|
*
|
||||||
|
* @return StreamsLeasesDeletionStrategy
|
||||||
|
*/
|
||||||
|
FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy();
|
||||||
|
}
|
||||||
|
|
@ -93,7 +93,6 @@ public interface RecordProcessorCheckpointer {
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||||
IllegalArgumentException;
|
IllegalArgumentException;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will checkpoint the progress at the provided sequenceNumber and subSequenceNumber, the latter for
|
* This method will checkpoint the progress at the provided sequenceNumber and subSequenceNumber, the latter for
|
||||||
* aggregated records produced with the Producer Library. This method is analogous to {@link #checkpoint()}
|
* aggregated records produced with the Producer Library. This method is analogous to {@link #checkpoint()}
|
||||||
|
|
@ -145,6 +144,32 @@ public interface RecordProcessorCheckpointer {
|
||||||
PreparedCheckpointer prepareCheckpoint()
|
PreparedCheckpointer prepareCheckpoint()
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method will record a pending checkpoint at the last data record that was delivered to the record processor.
|
||||||
|
* If the application fails over between calling prepareCheckpoint() and checkpoint(), the init() method of the next
|
||||||
|
* IRecordProcessor for this shard will be informed of the prepared sequence number and application state.
|
||||||
|
*
|
||||||
|
* Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having
|
||||||
|
* side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete.
|
||||||
|
* Use the sequence number and application state passed in to init() to behave idempotently.
|
||||||
|
*
|
||||||
|
* @param applicationState arbitrary application state that will be passed to the next record processor that
|
||||||
|
* processes the shard.
|
||||||
|
* @return an PreparedCheckpointer object that can be called later to persist the checkpoint.
|
||||||
|
*
|
||||||
|
* @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently.
|
||||||
|
* Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency.
|
||||||
|
* @throws ShutdownException The record processor instance has been shutdown. Another instance may have
|
||||||
|
* started processing some of these records already.
|
||||||
|
* The application should abort processing via this ShardRecordProcessor instance.
|
||||||
|
* @throws InvalidStateException Can't store pending checkpoint.
|
||||||
|
* Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist).
|
||||||
|
* @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The
|
||||||
|
* application can backoff and retry.
|
||||||
|
*/
|
||||||
|
PreparedCheckpointer prepareCheckpoint(byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will record a pending checkpoint at the at the provided record. This method is analogous to
|
* This method will record a pending checkpoint at the at the provided record. This method is analogous to
|
||||||
* {@link #prepareCheckpoint()} but provides the ability to specify the record at which to prepare the checkpoint.
|
* {@link #prepareCheckpoint()} but provides the ability to specify the record at which to prepare the checkpoint.
|
||||||
|
|
@ -174,6 +199,38 @@ public interface RecordProcessorCheckpointer {
|
||||||
PreparedCheckpointer prepareCheckpoint(Record record)
|
PreparedCheckpointer prepareCheckpoint(Record record)
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method will record a pending checkpoint at the at the provided record. This method is analogous to
|
||||||
|
* {@link #prepareCheckpoint()} but provides the ability to specify the record and application state at which to
|
||||||
|
* prepare the checkpoint.
|
||||||
|
*
|
||||||
|
* @param record A record at which to prepare checkpoint in this shard.
|
||||||
|
* @param applicationState arbitrary application state that will be passed to the next record processor that
|
||||||
|
* processes the shard.
|
||||||
|
*
|
||||||
|
* Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having
|
||||||
|
* side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete.
|
||||||
|
* Use the sequence number and application state passed in to init() to behave idempotently.
|
||||||
|
*
|
||||||
|
* @return an PreparedCheckpointer object that can be called later to persist the checkpoint.
|
||||||
|
*
|
||||||
|
* @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently.
|
||||||
|
* Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency.
|
||||||
|
* @throws ShutdownException The record processor instance has been shutdown. Another instance may have
|
||||||
|
* started processing some of these records already.
|
||||||
|
* The application should abort processing via this ShardRecordProcessor instance.
|
||||||
|
* @throws InvalidStateException Can't store pending checkpoint.
|
||||||
|
* Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist).
|
||||||
|
* @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The
|
||||||
|
* application can backoff and retry.
|
||||||
|
* @throws IllegalArgumentException The sequence number is invalid for one of the following reasons:
|
||||||
|
* 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the
|
||||||
|
* greatest sequence number seen by the associated record processor.
|
||||||
|
* 2.) It is not a valid sequence number for a record in this shard.
|
||||||
|
*/
|
||||||
|
PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to
|
* This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to
|
||||||
* {@link #prepareCheckpoint()} but provides the ability to specify the sequence number at which to checkpoint.
|
* {@link #prepareCheckpoint()} but provides the ability to specify the sequence number at which to checkpoint.
|
||||||
|
|
@ -200,6 +257,35 @@ public interface RecordProcessorCheckpointer {
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||||
IllegalArgumentException;
|
IllegalArgumentException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to
|
||||||
|
* {@link #prepareCheckpoint()} but provides the ability to specify the sequence number and application state
|
||||||
|
* at which to checkpoint.
|
||||||
|
*
|
||||||
|
* @param sequenceNumber A sequence number at which to prepare checkpoint in this shard.
|
||||||
|
* @param applicationState arbitrary application state that will be passed to the next record processor that
|
||||||
|
* processes the shard.
|
||||||
|
*
|
||||||
|
* @return an PreparedCheckpointer object that can be called later to persist the checkpoint.
|
||||||
|
*
|
||||||
|
* @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently.
|
||||||
|
* Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency.
|
||||||
|
* @throws ShutdownException The record processor instance has been shutdown. Another instance may have
|
||||||
|
* started processing some of these records already.
|
||||||
|
* The application should abort processing via this ShardRecordProcessor instance.
|
||||||
|
* @throws InvalidStateException Can't store pending checkpoint.
|
||||||
|
* Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist).
|
||||||
|
* @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The
|
||||||
|
* application can backoff and retry.
|
||||||
|
* @throws IllegalArgumentException The sequence number is invalid for one of the following reasons:
|
||||||
|
* 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the
|
||||||
|
* greatest sequence number seen by the associated record processor.
|
||||||
|
* 2.) It is not a valid sequence number for a record in this shard.
|
||||||
|
*/
|
||||||
|
PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||||
|
IllegalArgumentException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for
|
* This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for
|
||||||
* aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()}
|
* aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()}
|
||||||
|
|
@ -228,5 +314,36 @@ public interface RecordProcessorCheckpointer {
|
||||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||||
IllegalArgumentException;
|
IllegalArgumentException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for
|
||||||
|
* aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()}
|
||||||
|
* but provides the ability to specify the sequence number, subsequence number, and application state at which to
|
||||||
|
* checkpoint.
|
||||||
|
*
|
||||||
|
* @param sequenceNumber A sequence number at which to prepare checkpoint in this shard.
|
||||||
|
* @param subSequenceNumber A subsequence number at which to prepare checkpoint within this shard.
|
||||||
|
* @param applicationState arbitrary application state that will be passed to the next record processor that
|
||||||
|
* processes the shard.
|
||||||
|
*
|
||||||
|
* @return an PreparedCheckpointer object that can be called later to persist the checkpoint.
|
||||||
|
*
|
||||||
|
* @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently.
|
||||||
|
* Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency.
|
||||||
|
* @throws ShutdownException The record processor instance has been shutdown. Another instance may have
|
||||||
|
* started processing some of these records already.
|
||||||
|
* The application should abort processing via this ShardRecordProcessor instance.
|
||||||
|
* @throws InvalidStateException Can't store pending checkpoint.
|
||||||
|
* Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist).
|
||||||
|
* @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The
|
||||||
|
* application can backoff and retry.
|
||||||
|
* @throws IllegalArgumentException The sequence number is invalid for one of the following reasons:
|
||||||
|
* 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the
|
||||||
|
* greatest sequence number seen by the associated record processor.
|
||||||
|
* 2.) It is not a valid sequence number for a record in this shard.
|
||||||
|
*/
|
||||||
|
PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState)
|
||||||
|
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||||
|
IllegalArgumentException;
|
||||||
|
|
||||||
Checkpointer checkpointer();
|
Checkpointer checkpointer();
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,8 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.processor;
|
package software.amazon.kinesis.processor;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
|
@ -25,4 +27,13 @@ public interface ShardRecordProcessorFactory {
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
ShardRecordProcessor shardRecordProcessor();
|
ShardRecordProcessor shardRecordProcessor();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a new instance of the ShardRecordProcessor for a stream identifier
|
||||||
|
* @param streamIdentifier
|
||||||
|
* @return ShardRecordProcessor
|
||||||
|
*/
|
||||||
|
default ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) {
|
||||||
|
return shardRecordProcessor();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,48 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
||||||
|
public interface DataFetcherProviderConfig {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets stream identifier for dataFetcher.
|
||||||
|
*/
|
||||||
|
StreamIdentifier getStreamIdentifier();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets shard id.
|
||||||
|
*/
|
||||||
|
String getShardId();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets current instance of metrics factory.
|
||||||
|
*/
|
||||||
|
MetricsFactory getMetricsFactory();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets current max records allowed to process at a given time.
|
||||||
|
*/
|
||||||
|
Integer getMaxRecords();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets timeout for kinesis request.
|
||||||
|
*/
|
||||||
|
Duration getKinesisRequestTimeout();
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,47 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class DataRetrievalUtil {
|
||||||
|
|
||||||
|
public static boolean isValidResult(String shardEndIndicator, List<ChildShard> childShards) {
|
||||||
|
// shardEndIndicator is nextShardIterator for GetRecordsResponse, and is continuationSequenceNumber for SubscribeToShardEvent
|
||||||
|
// There are two valid scenarios for the shardEndIndicator and childShards combination.
|
||||||
|
// 1. ShardEnd scenario: shardEndIndicator should be null and childShards should be a non-empty list.
|
||||||
|
// 2. Non-ShardEnd scenario: shardEndIndicator should be non-null and childShards should be null or an empty list.
|
||||||
|
// Otherwise, the retrieval result is invalid.
|
||||||
|
if (shardEndIndicator == null && CollectionUtils.isNullOrEmpty(childShards) ||
|
||||||
|
shardEndIndicator != null && !CollectionUtils.isNullOrEmpty(childShards)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// For ShardEnd scenario, for each childShard we should validate if parentShards are available.
|
||||||
|
// Missing parentShards can cause issues with creating leases for childShards during ShardConsumer shutdown.
|
||||||
|
if (!CollectionUtils.isNullOrEmpty(childShards)) {
|
||||||
|
for (ChildShard childShard : childShards) {
|
||||||
|
if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -14,7 +14,9 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.retrieval;
|
package software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
|
import software.amazon.kinesis.retrieval.polling.DataFetcher;
|
||||||
import software.amazon.kinesis.retrieval.polling.KinesisDataFetcher;
|
import software.amazon.kinesis.retrieval.polling.KinesisDataFetcher;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -47,9 +49,27 @@ public interface GetRecordsRetrievalStrategy {
|
||||||
boolean isShutdown();
|
boolean isShutdown();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the KinesisDataFetcher used to records from Kinesis.
|
* Returns a DataFetcher used to records from Kinesis.
|
||||||
*
|
*
|
||||||
* @return KinesisDataFetcher
|
* @return DataFetcher
|
||||||
*/
|
*/
|
||||||
KinesisDataFetcher getDataFetcher();
|
KinesisDataFetcher getDataFetcher();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a DataFetcher override if applicable, else empty for retrieving records from Kinesis.
|
||||||
|
*
|
||||||
|
* @return Optional<DataFetcher>
|
||||||
|
*/
|
||||||
|
default Optional<DataFetcher> getDataFetcherOverride() {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a dataFetcher by first checking for an override if it exists, else using the default data fetcher.
|
||||||
|
*
|
||||||
|
* @return DataFetcher
|
||||||
|
*/
|
||||||
|
default DataFetcher dataFetcher() {
|
||||||
|
return getDataFetcherOverride().orElse(getDataFetcher());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,45 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import lombok.Data;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configuration needed for custom data fetchers
|
||||||
|
*/
|
||||||
|
@Data
|
||||||
|
public class KinesisDataFetcherProviderConfig implements DataFetcherProviderConfig {
|
||||||
|
|
||||||
|
@NonNull
|
||||||
|
private StreamIdentifier streamIdentifier;
|
||||||
|
|
||||||
|
@NonNull
|
||||||
|
private String shardId;
|
||||||
|
|
||||||
|
@NonNull
|
||||||
|
private MetricsFactory metricsFactory;
|
||||||
|
|
||||||
|
@NonNull
|
||||||
|
private Integer maxRecords;
|
||||||
|
|
||||||
|
@NonNull
|
||||||
|
private Duration kinesisRequestTimeout;
|
||||||
|
}
|
||||||
|
|
@ -15,18 +15,29 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.retrieval;
|
package software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
import lombok.Data;
|
import lombok.EqualsAndHashCode;
|
||||||
|
import lombok.Getter;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
|
import lombok.Setter;
|
||||||
|
import lombok.ToString;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
|
import software.amazon.awssdk.utils.Either;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStream;
|
import software.amazon.kinesis.common.InitialPositionInStream;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.processor.MultiStreamTracker;
|
||||||
import software.amazon.kinesis.retrieval.fanout.FanOutConfig;
|
import software.amazon.kinesis.retrieval.fanout.FanOutConfig;
|
||||||
|
import software.amazon.kinesis.retrieval.polling.PollingConfig;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used by the KCL to configure the retrieval of records from Kinesis.
|
* Used by the KCL to configure the retrieval of records from Kinesis.
|
||||||
*/
|
*/
|
||||||
@Data
|
@Getter
|
||||||
|
@Setter
|
||||||
|
@ToString
|
||||||
|
@EqualsAndHashCode
|
||||||
@Accessors(fluent = true)
|
@Accessors(fluent = true)
|
||||||
public class RetrievalConfig {
|
public class RetrievalConfig {
|
||||||
/**
|
/**
|
||||||
|
|
@ -42,15 +53,15 @@ public class RetrievalConfig {
|
||||||
@NonNull
|
@NonNull
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
|
|
||||||
/**
|
|
||||||
* The name of the stream to process records from.
|
|
||||||
*/
|
|
||||||
@NonNull
|
|
||||||
private final String streamName;
|
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String applicationName;
|
private final String applicationName;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* AppStreamTracker either for multi stream tracking or single stream
|
||||||
|
*/
|
||||||
|
private Either<MultiStreamTracker, StreamConfig> appStreamTracker;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Backoff time between consecutive ListShards calls.
|
* Backoff time between consecutive ListShards calls.
|
||||||
*
|
*
|
||||||
|
|
@ -84,15 +95,77 @@ public class RetrievalConfig {
|
||||||
|
|
||||||
private RetrievalFactory retrievalFactory;
|
private RetrievalFactory retrievalFactory;
|
||||||
|
|
||||||
public RetrievalFactory retrievalFactory() {
|
public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName,
|
||||||
|
@NonNull String applicationName) {
|
||||||
|
this.kinesisClient = kinesisAsyncClient;
|
||||||
|
this.appStreamTracker = Either
|
||||||
|
.right(new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStreamExtended));
|
||||||
|
this.applicationName = applicationName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull MultiStreamTracker multiStreamTracker,
|
||||||
|
@NonNull String applicationName) {
|
||||||
|
this.kinesisClient = kinesisAsyncClient;
|
||||||
|
this.appStreamTracker = Either.left(multiStreamTracker);
|
||||||
|
this.applicationName = applicationName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) {
|
||||||
|
final StreamConfig[] streamConfig = new StreamConfig[1];
|
||||||
|
this.appStreamTracker.apply(multiStreamTracker -> {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Cannot set initialPositionInStreamExtended when multiStreamTracker is set");
|
||||||
|
}, sc -> streamConfig[0] = sc);
|
||||||
|
this.appStreamTracker = Either
|
||||||
|
.right(new StreamConfig(streamConfig[0].streamIdentifier(), initialPositionInStreamExtended));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetrievalConfig retrievalSpecificConfig(RetrievalSpecificConfig retrievalSpecificConfig) {
|
||||||
|
this.retrievalSpecificConfig = retrievalSpecificConfig;
|
||||||
|
validateFanoutConfig();
|
||||||
|
validatePollingConfig();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetrievalFactory retrievalFactory() {
|
||||||
if (retrievalFactory == null) {
|
if (retrievalFactory == null) {
|
||||||
if (retrievalSpecificConfig == null) {
|
if (retrievalSpecificConfig == null) {
|
||||||
retrievalSpecificConfig = new FanOutConfig(kinesisClient()).streamName(streamName())
|
retrievalSpecificConfig = new FanOutConfig(kinesisClient())
|
||||||
.applicationName(applicationName());
|
.applicationName(applicationName());
|
||||||
|
retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig,
|
||||||
|
streamConfig -> ((FanOutConfig) retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName()));
|
||||||
}
|
}
|
||||||
retrievalFactory = retrievalSpecificConfig.retrievalFactory();
|
retrievalFactory = retrievalSpecificConfig.retrievalFactory();
|
||||||
}
|
}
|
||||||
return retrievalFactory;
|
return retrievalFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void validateFanoutConfig() {
|
||||||
|
// If we are in multistream mode and if retrievalSpecificConfig is an instance of FanOutConfig and if consumerArn is set throw exception.
|
||||||
|
boolean isFanoutConfig = retrievalSpecificConfig instanceof FanOutConfig;
|
||||||
|
boolean isInvalidFanoutConfig = isFanoutConfig && appStreamTracker.map(
|
||||||
|
multiStreamTracker -> ((FanOutConfig) retrievalSpecificConfig).consumerArn() != null
|
||||||
|
|| ((FanOutConfig) retrievalSpecificConfig).streamName() != null,
|
||||||
|
streamConfig -> streamConfig.streamIdentifier() == null
|
||||||
|
|| streamConfig.streamIdentifier().streamName() == null);
|
||||||
|
if(isInvalidFanoutConfig) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Invalid config: Either in multi-stream mode with streamName/consumerArn configured or in single-stream mode with no streamName configured");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validatePollingConfig() {
|
||||||
|
boolean isPollingConfig = retrievalSpecificConfig instanceof PollingConfig;
|
||||||
|
boolean isInvalidPollingConfig = isPollingConfig && appStreamTracker.map(
|
||||||
|
multiStreamTracker ->
|
||||||
|
((PollingConfig) retrievalSpecificConfig).streamName() != null,
|
||||||
|
streamConfig ->
|
||||||
|
streamConfig.streamIdentifier() == null || streamConfig.streamIdentifier().streamName() == null);
|
||||||
|
|
||||||
|
if (isInvalidPollingConfig) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Invalid config: Either in multi-stream mode with streamName configured or in single-stream mode with no streamName configured");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.retrieval;
|
package software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
|
||||||
|
|
@ -24,5 +25,10 @@ import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
public interface RetrievalFactory {
|
public interface RetrievalFactory {
|
||||||
GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(ShardInfo shardInfo, MetricsFactory metricsFactory);
|
GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(ShardInfo shardInfo, MetricsFactory metricsFactory);
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory);
|
RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory);
|
||||||
|
|
||||||
|
default RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, StreamConfig streamConfig, MetricsFactory metricsFactory) {
|
||||||
|
return createGetRecordsCache(shardInfo, metricsFactory);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,9 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.retrieval;
|
package software.amazon.kinesis.retrieval;
|
||||||
|
|
||||||
|
import java.util.function.Function;
|
||||||
|
import software.amazon.kinesis.retrieval.polling.DataFetcher;
|
||||||
|
|
||||||
public interface RetrievalSpecificConfig {
|
public interface RetrievalSpecificConfig {
|
||||||
/**
|
/**
|
||||||
* Creates and returns a retrieval factory for the specific configuration
|
* Creates and returns a retrieval factory for the specific configuration
|
||||||
|
|
|
||||||
|
|
@ -80,17 +80,12 @@ public class FanOutConfig implements RetrievalSpecificConfig {
|
||||||
*/
|
*/
|
||||||
private long retryBackoffMillis = 1000;
|
private long retryBackoffMillis = 1000;
|
||||||
|
|
||||||
@Override
|
@Override public RetrievalFactory retrievalFactory() {
|
||||||
public RetrievalFactory retrievalFactory() {
|
return new FanOutRetrievalFactory(kinesisClient, streamName, consumerArn, this::getOrCreateConsumerArn);
|
||||||
return new FanOutRetrievalFactory(kinesisClient, getOrCreateConsumerArn());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getOrCreateConsumerArn() {
|
private String getOrCreateConsumerArn(String streamName) {
|
||||||
if (consumerArn != null) {
|
FanOutConsumerRegistration registration = createConsumerRegistration(streamName);
|
||||||
return consumerArn;
|
|
||||||
}
|
|
||||||
|
|
||||||
FanOutConsumerRegistration registration = createConsumerRegistration();
|
|
||||||
try {
|
try {
|
||||||
return registration.getOrCreateStreamConsumerArn();
|
return registration.getOrCreateStreamConsumerArn();
|
||||||
} catch (DependencyException e) {
|
} catch (DependencyException e) {
|
||||||
|
|
@ -98,10 +93,10 @@ public class FanOutConfig implements RetrievalSpecificConfig {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private FanOutConsumerRegistration createConsumerRegistration() {
|
private FanOutConsumerRegistration createConsumerRegistration(String streamName) {
|
||||||
String consumerToCreate = ObjectUtils.firstNonNull(consumerName(), applicationName());
|
String consumerToCreate = ObjectUtils.firstNonNull(consumerName(), applicationName());
|
||||||
return createConsumerRegistration(kinesisClient(),
|
return createConsumerRegistration(kinesisClient(),
|
||||||
Preconditions.checkNotNull(streamName(), "streamName must be set for consumer creation"),
|
Preconditions.checkNotNull(streamName, "streamName must be set for consumer creation"),
|
||||||
Preconditions.checkNotNull(consumerToCreate,
|
Preconditions.checkNotNull(consumerToCreate,
|
||||||
"applicationName or consumerName must be set for consumer creation"));
|
"applicationName or consumerName must be set for consumer creation"));
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -76,7 +76,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
try {
|
try {
|
||||||
response = describeStreamConsumer();
|
response = describeStreamConsumer();
|
||||||
} catch (ResourceNotFoundException e) {
|
} catch (ResourceNotFoundException e) {
|
||||||
log.info("StreamConsumer not found, need to create it.");
|
log.info("{} : StreamConsumer not found, need to create it.", streamName);
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2. If not, register consumer
|
// 2. If not, register consumer
|
||||||
|
|
@ -92,7 +92,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
break;
|
break;
|
||||||
} catch (LimitExceededException e) {
|
} catch (LimitExceededException e) {
|
||||||
// TODO: Figure out internal service exceptions
|
// TODO: Figure out internal service exceptions
|
||||||
log.debug("RegisterStreamConsumer call got throttled will retry.");
|
log.debug("{} : RegisterStreamConsumer call got throttled will retry.", streamName);
|
||||||
finalException = e;
|
finalException = e;
|
||||||
}
|
}
|
||||||
retries--;
|
retries--;
|
||||||
|
|
@ -104,7 +104,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
}
|
}
|
||||||
} catch (ResourceInUseException e) {
|
} catch (ResourceInUseException e) {
|
||||||
// Consumer is present, call DescribeStreamConsumer
|
// Consumer is present, call DescribeStreamConsumer
|
||||||
log.debug("Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again.");
|
log.debug("{} : Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again.", streamName);
|
||||||
response = describeStreamConsumer();
|
response = describeStreamConsumer();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -160,17 +160,17 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
while (!ConsumerStatus.ACTIVE.equals(status) && retries > 0) {
|
while (!ConsumerStatus.ACTIVE.equals(status) && retries > 0) {
|
||||||
status = describeStreamConsumer().consumerDescription().consumerStatus();
|
status = describeStreamConsumer().consumerDescription().consumerStatus();
|
||||||
retries--;
|
retries--;
|
||||||
log.info(String.format("Waiting for StreamConsumer %s to have ACTIVE status...", streamConsumerName));
|
log.info("{} : Waiting for StreamConsumer {} to have ACTIVE status...", streamName, streamConsumerName);
|
||||||
Thread.sleep(retryBackoffMillis);
|
Thread.sleep(retryBackoffMillis);
|
||||||
}
|
}
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
log.debug("Thread was interrupted while fetching StreamConsumer status, moving on.");
|
log.debug("{} : Thread was interrupted while fetching StreamConsumer status, moving on.", streamName);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!ConsumerStatus.ACTIVE.equals(status)) {
|
if (!ConsumerStatus.ACTIVE.equals(status)) {
|
||||||
final String message = String.format(
|
final String message = String.format(
|
||||||
"Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.",
|
"%s : Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.",
|
||||||
streamConsumerName, status);
|
streamName, streamConsumerName, status);
|
||||||
log.error(message);
|
log.error(message);
|
||||||
throw new IllegalStateException(message);
|
throw new IllegalStateException(message);
|
||||||
}
|
}
|
||||||
|
|
@ -211,7 +211,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
throw new DependencyException(e);
|
throw new DependencyException(e);
|
||||||
}
|
}
|
||||||
} catch (LimitExceededException e) {
|
} catch (LimitExceededException e) {
|
||||||
log.info("Throttled while calling {} API, will backoff.", apiName);
|
log.info("{} : Throttled while calling {} API, will backoff.", streamName, apiName);
|
||||||
try {
|
try {
|
||||||
Thread.sleep(retryBackoffMillis + (long) (Math.random() * 100));
|
Thread.sleep(retryBackoffMillis + (long) (Math.random() * 100));
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
|
|
@ -224,7 +224,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration {
|
||||||
|
|
||||||
if (finalException == null) {
|
if (finalException == null) {
|
||||||
throw new IllegalStateException(
|
throw new IllegalStateException(
|
||||||
String.format("Finished all retries and no exception was caught while calling %s", apiName));
|
String.format("%s : Finished all retries and no exception was caught while calling %s", streamName, apiName));
|
||||||
}
|
}
|
||||||
|
|
||||||
throw finalException;
|
throw finalException;
|
||||||
|
|
|
||||||
|
|
@ -27,17 +27,20 @@ import org.reactivestreams.Subscriber;
|
||||||
import org.reactivestreams.Subscription;
|
import org.reactivestreams.Subscription;
|
||||||
import software.amazon.awssdk.core.async.SdkPublisher;
|
import software.amazon.awssdk.core.async.SdkPublisher;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
|
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
|
||||||
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream;
|
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream;
|
||||||
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
|
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
|
||||||
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponse;
|
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponse;
|
||||||
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
|
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.awssdk.utils.Either;
|
import software.amazon.awssdk.utils.Either;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
||||||
import software.amazon.kinesis.common.RequestDetails;
|
import software.amazon.kinesis.common.RequestDetails;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
||||||
import software.amazon.kinesis.retrieval.BatchUniqueIdentifier;
|
import software.amazon.kinesis.retrieval.BatchUniqueIdentifier;
|
||||||
import software.amazon.kinesis.retrieval.IteratorBuilder;
|
import software.amazon.kinesis.retrieval.IteratorBuilder;
|
||||||
|
|
@ -51,7 +54,6 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
|
@ -59,8 +61,8 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired;
|
import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired;
|
||||||
|
import static software.amazon.kinesis.retrieval.DataRetrievalUtil.isValidResult;
|
||||||
|
|
||||||
@RequiredArgsConstructor
|
|
||||||
@Slf4j
|
@Slf4j
|
||||||
@KinesisClientInternalApi
|
@KinesisClientInternalApi
|
||||||
public class FanOutRecordsPublisher implements RecordsPublisher {
|
public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
@ -73,7 +75,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
private final KinesisAsyncClient kinesis;
|
private final KinesisAsyncClient kinesis;
|
||||||
private final String shardId;
|
private final String shardId;
|
||||||
private final String consumerArn;
|
private final String consumerArn;
|
||||||
|
private final String streamAndShardId;
|
||||||
private final Object lockObject = new Object();
|
private final Object lockObject = new Object();
|
||||||
|
|
||||||
private final AtomicInteger subscribeToShardId = new AtomicInteger(0);
|
private final AtomicInteger subscribeToShardId = new AtomicInteger(0);
|
||||||
|
|
@ -91,11 +93,25 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
private RequestDetails lastSuccessfulRequestDetails = new RequestDetails();
|
private RequestDetails lastSuccessfulRequestDetails = new RequestDetails();
|
||||||
|
|
||||||
|
public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn) {
|
||||||
|
this.kinesis = kinesis;
|
||||||
|
this.shardId = shardId;
|
||||||
|
this.consumerArn = consumerArn;
|
||||||
|
this.streamAndShardId = shardId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn, String streamIdentifierSer) {
|
||||||
|
this.kinesis = kinesis;
|
||||||
|
this.shardId = shardId;
|
||||||
|
this.consumerArn = consumerArn;
|
||||||
|
this.streamAndShardId = streamIdentifierSer + ":" + shardId;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void start(ExtendedSequenceNumber extendedSequenceNumber,
|
public void start(ExtendedSequenceNumber extendedSequenceNumber,
|
||||||
InitialPositionInStreamExtended initialPositionInStreamExtended) {
|
InitialPositionInStreamExtended initialPositionInStreamExtended) {
|
||||||
synchronized (lockObject) {
|
synchronized (lockObject) {
|
||||||
log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", shardId,
|
log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", streamAndShardId,
|
||||||
extendedSequenceNumber, initialPositionInStreamExtended);
|
extendedSequenceNumber, initialPositionInStreamExtended);
|
||||||
this.initialPositionInStreamExtended = initialPositionInStreamExtended;
|
this.initialPositionInStreamExtended = initialPositionInStreamExtended;
|
||||||
this.currentSequenceNumber = extendedSequenceNumber.sequenceNumber();
|
this.currentSequenceNumber = extendedSequenceNumber.sequenceNumber();
|
||||||
|
|
@ -174,7 +190,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
// It is now safe to remove the element
|
// It is now safe to remove the element
|
||||||
recordsDeliveryQueue.poll();
|
recordsDeliveryQueue.poll();
|
||||||
// Take action based on the time spent by the event in queue.
|
// Take action based on the time spent by the event in queue.
|
||||||
takeDelayedDeliveryActionIfRequired(shardId, recordsRetrievedContext.getEnqueueTimestamp(), log);
|
takeDelayedDeliveryActionIfRequired(streamAndShardId, recordsRetrievedContext.getEnqueueTimestamp(), log);
|
||||||
// Update current sequence number for the successfully delivered event.
|
// Update current sequence number for the successfully delivered event.
|
||||||
currentSequenceNumber = ((FanoutRecordsRetrieved)recordsRetrieved).continuationSequenceNumber();
|
currentSequenceNumber = ((FanoutRecordsRetrieved)recordsRetrieved).continuationSequenceNumber();
|
||||||
// Update the triggering flow for post scheduling upstream request.
|
// Update the triggering flow for post scheduling upstream request.
|
||||||
|
|
@ -190,13 +206,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (flow != null && recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()
|
if (flow != null && recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()
|
||||||
.equals(flow.getSubscribeToShardId())) {
|
.equals(flow.getSubscribeToShardId())) {
|
||||||
log.error(
|
log.error(
|
||||||
"{}: Received unexpected ack for the active subscription {}. Throwing. ",
|
"{}: Received unexpected ack for the active subscription {}. Throwing. ", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier());
|
||||||
shardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier());
|
|
||||||
throw new IllegalStateException("Unexpected ack for the active subscription");
|
throw new IllegalStateException("Unexpected ack for the active subscription");
|
||||||
}
|
}
|
||||||
// Otherwise publisher received a stale ack.
|
// Otherwise publisher received a stale ack.
|
||||||
else {
|
else {
|
||||||
log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", shardId,
|
log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", streamAndShardId,
|
||||||
recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier());
|
recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -219,10 +234,10 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
} catch (IllegalStateException e) {
|
} catch (IllegalStateException e) {
|
||||||
|
|
||||||
log.warn("{}: Unable to enqueue the payload due to capacity restrictions in delivery queue with remaining capacity {}. Last successful request details -- {}",
|
log.warn("{}: Unable to enqueue the payload due to capacity restrictions in delivery queue with remaining capacity {}. Last successful request details -- {}",
|
||||||
shardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails);
|
streamAndShardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails);
|
||||||
throw e;
|
throw e;
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
log.error("{}: Unable to deliver event to the shard consumer.", shardId, t);
|
log.error("{}: Unable to deliver event to the shard consumer.", streamAndShardId, t);
|
||||||
throw t;
|
throw t;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -290,7 +305,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
String instanceId = shardId + "-" + subscribeInvocationId;
|
String instanceId = shardId + "-" + subscribeInvocationId;
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#subscribeToShard) @ {} id: {} -- Starting subscribe to shard",
|
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#subscribeToShard) @ {} id: {} -- Starting subscribe to shard",
|
||||||
shardId, connectionStart, instanceId);
|
streamAndShardId, connectionStart, instanceId);
|
||||||
flow = new RecordFlow(this, connectionStart, instanceId);
|
flow = new RecordFlow(this, connectionStart, instanceId);
|
||||||
kinesis.subscribeToShard(request, flow);
|
kinesis.subscribeToShard(request, flow);
|
||||||
}
|
}
|
||||||
|
|
@ -303,12 +318,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if(hasValidFlow()) {
|
if(hasValidFlow()) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- Subscriber is null." +
|
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- Subscriber is null." +
|
||||||
" Last successful request details -- {}", shardId, flow.connectionStartedAt,
|
" Last successful request details -- {}", streamAndShardId, flow.connectionStartedAt,
|
||||||
flow.subscribeToShardId, lastSuccessfulRequestDetails);
|
flow.subscribeToShardId, lastSuccessfulRequestDetails);
|
||||||
} else {
|
} else {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) -- Subscriber and flow are null." +
|
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) -- Subscriber and flow are null." +
|
||||||
" Last successful request details -- {}", shardId, lastSuccessfulRequestDetails);
|
" Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails);
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -320,8 +335,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (flow != null) {
|
if (flow != null) {
|
||||||
String logMessage = String.format(
|
String logMessage = String.format(
|
||||||
"%s: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ %s id: %s -- %s." +
|
"%s: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ %s id: %s -- %s." +
|
||||||
" Last successful request details -- %s",
|
" Last successful request details -- %s", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails);
|
||||||
shardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails);
|
|
||||||
switch (category.throwableType) {
|
switch (category.throwableType) {
|
||||||
case READ_TIMEOUT:
|
case READ_TIMEOUT:
|
||||||
log.debug(logMessage, propagationThrowable);
|
log.debug(logMessage, propagationThrowable);
|
||||||
|
|
@ -339,13 +353,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
}
|
}
|
||||||
flow.cancel();
|
flow.cancel();
|
||||||
}
|
}
|
||||||
log.debug("{}: availableQueueSpace zeroing from {}", shardId, availableQueueSpace);
|
log.debug("{}: availableQueueSpace zeroing from {}", streamAndShardId, availableQueueSpace);
|
||||||
availableQueueSpace = 0;
|
availableQueueSpace = 0;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
handleFlowError(propagationThrowable, triggeringFlow);
|
handleFlowError(propagationThrowable, triggeringFlow);
|
||||||
} catch (Throwable innerThrowable) {
|
} catch (Throwable innerThrowable) {
|
||||||
log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}", shardId, lastSuccessfulRequestDetails, innerThrowable);
|
log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}",
|
||||||
|
streamAndShardId, lastSuccessfulRequestDetails, innerThrowable);
|
||||||
}
|
}
|
||||||
subscriber = null;
|
subscriber = null;
|
||||||
flow = null;
|
flow = null;
|
||||||
|
|
@ -353,7 +368,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (triggeringFlow != null) {
|
if (triggeringFlow != null) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- {} -> triggeringFlow wasn't the active flow. Didn't dispatch error",
|
"{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- {} -> triggeringFlow wasn't the active flow. Didn't dispatch error",
|
||||||
shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId,
|
streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId,
|
||||||
category.throwableTypeString);
|
category.throwableTypeString);
|
||||||
triggeringFlow.cancel();
|
triggeringFlow.cancel();
|
||||||
}
|
}
|
||||||
|
|
@ -367,7 +382,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
// Clear any lingering records in the queue.
|
// Clear any lingering records in the queue.
|
||||||
if (!recordsDeliveryQueue.isEmpty()) {
|
if (!recordsDeliveryQueue.isEmpty()) {
|
||||||
log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of"
|
log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of"
|
||||||
+ "previous subscription - {}. Last successful request details -- {}", shardId, subscribeToShardId, lastSuccessfulRequestDetails);
|
+ "previous subscription - {}. Last successful request details -- {}", streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails);
|
||||||
recordsDeliveryQueue.clear();
|
recordsDeliveryQueue.clear();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -383,11 +398,11 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (t.getCause() instanceof ResourceNotFoundException) {
|
if (t.getCause() instanceof ResourceNotFoundException) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: Could not call SubscribeToShard successfully because shard no longer exists. Marking shard for completion.",
|
"{}: Could not call SubscribeToShard successfully because shard no longer exists. Marking shard for completion.",
|
||||||
shardId);
|
streamAndShardId);
|
||||||
// The ack received for this onNext event will be ignored by the publisher as the global flow object should
|
// The ack received for this onNext event will be ignored by the publisher as the global flow object should
|
||||||
// be either null or renewed when the ack's flow identifier is evaluated.
|
// be either null or renewed when the ack's flow identifier is evaluated.
|
||||||
FanoutRecordsRetrieved response = new FanoutRecordsRetrieved(
|
FanoutRecordsRetrieved response = new FanoutRecordsRetrieved(
|
||||||
ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true).build(), null,
|
ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true).childShards(Collections.emptyList()).build(), null,
|
||||||
triggeringFlow != null ? triggeringFlow.getSubscribeToShardId() : shardId + "-no-flow-found");
|
triggeringFlow != null ? triggeringFlow.getSubscribeToShardId() : shardId + "-no-flow-found");
|
||||||
subscriber.onNext(response);
|
subscriber.onNext(response);
|
||||||
subscriber.onComplete();
|
subscriber.onComplete();
|
||||||
|
|
@ -452,7 +467,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (!hasValidSubscriber()) {
|
if (!hasValidSubscriber()) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Subscriber is null.",
|
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Subscriber is null.",
|
||||||
shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
triggeringFlow.cancel();
|
triggeringFlow.cancel();
|
||||||
if (flow != null) {
|
if (flow != null) {
|
||||||
flow.cancel();
|
flow.cancel();
|
||||||
|
|
@ -462,23 +477,36 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (!isActiveFlow(triggeringFlow)) {
|
if (!isActiveFlow(triggeringFlow)) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Received records for an inactive flow.",
|
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Received records for an inactive flow.",
|
||||||
shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
// If recordBatchEvent is not valid event, RuntimeException will be thrown here and trigger the errorOccurred call.
|
||||||
|
// Since the triggeringFlow is active flow, it will then trigger the handleFlowError call.
|
||||||
|
// Since the exception is not ResourceNotFoundException, it will trigger onError in the ShardConsumerSubscriber.
|
||||||
|
// The ShardConsumerSubscriber will finally cancel the subscription.
|
||||||
|
if (!isValidResult(recordBatchEvent.continuationSequenceNumber(), recordBatchEvent.childShards())) {
|
||||||
|
throw new InvalidStateException("RecordBatchEvent for flow " + triggeringFlow.toString() + " is invalid."
|
||||||
|
+ " event.continuationSequenceNumber: " + recordBatchEvent.continuationSequenceNumber()
|
||||||
|
+ ". event.childShards: " + recordBatchEvent.childShards());
|
||||||
|
}
|
||||||
|
|
||||||
List<KinesisClientRecord> records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord)
|
List<KinesisClientRecord> records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
ProcessRecordsInput input = ProcessRecordsInput.builder().cacheEntryTime(Instant.now())
|
ProcessRecordsInput input = ProcessRecordsInput.builder()
|
||||||
|
.cacheEntryTime(Instant.now())
|
||||||
.millisBehindLatest(recordBatchEvent.millisBehindLatest())
|
.millisBehindLatest(recordBatchEvent.millisBehindLatest())
|
||||||
.isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null).records(records).build();
|
.isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null)
|
||||||
|
.records(records)
|
||||||
|
.childShards(recordBatchEvent.childShards())
|
||||||
|
.build();
|
||||||
FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input,
|
FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input,
|
||||||
recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId);
|
recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId);
|
||||||
|
|
||||||
try {
|
|
||||||
bufferCurrentEventAndScheduleIfRequired(recordsRetrieved, triggeringFlow);
|
bufferCurrentEventAndScheduleIfRequired(recordsRetrieved, triggeringFlow);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
log.warn("{}: Unable to buffer or schedule onNext for subscriber. Failing publisher." +
|
log.warn("{}: Unable to buffer or schedule onNext for subscriber. Failing publisher." +
|
||||||
" Last successful request details -- {}", shardId, lastSuccessfulRequestDetails);
|
" Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails);
|
||||||
errorOccurred(triggeringFlow, t);
|
errorOccurred(triggeringFlow, t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -488,7 +516,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (availableQueueSpace <= 0) {
|
if (availableQueueSpace <= 0) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Attempted to decrement availableQueueSpace to below 0",
|
"{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Attempted to decrement availableQueueSpace to below 0",
|
||||||
shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
} else {
|
} else {
|
||||||
availableQueueSpace--;
|
availableQueueSpace--;
|
||||||
if (availableQueueSpace > 0) {
|
if (availableQueueSpace > 0) {
|
||||||
|
|
@ -503,12 +531,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
private void onComplete(RecordFlow triggeringFlow) {
|
private void onComplete(RecordFlow triggeringFlow) {
|
||||||
synchronized (lockObject) {
|
synchronized (lockObject) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId,
|
log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", streamAndShardId,
|
||||||
triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
|
|
||||||
triggeringFlow.cancel();
|
triggeringFlow.cancel();
|
||||||
if (!hasValidSubscriber()) {
|
if (!hasValidSubscriber()) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId,
|
log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}",
|
||||||
|
streamAndShardId,
|
||||||
triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -516,15 +545,15 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (!isActiveFlow(triggeringFlow)) {
|
if (!isActiveFlow(triggeringFlow)) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {} -- Received spurious onComplete from unexpected flow. Ignoring.",
|
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {} -- Received spurious onComplete from unexpected flow. Ignoring.",
|
||||||
shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (currentSequenceNumber != null) {
|
if (currentSequenceNumber != null) {
|
||||||
log.debug("{}: Shard hasn't ended. Resubscribing.", shardId);
|
log.debug("{}: Shard hasn't ended. Resubscribing.", streamAndShardId);
|
||||||
subscribeToShard(currentSequenceNumber);
|
subscribeToShard(currentSequenceNumber);
|
||||||
} else {
|
} else {
|
||||||
log.debug("{}: Shard has ended completing subscriber.", shardId);
|
log.debug("{}: Shard has ended completing subscriber.", streamAndShardId);
|
||||||
subscriber.onComplete();
|
subscriber.onComplete();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -536,7 +565,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (subscriber != null) {
|
if (subscriber != null) {
|
||||||
log.error(
|
log.error(
|
||||||
"{}: A subscribe occurred while there was an active subscriber. Sending error to current subscriber",
|
"{}: A subscribe occurred while there was an active subscriber. Sending error to current subscriber",
|
||||||
shardId);
|
streamAndShardId);
|
||||||
MultipleSubscriberException multipleSubscriberException = new MultipleSubscriberException();
|
MultipleSubscriberException multipleSubscriberException = new MultipleSubscriberException();
|
||||||
|
|
||||||
//
|
//
|
||||||
|
|
@ -575,7 +604,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (subscriber != s) {
|
if (subscriber != s) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: (FanOutRecordsPublisher/Subscription#request) - Rejected an attempt to request({}), because subscribers don't match. Last successful request details -- {}",
|
"{}: (FanOutRecordsPublisher/Subscription#request) - Rejected an attempt to request({}), because subscribers don't match. Last successful request details -- {}",
|
||||||
shardId, n, lastSuccessfulRequestDetails);
|
streamAndShardId, n, lastSuccessfulRequestDetails);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (flow == null) {
|
if (flow == null) {
|
||||||
|
|
@ -584,7 +613,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
//
|
//
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: (FanOutRecordsPublisher/Subscription#request) - Request called for a null flow.",
|
"{}: (FanOutRecordsPublisher/Subscription#request) - Request called for a null flow.",
|
||||||
shardId);
|
streamAndShardId);
|
||||||
errorOccurred(flow, new IllegalStateException("Attempted to request on a null flow."));
|
errorOccurred(flow, new IllegalStateException("Attempted to request on a null flow."));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -602,19 +631,19 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (subscriber != s) {
|
if (subscriber != s) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: (FanOutRecordsPublisher/Subscription#cancel) - Rejected attempt to cancel subscription, because subscribers don't match. Last successful request details -- {}",
|
"{}: (FanOutRecordsPublisher/Subscription#cancel) - Rejected attempt to cancel subscription, because subscribers don't match. Last successful request details -- {}",
|
||||||
shardId, lastSuccessfulRequestDetails);
|
streamAndShardId, lastSuccessfulRequestDetails);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (!hasValidSubscriber()) {
|
if (!hasValidSubscriber()) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: (FanOutRecordsPublisher/Subscription#cancel) - Cancelled called even with an invalid subscriber. Last successful request details -- {}",
|
"{}: (FanOutRecordsPublisher/Subscription#cancel) - Cancelled called even with an invalid subscriber. Last successful request details -- {}",
|
||||||
shardId, lastSuccessfulRequestDetails);
|
streamAndShardId, lastSuccessfulRequestDetails);
|
||||||
}
|
}
|
||||||
subscriber = null;
|
subscriber = null;
|
||||||
if (flow != null) {
|
if (flow != null) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher/Subscription#cancel) @ {} id: {}",
|
"{}: [SubscriptionLifetime]: (FanOutRecordsPublisher/Subscription#cancel) @ {} id: {}",
|
||||||
shardId, flow.connectionStartedAt, flow.subscribeToShardId);
|
streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId);
|
||||||
flow.cancel();
|
flow.cancel();
|
||||||
availableQueueSpace = 0;
|
availableQueueSpace = 0;
|
||||||
}
|
}
|
||||||
|
|
@ -703,12 +732,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
public void onEventStream(SdkPublisher<SubscribeToShardEventStream> publisher) {
|
public void onEventStream(SdkPublisher<SubscribeToShardEventStream> publisher) {
|
||||||
synchronized (parent.lockObject) {
|
synchronized (parent.lockObject) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- Subscribe",
|
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- Subscribe",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
if (!parent.isActiveFlow(this)) {
|
if (!parent.isActiveFlow(this)) {
|
||||||
this.isDisposed = true;
|
this.isDisposed = true;
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- parent is disposed",
|
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- parent is disposed",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
parent.rejectSubscription(publisher);
|
parent.rejectSubscription(publisher);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -716,7 +745,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
try {
|
try {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- creating record subscription",
|
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- creating record subscription",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
subscription = new RecordSubscription(parent, this, connectionStartedAt, subscribeToShardId);
|
subscription = new RecordSubscription(parent, this, connectionStartedAt, subscribeToShardId);
|
||||||
publisher.subscribe(subscription);
|
publisher.subscribe(subscription);
|
||||||
|
|
||||||
|
|
@ -727,7 +756,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- throwable during record subscription: {}",
|
"{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- throwable during record subscription: {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage());
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage());
|
||||||
parent.errorOccurred(this, t);
|
parent.errorOccurred(this, t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -736,7 +765,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
@Override
|
@Override
|
||||||
public void responseReceived(SubscribeToShardResponse response) {
|
public void responseReceived(SubscribeToShardResponse response) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#responseReceived) @ {} id: {} -- Response received. Request id - {}",
|
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#responseReceived) @ {} id: {} -- Response received. Request id - {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId());
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId());
|
||||||
|
|
||||||
final RequestDetails requestDetails = new RequestDetails(response.responseMetadata().requestId(), connectionStartedAt.toString());
|
final RequestDetails requestDetails = new RequestDetails(response.responseMetadata().requestId(), connectionStartedAt.toString());
|
||||||
parent.setLastSuccessfulRequestDetails(requestDetails);
|
parent.setLastSuccessfulRequestDetails(requestDetails);
|
||||||
|
|
@ -759,12 +788,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
synchronized (parent.lockObject) {
|
synchronized (parent.lockObject) {
|
||||||
|
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- {}: {}",
|
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- {}: {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
||||||
throwable.getMessage());
|
throwable.getMessage());
|
||||||
if (this.isDisposed) {
|
if (this.isDisposed) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- This flow has been disposed, not dispatching error. {}: {}",
|
"{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- This flow has been disposed, not dispatching error. {}: {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
||||||
throwable.getMessage());
|
throwable.getMessage());
|
||||||
this.isErrorDispatched = true;
|
this.isErrorDispatched = true;
|
||||||
}
|
}
|
||||||
|
|
@ -775,7 +804,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
} else {
|
} else {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- An error has previously been dispatched, not dispatching this error {}: {}",
|
"{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- An error has previously been dispatched, not dispatching this error {}: {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
|
||||||
throwable.getMessage());
|
throwable.getMessage());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -802,7 +831,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: Unable to enqueue the {} shutdown event due to capacity restrictions in delivery queue with remaining capacity {}. Ignoring. Last successful request details -- {}",
|
"{}: Unable to enqueue the {} shutdown event due to capacity restrictions in delivery queue with remaining capacity {}. Ignoring. Last successful request details -- {}",
|
||||||
parent.shardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(),
|
parent.streamAndShardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(),
|
||||||
parent.lastSuccessfulRequestDetails, subscriptionShutdownEvent.getShutdownEventThrowableOptional());
|
parent.lastSuccessfulRequestDetails, subscriptionShutdownEvent.getShutdownEventThrowableOptional());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -810,7 +839,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
private void executeComplete() {
|
private void executeComplete() {
|
||||||
synchronized (parent.lockObject) {
|
synchronized (parent.lockObject) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Connection completed",
|
log.debug("{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Connection completed",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
|
|
||||||
if (isCancelled) {
|
if (isCancelled) {
|
||||||
//
|
//
|
||||||
|
|
@ -820,13 +849,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
// subscription, which was cancelled for a reason (usually queue overflow).
|
// subscription, which was cancelled for a reason (usually queue overflow).
|
||||||
//
|
//
|
||||||
log.warn("{}: complete called on a cancelled subscription. Ignoring completion. Last successful request details -- {}",
|
log.warn("{}: complete called on a cancelled subscription. Ignoring completion. Last successful request details -- {}",
|
||||||
parent.shardId, parent.lastSuccessfulRequestDetails);
|
parent.streamAndShardId, parent.lastSuccessfulRequestDetails);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (this.isDisposed) {
|
if (this.isDisposed) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- This flow has been disposed not dispatching completion. Last successful request details -- {}",
|
"{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- This flow has been disposed not dispatching completion. Last successful request details -- {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -844,7 +873,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
log.error(
|
log.error(
|
||||||
"{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Exception while trying to cancel failed subscription: {}",
|
"{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Exception while trying to cancel failed subscription: {}",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -885,14 +914,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
public void cancel() {
|
public void cancel() {
|
||||||
synchronized (parent.lockObject) {
|
synchronized (parent.lockObject) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- Cancel called",
|
log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- Cancel called",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
flow.isCancelled = true;
|
flow.isCancelled = true;
|
||||||
if (subscription != null) {
|
if (subscription != null) {
|
||||||
subscription.cancel();
|
subscription.cancel();
|
||||||
} else {
|
} else {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- SDK subscription is null",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- SDK subscription is null",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -906,21 +935,21 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (flow.isCancelled) {
|
if (flow.isCancelled) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Subscription was cancelled before onSubscribe",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Subscription was cancelled before onSubscribe",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
}
|
}
|
||||||
if (flow.isDisposed) {
|
if (flow.isDisposed) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow has been disposed cancelling subscribe",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow has been disposed cancelling subscribe",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
}
|
}
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow requires cancelling",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow requires cancelling",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
cancel();
|
cancel();
|
||||||
}
|
}
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Outstanding: {} items so requesting an item",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Outstanding: {} items so requesting an item",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace);
|
||||||
if (parent.availableQueueSpace > 0) {
|
if (parent.availableQueueSpace > 0) {
|
||||||
request(1);
|
request(1);
|
||||||
}
|
}
|
||||||
|
|
@ -933,7 +962,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
if (flow.shouldSubscriptionCancel()) {
|
if (flow.shouldSubscriptionCancel()) {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onNext) @ {} id: {} -- RecordFlow requires cancelling",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onNext) @ {} id: {} -- RecordFlow requires cancelling",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
cancel();
|
cancel();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
@ -948,7 +977,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onError(Throwable t) {
|
public void onError(Throwable t) {
|
||||||
log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.shardId,
|
log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.streamAndShardId,
|
||||||
connectionStartedAt, subscribeToShardId, t.getClass().getName(), t.getMessage());
|
connectionStartedAt, subscribeToShardId, t.getClass().getName(), t.getMessage());
|
||||||
|
|
||||||
//
|
//
|
||||||
|
|
@ -961,7 +990,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
|
||||||
public void onComplete() {
|
public void onComplete() {
|
||||||
log.debug(
|
log.debug(
|
||||||
"{}: [SubscriptionLifetime]: (RecordSubscription#onComplete) @ {} id: {} -- Allowing RecordFlow to call onComplete",
|
"{}: [SubscriptionLifetime]: (RecordSubscription#onComplete) @ {} id: {} -- Allowing RecordFlow to call onComplete",
|
||||||
parent.shardId, connectionStartedAt, subscribeToShardId);
|
parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -19,18 +19,29 @@ import lombok.NonNull;
|
||||||
import lombok.RequiredArgsConstructor;
|
import lombok.RequiredArgsConstructor;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
@RequiredArgsConstructor
|
@RequiredArgsConstructor
|
||||||
@KinesisClientInternalApi
|
@KinesisClientInternalApi
|
||||||
public class FanOutRetrievalFactory implements RetrievalFactory {
|
public class FanOutRetrievalFactory implements RetrievalFactory {
|
||||||
|
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
private final String consumerArn;
|
private final String defaultStreamName;
|
||||||
|
private final String defaultConsumerArn;
|
||||||
|
private final Function<String, String> consumerArnCreator;
|
||||||
|
|
||||||
|
private Map<StreamIdentifier, String> implicitConsumerArnTracker = new HashMap<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo,
|
public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo,
|
||||||
|
|
@ -40,7 +51,28 @@ public class FanOutRetrievalFactory implements RetrievalFactory {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo,
|
public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo,
|
||||||
|
final StreamConfig streamConfig,
|
||||||
final MetricsFactory metricsFactory) {
|
final MetricsFactory metricsFactory) {
|
||||||
return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), consumerArn);
|
final Optional<String> streamIdentifierStr = shardInfo.streamIdentifierSerOpt();
|
||||||
|
if(streamIdentifierStr.isPresent()) {
|
||||||
|
final StreamIdentifier streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get());
|
||||||
|
return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(),
|
||||||
|
getOrCreateConsumerArn(streamIdentifier, streamConfig.consumerArn()),
|
||||||
|
streamIdentifierStr.get());
|
||||||
|
} else {
|
||||||
|
final StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance(defaultStreamName);
|
||||||
|
return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(),
|
||||||
|
getOrCreateConsumerArn(streamIdentifier, defaultConsumerArn));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory) {
|
||||||
|
throw new UnsupportedOperationException("FanoutRetrievalFactory needs StreamConfig Info");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getOrCreateConsumerArn(StreamIdentifier streamIdentifier, String consumerArn) {
|
||||||
|
return consumerArn != null ? consumerArn : implicitConsumerArnTracker
|
||||||
|
.computeIfAbsent(streamIdentifier, sId -> consumerArnCreator.apply(sId.streamName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -141,6 +141,11 @@ public class ExtendedSequenceNumber implements Comparable<ExtendedSequenceNumber
|
||||||
return subSequenceNumber;
|
return subSequenceNumber;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public boolean isShardEnd() {
|
||||||
|
return sequenceNumber.equals(SentinelCheckpoint.SHARD_END.toString());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
|
|
|
||||||
|
|
@ -67,6 +67,7 @@ public class BlockingRecordsPublisher implements RecordsPublisher {
|
||||||
return ProcessRecordsInput.builder()
|
return ProcessRecordsInput.builder()
|
||||||
.records(records)
|
.records(records)
|
||||||
.millisBehindLatest(getRecordsResult.millisBehindLatest())
|
.millisBehindLatest(getRecordsResult.millisBehindLatest())
|
||||||
|
.childShards(getRecordsResult.childShards())
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,124 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2019 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
|
||||||
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.retrieval.DataFetcherResult;
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
public interface DataFetcher {
|
||||||
|
/**
|
||||||
|
* Get records from the current position in the stream (up to maxRecords).
|
||||||
|
*
|
||||||
|
* @return list of records of up to maxRecords size
|
||||||
|
*/
|
||||||
|
DataFetcherResult getRecords();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number.
|
||||||
|
*
|
||||||
|
* @param initialCheckpoint Current checkpoint sequence number for this shard.
|
||||||
|
* @param initialPositionInStream The initialPositionInStream.
|
||||||
|
*/
|
||||||
|
void initialize(String initialCheckpoint,
|
||||||
|
InitialPositionInStreamExtended initialPositionInStream);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number as an
|
||||||
|
* ExtendedSequenceNumber.
|
||||||
|
*
|
||||||
|
* @param initialCheckpoint Current checkpoint sequence number for this shard.
|
||||||
|
* @param initialPositionInStream The initialPositionInStream.
|
||||||
|
*/
|
||||||
|
void initialize(ExtendedSequenceNumber initialCheckpoint,
|
||||||
|
InitialPositionInStreamExtended initialPositionInStream);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Advances this KinesisDataFetcher's internal iterator to be at the passed-in sequence number.
|
||||||
|
*
|
||||||
|
* @param sequenceNumber advance the iterator to the record at this sequence number.
|
||||||
|
* @param initialPositionInStream The initialPositionInStream.
|
||||||
|
*/
|
||||||
|
void advanceIteratorTo(String sequenceNumber,
|
||||||
|
InitialPositionInStreamExtended initialPositionInStream);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last
|
||||||
|
* records call.
|
||||||
|
*/
|
||||||
|
void restartIterator();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Resets the iterator by setting shardIterator, sequenceNumber and the position in the stream.
|
||||||
|
*
|
||||||
|
* @param shardIterator set the current shard iterator.
|
||||||
|
* @param sequenceNumber reset the iterator to the record at this sequence number.
|
||||||
|
* @param initialPositionInStream the current position in the stream to reset the iterator to.
|
||||||
|
*/
|
||||||
|
void resetIterator(String shardIterator, String sequenceNumber, InitialPositionInStreamExtended initialPositionInStream);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieves the response based on the request.
|
||||||
|
*
|
||||||
|
* @param request the current get records request used to receive a response.
|
||||||
|
* @return GetRecordsResponse response for getRecords
|
||||||
|
*/
|
||||||
|
GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws Exception;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retrieves the next get records request based on the current iterator.
|
||||||
|
*
|
||||||
|
* @param nextIterator specify the iterator to get the next record request
|
||||||
|
* @return {@link GetRecordsRequest}
|
||||||
|
*/
|
||||||
|
GetRecordsRequest getGetRecordsRequest(String nextIterator);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the next iterator based on the request.
|
||||||
|
*
|
||||||
|
* @param request used to obtain the next shard iterator
|
||||||
|
* @return next iterator string
|
||||||
|
*/
|
||||||
|
String getNextIterator(GetShardIteratorRequest request) throws ExecutionException, InterruptedException, TimeoutException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the next set of records based on the iterator.
|
||||||
|
*
|
||||||
|
* @param nextIterator specified shard iterator for getting the next set of records
|
||||||
|
* @return {@link GetRecordsResponse}
|
||||||
|
*/
|
||||||
|
GetRecordsResponse getRecords(@NonNull String nextIterator);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the current account and stream information.
|
||||||
|
*
|
||||||
|
* @return {@link StreamIdentifier}
|
||||||
|
*/
|
||||||
|
StreamIdentifier getStreamIdentifier();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if shardEnd is reached.
|
||||||
|
* @return boolean to determine whether shard end is reached
|
||||||
|
*/
|
||||||
|
boolean isShardEndReached();
|
||||||
|
}
|
||||||
|
|
@ -14,82 +14,105 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.retrieval.polling;
|
package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
|
||||||
|
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
|
|
||||||
import lombok.AccessLevel;
|
import lombok.AccessLevel;
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import software.amazon.awssdk.core.exception.SdkException;
|
import software.amazon.awssdk.core.exception.SdkException;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
|
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse;
|
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse;
|
||||||
import software.amazon.awssdk.services.kinesis.model.KinesisException;
|
import software.amazon.awssdk.services.kinesis.model.KinesisException;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.FutureUtils;
|
import software.amazon.kinesis.common.FutureUtils;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
import software.amazon.kinesis.common.KinesisRequestsBuilder;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsLevel;
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
import software.amazon.kinesis.metrics.MetricsScope;
|
import software.amazon.kinesis.metrics.MetricsScope;
|
||||||
import software.amazon.kinesis.metrics.MetricsUtil;
|
import software.amazon.kinesis.metrics.MetricsUtil;
|
||||||
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
import software.amazon.kinesis.retrieval.AWSExceptionManager;
|
||||||
|
import software.amazon.kinesis.retrieval.DataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.DataFetcherResult;
|
import software.amazon.kinesis.retrieval.DataFetcherResult;
|
||||||
|
import software.amazon.kinesis.retrieval.DataRetrievalUtil;
|
||||||
import software.amazon.kinesis.retrieval.IteratorBuilder;
|
import software.amazon.kinesis.retrieval.IteratorBuilder;
|
||||||
|
import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import static software.amazon.kinesis.retrieval.DataRetrievalUtil.isValidResult;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used to get data from Amazon Kinesis. Tracks iterator state internally.
|
* Used to get data from Amazon Kinesis. Tracks iterator state internally.
|
||||||
*/
|
*/
|
||||||
@Slf4j
|
@Slf4j
|
||||||
@KinesisClientInternalApi
|
@KinesisClientInternalApi
|
||||||
public class KinesisDataFetcher {
|
public class KinesisDataFetcher implements DataFetcher {
|
||||||
|
|
||||||
private static final String METRICS_PREFIX = "KinesisDataFetcher";
|
private static final String METRICS_PREFIX = "KinesisDataFetcher";
|
||||||
private static final String OPERATION = "ProcessTask";
|
private static final String OPERATION = "ProcessTask";
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
@NonNull
|
@NonNull @Getter
|
||||||
private final String streamName;
|
private final StreamIdentifier streamIdentifier;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final String shardId;
|
private final String shardId;
|
||||||
private final int maxRecords;
|
private final int maxRecords;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final MetricsFactory metricsFactory;
|
private final MetricsFactory metricsFactory;
|
||||||
private final Duration maxFutureWait;
|
private final Duration maxFutureWait;
|
||||||
|
private final String streamAndShardId;
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) {
|
public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) {
|
||||||
this(kinesisClient, streamName, shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT);
|
this(kinesisClient, new KinesisDataFetcherProviderConfig(
|
||||||
|
StreamIdentifier.singleStreamInstance(streamName),
|
||||||
|
shardId,
|
||||||
|
metricsFactory,
|
||||||
|
maxRecords,
|
||||||
|
PollingConfig.DEFAULT_REQUEST_TIMEOUT
|
||||||
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory, Duration maxFutureWait) {
|
/**
|
||||||
this.kinesisClient = kinesisClient;
|
* Note: This method has package level access for testing purposes.
|
||||||
this.streamName = streamName;
|
*
|
||||||
this.shardId = shardId;
|
|
||||||
this.maxRecords = maxRecords;
|
|
||||||
this.metricsFactory = metricsFactory;
|
|
||||||
this.maxFutureWait = maxFutureWait;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Note: This method has package level access for testing purposes.
|
|
||||||
* @return nextIterator
|
* @return nextIterator
|
||||||
*/
|
*/
|
||||||
@Getter(AccessLevel.PACKAGE)
|
@Getter(AccessLevel.PACKAGE)
|
||||||
private String nextIterator;
|
private String nextIterator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs KinesisDataFetcher.
|
||||||
|
*
|
||||||
|
* @param kinesisClient
|
||||||
|
* @param kinesisDataFetcherProviderConfig
|
||||||
|
*/
|
||||||
|
public KinesisDataFetcher(KinesisAsyncClient kinesisClient, DataFetcherProviderConfig kinesisDataFetcherProviderConfig) {
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.maxFutureWait = kinesisDataFetcherProviderConfig.getKinesisRequestTimeout();
|
||||||
|
this.maxRecords = kinesisDataFetcherProviderConfig.getMaxRecords();
|
||||||
|
this.metricsFactory = kinesisDataFetcherProviderConfig.getMetricsFactory();
|
||||||
|
this.shardId = kinesisDataFetcherProviderConfig.getShardId();
|
||||||
|
this.streamIdentifier = kinesisDataFetcherProviderConfig.getStreamIdentifier();
|
||||||
|
this.streamAndShardId = streamIdentifier.serialize() + ":" + shardId;
|
||||||
|
}
|
||||||
|
|
||||||
@Getter
|
@Getter
|
||||||
private boolean isShardEndReached;
|
private boolean isShardEndReached;
|
||||||
private boolean isInitialized;
|
private boolean isInitialized;
|
||||||
|
|
@ -101,6 +124,7 @@ public class KinesisDataFetcher {
|
||||||
*
|
*
|
||||||
* @return list of records of up to maxRecords size
|
* @return list of records of up to maxRecords size
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public DataFetcherResult getRecords() {
|
public DataFetcherResult getRecords() {
|
||||||
if (!isInitialized) {
|
if (!isInitialized) {
|
||||||
throw new IllegalArgumentException("KinesisDataFetcher.records called before initialization.");
|
throw new IllegalArgumentException("KinesisDataFetcher.records called before initialization.");
|
||||||
|
|
@ -110,7 +134,7 @@ public class KinesisDataFetcher {
|
||||||
try {
|
try {
|
||||||
return new AdvancingResult(getRecords(nextIterator));
|
return new AdvancingResult(getRecords(nextIterator));
|
||||||
} catch (ResourceNotFoundException e) {
|
} catch (ResourceNotFoundException e) {
|
||||||
log.info("Caught ResourceNotFoundException when fetching records for shard {}", shardId);
|
log.info("Caught ResourceNotFoundException when fetching records for shard {}", streamAndShardId);
|
||||||
return TERMINAL_RESULT;
|
return TERMINAL_RESULT;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -121,8 +145,12 @@ public class KinesisDataFetcher {
|
||||||
final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() {
|
final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() {
|
||||||
@Override
|
@Override
|
||||||
public GetRecordsResponse getResult() {
|
public GetRecordsResponse getResult() {
|
||||||
return GetRecordsResponse.builder().millisBehindLatest(null).records(Collections.emptyList())
|
return GetRecordsResponse.builder()
|
||||||
.nextShardIterator(null).build();
|
.millisBehindLatest(null)
|
||||||
|
.records(Collections.emptyList())
|
||||||
|
.nextShardIterator(null)
|
||||||
|
.childShards(Collections.emptyList())
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -170,16 +198,18 @@ public class KinesisDataFetcher {
|
||||||
* @param initialCheckpoint Current checkpoint sequence number for this shard.
|
* @param initialCheckpoint Current checkpoint sequence number for this shard.
|
||||||
* @param initialPositionInStream The initialPositionInStream.
|
* @param initialPositionInStream The initialPositionInStream.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void initialize(final String initialCheckpoint,
|
public void initialize(final String initialCheckpoint,
|
||||||
final InitialPositionInStreamExtended initialPositionInStream) {
|
final InitialPositionInStreamExtended initialPositionInStream) {
|
||||||
log.info("Initializing shard {} with {}", shardId, initialCheckpoint);
|
log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint);
|
||||||
advanceIteratorTo(initialCheckpoint, initialPositionInStream);
|
advanceIteratorTo(initialCheckpoint, initialPositionInStream);
|
||||||
isInitialized = true;
|
isInitialized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void initialize(final ExtendedSequenceNumber initialCheckpoint,
|
public void initialize(final ExtendedSequenceNumber initialCheckpoint,
|
||||||
final InitialPositionInStreamExtended initialPositionInStream) {
|
final InitialPositionInStreamExtended initialPositionInStream) {
|
||||||
log.info("Initializing shard {} with {}", shardId, initialCheckpoint.sequenceNumber());
|
log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint.sequenceNumber());
|
||||||
advanceIteratorTo(initialCheckpoint.sequenceNumber(), initialPositionInStream);
|
advanceIteratorTo(initialCheckpoint.sequenceNumber(), initialPositionInStream);
|
||||||
isInitialized = true;
|
isInitialized = true;
|
||||||
}
|
}
|
||||||
|
|
@ -190,6 +220,7 @@ public class KinesisDataFetcher {
|
||||||
* @param sequenceNumber advance the iterator to the record at this sequence number.
|
* @param sequenceNumber advance the iterator to the record at this sequence number.
|
||||||
* @param initialPositionInStream The initialPositionInStream.
|
* @param initialPositionInStream The initialPositionInStream.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void advanceIteratorTo(final String sequenceNumber,
|
public void advanceIteratorTo(final String sequenceNumber,
|
||||||
final InitialPositionInStreamExtended initialPositionInStream) {
|
final InitialPositionInStreamExtended initialPositionInStream) {
|
||||||
if (sequenceNumber == null) {
|
if (sequenceNumber == null) {
|
||||||
|
|
@ -199,21 +230,20 @@ public class KinesisDataFetcher {
|
||||||
final AWSExceptionManager exceptionManager = createExceptionManager();
|
final AWSExceptionManager exceptionManager = createExceptionManager();
|
||||||
|
|
||||||
GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder()
|
GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder()
|
||||||
.streamName(streamName).shardId(shardId);
|
.streamName(streamIdentifier.streamName()).shardId(shardId);
|
||||||
GetShardIteratorRequest request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream)
|
GetShardIteratorRequest request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// TODO: Check if this metric is fine to be added
|
// TODO: Check if this metric is fine to be added
|
||||||
final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION);
|
final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION);
|
||||||
|
MetricsUtil.addStreamId(metricsScope, streamIdentifier);
|
||||||
MetricsUtil.addShardId(metricsScope, shardId);
|
MetricsUtil.addShardId(metricsScope, shardId);
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
try {
|
try {
|
||||||
final GetShardIteratorResponse result = FutureUtils
|
nextIterator = getNextIterator(request);
|
||||||
.resolveOrCancelFuture(kinesisClient.getShardIterator(request), maxFutureWait);
|
|
||||||
nextIterator = result.shardIterator();
|
|
||||||
success = true;
|
success = true;
|
||||||
} catch (ExecutionException e) {
|
} catch (ExecutionException e) {
|
||||||
throw exceptionManager.apply(e.getCause());
|
throw exceptionManager.apply(e.getCause());
|
||||||
|
|
@ -224,7 +254,7 @@ public class KinesisDataFetcher {
|
||||||
throw new RetryableRetrievalException(e.getMessage(), e);
|
throw new RetryableRetrievalException(e.getMessage(), e);
|
||||||
}
|
}
|
||||||
} catch (ResourceNotFoundException e) {
|
} catch (ResourceNotFoundException e) {
|
||||||
log.info("Caught ResourceNotFoundException when getting an iterator for shard {}", shardId, e);
|
log.info("Caught ResourceNotFoundException when getting an iterator for shard {}", streamAndShardId, e);
|
||||||
nextIterator = null;
|
nextIterator = null;
|
||||||
} finally {
|
} finally {
|
||||||
MetricsUtil.addSuccessAndLatency(metricsScope, String.format("%s.%s", METRICS_PREFIX, "getShardIterator"),
|
MetricsUtil.addSuccessAndLatency(metricsScope, String.format("%s.%s", METRICS_PREFIX, "getShardIterator"),
|
||||||
|
|
@ -243,6 +273,7 @@ public class KinesisDataFetcher {
|
||||||
* Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last
|
* Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last
|
||||||
* records call.
|
* records call.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void restartIterator() {
|
public void restartIterator() {
|
||||||
if (StringUtils.isEmpty(lastKnownSequenceNumber) || initialPositionInStream == null) {
|
if (StringUtils.isEmpty(lastKnownSequenceNumber) || initialPositionInStream == null) {
|
||||||
throw new IllegalStateException(
|
throw new IllegalStateException(
|
||||||
|
|
@ -251,31 +282,57 @@ public class KinesisDataFetcher {
|
||||||
advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream);
|
advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void resetIterator(String shardIterator, String sequenceNumber, InitialPositionInStreamExtended initialPositionInStream) {
|
public void resetIterator(String shardIterator, String sequenceNumber, InitialPositionInStreamExtended initialPositionInStream) {
|
||||||
this.nextIterator = shardIterator;
|
this.nextIterator = shardIterator;
|
||||||
this.lastKnownSequenceNumber = sequenceNumber;
|
this.lastKnownSequenceNumber = sequenceNumber;
|
||||||
this.initialPositionInStream = initialPositionInStream;
|
this.initialPositionInStream = initialPositionInStream;
|
||||||
}
|
}
|
||||||
|
|
||||||
private GetRecordsResponse getRecords(@NonNull final String nextIterator) {
|
@Override
|
||||||
final AWSExceptionManager exceptionManager = createExceptionManager();
|
public GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws ExecutionException, InterruptedException, TimeoutException {
|
||||||
GetRecordsRequest request = KinesisRequestsBuilder.getRecordsRequestBuilder().shardIterator(nextIterator)
|
final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request),
|
||||||
|
maxFutureWait);
|
||||||
|
if (!isValidResult(response.nextShardIterator(), response.childShards())) {
|
||||||
|
throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId
|
||||||
|
+ ". nextShardIterator: " + response.nextShardIterator()
|
||||||
|
+ ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator.");
|
||||||
|
}
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public GetRecordsRequest getGetRecordsRequest(String nextIterator) {
|
||||||
|
return KinesisRequestsBuilder.getRecordsRequestBuilder().shardIterator(nextIterator)
|
||||||
.limit(maxRecords).build();
|
.limit(maxRecords).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getNextIterator(GetShardIteratorRequest request) throws ExecutionException, InterruptedException, TimeoutException {
|
||||||
|
final GetShardIteratorResponse result = FutureUtils
|
||||||
|
.resolveOrCancelFuture(kinesisClient.getShardIterator(request), maxFutureWait);
|
||||||
|
return result.shardIterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public GetRecordsResponse getRecords(@NonNull final String nextIterator) {
|
||||||
|
final AWSExceptionManager exceptionManager = createExceptionManager();
|
||||||
|
GetRecordsRequest request = getGetRecordsRequest(nextIterator);
|
||||||
|
|
||||||
final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION);
|
final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION);
|
||||||
|
MetricsUtil.addStreamId(metricsScope, streamIdentifier);
|
||||||
MetricsUtil.addShardId(metricsScope, shardId);
|
MetricsUtil.addShardId(metricsScope, shardId);
|
||||||
boolean success = false ;
|
boolean success = false ;
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
try {
|
try {
|
||||||
final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request),
|
final GetRecordsResponse response = getGetRecordsResponse(request);
|
||||||
maxFutureWait);
|
|
||||||
success = true;
|
success = true;
|
||||||
return response;
|
return response;
|
||||||
} catch (ExecutionException e) {
|
} catch (ExecutionException e) {
|
||||||
throw exceptionManager.apply(e.getCause());
|
throw exceptionManager.apply(e.getCause());
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
// TODO: Check behavior
|
// TODO: Check behavior
|
||||||
log.debug("Interrupt called on metod, shutdown initiated");
|
log.debug("{} : Interrupt called on method, shutdown initiated", streamAndShardId);
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
} catch (TimeoutException e) {
|
} catch (TimeoutException e) {
|
||||||
throw new RetryableRetrievalException(e.getMessage(), e);
|
throw new RetryableRetrievalException(e.getMessage(), e);
|
||||||
|
|
|
||||||
|
|
@ -17,31 +17,47 @@ package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.function.Function;
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
|
import lombok.EqualsAndHashCode;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
|
import lombok.Setter;
|
||||||
|
import lombok.ToString;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
|
||||||
|
import software.amazon.kinesis.retrieval.DataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalSpecificConfig;
|
import software.amazon.kinesis.retrieval.RetrievalSpecificConfig;
|
||||||
|
|
||||||
@Accessors(fluent = true)
|
@Accessors(fluent = true)
|
||||||
@Data
|
|
||||||
@Getter
|
@Getter
|
||||||
|
@Setter
|
||||||
|
@ToString
|
||||||
|
@EqualsAndHashCode
|
||||||
public class PollingConfig implements RetrievalSpecificConfig {
|
public class PollingConfig implements RetrievalSpecificConfig {
|
||||||
|
|
||||||
public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofSeconds(30);
|
public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofSeconds(30);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configurable functional interface to override the existing DataFetcher.
|
||||||
|
*/
|
||||||
|
Function<DataFetcherProviderConfig, DataFetcher> dataFetcherProvider;
|
||||||
/**
|
/**
|
||||||
* Name of the Kinesis stream.
|
* Name of the Kinesis stream.
|
||||||
*
|
*
|
||||||
* @return String
|
* @return String
|
||||||
*/
|
*/
|
||||||
@NonNull
|
private String streamName;
|
||||||
private final String streamName;
|
|
||||||
|
/**
|
||||||
|
* @param kinesisClient Client used to access Kinesis services.
|
||||||
|
*/
|
||||||
|
public PollingConfig(KinesisAsyncClient kinesisClient) {
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Client used to access to Kinesis service.
|
* Client used to access to Kinesis service.
|
||||||
|
|
@ -60,6 +76,15 @@ public class PollingConfig implements RetrievalSpecificConfig {
|
||||||
*/
|
*/
|
||||||
private int maxRecords = 10000;
|
private int maxRecords = 10000;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param streamName Name of Kinesis stream.
|
||||||
|
* @param kinesisClient Client used to access Kinesis serivces.
|
||||||
|
*/
|
||||||
|
public PollingConfig(String streamName, KinesisAsyncClient kinesisClient) {
|
||||||
|
this.kinesisClient = kinesisClient;
|
||||||
|
this.streamName = streamName;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The value for how long the ShardConsumer should sleep if no records are returned from the call to
|
* The value for how long the ShardConsumer should sleep if no records are returned from the call to
|
||||||
* {@link KinesisAsyncClient#getRecords(GetRecordsRequest)}.
|
* {@link KinesisAsyncClient#getRecords(GetRecordsRequest)}.
|
||||||
|
|
@ -105,6 +130,6 @@ public class PollingConfig implements RetrievalSpecificConfig {
|
||||||
@Override
|
@Override
|
||||||
public RetrievalFactory retrievalFactory() {
|
public RetrievalFactory retrievalFactory() {
|
||||||
return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory,
|
return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory,
|
||||||
maxRecords(), kinesisRequestTimeout);
|
maxRecords(), kinesisRequestTimeout, dataFetcherProvider);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,8 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.retrieval.polling;
|
package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
@ -25,21 +27,17 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import lombok.AccessLevel;
|
import lombok.AccessLevel;
|
||||||
|
import lombok.Data;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
|
import lombok.NonNull;
|
||||||
import lombok.Setter;
|
import lombok.Setter;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.commons.lang3.Validate;
|
import org.apache.commons.lang3.Validate;
|
||||||
import org.reactivestreams.Subscriber;
|
import org.reactivestreams.Subscriber;
|
||||||
import org.reactivestreams.Subscription;
|
import org.reactivestreams.Subscription;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
|
|
||||||
import lombok.Data;
|
|
||||||
import lombok.NonNull;
|
|
||||||
import lombok.experimental.Accessors;
|
|
||||||
import lombok.extern.slf4j.Slf4j;
|
|
||||||
import software.amazon.awssdk.core.exception.SdkException;
|
import software.amazon.awssdk.core.exception.SdkException;
|
||||||
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
|
import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
|
||||||
|
|
@ -47,6 +45,7 @@ import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
import software.amazon.kinesis.common.RequestDetails;
|
import software.amazon.kinesis.common.RequestDetails;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsLevel;
|
import software.amazon.kinesis.metrics.MetricsLevel;
|
||||||
|
|
@ -61,7 +60,6 @@ import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RecordsRetrieved;
|
import software.amazon.kinesis.retrieval.RecordsRetrieved;
|
||||||
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
import software.amazon.kinesis.retrieval.RetryableRetrievalException;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired;
|
import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -91,7 +89,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon;
|
private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon;
|
||||||
private boolean started = false;
|
private boolean started = false;
|
||||||
private final String operation;
|
private final String operation;
|
||||||
private final String shardId;
|
private final StreamIdentifier streamId;
|
||||||
|
private final String streamAndShardId;
|
||||||
private Subscriber<? super RecordsRetrieved> subscriber;
|
private Subscriber<? super RecordsRetrieved> subscriber;
|
||||||
@VisibleForTesting @Getter
|
@VisibleForTesting @Getter
|
||||||
private final PublisherSession publisherSession;
|
private final PublisherSession publisherSession;
|
||||||
|
|
@ -108,7 +107,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
@VisibleForTesting @Getter
|
@VisibleForTesting @Getter
|
||||||
private final LinkedBlockingQueue<PrefetchRecordsRetrieved> prefetchRecordsQueue;
|
private final LinkedBlockingQueue<PrefetchRecordsRetrieved> prefetchRecordsQueue;
|
||||||
private final PrefetchCounters prefetchCounters;
|
private final PrefetchCounters prefetchCounters;
|
||||||
private final KinesisDataFetcher dataFetcher;
|
private final DataFetcher dataFetcher;
|
||||||
private InitialPositionInStreamExtended initialPositionInStreamExtended;
|
private InitialPositionInStreamExtended initialPositionInStreamExtended;
|
||||||
private String highestSequenceNumber;
|
private String highestSequenceNumber;
|
||||||
|
|
||||||
|
|
@ -135,11 +134,11 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
// Handle records delivery ack and execute nextEventDispatchAction.
|
// Handle records delivery ack and execute nextEventDispatchAction.
|
||||||
// This method is not thread-safe and needs to be called after acquiring a monitor.
|
// This method is not thread-safe and needs to be called after acquiring a monitor.
|
||||||
void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String shardId, Runnable nextEventDispatchAction) {
|
void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String streamAndShardId, Runnable nextEventDispatchAction) {
|
||||||
final PrefetchRecordsRetrieved recordsToCheck = peekNextRecord();
|
final PrefetchRecordsRetrieved recordsToCheck = peekNextRecord();
|
||||||
// Verify if the ack matches the head of the queue and evict it.
|
// Verify if the ack matches the head of the queue and evict it.
|
||||||
if (recordsToCheck != null && recordsToCheck.batchUniqueIdentifier().equals(recordsDeliveryAck.batchUniqueIdentifier())) {
|
if (recordsToCheck != null && recordsToCheck.batchUniqueIdentifier().equals(recordsDeliveryAck.batchUniqueIdentifier())) {
|
||||||
evictPublishedRecordAndUpdateDemand(shardId);
|
evictPublishedRecordAndUpdateDemand(streamAndShardId);
|
||||||
nextEventDispatchAction.run();
|
nextEventDispatchAction.run();
|
||||||
} else {
|
} else {
|
||||||
// Log and ignore any other ack received. As long as an ack is received for head of the queue
|
// Log and ignore any other ack received. As long as an ack is received for head of the queue
|
||||||
|
|
@ -148,21 +147,21 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
final BatchUniqueIdentifier peekedBatchUniqueIdentifier =
|
final BatchUniqueIdentifier peekedBatchUniqueIdentifier =
|
||||||
recordsToCheck == null ? null : recordsToCheck.batchUniqueIdentifier();
|
recordsToCheck == null ? null : recordsToCheck.batchUniqueIdentifier();
|
||||||
log.info("{} : Received a stale notification with id {} instead of expected id {} at {}. Will ignore.",
|
log.info("{} : Received a stale notification with id {} instead of expected id {} at {}. Will ignore.",
|
||||||
shardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now());
|
streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Evict the published record from the prefetch queue.
|
// Evict the published record from the prefetch queue.
|
||||||
// This method is not thread-safe and needs to be called after acquiring a monitor.
|
// This method is not thread-safe and needs to be called after acquiring a monitor.
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
RecordsRetrieved evictPublishedRecordAndUpdateDemand(String shardId) {
|
RecordsRetrieved evictPublishedRecordAndUpdateDemand(String streamAndShardId) {
|
||||||
final PrefetchRecordsRetrieved result = prefetchRecordsQueue.poll();
|
final PrefetchRecordsRetrieved result = prefetchRecordsQueue.poll();
|
||||||
if (result != null) {
|
if (result != null) {
|
||||||
updateDemandTrackersOnPublish(result);
|
updateDemandTrackersOnPublish(result);
|
||||||
} else {
|
} else {
|
||||||
log.info(
|
log.info(
|
||||||
"{}: No record batch found while evicting from the prefetch queue. This indicates the prefetch buffer"
|
"{}: No record batch found while evicting from the prefetch queue. This indicates the prefetch buffer"
|
||||||
+ "was reset.", shardId);
|
+ " was reset.", streamAndShardId);
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
@ -215,14 +214,15 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
this.maxByteSize = maxByteSize;
|
this.maxByteSize = maxByteSize;
|
||||||
this.maxRecordsCount = maxRecordsCount;
|
this.maxRecordsCount = maxRecordsCount;
|
||||||
this.publisherSession = new PublisherSession(new LinkedBlockingQueue<>(this.maxPendingProcessRecordsInput),
|
this.publisherSession = new PublisherSession(new LinkedBlockingQueue<>(this.maxPendingProcessRecordsInput),
|
||||||
new PrefetchCounters(), this.getRecordsRetrievalStrategy.getDataFetcher());
|
new PrefetchCounters(), this.getRecordsRetrievalStrategy.dataFetcher());
|
||||||
this.executorService = executorService;
|
this.executorService = executorService;
|
||||||
this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory);
|
this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory);
|
||||||
this.idleMillisBetweenCalls = idleMillisBetweenCalls;
|
this.idleMillisBetweenCalls = idleMillisBetweenCalls;
|
||||||
this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon();
|
this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon();
|
||||||
Validate.notEmpty(operation, "Operation cannot be empty");
|
Validate.notEmpty(operation, "Operation cannot be empty");
|
||||||
this.operation = operation;
|
this.operation = operation;
|
||||||
this.shardId = shardId;
|
this.streamId = this.getRecordsRetrievalStrategy.dataFetcher().getStreamIdentifier();
|
||||||
|
this.streamAndShardId = this.streamId.serialize() + ":" + shardId;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -230,12 +230,12 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
if (executorService.isShutdown()) {
|
if (executorService.isShutdown()) {
|
||||||
throw new IllegalStateException("ExecutorService has been shutdown.");
|
throw new IllegalStateException("ExecutorService has been shutdown.");
|
||||||
}
|
}
|
||||||
|
|
||||||
publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended);
|
|
||||||
|
|
||||||
if (!started) {
|
if (!started) {
|
||||||
log.info("{} : Starting prefetching thread.", shardId);
|
log.info("{} : Starting Prefetching thread and initializing publisher session.", streamAndShardId);
|
||||||
|
publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended);
|
||||||
executorService.execute(defaultGetRecordsCacheDaemon);
|
executorService.execute(defaultGetRecordsCacheDaemon);
|
||||||
|
} else {
|
||||||
|
log.info("{} : Skipping publisher start as it was already started.", streamAndShardId);
|
||||||
}
|
}
|
||||||
started = true;
|
started = true;
|
||||||
}
|
}
|
||||||
|
|
@ -304,9 +304,9 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void notify(RecordsDeliveryAck recordsDeliveryAck) {
|
public synchronized void notify(RecordsDeliveryAck recordsDeliveryAck) {
|
||||||
publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, shardId, () -> drainQueueForRequests());
|
publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, streamAndShardId, () -> drainQueueForRequests());
|
||||||
// Take action based on the time spent by the event in queue.
|
// Take action based on the time spent by the event in queue.
|
||||||
takeDelayedDeliveryActionIfRequired(shardId, lastEventDeliveryTime, log);
|
takeDelayedDeliveryActionIfRequired(streamAndShardId, lastEventDeliveryTime, log);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Note : Do not make this method synchronous as notify() will not be able to evict any entry from the queue.
|
// Note : Do not make this method synchronous as notify() will not be able to evict any entry from the queue.
|
||||||
|
|
@ -403,7 +403,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
public void run() {
|
public void run() {
|
||||||
while (!isShutdown) {
|
while (!isShutdown) {
|
||||||
if (Thread.currentThread().isInterrupted()) {
|
if (Thread.currentThread().isInterrupted()) {
|
||||||
log.warn("{} : Prefetch thread was interrupted.", shardId);
|
log.warn("{} : Prefetch thread was interrupted.", streamAndShardId);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -411,7 +411,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
try {
|
try {
|
||||||
makeRetrievalAttempt();
|
makeRetrievalAttempt();
|
||||||
} catch(PositionResetException pre) {
|
} catch(PositionResetException pre) {
|
||||||
log.debug("{} : Position was reset while attempting to add item to queue.", shardId);
|
log.debug("{} : Position was reset while attempting to add item to queue.", streamAndShardId);
|
||||||
} finally {
|
} finally {
|
||||||
resetLock.readLock().unlock();
|
resetLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
|
|
@ -435,7 +435,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
.records(records)
|
.records(records)
|
||||||
.millisBehindLatest(getRecordsResult.millisBehindLatest())
|
.millisBehindLatest(getRecordsResult.millisBehindLatest())
|
||||||
.cacheEntryTime(lastSuccessfulCall)
|
.cacheEntryTime(lastSuccessfulCall)
|
||||||
.isAtShardEnd(getRecordsRetrievalStrategy.getDataFetcher().isShardEndReached())
|
.isAtShardEnd(getRecordsRetrievalStrategy.dataFetcher().isShardEndReached())
|
||||||
|
.childShards(getRecordsResult.childShards())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
PrefetchRecordsRetrieved recordsRetrieved = new PrefetchRecordsRetrieved(processRecordsInput,
|
PrefetchRecordsRetrieved recordsRetrieved = new PrefetchRecordsRetrieved(processRecordsInput,
|
||||||
|
|
@ -447,23 +448,24 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
} catch (PositionResetException pse) {
|
} catch (PositionResetException pse) {
|
||||||
throw pse;
|
throw pse;
|
||||||
} catch (RetryableRetrievalException rre) {
|
} catch (RetryableRetrievalException rre) {
|
||||||
log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", shardId);
|
log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", streamAndShardId);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", shardId);
|
log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", streamAndShardId);
|
||||||
} catch (ExpiredIteratorException e) {
|
} catch (ExpiredIteratorException e) {
|
||||||
log.info("{} : records threw ExpiredIteratorException - restarting"
|
log.info("{} : records threw ExpiredIteratorException - restarting"
|
||||||
+ " after greatest seqNum passed to customer", shardId, e);
|
+ " after greatest seqNum passed to customer", streamAndShardId, e);
|
||||||
|
|
||||||
|
MetricsUtil.addStreamId(scope, streamId);
|
||||||
scope.addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
scope.addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.COUNT, MetricsLevel.SUMMARY);
|
||||||
|
|
||||||
publisherSession.dataFetcher().restartIterator();
|
publisherSession.dataFetcher().restartIterator();
|
||||||
} catch (SdkException e) {
|
} catch (SdkException e) {
|
||||||
log.error("{} : Exception thrown while fetching records from Kinesis", shardId, e);
|
log.error("{} : Exception thrown while fetching records from Kinesis", streamAndShardId, e);
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
log.error("{} : Unexpected exception was thrown. This could probably be an issue or a bug." +
|
log.error("{} : Unexpected exception was thrown. This could probably be an issue or a bug." +
|
||||||
" Please search for the exception/error online to check what is going on. If the " +
|
" Please search for the exception/error online to check what is going on. If the " +
|
||||||
"issue persists or is a recurring problem, feel free to open an issue on, " +
|
"issue persists or is a recurring problem, feel free to open an issue on, " +
|
||||||
"https://github.com/awslabs/amazon-kinesis-client.", shardId, e);
|
"https://github.com/awslabs/amazon-kinesis-client.", streamAndShardId, e);
|
||||||
} finally {
|
} finally {
|
||||||
MetricsUtil.endScope(scope);
|
MetricsUtil.endScope(scope);
|
||||||
}
|
}
|
||||||
|
|
@ -475,7 +477,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
publisherSession.prefetchCounters().waitForConsumer();
|
publisherSession.prefetchCounters().waitForConsumer();
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
log.info("{} : Thread was interrupted while waiting for the consumer. " +
|
log.info("{} : Thread was interrupted while waiting for the consumer. " +
|
||||||
"Shutdown has probably been started", shardId);
|
"Shutdown has probably been started", streamAndShardId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -522,14 +524,14 @@ public class PrefetchRecordsPublisher implements RecordsPublisher {
|
||||||
|
|
||||||
public synchronized void waitForConsumer() throws InterruptedException {
|
public synchronized void waitForConsumer() throws InterruptedException {
|
||||||
if (!shouldGetNewRecords()) {
|
if (!shouldGetNewRecords()) {
|
||||||
log.debug("{} : Queue is full waiting for consumer for {} ms", shardId, idleMillisBetweenCalls);
|
log.debug("{} : Queue is full waiting for consumer for {} ms", streamAndShardId, idleMillisBetweenCalls);
|
||||||
this.wait(idleMillisBetweenCalls);
|
this.wait(idleMillisBetweenCalls);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized boolean shouldGetNewRecords() {
|
public synchronized boolean shouldGetNewRecords() {
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("{} : Current Prefetch Counter States: {}", shardId, this.toString());
|
log.debug("{} : Current Prefetch Counter States: {}", streamAndShardId, this.toString());
|
||||||
}
|
}
|
||||||
return size < maxRecordsCount && byteSize < maxByteSize;
|
return size < maxRecordsCount && byteSize < maxByteSize;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,19 +15,22 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.retrieval.polling;
|
package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.function.Function;
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.retrieval.DataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
||||||
|
import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
||||||
|
|
||||||
import java.time.Duration;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
|
@ -41,29 +44,71 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory {
|
||||||
private final KinesisAsyncClient kinesisClient;
|
private final KinesisAsyncClient kinesisClient;
|
||||||
@NonNull
|
@NonNull
|
||||||
private final RecordsFetcherFactory recordsFetcherFactory;
|
private final RecordsFetcherFactory recordsFetcherFactory;
|
||||||
// private final long listShardsBackoffTimeInMillis;
|
|
||||||
// private final int maxListShardsRetryAttempts;
|
|
||||||
private final int maxRecords;
|
private final int maxRecords;
|
||||||
private final Duration kinesisRequestTimeout;
|
private final Duration kinesisRequestTimeout;
|
||||||
|
|
||||||
public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords, Duration kinesisRequestTimeout) {
|
private final Function<DataFetcherProviderConfig, DataFetcher> dataFetcherProvider;
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
public SynchronousBlockingRetrievalFactory(String streamName,
|
||||||
|
KinesisAsyncClient kinesisClient,
|
||||||
|
RecordsFetcherFactory recordsFetcherFactory,
|
||||||
|
int maxRecords,
|
||||||
|
Duration kinesisRequestTimeout) {
|
||||||
|
this(streamName,
|
||||||
|
kinesisClient,
|
||||||
|
recordsFetcherFactory,
|
||||||
|
maxRecords,
|
||||||
|
kinesisRequestTimeout,
|
||||||
|
defaultDataFetcherProvider(kinesisClient));
|
||||||
|
}
|
||||||
|
|
||||||
|
public SynchronousBlockingRetrievalFactory(String streamName,
|
||||||
|
KinesisAsyncClient kinesisClient,
|
||||||
|
RecordsFetcherFactory recordsFetcherFactory,
|
||||||
|
int maxRecords,
|
||||||
|
Duration kinesisRequestTimeout,
|
||||||
|
Function<DataFetcherProviderConfig, DataFetcher> dataFetcherProvider) {
|
||||||
this.streamName = streamName;
|
this.streamName = streamName;
|
||||||
this.kinesisClient = kinesisClient;
|
this.kinesisClient = kinesisClient;
|
||||||
this.recordsFetcherFactory = recordsFetcherFactory;
|
this.recordsFetcherFactory = recordsFetcherFactory;
|
||||||
this.maxRecords = maxRecords;
|
this.maxRecords = maxRecords;
|
||||||
this.kinesisRequestTimeout = kinesisRequestTimeout;
|
this.kinesisRequestTimeout = kinesisRequestTimeout;
|
||||||
|
this.dataFetcherProvider = dataFetcherProvider == null ?
|
||||||
|
defaultDataFetcherProvider(kinesisClient) : dataFetcherProvider;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords) {
|
public SynchronousBlockingRetrievalFactory(String streamName,
|
||||||
|
KinesisAsyncClient kinesisClient,
|
||||||
|
RecordsFetcherFactory recordsFetcherFactory,
|
||||||
|
int maxRecords) {
|
||||||
this(streamName, kinesisClient, recordsFetcherFactory, maxRecords, PollingConfig.DEFAULT_REQUEST_TIMEOUT);
|
this(streamName, kinesisClient, recordsFetcherFactory, maxRecords, PollingConfig.DEFAULT_REQUEST_TIMEOUT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static Function<DataFetcherProviderConfig, DataFetcher> defaultDataFetcherProvider(
|
||||||
|
KinesisAsyncClient kinesisClient) {
|
||||||
|
return dataFetcherProviderConfig -> new KinesisDataFetcher(kinesisClient, dataFetcherProviderConfig);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo,
|
public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo,
|
||||||
@NonNull final MetricsFactory metricsFactory) {
|
@NonNull final MetricsFactory metricsFactory) {
|
||||||
return new SynchronousGetRecordsRetrievalStrategy(
|
final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ?
|
||||||
new KinesisDataFetcher(kinesisClient, streamName, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout));
|
StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) :
|
||||||
|
StreamIdentifier.singleStreamInstance(streamName);
|
||||||
|
|
||||||
|
final DataFetcherProviderConfig kinesisDataFetcherProviderConfig = new KinesisDataFetcherProviderConfig(
|
||||||
|
streamIdentifier,
|
||||||
|
shardInfo.shardId(),
|
||||||
|
metricsFactory,
|
||||||
|
maxRecords,
|
||||||
|
kinesisRequestTimeout);
|
||||||
|
|
||||||
|
final DataFetcher dataFetcher = this.dataFetcherProvider.apply(kinesisDataFetcherProviderConfig);
|
||||||
|
|
||||||
|
return new SynchronousGetRecordsRetrievalStrategy(dataFetcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.retrieval.polling;
|
package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
import lombok.Data;
|
import lombok.Data;
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
|
||||||
|
|
@ -26,8 +27,9 @@ import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
||||||
@Data
|
@Data
|
||||||
@KinesisClientInternalApi
|
@KinesisClientInternalApi
|
||||||
public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy {
|
public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy {
|
||||||
|
|
||||||
@NonNull
|
@NonNull
|
||||||
private final KinesisDataFetcher dataFetcher;
|
private final DataFetcher dataFetcher;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public GetRecordsResponse getRecords(final int maxRecords) {
|
public GetRecordsResponse getRecords(final int maxRecords) {
|
||||||
|
|
@ -48,6 +50,11 @@ public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetriev
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KinesisDataFetcher getDataFetcher() {
|
public KinesisDataFetcher getDataFetcher() {
|
||||||
|
throw new UnsupportedOperationException("Deprecated. Use dataFetcher() to retrieve a dataFetcher");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DataFetcher dataFetcher() {
|
||||||
return dataFetcher;
|
return dataFetcher;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,13 +17,14 @@ package software.amazon.kinesis.retrieval.polling;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
import lombok.NonNull;
|
import lombok.NonNull;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy;
|
||||||
|
import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig;
|
||||||
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
import software.amazon.kinesis.retrieval.RecordsFetcherFactory;
|
||||||
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
import software.amazon.kinesis.retrieval.RecordsPublisher;
|
||||||
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
import software.amazon.kinesis.retrieval.RetrievalFactory;
|
||||||
|
|
@ -65,11 +66,20 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory
|
||||||
this.maxFutureWait = maxFutureWait;
|
this.maxFutureWait = maxFutureWait;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo,
|
||||||
public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo,
|
|
||||||
@NonNull final MetricsFactory metricsFactory) {
|
@NonNull final MetricsFactory metricsFactory) {
|
||||||
return new SynchronousGetRecordsRetrievalStrategy(new KinesisDataFetcher(kinesisClient, streamName,
|
final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ?
|
||||||
shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait));
|
StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) :
|
||||||
|
StreamIdentifier.singleStreamInstance(streamName);
|
||||||
|
|
||||||
|
return new SynchronousGetRecordsRetrievalStrategy(
|
||||||
|
new KinesisDataFetcher(kinesisClient, new KinesisDataFetcherProviderConfig(
|
||||||
|
streamIdentifier,
|
||||||
|
shardInfo.shardId(),
|
||||||
|
metricsFactory,
|
||||||
|
maxRecords,
|
||||||
|
maxFutureWait
|
||||||
|
)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -90,6 +90,26 @@ public class CheckpointerTest {
|
||||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testInitialPrepareCheckpointWithApplicationState() throws Exception {
|
||||||
|
String sequenceNumber = "1";
|
||||||
|
String pendingCheckpointValue = "99999";
|
||||||
|
String shardId = "myShardId";
|
||||||
|
byte[] applicationState = "applicationState".getBytes();
|
||||||
|
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(sequenceNumber);
|
||||||
|
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken);
|
||||||
|
|
||||||
|
ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue);
|
||||||
|
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), testConcurrencyToken,
|
||||||
|
applicationState);
|
||||||
|
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
|
Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public final void testAdvancingPrepareCheckpoint() throws Exception {
|
public final void testAdvancingPrepareCheckpoint() throws Exception {
|
||||||
String shardId = "myShardId";
|
String shardId = "myShardId";
|
||||||
|
|
@ -107,6 +127,26 @@ public class CheckpointerTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testAdvancingPrepareCheckpointWithApplicationState() throws Exception {
|
||||||
|
String shardId = "myShardId";
|
||||||
|
String checkpointValue = "12345";
|
||||||
|
byte[] applicationState = "applicationState".getBytes();
|
||||||
|
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue);
|
||||||
|
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), testConcurrencyToken);
|
||||||
|
|
||||||
|
for (Integer i = 0; i < 10; i++) {
|
||||||
|
String sequenceNumber = i.toString();
|
||||||
|
ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber);
|
||||||
|
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken,
|
||||||
|
applicationState);
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
|
Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
|
Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public final void testPrepareAndSetCheckpoint() throws Exception {
|
public final void testPrepareAndSetCheckpoint() throws Exception {
|
||||||
String checkpointValue = "12345";
|
String checkpointValue = "12345";
|
||||||
|
|
@ -134,4 +174,35 @@ public class CheckpointerTest {
|
||||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testPrepareAndSetCheckpointWithApplicationState() throws Exception {
|
||||||
|
String checkpointValue = "12345";
|
||||||
|
String shardId = "testShardId-1";
|
||||||
|
String concurrencyToken = "token-1";
|
||||||
|
String pendingCheckpointValue = "99999";
|
||||||
|
byte[] applicationState = "applicationState".getBytes();
|
||||||
|
|
||||||
|
// set initial checkpoint
|
||||||
|
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue);
|
||||||
|
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken);
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
|
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
|
|
||||||
|
// prepare checkpoint
|
||||||
|
ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue);
|
||||||
|
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken, applicationState);
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||||
|
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
|
Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState());
|
||||||
|
|
||||||
|
// do checkpoint
|
||||||
|
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken);
|
||||||
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||||
|
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint());
|
||||||
|
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint());
|
||||||
|
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpointState());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,7 +18,6 @@ import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import software.amazon.kinesis.exceptions.KinesisClientLibException;
|
import software.amazon.kinesis.exceptions.KinesisClientLibException;
|
||||||
import software.amazon.kinesis.checkpoint.Checkpoint;
|
|
||||||
import software.amazon.kinesis.processor.Checkpointer;
|
import software.amazon.kinesis.processor.Checkpointer;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
|
@ -32,6 +31,7 @@ public class InMemoryCheckpointer implements Checkpointer {
|
||||||
private Map<String, ExtendedSequenceNumber> checkpoints = new HashMap<>();
|
private Map<String, ExtendedSequenceNumber> checkpoints = new HashMap<>();
|
||||||
private Map<String, ExtendedSequenceNumber> flushpoints = new HashMap<>();
|
private Map<String, ExtendedSequenceNumber> flushpoints = new HashMap<>();
|
||||||
private Map<String, ExtendedSequenceNumber> pendingCheckpoints = new HashMap<>();
|
private Map<String, ExtendedSequenceNumber> pendingCheckpoints = new HashMap<>();
|
||||||
|
private Map<String, byte[]> pendingCheckpointStates = new HashMap<>();
|
||||||
|
|
||||||
private String operation;
|
private String operation;
|
||||||
|
|
||||||
|
|
@ -39,14 +39,15 @@ public class InMemoryCheckpointer implements Checkpointer {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken)
|
public void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken)
|
||||||
throws KinesisClientLibException {
|
throws KinesisClientLibException {
|
||||||
checkpoints.put(shardId, checkpointValue);
|
checkpoints.put(leaseKey, checkpointValue);
|
||||||
flushpoints.put(shardId, checkpointValue);
|
flushpoints.put(leaseKey, checkpointValue);
|
||||||
pendingCheckpoints.remove(shardId);
|
pendingCheckpoints.remove(leaseKey);
|
||||||
|
pendingCheckpointStates.remove(leaseKey);
|
||||||
|
|
||||||
if (log.isDebugEnabled()) {
|
if (log.isDebugEnabled()) {
|
||||||
log.debug("shardId: {} checkpoint: {}", shardId, checkpointValue);
|
log.debug("shardId: {} checkpoint: {}", leaseKey, checkpointValue);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -55,25 +56,32 @@ public class InMemoryCheckpointer implements Checkpointer {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException {
|
public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException {
|
||||||
ExtendedSequenceNumber checkpoint = flushpoints.get(shardId);
|
ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey);
|
||||||
log.debug("checkpoint shardId: {} checkpoint: {}", shardId, checkpoint);
|
log.debug("checkpoint shardId: {} checkpoint: {}", leaseKey, checkpoint);
|
||||||
return checkpoint;
|
return checkpoint;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken)
|
public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken)
|
||||||
throws KinesisClientLibException {
|
throws KinesisClientLibException {
|
||||||
pendingCheckpoints.put(shardId, pendingCheckpoint);
|
prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException {
|
public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException {
|
||||||
ExtendedSequenceNumber checkpoint = flushpoints.get(shardId);
|
pendingCheckpoints.put(leaseKey, pendingCheckpoint);
|
||||||
ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(shardId);
|
pendingCheckpointStates.put(leaseKey, pendingCheckpointState);
|
||||||
|
}
|
||||||
|
|
||||||
Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint);
|
@Override
|
||||||
log.debug("getCheckpointObject shardId: {}, {}", shardId, checkpointObj);
|
public Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException {
|
||||||
|
ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey);
|
||||||
|
ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(leaseKey);
|
||||||
|
byte[] pendingCheckpointState = pendingCheckpointStates.get(leaseKey);
|
||||||
|
|
||||||
|
Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint, pendingCheckpointState);
|
||||||
|
log.debug("getCheckpointObject shardId: {}, {}", leaseKey, checkpointObj);
|
||||||
return checkpointObj;
|
return checkpointObj;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,136 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.mockito.Matchers.booleanThat;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
import static software.amazon.kinesis.coordinator.DeterministicShuffleShardSyncLeaderDecider.DETERMINISTIC_SHUFFLE_SEED;
|
||||||
|
|
||||||
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
|
public class DeterministicShuffleShardSyncLeaderDeciderTest {
|
||||||
|
private static final String LEASE_KEY = "lease_key";
|
||||||
|
private static final String LEASE_OWNER = "lease_owner";
|
||||||
|
private static final String WORKER_ID = "worker-id";
|
||||||
|
|
||||||
|
private DeterministicShuffleShardSyncLeaderDecider leaderDecider;
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private LeaseRefresher leaseRefresher;
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private ScheduledExecutorService scheduledExecutorService;
|
||||||
|
|
||||||
|
private int numShardSyncWorkers;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
numShardSyncWorkers = 1;
|
||||||
|
leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLeaderElectionWithNullLeases() {
|
||||||
|
boolean isLeader = leaderDecider.isLeader(WORKER_ID);
|
||||||
|
assertTrue("IsLeader should return true if leaders is null", isLeader);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLeaderElectionWithEmptyLeases() throws Exception {
|
||||||
|
when(leaseRefresher.listLeases()).thenReturn(new ArrayList<>());
|
||||||
|
boolean isLeader = leaderDecider.isLeader(WORKER_ID);
|
||||||
|
assertTrue("IsLeader should return true if no leases are returned", isLeader);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testleaderElectionWithEmptyOwnerLeases() throws Exception {
|
||||||
|
List<Lease> leases = getLeases(5, true, true, true);
|
||||||
|
when(leaseRefresher.listLeases()).thenReturn(leases);
|
||||||
|
boolean isLeader = leaderDecider.isLeader(WORKER_ID);
|
||||||
|
assertTrue("IsLeader should return true if leases have no owner", isLeader);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testElectedLeadersAsPerExpectedShufflingOrder()
|
||||||
|
throws Exception {
|
||||||
|
List<Lease> leases = getLeases(5, false /*emptyLeaseOwner */,false /* duplicateLeaseOwner */, true /* activeLeases */);
|
||||||
|
when(leaseRefresher.listLeases()).thenReturn(leases);
|
||||||
|
Set<String> expectedLeaders = getExpectedLeaders(leases);
|
||||||
|
for (String leader : expectedLeaders) {
|
||||||
|
assertTrue(leaderDecider.isLeader(leader));
|
||||||
|
}
|
||||||
|
for (Lease lease : leases) {
|
||||||
|
if (!expectedLeaders.contains(lease.leaseOwner())) {
|
||||||
|
assertFalse(leaderDecider.isLeader(lease.leaseOwner()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testElectedLeadersAsPerExpectedShufflingOrderWhenUniqueWorkersLessThanMaxLeaders() {
|
||||||
|
this.numShardSyncWorkers = 5; // More than number of unique lease owners
|
||||||
|
leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers);
|
||||||
|
List<Lease> leases = getLeases(3, false /*emptyLeaseOwner */, false /* duplicateLeaseOwner */, true /* activeLeases */);
|
||||||
|
Set<String> expectedLeaders = getExpectedLeaders(leases);
|
||||||
|
// All lease owners should be present in expected leaders set, and they should all be leaders.
|
||||||
|
for (Lease lease : leases) {
|
||||||
|
assertTrue(leaderDecider.isLeader(lease.leaseOwner()));
|
||||||
|
assertTrue(expectedLeaders.contains(lease.leaseOwner()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<Lease> getLeases(int count, boolean emptyLeaseOwner, boolean duplicateLeaseOwner, boolean activeLeases) {
|
||||||
|
List<Lease> leases = new ArrayList<>();
|
||||||
|
for (int i = 0; i < count; i++) {
|
||||||
|
Lease lease = new Lease();
|
||||||
|
lease.leaseKey(LEASE_KEY + i);
|
||||||
|
lease.checkpoint(activeLeases ? ExtendedSequenceNumber.LATEST : ExtendedSequenceNumber.SHARD_END);
|
||||||
|
lease.leaseCounter(new Random().nextLong());
|
||||||
|
if (!emptyLeaseOwner) {
|
||||||
|
lease.leaseOwner(LEASE_OWNER + (duplicateLeaseOwner ? "" : i));
|
||||||
|
}
|
||||||
|
leases.add(lease);
|
||||||
|
}
|
||||||
|
return leases;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Set<String> getExpectedLeaders(List<Lease> leases) {
|
||||||
|
List<String> uniqueHosts = leases.stream().filter(lease -> lease.leaseOwner() != null)
|
||||||
|
.map(Lease::leaseOwner).distinct().sorted().collect(Collectors.toList());
|
||||||
|
|
||||||
|
Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED));
|
||||||
|
int numWorkers = Math.min(uniqueHosts.size(), this.numShardSyncWorkers);
|
||||||
|
return new HashSet<>(uniqueHosts.subList(0, numWorkers));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,589 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.coordinator;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.HashKeyRange;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
|
import software.amazon.awssdk.utils.CollectionUtils;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
|
import software.amazon.kinesis.leases.MultiStreamLease;
|
||||||
|
import software.amazon.kinesis.leases.ShardDetector;
|
||||||
|
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
||||||
|
import software.amazon.kinesis.metrics.NullMetricsFactory;
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.math.BigInteger;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
import static software.amazon.kinesis.common.HashKeyRangeForLease.deserialize;
|
||||||
|
import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MAX_HASH_KEY;
|
||||||
|
import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MIN_HASH_KEY;
|
||||||
|
import static software.amazon.kinesis.leases.LeaseManagementConfig.DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY;
|
||||||
|
|
||||||
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
|
|
||||||
|
public class PeriodicShardSyncManagerTest {
|
||||||
|
|
||||||
|
private StreamIdentifier streamIdentifier;
|
||||||
|
private PeriodicShardSyncManager periodicShardSyncManager;
|
||||||
|
@Mock
|
||||||
|
private LeaderDecider leaderDecider;
|
||||||
|
@Mock
|
||||||
|
private LeaseRefresher leaseRefresher;
|
||||||
|
@Mock
|
||||||
|
Map<StreamIdentifier, StreamConfig> currentStreamConfigMap;
|
||||||
|
@Mock
|
||||||
|
Function<StreamConfig, ShardSyncTaskManager> shardSyncTaskManagerProvider;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
streamIdentifier = StreamIdentifier.multiStreamInstance("123:stream:456");
|
||||||
|
periodicShardSyncManager = new PeriodicShardSyncManager("worker", leaderDecider, leaseRefresher, currentStreamConfigMap,
|
||||||
|
shardSyncTaskManagerProvider, true, new NullMetricsFactory(), 2 * 60 * 1000, 3);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForFailureWhenHashRangesAreIncomplete() {
|
||||||
|
List<Lease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize("0", "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("25", MAX_HASH_KEY.toString())); // Missing interval here
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
Lease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertTrue(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenHashRangesAreComplete() {
|
||||||
|
List<Lease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize("0", "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
Lease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenUnSortedHashRangesAreComplete() {
|
||||||
|
List<Lease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("0", "1"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
Lease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() {
|
||||||
|
List<Lease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize("0", "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
add(deserialize("24", "45"));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
Lease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, null).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, new ArrayList<>()).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsNotInitiatedWhenConfidenceFactorIsNotReached() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23")); // Hole between 23 and 25
|
||||||
|
add(deserialize("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsReached() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23")); // Hole between 23 and 25
|
||||||
|
add(deserialize("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenHoleIsDueToShardEnd() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23")); // introducing hole here through SHARD_END checkpoint
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
if(lease.hashKeyRangeForLease().startingHashKey().toString().equals("4")) {
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
} else {
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
}
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenNoLeasesAreUsedDueToShardEnd() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsNotInitiatedWhenHoleShifts() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23")); // Hole between 23 and 25
|
||||||
|
add(deserialize("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
List<Lease> multiStreamLeases2 = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3")); // Hole between 3 and 5
|
||||||
|
add(deserialize("5", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
// Resetting the holes
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsNotInitiatedWhenHoleShiftsMoreThanOnce() {
|
||||||
|
List<Lease> multiStreamLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("6", "23")); // Hole between 23 and 25
|
||||||
|
add(deserialize("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
List<Lease> multiStreamLeases2 = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3")); // Hole between 3 and 5
|
||||||
|
add(deserialize("5", "23"));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
// Resetting the holes
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync()));
|
||||||
|
// Resetting the holes
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSync() {
|
||||||
|
ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class);
|
||||||
|
ShardDetector shardDetector = mock(ShardDetector.class);
|
||||||
|
when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager);
|
||||||
|
when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
|
||||||
|
|
||||||
|
final int[] shardCounter = { 0 };
|
||||||
|
List<HashKeyRangeForLease> hashKeyRangeForLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("4", "20"));
|
||||||
|
add(deserialize("21", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}};
|
||||||
|
|
||||||
|
List<Shard> kinesisShards = hashKeyRangeForLeases.stream()
|
||||||
|
.map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange(
|
||||||
|
HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey())
|
||||||
|
.endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(shardDetector.listShards()).thenReturn(kinesisShards);
|
||||||
|
|
||||||
|
final int[] leaseCounter = { 0 };
|
||||||
|
List<Lease> multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0])));
|
||||||
|
lease.shardId("shard-"+(leaseCounter[0]));
|
||||||
|
// Setting the hashrange only for last two leases
|
||||||
|
if(leaseCounter[0] >= 3) {
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
}
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Assert that shard sync should never trigger
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
|
||||||
|
// Assert that all the leases now has hashRanges set.
|
||||||
|
for(Lease lease : multiStreamLeases) {
|
||||||
|
Assert.assertNotNull(lease.hashKeyRangeForLease());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSyncInHoleScenario() {
|
||||||
|
ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class);
|
||||||
|
ShardDetector shardDetector = mock(ShardDetector.class);
|
||||||
|
when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager);
|
||||||
|
when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
|
||||||
|
|
||||||
|
final int[] shardCounter = { 0 };
|
||||||
|
List<HashKeyRangeForLease> hashKeyRangeForLeases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("5", "20")); // Hole between 3 and 5
|
||||||
|
add(deserialize("21", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}};
|
||||||
|
|
||||||
|
List<Shard> kinesisShards = hashKeyRangeForLeases.stream()
|
||||||
|
.map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange(
|
||||||
|
HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey())
|
||||||
|
.endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(shardDetector.listShards()).thenReturn(kinesisShards);
|
||||||
|
|
||||||
|
final int[] leaseCounter = { 0 };
|
||||||
|
List<Lease> multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> {
|
||||||
|
MultiStreamLease lease = new MultiStreamLease();
|
||||||
|
lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0])));
|
||||||
|
lease.shardId("shard-"+(leaseCounter[0]));
|
||||||
|
// Setting the hashrange only for last two leases
|
||||||
|
if(leaseCounter[0] >= 3) {
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
}
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Assert that shard sync should never trigger
|
||||||
|
IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert
|
||||||
|
.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()));
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync());
|
||||||
|
|
||||||
|
// Assert that all the leases now has hashRanges set.
|
||||||
|
for(Lease lease : multiStreamLeases) {
|
||||||
|
Assert.assertNotNull(lease.hashKeyRangeForLease());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidSplitHierarchyTreeTheHashRangesAreAlwaysComplete() {
|
||||||
|
for(int i=0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<Lease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.SPLIT, maxInitialLeaseCount, false);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
// System.out.println(
|
||||||
|
// leases.stream().map(l -> l.checkpoint().sequenceNumber() + ":" + l.hashKeyRangeForLease()).collect(Collectors.toList()));
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidMergeHierarchyTreeTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<Lease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, false);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidReshardHierarchyTreeTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<Lease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, false);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidMergeHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<Lease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, true);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidReshardHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<Lease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, true);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
private List<Lease> generateInitialLeases(int initialShardCount) {
|
||||||
|
long hashRangeInternalMax = 10000000;
|
||||||
|
List<Lease> initialLeases = new ArrayList<>();
|
||||||
|
long leaseStartKey = 0;
|
||||||
|
for (int i = 1; i <= initialShardCount; i++) {
|
||||||
|
final Lease lease = new Lease();
|
||||||
|
long leaseEndKey;
|
||||||
|
if (i != initialShardCount) {
|
||||||
|
leaseEndKey = (hashRangeInternalMax / initialShardCount) * i;
|
||||||
|
lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", leaseEndKey + ""));
|
||||||
|
} else {
|
||||||
|
leaseEndKey = 0;
|
||||||
|
lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", MAX_HASH_KEY.toString()));
|
||||||
|
}
|
||||||
|
lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
lease.leaseKey("shard-" + i);
|
||||||
|
initialLeases.add(lease);
|
||||||
|
leaseStartKey = leaseEndKey + 1;
|
||||||
|
}
|
||||||
|
return initialLeases;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reshard(List<Lease> initialLeases, int depth, ReshardType reshardType, int leaseCounter,
|
||||||
|
boolean shouldKeepSomeParentsInProgress) {
|
||||||
|
for (int i = 0; i < depth; i++) {
|
||||||
|
if (reshardType == ReshardType.SPLIT) {
|
||||||
|
leaseCounter = split(initialLeases, leaseCounter);
|
||||||
|
} else if (reshardType == ReshardType.MERGE) {
|
||||||
|
leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress);
|
||||||
|
} else {
|
||||||
|
if (isHeads()) {
|
||||||
|
leaseCounter = split(initialLeases, leaseCounter);
|
||||||
|
} else {
|
||||||
|
leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private int merge(List<Lease> initialLeases, int leaseCounter, boolean shouldKeepSomeParentsInProgress) {
|
||||||
|
List<Lease> leasesEligibleForMerge = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds()))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
// System.out.println("Leases to merge : " + leasesEligibleForMerge);
|
||||||
|
int leasesToMerge = (int) ((leasesEligibleForMerge.size() - 1) / 2.0 * Math.random());
|
||||||
|
for (int i = 0; i < leasesToMerge; i += 2) {
|
||||||
|
Lease parent1 = leasesEligibleForMerge.get(i);
|
||||||
|
Lease parent2 = leasesEligibleForMerge.get(i + 1);
|
||||||
|
if(parent2.hashKeyRangeForLease().startingHashKey().subtract(parent1.hashKeyRangeForLease().endingHashKey()).equals(BigInteger.ONE))
|
||||||
|
{
|
||||||
|
parent1.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) {
|
||||||
|
// System.out.println("Deciding to keep parent in progress : " + parent2);
|
||||||
|
parent2.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
}
|
||||||
|
Lease child = new Lease();
|
||||||
|
child.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child.leaseKey("shard-" + (++leaseCounter));
|
||||||
|
// System.out.println("Parent " + parent1 + " and " + parent2 + " merges into " + child);
|
||||||
|
child.hashKeyRange(new HashKeyRangeForLease(parent1.hashKeyRangeForLease().startingHashKey(),
|
||||||
|
parent2.hashKeyRangeForLease().endingHashKey()));
|
||||||
|
parent1.childShardIds(Collections.singletonList(child.leaseKey()));
|
||||||
|
parent2.childShardIds(Collections.singletonList(child.leaseKey()));
|
||||||
|
child.parentShardIds(Sets.newHashSet(parent1.leaseKey(), parent2.leaseKey()));
|
||||||
|
|
||||||
|
initialLeases.add(child);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return leaseCounter;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int split(List<Lease> initialLeases, int leaseCounter) {
|
||||||
|
List<Lease> leasesEligibleForSplit = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds()))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
// System.out.println("Leases to split : " + leasesEligibleForSplit);
|
||||||
|
int leasesToSplit = (int) (leasesEligibleForSplit.size() * Math.random());
|
||||||
|
for (int i = 0; i < leasesToSplit; i++) {
|
||||||
|
Lease parent = leasesEligibleForSplit.get(i);
|
||||||
|
parent.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
Lease child1 = new Lease();
|
||||||
|
child1.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child1.hashKeyRange(new HashKeyRangeForLease(parent.hashKeyRangeForLease().startingHashKey(),
|
||||||
|
parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey())
|
||||||
|
.divide(new BigInteger("2"))));
|
||||||
|
child1.leaseKey("shard-" + (++leaseCounter));
|
||||||
|
Lease child2 = new Lease();
|
||||||
|
child2.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child2.hashKeyRange(new HashKeyRangeForLease(
|
||||||
|
parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey())
|
||||||
|
.divide(new BigInteger("2")).add(new BigInteger("1")),
|
||||||
|
parent.hashKeyRangeForLease().endingHashKey()));
|
||||||
|
child2.leaseKey("shard-" + (++leaseCounter));
|
||||||
|
|
||||||
|
child1.parentShardIds(Sets.newHashSet(parent.leaseKey()));
|
||||||
|
child2.parentShardIds(Sets.newHashSet(parent.leaseKey()));
|
||||||
|
parent.childShardIds(Lists.newArrayList(child1.leaseKey(), child2.leaseKey()));
|
||||||
|
|
||||||
|
// System.out.println("Parent " + parent + " splits into " + child1 + " and " + child2);
|
||||||
|
|
||||||
|
initialLeases.add(child1);
|
||||||
|
initialLeases.add(child2);
|
||||||
|
}
|
||||||
|
return leaseCounter;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isHeads() {
|
||||||
|
return Math.random() <= 0.5;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isOneFromDiceRoll() {
|
||||||
|
return Math.random() <= 0.16;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private enum ReshardType {
|
||||||
|
SPLIT,
|
||||||
|
MERGE,
|
||||||
|
ANY
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -22,8 +22,10 @@ import static org.junit.Assert.assertNotSame;
|
||||||
import static org.junit.Assert.assertSame;
|
import static org.junit.Assert.assertSame;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Matchers.anyString;
|
||||||
import static org.mockito.Matchers.eq;
|
import static org.mockito.Matchers.eq;
|
||||||
import static org.mockito.Matchers.same;
|
import static org.mockito.Matchers.same;
|
||||||
|
import static org.mockito.Mockito.atLeast;
|
||||||
import static org.mockito.Mockito.doCallRealMethod;
|
import static org.mockito.Mockito.doCallRealMethod;
|
||||||
import static org.mockito.Mockito.doNothing;
|
import static org.mockito.Mockito.doNothing;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
@ -32,14 +34,27 @@ import static org.mockito.Mockito.spy;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
import static org.mockito.internal.verification.VerificationModeFactory.atMost;
|
||||||
|
import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.*;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.RejectedExecutionException;
|
import java.util.concurrent.RejectedExecutionException;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
|
import com.google.common.base.Joiner;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
import io.reactivex.plugins.RxJavaPlugins;
|
import io.reactivex.plugins.RxJavaPlugins;
|
||||||
|
import lombok.RequiredArgsConstructor;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
|
|
@ -52,7 +67,14 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import software.amazon.kinesis.checkpoint.Checkpoint;
|
import software.amazon.kinesis.checkpoint.Checkpoint;
|
||||||
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
import software.amazon.kinesis.checkpoint.CheckpointConfig;
|
||||||
import software.amazon.kinesis.checkpoint.CheckpointFactory;
|
import software.amazon.kinesis.checkpoint.CheckpointFactory;
|
||||||
|
import software.amazon.kinesis.common.InitialPositionInStream;
|
||||||
|
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
|
||||||
|
import software.amazon.kinesis.common.StreamConfig;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.exceptions.KinesisClientLibException;
|
||||||
import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException;
|
import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException;
|
||||||
|
import software.amazon.kinesis.leases.LeaseCleanupManager;
|
||||||
|
import software.amazon.kinesis.leases.HierarchicalShardSyncer;
|
||||||
import software.amazon.kinesis.leases.LeaseCoordinator;
|
import software.amazon.kinesis.leases.LeaseCoordinator;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
import software.amazon.kinesis.leases.LeaseManagementConfig;
|
||||||
import software.amazon.kinesis.leases.LeaseManagementFactory;
|
import software.amazon.kinesis.leases.LeaseManagementFactory;
|
||||||
|
|
@ -61,8 +83,12 @@ import software.amazon.kinesis.leases.ShardDetector;
|
||||||
import software.amazon.kinesis.leases.ShardInfo;
|
import software.amazon.kinesis.leases.ShardInfo;
|
||||||
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
import software.amazon.kinesis.leases.ShardSyncTaskManager;
|
||||||
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher;
|
import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import software.amazon.kinesis.lifecycle.LifecycleConfig;
|
import software.amazon.kinesis.lifecycle.LifecycleConfig;
|
||||||
import software.amazon.kinesis.lifecycle.ShardConsumer;
|
import software.amazon.kinesis.lifecycle.ShardConsumer;
|
||||||
|
import software.amazon.kinesis.lifecycle.TaskResult;
|
||||||
import software.amazon.kinesis.lifecycle.events.InitializationInput;
|
import software.amazon.kinesis.lifecycle.events.InitializationInput;
|
||||||
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
|
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
|
||||||
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
|
||||||
|
|
@ -71,6 +97,7 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput;
|
||||||
import software.amazon.kinesis.metrics.MetricsFactory;
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
import software.amazon.kinesis.metrics.MetricsConfig;
|
import software.amazon.kinesis.metrics.MetricsConfig;
|
||||||
import software.amazon.kinesis.processor.Checkpointer;
|
import software.amazon.kinesis.processor.Checkpointer;
|
||||||
|
import software.amazon.kinesis.processor.MultiStreamTracker;
|
||||||
import software.amazon.kinesis.processor.ProcessorConfig;
|
import software.amazon.kinesis.processor.ProcessorConfig;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
|
||||||
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
import software.amazon.kinesis.processor.ShardRecordProcessor;
|
||||||
|
|
@ -89,6 +116,9 @@ public class SchedulerTest {
|
||||||
private final String applicationName = "applicationName";
|
private final String applicationName = "applicationName";
|
||||||
private final String streamName = "streamName";
|
private final String streamName = "streamName";
|
||||||
private final String namespace = "testNamespace";
|
private final String namespace = "testNamespace";
|
||||||
|
private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 5 * 1000L;
|
||||||
|
private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L;
|
||||||
|
private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L;
|
||||||
|
|
||||||
private Scheduler scheduler;
|
private Scheduler scheduler;
|
||||||
private ShardRecordProcessorFactory shardRecordProcessorFactory;
|
private ShardRecordProcessorFactory shardRecordProcessorFactory;
|
||||||
|
|
@ -122,24 +152,54 @@ public class SchedulerTest {
|
||||||
private Checkpointer checkpoint;
|
private Checkpointer checkpoint;
|
||||||
@Mock
|
@Mock
|
||||||
private WorkerStateChangeListener workerStateChangeListener;
|
private WorkerStateChangeListener workerStateChangeListener;
|
||||||
|
@Mock
|
||||||
|
private MultiStreamTracker multiStreamTracker;
|
||||||
|
@Mock
|
||||||
|
private LeaseCleanupManager leaseCleanupManager;
|
||||||
|
|
||||||
|
private Map<StreamIdentifier, ShardSyncTaskManager> shardSyncTaskManagerMap;
|
||||||
|
private Map<StreamIdentifier, ShardDetector> shardDetectorMap;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() {
|
public void setup() {
|
||||||
|
shardSyncTaskManagerMap = new HashMap<>();
|
||||||
|
shardDetectorMap = new HashMap<>();
|
||||||
shardRecordProcessorFactory = new TestShardRecordProcessorFactory();
|
shardRecordProcessorFactory = new TestShardRecordProcessorFactory();
|
||||||
|
|
||||||
checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory());
|
checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory());
|
||||||
coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L).workerStateChangeListener(workerStateChangeListener);
|
coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L).workerStateChangeListener(workerStateChangeListener);
|
||||||
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName,
|
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName,
|
||||||
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory());
|
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, false));
|
||||||
lifecycleConfig = new LifecycleConfig();
|
lifecycleConfig = new LifecycleConfig();
|
||||||
metricsConfig = new MetricsConfig(cloudWatchClient, namespace);
|
metricsConfig = new MetricsConfig(cloudWatchClient, namespace);
|
||||||
processorConfig = new ProcessorConfig(shardRecordProcessorFactory);
|
processorConfig = new ProcessorConfig(shardRecordProcessorFactory);
|
||||||
retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName)
|
retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName)
|
||||||
.retrievalFactory(retrievalFactory);
|
.retrievalFactory(retrievalFactory);
|
||||||
|
|
||||||
|
final List<StreamConfig> streamConfigList = new ArrayList<StreamConfig>() {{
|
||||||
|
add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream1:1"), InitialPositionInStreamExtended.newInitialPosition(
|
||||||
|
InitialPositionInStream.LATEST)));
|
||||||
|
add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream2:2"), InitialPositionInStreamExtended.newInitialPosition(
|
||||||
|
InitialPositionInStream.LATEST)));
|
||||||
|
add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream1:1"), InitialPositionInStreamExtended.newInitialPosition(
|
||||||
|
InitialPositionInStream.LATEST)));
|
||||||
|
add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream2:3"), InitialPositionInStreamExtended.newInitialPosition(
|
||||||
|
InitialPositionInStream.LATEST)));
|
||||||
|
}};
|
||||||
|
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList);
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy())
|
||||||
|
.thenReturn(new AutoDetectionAndDeferredDeletionStrategy() {
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
});
|
||||||
when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher);
|
when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher);
|
||||||
when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
|
when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
|
||||||
when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher);
|
when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(new HierarchicalShardSyncer());
|
||||||
|
when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null));
|
||||||
|
when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(StreamConfig.class), any(MetricsFactory.class))).thenReturn(recordsPublisher);
|
||||||
|
when(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class));
|
||||||
|
|
||||||
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
metricsConfig, processorConfig, retrievalConfig);
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
|
@ -162,9 +222,9 @@ public class SchedulerTest {
|
||||||
final String shardId = "shardId-000000000000";
|
final String shardId = "shardId-000000000000";
|
||||||
final String concurrencyToken = "concurrencyToken";
|
final String concurrencyToken = "concurrencyToken";
|
||||||
final ShardInfo shardInfo = new ShardInfo(shardId, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON);
|
final ShardInfo shardInfo = new ShardInfo(shardId, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory);
|
final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
assertNotNull(shardConsumer1);
|
assertNotNull(shardConsumer1);
|
||||||
final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory);
|
final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
assertNotNull(shardConsumer2);
|
assertNotNull(shardConsumer2);
|
||||||
|
|
||||||
assertSame(shardConsumer1, shardConsumer2);
|
assertSame(shardConsumer1, shardConsumer2);
|
||||||
|
|
@ -172,7 +232,7 @@ public class SchedulerTest {
|
||||||
final String anotherConcurrencyToken = "anotherConcurrencyToken";
|
final String anotherConcurrencyToken = "anotherConcurrencyToken";
|
||||||
final ShardInfo shardInfo2 = new ShardInfo(shardId, anotherConcurrencyToken, null,
|
final ShardInfo shardInfo2 = new ShardInfo(shardId, anotherConcurrencyToken, null,
|
||||||
ExtendedSequenceNumber.TRIM_HORIZON);
|
ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory);
|
final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
assertNotNull(shardConsumer3);
|
assertNotNull(shardConsumer3);
|
||||||
|
|
||||||
assertNotSame(shardConsumer1, shardConsumer3);
|
assertNotSame(shardConsumer1, shardConsumer3);
|
||||||
|
|
@ -194,7 +254,7 @@ public class SchedulerTest {
|
||||||
final List<ShardInfo> secondShardInfo = Collections.singletonList(
|
final List<ShardInfo> secondShardInfo = Collections.singletonList(
|
||||||
new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber));
|
new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber));
|
||||||
|
|
||||||
final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null);
|
final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null);
|
||||||
|
|
||||||
when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo);
|
when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo);
|
||||||
when(checkpoint.getCheckpointObject(eq(shardId))).thenReturn(firstCheckpoint);
|
when(checkpoint.getCheckpointObject(eq(shardId))).thenReturn(firstCheckpoint);
|
||||||
|
|
@ -204,9 +264,9 @@ public class SchedulerTest {
|
||||||
schedulerSpy.runProcessLoop();
|
schedulerSpy.runProcessLoop();
|
||||||
schedulerSpy.runProcessLoop();
|
schedulerSpy.runProcessLoop();
|
||||||
|
|
||||||
verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory));
|
verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager));
|
||||||
verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory));
|
verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager));
|
||||||
verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory));
|
verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager));
|
||||||
verify(checkpoint).getCheckpointObject(eq(shardId));
|
verify(checkpoint).getCheckpointObject(eq(shardId));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -222,10 +282,10 @@ public class SchedulerTest {
|
||||||
ExtendedSequenceNumber.TRIM_HORIZON);
|
ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
final ShardInfo shardInfo1 = new ShardInfo(shard1, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON);
|
final ShardInfo shardInfo1 = new ShardInfo(shard1, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
|
||||||
final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory);
|
final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
final ShardConsumer shardConsumer0WithAnotherConcurrencyToken =
|
final ShardConsumer shardConsumer0WithAnotherConcurrencyToken =
|
||||||
scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory);
|
scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory);
|
final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory, leaseCleanupManager);
|
||||||
|
|
||||||
Set<ShardInfo> shards = new HashSet<>();
|
Set<ShardInfo> shards = new HashSet<>();
|
||||||
shards.add(shardInfo0);
|
shards.add(shardInfo0);
|
||||||
|
|
@ -242,27 +302,504 @@ public class SchedulerTest {
|
||||||
@Test
|
@Test
|
||||||
public final void testInitializationFailureWithRetries() throws Exception {
|
public final void testInitializationFailureWithRetries() throws Exception {
|
||||||
doNothing().when(leaseCoordinator).initialize();
|
doNothing().when(leaseCoordinator).initialize();
|
||||||
when(shardDetector.listShards()).thenThrow(new RuntimeException());
|
when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException());
|
||||||
|
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName,
|
||||||
|
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true));
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
scheduler.run();
|
scheduler.run();
|
||||||
|
|
||||||
verify(shardDetector, times(coordinatorConfig.maxInitializationAttempts())).listShards();
|
verify(dynamoDBLeaseRefresher, times(coordinatorConfig.maxInitializationAttempts())).isLeaseTableEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public final void testInitializationFailureWithRetriesWithConfiguredMaxInitializationAttempts() throws Exception {
|
public final void testInitializationFailureWithRetriesWithConfiguredMaxInitializationAttempts() throws Exception {
|
||||||
final int maxInitializationAttempts = 5;
|
final int maxInitializationAttempts = 5;
|
||||||
coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts);
|
coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts);
|
||||||
|
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName,
|
||||||
|
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true));
|
||||||
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
metricsConfig, processorConfig, retrievalConfig);
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
|
||||||
doNothing().when(leaseCoordinator).initialize();
|
doNothing().when(leaseCoordinator).initialize();
|
||||||
when(shardDetector.listShards()).thenThrow(new RuntimeException());
|
when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException());
|
||||||
|
|
||||||
scheduler.run();
|
scheduler.run();
|
||||||
|
|
||||||
// verify initialization was retried for maxInitializationAttempts times
|
// verify initialization was retried for maxInitializationAttempts times
|
||||||
verify(shardDetector, times(maxInitializationAttempts)).listShards();
|
verify(dynamoDBLeaseRefresher, times(maxInitializationAttempts)).isLeaseTableEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamInitialization() throws ProvisionedThroughputException, DependencyException {
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient,
|
||||||
|
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true));
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
scheduler.initialize();
|
||||||
|
shardDetectorMap.values().stream()
|
||||||
|
.forEach(shardDetector -> verify(shardDetector, times(1)).listShards());
|
||||||
|
shardSyncTaskManagerMap.values().stream()
|
||||||
|
.forEach(shardSyncTM -> verify(shardSyncTM, times(1)).hierarchicalShardSyncer());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamInitializationWithFailures() {
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient,
|
||||||
|
workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true));
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
scheduler.initialize();
|
||||||
|
// Note : As of today we retry for all streams in the next attempt. Hence the retry for each stream will vary.
|
||||||
|
// At the least we expect 2 retries for each stream. Since there are 4 streams, we expect at most
|
||||||
|
// the number of calls to be 5.
|
||||||
|
shardDetectorMap.values().stream()
|
||||||
|
.forEach(shardDetector -> verify(shardDetector, atLeast(2)).listShards());
|
||||||
|
shardDetectorMap.values().stream()
|
||||||
|
.forEach(shardDetector -> verify(shardDetector, atMost(5)).listShards());
|
||||||
|
shardSyncTaskManagerMap.values().stream()
|
||||||
|
.forEach(shardSyncTM -> verify(shardSyncTM, atLeast(2)).hierarchicalShardSyncer());
|
||||||
|
shardSyncTaskManagerMap.values().stream()
|
||||||
|
.forEach(shardSyncTM -> verify(shardSyncTM, atMost(5)).hierarchicalShardSyncer());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamConsumersAreBuiltOncePerAccountStreamShard() throws KinesisClientLibException {
|
||||||
|
final String shardId = "shardId-000000000000";
|
||||||
|
final String concurrencyToken = "concurrencyToken";
|
||||||
|
final ExtendedSequenceNumber firstSequenceNumber = ExtendedSequenceNumber.TRIM_HORIZON;
|
||||||
|
final ExtendedSequenceNumber secondSequenceNumber = new ExtendedSequenceNumber("1000");
|
||||||
|
final ExtendedSequenceNumber finalSequenceNumber = new ExtendedSequenceNumber("2000");
|
||||||
|
|
||||||
|
final List<ShardInfo> initialShardInfo = multiStreamTracker.streamConfigList().stream()
|
||||||
|
.map(sc -> new ShardInfo(shardId, concurrencyToken, null, firstSequenceNumber,
|
||||||
|
sc.streamIdentifier().serialize())).collect(Collectors.toList());
|
||||||
|
final List<ShardInfo> firstShardInfo = multiStreamTracker.streamConfigList().stream()
|
||||||
|
.map(sc -> new ShardInfo(shardId, concurrencyToken, null, secondSequenceNumber,
|
||||||
|
sc.streamIdentifier().serialize())).collect(Collectors.toList());
|
||||||
|
final List<ShardInfo> secondShardInfo = multiStreamTracker.streamConfigList().stream()
|
||||||
|
.map(sc -> new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber,
|
||||||
|
sc.streamIdentifier().serialize())).collect(Collectors.toList());
|
||||||
|
|
||||||
|
final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null);
|
||||||
|
|
||||||
|
when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo);
|
||||||
|
when(checkpoint.getCheckpointObject(anyString())).thenReturn(firstCheckpoint);
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
Scheduler schedulerSpy = spy(scheduler);
|
||||||
|
schedulerSpy.runProcessLoop();
|
||||||
|
schedulerSpy.runProcessLoop();
|
||||||
|
schedulerSpy.runProcessLoop();
|
||||||
|
|
||||||
|
initialShardInfo.stream().forEach(
|
||||||
|
shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), same(leaseCleanupManager)));
|
||||||
|
firstShardInfo.stream().forEach(
|
||||||
|
shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)));
|
||||||
|
secondShardInfo.stream().forEach(
|
||||||
|
shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNoStreamsAreSyncedWhenStreamsAreNotRefreshed()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Assert.assertTrue("SyncedStreams should be empty", syncedStreams.isEmpty());
|
||||||
|
Assert.assertEquals(new HashSet(streamConfigList1), new HashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamOnlyNewStreamsAreSynced()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Set<StreamIdentifier> expectedSyncedStreams = IntStream.range(5, 7).mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect(
|
||||||
|
Collectors.toCollection(HashSet::new));
|
||||||
|
Assert.assertEquals(expectedSyncedStreams, syncedStreams);
|
||||||
|
Assert.assertEquals(Sets.newHashSet(streamConfigList2),
|
||||||
|
Sets.newHashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyAutoDeletionStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() {
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyNoDeletionStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy());
|
||||||
|
testMultiStreamStaleStreamsAreNotDeletedImmediately(false, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamStaleStreamsAreNotDeletedImmediately(false, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy2()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect(
|
||||||
|
Collectors.toCollection(ArrayList::new));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void testMultiStreamStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion,
|
||||||
|
boolean onlyStreamsDeletionNotLeases)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Set<StreamIdentifier> expectedPendingStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect(
|
||||||
|
Collectors.toCollection(HashSet::new));
|
||||||
|
Set<StreamIdentifier> expectedSyncedStreams = onlyStreamsDeletionNotLeases ? expectedPendingStreams : Sets.newHashSet();
|
||||||
|
Assert.assertEquals(expectedSyncedStreams, syncedStreams);
|
||||||
|
Assert.assertEquals(Sets.newHashSet(onlyStreamsDeletionNotLeases ? streamConfigList2 : streamConfigList1),
|
||||||
|
Sets.newHashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams : Sets.newHashSet(),
|
||||||
|
scheduler.staleStreamDeletionMap().keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithAutoDetectionStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() {
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
HashSet<StreamConfig> currentStreamConfigMapOverride = IntStream.range(1, 5).mapToObj(
|
||||||
|
streamId -> new StreamConfig(StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
|
testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(false, currentStreamConfigMapOverride);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy2()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect(
|
||||||
|
Collectors.toCollection(ArrayList::new));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(boolean expectSyncedStreams, Set<StreamConfig> currentStreamConfigMapOverride)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Set<StreamIdentifier> expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect(
|
||||||
|
Collectors.toCollection(HashSet::new));
|
||||||
|
Assert.assertEquals(expectSyncedStreams ? expectedSyncedStreams : Sets.newHashSet(), syncedStreams);
|
||||||
|
Assert.assertEquals(currentStreamConfigMapOverride == null ? Sets.newHashSet(streamConfigList2) : currentStreamConfigMapOverride,
|
||||||
|
Sets.newHashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
Assert.assertEquals(Sets.newHashSet(),
|
||||||
|
scheduler.staleStreamDeletionMap().keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithAutoDetectionStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() {
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithNoDeletionStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy());
|
||||||
|
testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy2()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() {
|
||||||
|
@Override public List<StreamIdentifier> streamIdentifiersForLeaseCleanup() {
|
||||||
|
return IntStream.range(1, 3)
|
||||||
|
.mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)))
|
||||||
|
.collect(Collectors.toCollection(ArrayList::new));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ofHours(1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion,
|
||||||
|
boolean onlyStreamsNoLeasesDeletion)
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Set<StreamIdentifier> expectedSyncedStreams;
|
||||||
|
Set<StreamIdentifier> expectedPendingStreams = IntStream.range(1, 3)
|
||||||
|
.mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)))
|
||||||
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
|
|
||||||
|
if(onlyStreamsNoLeasesDeletion) {
|
||||||
|
expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7))
|
||||||
|
.mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)))
|
||||||
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
|
} else {
|
||||||
|
expectedSyncedStreams = IntStream.range(5, 7)
|
||||||
|
.mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)))
|
||||||
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals(expectedSyncedStreams, syncedStreams);
|
||||||
|
List<StreamConfig> expectedCurrentStreamConfigs;
|
||||||
|
if(onlyStreamsNoLeasesDeletion) {
|
||||||
|
expectedCurrentStreamConfigs = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
} else {
|
||||||
|
expectedCurrentStreamConfigs = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
}
|
||||||
|
Assert.assertEquals(Sets.newHashSet(expectedCurrentStreamConfigs),
|
||||||
|
Sets.newHashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams: Sets.newHashSet(),
|
||||||
|
scheduler.staleStreamDeletionMap().keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreDeletedAfterDefermentPeriod()
|
||||||
|
throws DependencyException, ProvisionedThroughputException, InvalidStateException {
|
||||||
|
List<StreamConfig> streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
List<StreamConfig> streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig(
|
||||||
|
StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)),
|
||||||
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)))
|
||||||
|
.collect(Collectors.toCollection(LinkedList::new));
|
||||||
|
retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName)
|
||||||
|
.retrievalFactory(retrievalFactory);
|
||||||
|
when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2);
|
||||||
|
scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig));
|
||||||
|
when(scheduler.shouldSyncStreamsNow()).thenReturn(true);
|
||||||
|
when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() {
|
||||||
|
@Override public Duration waitPeriodToDeleteFormerStreams() {
|
||||||
|
return Duration.ZERO;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
Set<StreamIdentifier> syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases();
|
||||||
|
Set<StreamIdentifier> expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7))
|
||||||
|
.mapToObj(streamId -> StreamIdentifier.multiStreamInstance(
|
||||||
|
Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)))
|
||||||
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
|
Assert.assertEquals(expectedSyncedStreams, syncedStreams);
|
||||||
|
Assert.assertEquals(Sets.newHashSet(streamConfigList2),
|
||||||
|
Sets.newHashSet(scheduler.currentStreamConfigMap().values()));
|
||||||
|
Assert.assertEquals(Sets.newHashSet(),
|
||||||
|
scheduler.staleStreamDeletionMap().keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testInitializationWaitsWhenLeaseTableIsEmpty() throws Exception {
|
||||||
|
final int maxInitializationAttempts = 1;
|
||||||
|
coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts);
|
||||||
|
coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false);
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
|
||||||
|
doNothing().when(leaseCoordinator).initialize();
|
||||||
|
when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true);
|
||||||
|
|
||||||
|
long startTime = System.currentTimeMillis();
|
||||||
|
scheduler.shouldInitiateLeaseSync();
|
||||||
|
long endTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS);
|
||||||
|
assertTrue(endTime - startTime < (MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS + LEASE_TABLE_CHECK_FREQUENCY_MILLIS));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public final void testInitializationDoesntWaitWhenLeaseTableIsNotEmpty() throws Exception {
|
||||||
|
final int maxInitializationAttempts = 1;
|
||||||
|
coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts);
|
||||||
|
coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false);
|
||||||
|
scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
|
||||||
|
metricsConfig, processorConfig, retrievalConfig);
|
||||||
|
|
||||||
|
doNothing().when(leaseCoordinator).initialize();
|
||||||
|
when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false);
|
||||||
|
|
||||||
|
long startTime = System.currentTimeMillis();
|
||||||
|
scheduler.shouldInitiateLeaseSync();
|
||||||
|
long endTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
assertTrue(endTime - startTime < MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -498,9 +1035,20 @@ public class SchedulerTest {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) {
|
||||||
|
return shardRecordProcessor();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@RequiredArgsConstructor
|
||||||
private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory {
|
private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory {
|
||||||
|
|
||||||
|
private final boolean shardSyncFirstAttemptFailure;
|
||||||
|
private final boolean shouldReturnDefaultShardSyncTaskmanager;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LeaseCoordinator createLeaseCoordinator(MetricsFactory metricsFactory) {
|
public LeaseCoordinator createLeaseCoordinator(MetricsFactory metricsFactory) {
|
||||||
return leaseCoordinator;
|
return leaseCoordinator;
|
||||||
|
|
@ -511,6 +1059,29 @@ public class SchedulerTest {
|
||||||
return shardSyncTaskManager;
|
return shardSyncTaskManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory,
|
||||||
|
StreamConfig streamConfig) {
|
||||||
|
if(shouldReturnDefaultShardSyncTaskmanager) {
|
||||||
|
return shardSyncTaskManager;
|
||||||
|
}
|
||||||
|
final ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class);
|
||||||
|
final ShardDetector shardDetector = mock(ShardDetector.class);
|
||||||
|
shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager);
|
||||||
|
shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector);
|
||||||
|
when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
|
||||||
|
final HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer();
|
||||||
|
when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(hierarchicalShardSyncer);
|
||||||
|
when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier());
|
||||||
|
when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null));
|
||||||
|
if(shardSyncFirstAttemptFailure) {
|
||||||
|
when(shardDetector.listShards())
|
||||||
|
.thenThrow(new RuntimeException("Service Exception"))
|
||||||
|
.thenReturn(Collections.EMPTY_LIST);
|
||||||
|
}
|
||||||
|
return shardSyncTaskManager;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DynamoDBLeaseRefresher createLeaseRefresher() {
|
public DynamoDBLeaseRefresher createLeaseRefresher() {
|
||||||
return dynamoDBLeaseRefresher;
|
return dynamoDBLeaseRefresher;
|
||||||
|
|
@ -520,6 +1091,16 @@ public class SchedulerTest {
|
||||||
public ShardDetector createShardDetector() {
|
public ShardDetector createShardDetector() {
|
||||||
return shardDetector;
|
return shardDetector;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardDetector createShardDetector(StreamConfig streamConfig) {
|
||||||
|
return shardDetectorMap.get(streamConfig.streamIdentifier());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) {
|
||||||
|
return leaseCleanupManager;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private class TestKinesisCheckpointFactory implements CheckpointFactory {
|
private class TestKinesisCheckpointFactory implements CheckpointFactory {
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,7 @@ import java.util.List;
|
||||||
|
|
||||||
import lombok.RequiredArgsConstructor;
|
import lombok.RequiredArgsConstructor;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
import software.amazon.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
|
@ -54,6 +55,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher {
|
||||||
DELETELEASE(9),
|
DELETELEASE(9),
|
||||||
DELETEALL(10),
|
DELETEALL(10),
|
||||||
UPDATELEASE(11),
|
UPDATELEASE(11),
|
||||||
|
LISTLEASESFORSTREAM(12),
|
||||||
NONE(Integer.MIN_VALUE);
|
NONE(Integer.MIN_VALUE);
|
||||||
|
|
||||||
private Integer index;
|
private Integer index;
|
||||||
|
|
@ -129,6 +131,13 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher {
|
||||||
return leaseRefresher.waitUntilLeaseTableExists(secondsBetweenPolls, timeoutSeconds);
|
return leaseRefresher.waitUntilLeaseTableExists(secondsBetweenPolls, timeoutSeconds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Lease> listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
throwExceptions("listLeasesForStream", ExceptionThrowingLeaseRefresherMethods.LISTLEASESFORSTREAM);
|
||||||
|
|
||||||
|
return leaseRefresher.listLeasesForStream(streamIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Lease> listLeases()
|
public List<Lease> listLeases()
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
|
@ -186,11 +195,11 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Lease getLease(String shardId)
|
public Lease getLease(String leaseKey)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
throwExceptions("getLease", ExceptionThrowingLeaseRefresherMethods.GETLEASE);
|
throwExceptions("getLease", ExceptionThrowingLeaseRefresherMethods.GETLEASE);
|
||||||
|
|
||||||
return leaseRefresher.getLease(shardId);
|
return leaseRefresher.getLease(leaseKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -207,7 +216,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ExtendedSequenceNumber getCheckpoint(final String shardId)
|
public ExtendedSequenceNumber getCheckpoint(final String leaseKey)
|
||||||
throws ProvisionedThroughputException, InvalidStateException, DependencyException {
|
throws ProvisionedThroughputException, InvalidStateException, DependencyException {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -15,17 +15,21 @@
|
||||||
|
|
||||||
package software.amazon.kinesis.leases;
|
package software.amazon.kinesis.leases;
|
||||||
|
|
||||||
import static org.hamcrest.CoreMatchers.equalTo;
|
import org.junit.Assert;
|
||||||
import static org.hamcrest.CoreMatchers.isA;
|
import org.junit.Before;
|
||||||
import static org.hamcrest.CoreMatchers.nullValue;
|
import org.junit.Rule;
|
||||||
import static org.junit.Assert.assertThat;
|
import org.junit.Test;
|
||||||
import static org.mockito.Matchers.any;
|
import org.junit.rules.ExpectedException;
|
||||||
import static org.mockito.Matchers.anyLong;
|
import org.junit.runner.RunWith;
|
||||||
import static org.mockito.Matchers.eq;
|
import org.mockito.Mock;
|
||||||
import static org.mockito.Mockito.never;
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
import static org.mockito.Mockito.times;
|
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
||||||
import static org.mockito.Mockito.verify;
|
import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
|
||||||
import static org.mockito.Mockito.when;
|
import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ResourceInUseException;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.Shard;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
@ -36,21 +40,16 @@ import java.util.concurrent.TimeoutException;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import org.junit.Before;
|
import static org.hamcrest.CoreMatchers.equalTo;
|
||||||
import org.junit.Rule;
|
import static org.hamcrest.CoreMatchers.isA;
|
||||||
import org.junit.Test;
|
import static org.hamcrest.CoreMatchers.nullValue;
|
||||||
import org.junit.rules.ExpectedException;
|
import static org.junit.Assert.assertThat;
|
||||||
import org.junit.runner.RunWith;
|
import static org.mockito.Matchers.any;
|
||||||
import org.mockito.Mock;
|
import static org.mockito.Matchers.anyLong;
|
||||||
import org.mockito.runners.MockitoJUnitRunner;
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
|
import static org.mockito.Mockito.verify;
|
||||||
import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
|
import static org.mockito.Mockito.when;
|
||||||
import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
|
|
||||||
import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
|
|
||||||
import software.amazon.awssdk.services.kinesis.model.ResourceInUseException;
|
|
||||||
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
|
||||||
import software.amazon.awssdk.services.kinesis.model.Shard;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
|
@ -143,20 +142,18 @@ public class KinesisShardDetectorTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = ResourceNotFoundException.class)
|
@Test
|
||||||
public void testListShardsResourceNotFound() {
|
public void testListShardsResourceNotFoundReturnsEmptyResponse() {
|
||||||
final CompletableFuture<ListShardsResponse> future = CompletableFuture.supplyAsync(() -> {
|
final CompletableFuture<ListShardsResponse> future = CompletableFuture.supplyAsync(() -> {
|
||||||
throw ResourceNotFoundException.builder().build();
|
throw ResourceNotFoundException.builder().build();
|
||||||
});
|
});
|
||||||
|
|
||||||
when(client.listShards(any(ListShardsRequest.class))).thenReturn(future);
|
when(client.listShards(any(ListShardsRequest.class))).thenReturn(future);
|
||||||
|
|
||||||
try {
|
List<Shard> shards = shardDetector.listShards();
|
||||||
shardDetector.listShards();
|
|
||||||
} finally {
|
Assert.assertEquals(0, shards.size());
|
||||||
verify(client).listShards(any(ListShardsRequest.class));
|
verify(client).listShards(any(ListShardsRequest.class));
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testListShardsTimesOut() throws Exception {
|
public void testListShardsTimesOut() throws Exception {
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,7 @@ import java.util.UUID;
|
||||||
|
|
||||||
import lombok.Setter;
|
import lombok.Setter;
|
||||||
import lombok.experimental.Accessors;
|
import lombok.experimental.Accessors;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
@Setter
|
@Setter
|
||||||
|
|
@ -34,9 +35,13 @@ public class LeaseBuilder {
|
||||||
private ExtendedSequenceNumber pendingCheckpoint;
|
private ExtendedSequenceNumber pendingCheckpoint;
|
||||||
private Long ownerSwitchesSinceCheckpoint = 0L;
|
private Long ownerSwitchesSinceCheckpoint = 0L;
|
||||||
private Set<String> parentShardIds = new HashSet<>();
|
private Set<String> parentShardIds = new HashSet<>();
|
||||||
|
private Set<String> childShardIds = new HashSet<>();
|
||||||
|
private byte[] pendingCheckpointState;
|
||||||
|
private HashKeyRangeForLease hashKeyRangeForLease;
|
||||||
|
|
||||||
public Lease build() {
|
public Lease build() {
|
||||||
return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos,
|
return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint,
|
||||||
checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds);
|
pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds,
|
||||||
|
pendingCheckpointState, hashKeyRangeForLease);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -0,0 +1,299 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ChildShard;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
|
||||||
|
import software.amazon.kinesis.common.StreamIdentifier;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion;
|
||||||
|
import software.amazon.kinesis.metrics.MetricsFactory;
|
||||||
|
import software.amazon.kinesis.metrics.NullMetricsFactory;
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
|
public class LeaseCleanupManagerTest {
|
||||||
|
|
||||||
|
private ShardInfo shardInfo;
|
||||||
|
private StreamIdentifier streamIdentifier;
|
||||||
|
private String concurrencyToken = "1234";
|
||||||
|
|
||||||
|
private String shardId = "shardId";
|
||||||
|
private String splitParent = "splitParent";
|
||||||
|
private String mergeParent1 = "mergeParent-1";
|
||||||
|
private String mergeParent2 = "mergeParent-2";
|
||||||
|
|
||||||
|
private Duration maxFutureWait = Duration.ofSeconds(1);
|
||||||
|
private long leaseCleanupIntervalMillis = Duration.ofSeconds(1).toMillis();
|
||||||
|
private long completedLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis();
|
||||||
|
private long garbageLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis();
|
||||||
|
private boolean cleanupLeasesOfCompletedShards = true;
|
||||||
|
private LeaseCleanupManager leaseCleanupManager;
|
||||||
|
private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory();
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private LeaseRefresher leaseRefresher;
|
||||||
|
@Mock
|
||||||
|
private LeaseCoordinator leaseCoordinator;
|
||||||
|
@Mock
|
||||||
|
private ShardDetector shardDetector;
|
||||||
|
@Mock
|
||||||
|
private ScheduledExecutorService deletionThreadPool;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
streamIdentifier = StreamIdentifier.singleStreamInstance("streamName");
|
||||||
|
leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool,
|
||||||
|
cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis,
|
||||||
|
garbageLeaseCleanupIntervalMillis);
|
||||||
|
|
||||||
|
when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
|
||||||
|
when(leaseCoordinator.updateLease(any(Lease.class), any(UUID.class), any(String.class), any(String.class))).thenReturn(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests subsequent calls to start {@link LeaseCleanupManager}.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testSubsequentStarts() {
|
||||||
|
leaseCleanupManager.start();
|
||||||
|
Assert.assertTrue(leaseCleanupManager.isRunning());
|
||||||
|
leaseCleanupManager.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that when both child shard leases are present, we are able to delete the parent shard for the completed
|
||||||
|
* shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testParentShardLeaseDeletedSplitCase() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that when both child shard leases are present, we are able to delete the parent shard for the completed
|
||||||
|
* shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testParentShardLeaseDeletedMergeCase() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that if cleanupLeasesOfCompletedShards is not enabled by the customer, then no leases are cleaned up for
|
||||||
|
* the completed shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testNoLeasesDeletedWhenNotEnabled() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
cleanupLeasesOfCompletedShards = false;
|
||||||
|
|
||||||
|
leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool,
|
||||||
|
cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis,
|
||||||
|
garbageLeaseCleanupIntervalMillis);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that if some of the child shard leases are missing, we fail fast and don't delete the parent shard lease
|
||||||
|
* for the completed shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testNoCleanupWhenSomeChildShardLeasesAreNotPresent() throws Exception {
|
||||||
|
List<ChildShard> childShards = childShardsForSplit();
|
||||||
|
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, ExtendedSequenceNumber.LATEST, false, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint TRIM_HORIZON),
|
||||||
|
* we don't delete them for the completed shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testParentShardLeaseNotDeletedWhenChildIsAtTrim() throws Exception {
|
||||||
|
testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint AT_TIMESTAMP),
|
||||||
|
* we don't delete them for the completed shard case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testParentShardLeaseNotDeletedWhenChildIsAtTimestamp() throws Exception {
|
||||||
|
testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.AT_TIMESTAMP);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber extendedSequenceNumber)
|
||||||
|
throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), extendedSequenceNumber, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that if a lease's parents are still present, we do not delete the lease.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testLeaseNotDeletedWhenParentsStillPresent() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.singleton("parent"),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests ResourceNotFound case for if a shard expires, that we delete the lease when shardExpired is found.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testLeaseDeletedWhenShardDoesNotExist() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId"));
|
||||||
|
|
||||||
|
testLeaseDeletedWhenShardDoesNotExist(heldLease);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests ResourceNotFound case when completed lease cleanup is disabled.
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testLeaseDeletedWhenShardDoesNotExistAndCleanupCompletedLeaseDisabled() throws Exception {
|
||||||
|
shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
|
||||||
|
ExtendedSequenceNumber.LATEST);
|
||||||
|
final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId"));
|
||||||
|
|
||||||
|
cleanupLeasesOfCompletedShards = false;
|
||||||
|
|
||||||
|
leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool,
|
||||||
|
cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis,
|
||||||
|
garbageLeaseCleanupIntervalMillis);
|
||||||
|
|
||||||
|
testLeaseDeletedWhenShardDoesNotExist(heldLease);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception {
|
||||||
|
when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
|
||||||
|
when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease);
|
||||||
|
when(shardDetector.getChildShards(any(String.class))).thenThrow(ResourceNotFoundException.class);
|
||||||
|
when(leaseRefresher.getLease(heldLease.leaseKey())).thenReturn(heldLease);
|
||||||
|
|
||||||
|
leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo, shardDetector));
|
||||||
|
leaseCleanupManager.cleanupLeases();
|
||||||
|
|
||||||
|
verify(shardDetector, times(1)).getChildShards(shardInfo.shardId());
|
||||||
|
verify(leaseRefresher, times(1)).deleteLease(heldLease);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List<ChildShard> childShards,
|
||||||
|
ExtendedSequenceNumber extendedSequenceNumber,
|
||||||
|
int expectedDeletedLeases) throws Exception {
|
||||||
|
verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, extendedSequenceNumber, true, expectedDeletedLeases);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List<ChildShard> childShards,
|
||||||
|
ExtendedSequenceNumber extendedSequenceNumber,
|
||||||
|
boolean childShardLeasesPresent,
|
||||||
|
int expectedDeletedLeases) throws Exception {
|
||||||
|
|
||||||
|
final Lease lease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", shardInfo.parentShardIds(),
|
||||||
|
childShards.stream().map(c -> c.shardId()).collect(Collectors.toSet()));
|
||||||
|
final List<Lease> childShardLeases = childShards.stream().map(c -> LeaseHelper.createLease(
|
||||||
|
ShardInfo.getLeaseKey(shardInfo, c.shardId()), "leaseOwner", Collections.singleton(shardInfo.shardId()),
|
||||||
|
Collections.emptyList(), extendedSequenceNumber)).collect(Collectors.toList());
|
||||||
|
|
||||||
|
final List<Lease> parentShardLeases = lease.parentShardIds().stream().map(p ->
|
||||||
|
LeaseHelper.createLease(ShardInfo.getLeaseKey(shardInfo, p), "leaseOwner", Collections.emptyList(),
|
||||||
|
Collections.singleton(shardInfo.shardId()), extendedSequenceNumber)).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseRefresher.getLease(lease.leaseKey())).thenReturn(lease);
|
||||||
|
for (Lease parentShardLease : parentShardLeases) {
|
||||||
|
when(leaseRefresher.getLease(parentShardLease.leaseKey())).thenReturn(parentShardLease);
|
||||||
|
}
|
||||||
|
if (childShardLeasesPresent) {
|
||||||
|
for (Lease childShardLease : childShardLeases) {
|
||||||
|
when(leaseRefresher.getLease(childShardLease.leaseKey())).thenReturn(childShardLease);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo, shardDetector));
|
||||||
|
leaseCleanupManager.cleanupLeases();
|
||||||
|
|
||||||
|
verify(shardDetector, times(1)).getChildShards(shardInfo.shardId());
|
||||||
|
verify(leaseRefresher, times(expectedDeletedLeases)).deleteLease(any(Lease.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<ChildShard> childShardsForSplit() {
|
||||||
|
List<String> parentShards = Arrays.asList(splitParent);
|
||||||
|
|
||||||
|
ChildShard leftChild = ChildShard.builder()
|
||||||
|
.shardId("leftChild")
|
||||||
|
.parentShards(parentShards)
|
||||||
|
.hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49"))
|
||||||
|
.build();
|
||||||
|
ChildShard rightChild = ChildShard.builder()
|
||||||
|
.shardId("rightChild")
|
||||||
|
.parentShards(parentShards)
|
||||||
|
.hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99"))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
return Arrays.asList(leftChild, rightChild);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<ChildShard> childShardsForMerge() {
|
||||||
|
List<String> parentShards = Arrays.asList(mergeParent1, mergeParent2);
|
||||||
|
|
||||||
|
ChildShard child = ChildShard.builder()
|
||||||
|
.shardId("onlyChild")
|
||||||
|
.parentShards(parentShards)
|
||||||
|
.hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "99"))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
return Collections.singletonList(child);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,44 @@
|
||||||
|
/*
|
||||||
|
* Copyright 2020 Amazon.com, Inc. or its affiliates.
|
||||||
|
* Licensed 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 software.amazon.kinesis.leases;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
public class LeaseHelper {
|
||||||
|
|
||||||
|
public static Lease createLease(String leaseKey, String leaseOwner, Collection<String> parentShardIds) {
|
||||||
|
return createLease(leaseKey, leaseOwner, parentShardIds, Collections.emptySet(), ExtendedSequenceNumber.LATEST);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Lease createLease(String leaseKey, String leaseOwner, Collection<String> parentShardIds, Collection<String> childShardIds) {
|
||||||
|
return createLease(leaseKey, leaseOwner, parentShardIds, childShardIds, ExtendedSequenceNumber.LATEST);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Lease createLease(String leaseKey, String leaseOwner, Collection<String> parentShardIds,
|
||||||
|
Collection<String> childShardIds, ExtendedSequenceNumber extendedSequenceNumber) {
|
||||||
|
Lease lease = new Lease();
|
||||||
|
lease.leaseKey(leaseKey);
|
||||||
|
lease.leaseOwner(leaseOwner);
|
||||||
|
lease.parentShardIds(parentShardIds);
|
||||||
|
lease.childShardIds(childShardIds);
|
||||||
|
lease.checkpoint(extendedSequenceNumber);
|
||||||
|
|
||||||
|
return lease;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -68,7 +68,7 @@ public class ShardObjectHelper {
|
||||||
String parentShardId,
|
String parentShardId,
|
||||||
String adjacentParentShardId,
|
String adjacentParentShardId,
|
||||||
SequenceNumberRange sequenceNumberRange) {
|
SequenceNumberRange sequenceNumberRange) {
|
||||||
return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, null);
|
return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, HashKeyRange.builder().startingHashKey("1").endingHashKey("100").build());
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Helper method to create a new shard object.
|
/** Helper method to create a new shard object.
|
||||||
|
|
|
||||||
|
|
@ -118,7 +118,8 @@ public class ShardSyncTaskIntegrationTest {
|
||||||
leaseRefresher.deleteAll();
|
leaseRefresher.deleteAll();
|
||||||
Set<String> shardIds = shardDetector.listShards().stream().map(Shard::shardId).collect(Collectors.toSet());
|
Set<String> shardIds = shardDetector.listShards().stream().map(Shard::shardId).collect(Collectors.toSet());
|
||||||
ShardSyncTask syncTask = new ShardSyncTask(shardDetector, leaseRefresher,
|
ShardSyncTask syncTask = new ShardSyncTask(shardDetector, leaseRefresher,
|
||||||
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), false, false, 0L,
|
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST),
|
||||||
|
false, true, false, 0L,
|
||||||
hierarchicalShardSyncer, NULL_METRICS_FACTORY);
|
hierarchicalShardSyncer, NULL_METRICS_FACTORY);
|
||||||
syncTask.call();
|
syncTask.call();
|
||||||
List<Lease> leases = leaseRefresher.listLeases();
|
List<Lease> leases = leaseRefresher.listLeases();
|
||||||
|
|
|
||||||
|
|
@ -127,16 +127,37 @@ public class DynamoDBLeaseCoordinatorIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
assertNotNull(lease);
|
assertNotNull(lease);
|
||||||
ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint");
|
final ExtendedSequenceNumber initialCheckpoint = new ExtendedSequenceNumber("initialCheckpoint");
|
||||||
|
final ExtendedSequenceNumber pendingCheckpoint = new ExtendedSequenceNumber("pendingCheckpoint");
|
||||||
|
final ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint");
|
||||||
|
final byte[] checkpointState = "checkpointState".getBytes();
|
||||||
|
|
||||||
// lease's leaseCounter is wrong at this point, but it shouldn't matter.
|
// lease's leaseCounter is wrong at this point, but it shouldn't matter.
|
||||||
|
assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), initialCheckpoint, lease.concurrencyToken()));
|
||||||
|
|
||||||
|
final Lease leaseFromDDBAtInitialCheckpoint = leaseRefresher.getLease(lease.leaseKey());
|
||||||
|
lease.leaseCounter(lease.leaseCounter() + 1);
|
||||||
|
lease.checkpoint(initialCheckpoint);
|
||||||
|
lease.leaseOwner(coordinator.workerIdentifier());
|
||||||
|
assertEquals(lease, leaseFromDDBAtInitialCheckpoint);
|
||||||
|
|
||||||
|
dynamoDBCheckpointer.prepareCheckpoint(lease.leaseKey(), pendingCheckpoint, lease.concurrencyToken().toString(), checkpointState);
|
||||||
|
|
||||||
|
final Lease leaseFromDDBAtPendingCheckpoint = leaseRefresher.getLease(lease.leaseKey());
|
||||||
|
lease.leaseCounter(lease.leaseCounter() + 1);
|
||||||
|
lease.checkpoint(initialCheckpoint);
|
||||||
|
lease.pendingCheckpoint(pendingCheckpoint);
|
||||||
|
lease.pendingCheckpointState(checkpointState);
|
||||||
|
assertEquals(lease, leaseFromDDBAtPendingCheckpoint);
|
||||||
|
|
||||||
assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), newCheckpoint, lease.concurrencyToken()));
|
assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), newCheckpoint, lease.concurrencyToken()));
|
||||||
|
|
||||||
Lease fromDynamo = leaseRefresher.getLease(lease.leaseKey());
|
final Lease leaseFromDDBAtNewCheckpoint = leaseRefresher.getLease(lease.leaseKey());
|
||||||
|
|
||||||
lease.leaseCounter(lease.leaseCounter() + 1);
|
lease.leaseCounter(lease.leaseCounter() + 1);
|
||||||
lease.checkpoint(newCheckpoint);
|
lease.checkpoint(newCheckpoint);
|
||||||
lease.leaseOwner(coordinator.workerIdentifier());
|
lease.pendingCheckpoint(null);
|
||||||
assertEquals(lease, fromDynamo);
|
lease.pendingCheckpointState(null);
|
||||||
|
assertEquals(lease, leaseFromDDBAtNewCheckpoint);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,21 @@
|
||||||
*/
|
*/
|
||||||
package software.amazon.kinesis.leases.dynamodb;
|
package software.amazon.kinesis.leases.dynamodb;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
import software.amazon.awssdk.services.kinesis.model.HashKeyRange;
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
|
import software.amazon.kinesis.leases.Lease;
|
||||||
|
import software.amazon.kinesis.leases.LeaseIntegrationTest;
|
||||||
|
import software.amazon.kinesis.leases.UpdateField;
|
||||||
|
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
|
@ -24,19 +39,6 @@ import static org.mockito.Matchers.eq;
|
||||||
import static org.mockito.Mockito.doNothing;
|
import static org.mockito.Mockito.doNothing;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.runner.RunWith;
|
|
||||||
import org.mockito.runners.MockitoJUnitRunner;
|
|
||||||
|
|
||||||
import software.amazon.kinesis.leases.Lease;
|
|
||||||
import software.amazon.kinesis.leases.LeaseIntegrationTest;
|
|
||||||
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
|
||||||
|
|
||||||
@RunWith(MockitoJUnitRunner.class)
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest {
|
public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest {
|
||||||
|
|
||||||
|
|
@ -71,7 +73,7 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest
|
||||||
Collection<Lease> expected = builder.build().values();
|
Collection<Lease> expected = builder.build().values();
|
||||||
|
|
||||||
// The / 3 here ensures that we will test Dynamo's paging mechanics.
|
// The / 3 here ensures that we will test Dynamo's paging mechanics.
|
||||||
List<Lease> actual = leaseRefresher.list(numRecordsToPut / 3);
|
List<Lease> actual = leaseRefresher.list(numRecordsToPut / 3, null);
|
||||||
|
|
||||||
for (Lease lease : actual) {
|
for (Lease lease : actual) {
|
||||||
assertNotNull(expected.remove(lease));
|
assertNotNull(expected.remove(lease));
|
||||||
|
|
@ -101,6 +103,38 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest
|
||||||
assertNull(actual);
|
assertNull(actual);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests leaseRefresher.updateLeaseWithMetaInfo() when the lease is deleted before updating it with meta info
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testDeleteLeaseThenUpdateLeaseWithMetaInfo() throws LeasingException {
|
||||||
|
TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher);
|
||||||
|
Lease lease = builder.withLease("1").build().get("1");
|
||||||
|
final String leaseKey = lease.leaseKey();
|
||||||
|
leaseRefresher.deleteLease(lease);
|
||||||
|
leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE);
|
||||||
|
final Lease deletedLease = leaseRefresher.getLease(leaseKey);
|
||||||
|
Assert.assertNull(deletedLease);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests leaseRefresher.updateLeaseWithMetaInfo() on hashKeyRange update
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testUpdateLeaseWithMetaInfo() throws LeasingException {
|
||||||
|
TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher);
|
||||||
|
Lease lease = builder.withLease("1").build().get("1");
|
||||||
|
final String leaseKey = lease.leaseKey();
|
||||||
|
final HashKeyRangeForLease hashKeyRangeForLease = HashKeyRangeForLease.fromHashKeyRange(HashKeyRange.builder()
|
||||||
|
.startingHashKey("1")
|
||||||
|
.endingHashKey("2")
|
||||||
|
.build());
|
||||||
|
lease.hashKeyRange(hashKeyRangeForLease);
|
||||||
|
leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE);
|
||||||
|
final Lease updatedLease = leaseRefresher.getLease(leaseKey);
|
||||||
|
Assert.assertEquals(lease, updatedLease);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests leaseRefresher.holdLease's success scenario.
|
* Tests leaseRefresher.holdLease's success scenario.
|
||||||
*/
|
*/
|
||||||
|
|
@ -239,6 +273,18 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest
|
||||||
assertNull(newLease);
|
assertNull(newLease);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUpdateLease() throws LeasingException {
|
||||||
|
TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher);
|
||||||
|
Lease lease = builder.withLease("1").build().get("1");
|
||||||
|
Lease updatedLease = lease.copy();
|
||||||
|
updatedLease.childShardIds(Collections.singleton("updatedChildShardId"));
|
||||||
|
|
||||||
|
leaseRefresher.updateLease(updatedLease);
|
||||||
|
Lease newLease = leaseRefresher.getLease(lease.leaseKey());
|
||||||
|
assertEquals(Collections.singleton("updatedChildShardId"), newLease.childShardIds());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests deleteLease when a lease does not exist.
|
* Tests deleteLease when a lease does not exist.
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -37,6 +37,7 @@ import org.junit.runner.RunWith;
|
||||||
import org.mockito.Mock;
|
import org.mockito.Mock;
|
||||||
import org.mockito.runners.MockitoJUnitRunner;
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
|
||||||
|
import software.amazon.kinesis.common.HashKeyRangeForLease;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
import software.amazon.kinesis.leases.LeaseRefresher;
|
import software.amazon.kinesis.leases.LeaseRefresher;
|
||||||
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
import software.amazon.kinesis.leases.exceptions.DependencyException;
|
||||||
|
|
@ -55,7 +56,8 @@ public class DynamoDBLeaseRenewerTest {
|
||||||
private LeaseRefresher leaseRefresher;
|
private LeaseRefresher leaseRefresher;
|
||||||
|
|
||||||
private static Lease newLease(String leaseKey) {
|
private static Lease newLease(String leaseKey) {
|
||||||
return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>());
|
return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null,
|
||||||
|
new HashSet<>(), new HashSet<>(), null, HashKeyRangeForLease.deserialize("1", "2"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|
|
||||||
|
|
@ -16,22 +16,17 @@ package software.amazon.kinesis.leases.dynamodb;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.mockito.runners.MockitoJUnitRunner;
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
import software.amazon.kinesis.leases.Lease;
|
import software.amazon.kinesis.leases.Lease;
|
||||||
import software.amazon.kinesis.leases.LeaseIntegrationTest;
|
import software.amazon.kinesis.leases.LeaseIntegrationTest;
|
||||||
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
import software.amazon.kinesis.leases.exceptions.LeasingException;
|
||||||
import software.amazon.kinesis.metrics.NullMetricsFactory;
|
import software.amazon.kinesis.metrics.NullMetricsFactory;
|
||||||
|
|
||||||
import static org.hamcrest.CoreMatchers.equalTo;
|
import static org.hamcrest.CoreMatchers.equalTo;
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertThat;
|
import static org.junit.Assert.assertThat;
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
@RunWith(MockitoJUnitRunner.class)
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest {
|
public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest {
|
||||||
|
|
@ -105,9 +100,30 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest {
|
||||||
|
|
||||||
builder.withLease("4", "bar").build();
|
builder.withLease("4", "bar").build();
|
||||||
|
|
||||||
|
// setting multiplier to unusually high number to avoid very old lease taking
|
||||||
|
taker.withVeryOldLeaseDurationNanosMultipler(5000000000L);
|
||||||
builder.takeMutateAssert(taker, 2);
|
builder.takeMutateAssert(taker, 2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verify that we take all very old leases by setting up an environment where there are 4 leases and 2 workers,
|
||||||
|
* only one of which holds a lease. This leaves 3 free leases. LeaseTaker should take all 3 leases since they
|
||||||
|
* are denoted as very old.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testVeryOldLeaseTaker() throws LeasingException {
|
||||||
|
TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher);
|
||||||
|
|
||||||
|
for (int i = 0; i < 3; i++) {
|
||||||
|
builder.withLease(Integer.toString(i), null);
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.withLease("4", "bar").build();
|
||||||
|
|
||||||
|
// setting multiplier to unusually high number to avoid very old lease taking
|
||||||
|
builder.takeMutateAssert(taker, 3);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Verify that when getAllLeases() is called, DynamoDBLeaseTaker
|
* Verify that when getAllLeases() is called, DynamoDBLeaseTaker
|
||||||
* - does not call listLeases()
|
* - does not call listLeases()
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,7 @@ import static org.junit.Assert.assertNotNull;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
|
|
@ -72,6 +73,7 @@ public class TestHarnessBuilder {
|
||||||
lease.leaseCounter(0L);
|
lease.leaseCounter(0L);
|
||||||
lease.leaseOwner(owner);
|
lease.leaseOwner(owner);
|
||||||
lease.parentShardIds(Collections.singleton("parentShardId"));
|
lease.parentShardIds(Collections.singleton("parentShardId"));
|
||||||
|
lease.childShardIds(new HashSet<>());
|
||||||
lease.leaseKey(shardId);
|
lease.leaseKey(shardId);
|
||||||
|
|
||||||
return lease;
|
return lease;
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
@ -40,6 +41,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
|
||||||
public class BlockOnParentShardTaskTest {
|
public class BlockOnParentShardTaskTest {
|
||||||
private final long backoffTimeInMillis = 50L;
|
private final long backoffTimeInMillis = 50L;
|
||||||
private final String shardId = "shardId-97";
|
private final String shardId = "shardId-97";
|
||||||
|
private final String streamId = "123:stream:146";
|
||||||
private final String concurrencyToken = "testToken";
|
private final String concurrencyToken = "testToken";
|
||||||
private final List<String> emptyParentShardIds = new ArrayList<String>();
|
private final List<String> emptyParentShardIds = new ArrayList<String>();
|
||||||
private ShardInfo shardInfo;
|
private ShardInfo shardInfo;
|
||||||
|
|
@ -73,7 +75,7 @@ public class BlockOnParentShardTaskTest {
|
||||||
* @throws DependencyException
|
* @throws DependencyException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public final void testCallWhenParentsHaveFinished()
|
public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinished()
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
|
||||||
ShardInfo shardInfo = null;
|
ShardInfo shardInfo = null;
|
||||||
|
|
@ -107,6 +109,50 @@ public class BlockOnParentShardTaskTest {
|
||||||
assertNull(result.getException());
|
assertNull(result.getException());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test call() when there are 1-2 parent shards that have been fully processed.
|
||||||
|
* @throws ProvisionedThroughputException
|
||||||
|
* @throws InvalidStateException
|
||||||
|
* @throws DependencyException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinishedMultiStream()
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
|
||||||
|
ShardInfo shardInfo = null;
|
||||||
|
BlockOnParentShardTask task = null;
|
||||||
|
String parent1LeaseKey = streamId + ":" + "shardId-1";
|
||||||
|
String parent2LeaseKey = streamId + ":" + "shardId-2";
|
||||||
|
String parent1ShardId = "shardId-1";
|
||||||
|
String parent2ShardId = "shardId-2";
|
||||||
|
List<String> parentShardIds = new ArrayList<>();
|
||||||
|
TaskResult result = null;
|
||||||
|
|
||||||
|
Lease parent1Lease = new Lease();
|
||||||
|
parent1Lease.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
Lease parent2Lease = new Lease();
|
||||||
|
parent2Lease.checkpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
|
||||||
|
LeaseRefresher leaseRefresher = mock(LeaseRefresher.class);
|
||||||
|
when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease);
|
||||||
|
when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease);
|
||||||
|
|
||||||
|
// test single parent
|
||||||
|
parentShardIds.add(parent1ShardId);
|
||||||
|
shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON,
|
||||||
|
streamId);
|
||||||
|
task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis);
|
||||||
|
result = task.call();
|
||||||
|
assertNull(result.getException());
|
||||||
|
|
||||||
|
// test two parents
|
||||||
|
parentShardIds.add(parent2ShardId);
|
||||||
|
shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId);
|
||||||
|
task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis);
|
||||||
|
result = task.call();
|
||||||
|
assertNull(result.getException());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test call() when there are 1-2 parent shards that have NOT been fully processed.
|
* Test call() when there are 1-2 parent shards that have NOT been fully processed.
|
||||||
* @throws ProvisionedThroughputException
|
* @throws ProvisionedThroughputException
|
||||||
|
|
@ -149,6 +195,50 @@ public class BlockOnParentShardTaskTest {
|
||||||
assertNotNull(result.getException());
|
assertNotNull(result.getException());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test call() when there are 1-2 parent shards that have NOT been fully processed.
|
||||||
|
* @throws ProvisionedThroughputException
|
||||||
|
* @throws InvalidStateException
|
||||||
|
* @throws DependencyException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public final void testCallWhenParentsHaveNotFinishedMultiStream()
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
|
||||||
|
ShardInfo shardInfo = null;
|
||||||
|
BlockOnParentShardTask task = null;
|
||||||
|
String parent1LeaseKey = streamId + ":" + "shardId-1";
|
||||||
|
String parent2LeaseKey = streamId + ":" + "shardId-2";
|
||||||
|
String parent1ShardId = "shardId-1";
|
||||||
|
String parent2ShardId = "shardId-2";
|
||||||
|
List<String> parentShardIds = new ArrayList<>();
|
||||||
|
TaskResult result = null;
|
||||||
|
|
||||||
|
Lease parent1Lease = new Lease();
|
||||||
|
parent1Lease.checkpoint(ExtendedSequenceNumber.LATEST);
|
||||||
|
Lease parent2Lease = new Lease();
|
||||||
|
// mock a sequence number checkpoint
|
||||||
|
parent2Lease.checkpoint(new ExtendedSequenceNumber("98182584034"));
|
||||||
|
|
||||||
|
LeaseRefresher leaseRefresher = mock(LeaseRefresher.class);
|
||||||
|
when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease);
|
||||||
|
when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease);
|
||||||
|
|
||||||
|
// test single parent
|
||||||
|
parentShardIds.add(parent1ShardId);
|
||||||
|
shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId);
|
||||||
|
task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis);
|
||||||
|
result = task.call();
|
||||||
|
assertNotNull(result.getException());
|
||||||
|
|
||||||
|
// test two parents
|
||||||
|
parentShardIds.add(parent2ShardId);
|
||||||
|
shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId);
|
||||||
|
task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis);
|
||||||
|
result = task.call();
|
||||||
|
assertNotNull(result.getException());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test call() with 1 parent shard before and after it is completely processed.
|
* Test call() with 1 parent shard before and after it is completely processed.
|
||||||
* @throws ProvisionedThroughputException
|
* @throws ProvisionedThroughputException
|
||||||
|
|
|
||||||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue