Add periodic hash range auditor, hole detection, and recovery (#43)
* Add periodic hash range auditor, hole detection, and recovery * Add unit tests for hash range hole recovery * Fix max hash range bug * Address PR feedback * Fix DDB hash key persistence bug
This commit is contained in:
parent
6b474b7390
commit
3a88a60a4e
15 changed files with 1191 additions and 32 deletions
|
|
@ -14,39 +14,98 @@
|
||||||
*/
|
*/
|
||||||
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.math.BigInteger;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
|
||||||
|
import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
|
||||||
|
import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
|
||||||
|
import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.UpdateField;
|
||||||
|
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
|
||||||
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
|
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
|
||||||
|
import com.amazonaws.services.kinesis.model.Shard;
|
||||||
|
import com.amazonaws.util.CollectionUtils;
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.collect.ComparisonChain;
|
||||||
import lombok.EqualsAndHashCode;
|
import lombok.EqualsAndHashCode;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
import org.apache.commons.lang3.Validate;
|
import org.apache.commons.lang3.Validate;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
|
import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.fromHashKeyRange;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The top level orchestrator for coordinating the periodic shard sync related
|
* The top level orchestrator for coordinating the periodic shard sync related activities. If the configured
|
||||||
* activities.
|
* {@link ShardSyncStrategyType} is PERIODIC, this class will be the main shard sync orchestrator. For non-PERIODIC
|
||||||
|
* strategies, this class will serve as an internal auditor that periodically checks if the full hash range is covered
|
||||||
|
* by currently held leases, and initiates a recovery shard sync if not.
|
||||||
*/
|
*/
|
||||||
@Getter
|
@Getter
|
||||||
@EqualsAndHashCode
|
@EqualsAndHashCode
|
||||||
class PeriodicShardSyncManager {
|
class PeriodicShardSyncManager {
|
||||||
private static final Log LOG = LogFactory.getLog(PeriodicShardSyncManager.class);
|
private static final Log LOG = LogFactory.getLog(PeriodicShardSyncManager.class);
|
||||||
private static final long INITIAL_DELAY = 0;
|
private static final long INITIAL_DELAY = 0;
|
||||||
private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000;
|
|
||||||
|
/** DEFAULT interval is used for PERIODIC {@link ShardSyncStrategyType}. */
|
||||||
|
private static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000L;
|
||||||
|
|
||||||
|
/** AUDITOR interval is used for non-PERIODIC {@link ShardSyncStrategyType} auditor processes. */
|
||||||
|
private static final long AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L;
|
||||||
|
|
||||||
|
/** Parameters for validating hash range completeness when running in auditor mode. */
|
||||||
|
@VisibleForTesting
|
||||||
|
static final BigInteger MIN_HASH_KEY = BigInteger.ZERO;
|
||||||
|
@VisibleForTesting
|
||||||
|
static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE);
|
||||||
|
@VisibleForTesting
|
||||||
|
static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3;
|
||||||
|
private final HashRangeHoleTracker hashRangeHoleTracker = new HashRangeHoleTracker();
|
||||||
|
|
||||||
private final String workerId;
|
private final String workerId;
|
||||||
private final LeaderDecider leaderDecider;
|
private final LeaderDecider leaderDecider;
|
||||||
private final ITask metricsEmittingShardSyncTask;
|
private final ITask metricsEmittingShardSyncTask;
|
||||||
private final ScheduledExecutorService shardSyncThreadPool;
|
private final ScheduledExecutorService shardSyncThreadPool;
|
||||||
|
private final ILeaseManager<KinesisClientLease> leaseManager;
|
||||||
|
private final IKinesisProxy kinesisProxy;
|
||||||
|
private final boolean isAuditorMode;
|
||||||
|
private final long periodicShardSyncIntervalMillis;
|
||||||
private boolean isRunning;
|
private boolean isRunning;
|
||||||
|
|
||||||
PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, IMetricsFactory metricsFactory) {
|
PeriodicShardSyncManager(String workerId,
|
||||||
this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory);
|
LeaderDecider leaderDecider,
|
||||||
|
ShardSyncTask shardSyncTask,
|
||||||
|
IMetricsFactory metricsFactory,
|
||||||
|
ILeaseManager<KinesisClientLease> leaseManager,
|
||||||
|
IKinesisProxy kinesisProxy,
|
||||||
|
boolean isAuditorMode) {
|
||||||
|
this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory,
|
||||||
|
leaseManager, kinesisProxy, isAuditorMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, ScheduledExecutorService shardSyncThreadPool, IMetricsFactory metricsFactory) {
|
PeriodicShardSyncManager(String workerId,
|
||||||
|
LeaderDecider leaderDecider,
|
||||||
|
ShardSyncTask shardSyncTask,
|
||||||
|
ScheduledExecutorService shardSyncThreadPool,
|
||||||
|
IMetricsFactory metricsFactory,
|
||||||
|
ILeaseManager<KinesisClientLease> leaseManager,
|
||||||
|
IKinesisProxy kinesisProxy,
|
||||||
|
boolean isAuditorMode) {
|
||||||
Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager.");
|
Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager.");
|
||||||
Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager.");
|
Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager.");
|
||||||
Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager.");
|
Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager.");
|
||||||
|
|
@ -54,6 +113,16 @@ class PeriodicShardSyncManager {
|
||||||
this.leaderDecider = leaderDecider;
|
this.leaderDecider = leaderDecider;
|
||||||
this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory);
|
this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory);
|
||||||
this.shardSyncThreadPool = shardSyncThreadPool;
|
this.shardSyncThreadPool = shardSyncThreadPool;
|
||||||
|
this.leaseManager = leaseManager;
|
||||||
|
this.kinesisProxy = kinesisProxy;
|
||||||
|
this.isAuditorMode = isAuditorMode;
|
||||||
|
if (isAuditorMode) {
|
||||||
|
Validate.notNull(this.leaseManager, "LeaseManager is required for non-PERIODIC shard sync strategies.");
|
||||||
|
Validate.notNull(this.kinesisProxy, "KinesisProxy is required for non-PERIODIC shard sync strategies.");
|
||||||
|
this.periodicShardSyncIntervalMillis = AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS;
|
||||||
|
} else {
|
||||||
|
this.periodicShardSyncIntervalMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized TaskResult start() {
|
public synchronized TaskResult start() {
|
||||||
|
|
@ -67,7 +136,7 @@ class PeriodicShardSyncManager {
|
||||||
};
|
};
|
||||||
|
|
||||||
shardSyncThreadPool
|
shardSyncThreadPool
|
||||||
.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS,
|
.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, periodicShardSyncIntervalMillis,
|
||||||
TimeUnit.MILLISECONDS);
|
TimeUnit.MILLISECONDS);
|
||||||
isRunning = true;
|
isRunning = true;
|
||||||
}
|
}
|
||||||
|
|
@ -95,11 +164,227 @@ class PeriodicShardSyncManager {
|
||||||
|
|
||||||
private void runShardSync() {
|
private void runShardSync() {
|
||||||
if (leaderDecider.isLeader(workerId)) {
|
if (leaderDecider.isLeader(workerId)) {
|
||||||
LOG.debug(String.format("WorkerId %s is a leader, running the shard sync task", workerId));
|
LOG.debug("WorkerId " + workerId + " is a leader, running the shard sync task");
|
||||||
metricsEmittingShardSyncTask.call();
|
|
||||||
|
try {
|
||||||
|
final ShardSyncResponse shardSyncResponse = checkForShardSync();
|
||||||
|
if (shardSyncResponse.shouldDoShardSync()) {
|
||||||
|
LOG.info("Periodic shard syncer initiating shard sync due to the reason - " +
|
||||||
|
shardSyncResponse.reasonForDecision());
|
||||||
|
metricsEmittingShardSyncTask.call();
|
||||||
|
} else {
|
||||||
|
LOG.info("Skipping shard sync due to the reason - " + shardSyncResponse.reasonForDecision());
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Caught exception while running periodic shard syncer.", e);
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
LOG.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId));
|
LOG.debug("WorkerId " + workerId + " is not a leader, not running the shard sync task");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
ShardSyncResponse checkForShardSync() throws DependencyException, InvalidStateException,
|
||||||
|
ProvisionedThroughputException {
|
||||||
|
|
||||||
|
if (!isAuditorMode) {
|
||||||
|
// If we are running with PERIODIC shard sync strategy, we should sync every time.
|
||||||
|
return new ShardSyncResponse(true, "Syncing every time with PERIODIC shard sync strategy.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Get current leases from DynamoDB.
|
||||||
|
final List<KinesisClientLease> currentLeases = leaseManager.listLeases();
|
||||||
|
|
||||||
|
if (CollectionUtils.isNullOrEmpty(currentLeases)) {
|
||||||
|
// If the current leases are null or empty, then we need to initiate a shard sync.
|
||||||
|
LOG.info("No leases found. Will trigger a shard sync.");
|
||||||
|
return new ShardSyncResponse(true, "No leases found.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check if there are any holes in the hash range covered by current leases. Return the first hole if present.
|
||||||
|
Optional<HashRangeHole> hashRangeHoleOpt = hasHoleInLeases(currentLeases);
|
||||||
|
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 use the high confidence factor
|
||||||
|
// to avoid shard sync on any holes during resharding and lease cleanups, or other intermittent issues.
|
||||||
|
final boolean hasHoleWithHighConfidence =
|
||||||
|
hashRangeHoleTracker.hashHighConfidenceOfHoleWith(hashRangeHoleOpt.get());
|
||||||
|
|
||||||
|
return new ShardSyncResponse(hasHoleWithHighConfidence,
|
||||||
|
"Detected the same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. " +
|
||||||
|
"Will initiate shard sync after reaching threshold: " + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY);
|
||||||
|
} else {
|
||||||
|
// If hole is not present, clear any previous hole tracking and return false.
|
||||||
|
hashRangeHoleTracker.reset();
|
||||||
|
return new ShardSyncResponse(false, "Hash range is complete.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Optional<HashRangeHole> hasHoleInLeases(List<KinesisClientLease> leases) {
|
||||||
|
// Filter out any leases with checkpoints other than SHARD_END
|
||||||
|
final List<KinesisClientLease> activeLeases = leases.stream()
|
||||||
|
.filter(lease -> lease.getCheckpoint() != null && !lease.getCheckpoint().isShardEnd())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
final List<KinesisClientLease> activeLeasesWithHashRanges = fillWithHashRangesIfRequired(activeLeases);
|
||||||
|
return checkForHoleInHashKeyRanges(activeLeasesWithHashRanges);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<KinesisClientLease> fillWithHashRangesIfRequired(List<KinesisClientLease> activeLeases) {
|
||||||
|
final List<KinesisClientLease> activeLeasesWithNoHashRanges = activeLeases.stream()
|
||||||
|
.filter(lease -> lease.getHashKeyRange() == null).collect(Collectors.toList());
|
||||||
|
|
||||||
|
if (activeLeasesWithNoHashRanges.isEmpty()) {
|
||||||
|
return activeLeases;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Fetch shards from Kinesis to fill in the in-memory hash ranges
|
||||||
|
final Map<String, Shard> kinesisShards = kinesisProxy.getShardList().stream()
|
||||||
|
.collect(Collectors.toMap(Shard::getShardId, shard -> shard));
|
||||||
|
|
||||||
|
return activeLeases.stream().map(lease -> {
|
||||||
|
if (lease.getHashKeyRange() == null) {
|
||||||
|
final String shardId = lease.getLeaseKey();
|
||||||
|
final Shard shard = kinesisShards.get(shardId);
|
||||||
|
if (shard == null) {
|
||||||
|
return lease;
|
||||||
|
}
|
||||||
|
lease.setHashKeyRange(fromHashKeyRange(shard.getHashKeyRange()));
|
||||||
|
|
||||||
|
try {
|
||||||
|
leaseManager.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.warn("Unable to update hash range information for lease " + lease.getLeaseKey() +
|
||||||
|
". This may result in explicit lease sync.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return lease;
|
||||||
|
}).filter(lease -> lease.getHashKeyRange() != null).collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
static Optional<HashRangeHole> checkForHoleInHashKeyRanges(List<KinesisClientLease> leasesWithHashKeyRanges) {
|
||||||
|
// Sort the hash ranges by starting hash key
|
||||||
|
final List<KinesisClientLease> sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges);
|
||||||
|
if (sortedLeasesWithHashKeyRanges.isEmpty()) {
|
||||||
|
LOG.error("No leases with valid hash ranges found.");
|
||||||
|
return Optional.of(new HashRangeHole());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Validate the hash range bounds
|
||||||
|
final KinesisClientLease minHashKeyLease = sortedLeasesWithHashKeyRanges.get(0);
|
||||||
|
final KinesisClientLease maxHashKeyLease =
|
||||||
|
sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1);
|
||||||
|
if (!minHashKeyLease.getHashKeyRange().startingHashKey().equals(MIN_HASH_KEY) ||
|
||||||
|
!maxHashKeyLease.getHashKeyRange().endingHashKey().equals(MAX_HASH_KEY)) {
|
||||||
|
LOG.error("Incomplete hash range found between " + minHashKeyLease + " and " + maxHashKeyLease);
|
||||||
|
return Optional.of(new HashRangeHole(minHashKeyLease.getHashKeyRange(), maxHashKeyLease.getHashKeyRange()));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check for any holes in the sorted hash range intervals
|
||||||
|
if (sortedLeasesWithHashKeyRanges.size() > 1) {
|
||||||
|
KinesisClientLease leftmostLeaseToReportInCaseOfHole = minHashKeyLease;
|
||||||
|
HashKeyRangeForLease leftLeaseHashRange = leftmostLeaseToReportInCaseOfHole.getHashKeyRange();
|
||||||
|
|
||||||
|
for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) {
|
||||||
|
final KinesisClientLease rightLease = sortedLeasesWithHashKeyRanges.get(i);
|
||||||
|
final HashKeyRangeForLease rightLeaseHashRange = rightLease.getHashKeyRange();
|
||||||
|
final BigInteger rangeDiff =
|
||||||
|
rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey());
|
||||||
|
// We have overlapping leases when rangeDiff is 0 or negative.
|
||||||
|
// signum() will be -1 for negative and 0 if value is 0.
|
||||||
|
// Merge the ranges for further tracking.
|
||||||
|
if (rangeDiff.signum() <= 0) {
|
||||||
|
leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(),
|
||||||
|
leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey()));
|
||||||
|
} else {
|
||||||
|
// We have non-overlapping leases when rangeDiff is positive. signum() will be 1 in this case.
|
||||||
|
// If rangeDiff is 1, then it is a continuous hash range. If not, there is a hole.
|
||||||
|
if (!rangeDiff.equals(BigInteger.ONE)) {
|
||||||
|
LOG.error("Incomplete hash range found between " + leftmostLeaseToReportInCaseOfHole +
|
||||||
|
" and " + rightLease);
|
||||||
|
return Optional.of(new HashRangeHole(leftmostLeaseToReportInCaseOfHole.getHashKeyRange(),
|
||||||
|
rightLease.getHashKeyRange()));
|
||||||
|
}
|
||||||
|
|
||||||
|
leftmostLeaseToReportInCaseOfHole = rightLease;
|
||||||
|
leftLeaseHashRange = rightLeaseHashRange;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
static List<KinesisClientLease> sortLeasesByHashRange(List<KinesisClientLease> leasesWithHashKeyRanges) {
|
||||||
|
if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) {
|
||||||
|
return leasesWithHashKeyRanges;
|
||||||
|
}
|
||||||
|
Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator());
|
||||||
|
return leasesWithHashKeyRanges;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Value
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
@VisibleForTesting
|
||||||
|
static class ShardSyncResponse {
|
||||||
|
private final boolean shouldDoShardSync;
|
||||||
|
private final String reasonForDecision;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Value
|
||||||
|
private static class HashRangeHole {
|
||||||
|
private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole;
|
||||||
|
private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole;
|
||||||
|
|
||||||
|
HashRangeHole() {
|
||||||
|
hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole,
|
||||||
|
HashKeyRangeForLease hashRangeAtEndOfPossibleHole) {
|
||||||
|
this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole;
|
||||||
|
this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class HashRangeHoleTracker {
|
||||||
|
private HashRangeHole hashRangeHole;
|
||||||
|
@Getter
|
||||||
|
private Integer numConsecutiveHoles;
|
||||||
|
|
||||||
|
public boolean hashHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) {
|
||||||
|
if (hashRangeHole.equals(this.hashRangeHole)) {
|
||||||
|
++this.numConsecutiveHoles;
|
||||||
|
} else {
|
||||||
|
this.hashRangeHole = hashRangeHole;
|
||||||
|
this.numConsecutiveHoles = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return numConsecutiveHoles >= CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void reset() {
|
||||||
|
this.hashRangeHole = null;
|
||||||
|
this.numConsecutiveHoles = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class HashKeyRangeComparator implements Comparator<KinesisClientLease>, Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(KinesisClientLease lease, KinesisClientLease otherLease) {
|
||||||
|
Validate.notNull(lease);
|
||||||
|
Validate.notNull(otherLease);
|
||||||
|
Validate.notNull(lease.getHashKeyRange());
|
||||||
|
Validate.notNull(otherLease.getHashKeyRange());
|
||||||
|
return ComparisonChain.start()
|
||||||
|
.compare(lease.getHashKeyRange().startingHashKey(), otherLease.getHashKeyRange().startingHashKey())
|
||||||
|
.compare(lease.getHashKeyRange().endingHashKey(), otherLease.getHashKeyRange().endingHashKey())
|
||||||
|
.result();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -577,8 +577,9 @@ public class Worker implements Runnable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create shard sync strategy and corresponding {@link LeaderDecider} based on provided configs. PERIODIC
|
* Create shard sync strategy and corresponding {@link LeaderDecider} based on provided configs. PERIODIC
|
||||||
* {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy. All other
|
* {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy, and does not permit
|
||||||
* {@link ShardSyncStrategyType}s permit only a default single-leader strategy.
|
* skipping shard syncs if the hash range is complete. All other {@link ShardSyncStrategyType}s permit only a
|
||||||
|
* default single-leader strategy, and will skip shard syncs unless a hole in the hash range is detected.
|
||||||
*/
|
*/
|
||||||
private void createShardSyncStrategy(ShardSyncStrategyType strategyType,
|
private void createShardSyncStrategy(ShardSyncStrategyType strategyType,
|
||||||
LeaderDecider leaderDecider,
|
LeaderDecider leaderDecider,
|
||||||
|
|
@ -587,7 +588,7 @@ public class Worker implements Runnable {
|
||||||
case PERIODIC:
|
case PERIODIC:
|
||||||
this.leaderDecider = getOrCreateLeaderDecider(leaderDecider);
|
this.leaderDecider = getOrCreateLeaderDecider(leaderDecider);
|
||||||
this.leaderElectedPeriodicShardSyncManager =
|
this.leaderElectedPeriodicShardSyncManager =
|
||||||
getOrCreatePeriodicShardSyncManager(periodicShardSyncManager);
|
getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, false);
|
||||||
this.shardSyncStrategy = createPeriodicShardSyncStrategy();
|
this.shardSyncStrategy = createPeriodicShardSyncStrategy();
|
||||||
break;
|
break;
|
||||||
case SHARD_END:
|
case SHARD_END:
|
||||||
|
|
@ -598,7 +599,7 @@ public class Worker implements Runnable {
|
||||||
}
|
}
|
||||||
this.leaderDecider = getOrCreateLeaderDecider(null);
|
this.leaderDecider = getOrCreateLeaderDecider(null);
|
||||||
this.leaderElectedPeriodicShardSyncManager =
|
this.leaderElectedPeriodicShardSyncManager =
|
||||||
getOrCreatePeriodicShardSyncManager(periodicShardSyncManager);
|
getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, true);
|
||||||
this.shardSyncStrategy = createShardEndShardSyncStrategy();
|
this.shardSyncStrategy = createShardEndShardSyncStrategy();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1255,9 +1256,10 @@ public class Worker implements Runnable {
|
||||||
Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT);
|
Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT);
|
||||||
}
|
}
|
||||||
|
|
||||||
private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager) {
|
/** A non-null PeriodicShardSyncManager can only provided from unit tests. Any application code will create the
|
||||||
// TODO: Configure periodicShardSyncManager with either mandatory shard sync (PERIODIC) or hash range
|
* PeriodicShardSyncManager for the first time here. */
|
||||||
// validation based shard sync (SHARD_END) based on configured shard sync strategy
|
private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager,
|
||||||
|
boolean isAuditorMode) {
|
||||||
if (periodicShardSyncManager != null) {
|
if (periodicShardSyncManager != null) {
|
||||||
return periodicShardSyncManager;
|
return periodicShardSyncManager;
|
||||||
}
|
}
|
||||||
|
|
@ -1272,7 +1274,10 @@ public class Worker implements Runnable {
|
||||||
SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC,
|
SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC,
|
||||||
shardSyncer,
|
shardSyncer,
|
||||||
null),
|
null),
|
||||||
metricsFactory);
|
metricsFactory,
|
||||||
|
leaseCoordinator.getLeaseManager(),
|
||||||
|
streamConfig.getStreamProxy(),
|
||||||
|
isAuditorMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -141,6 +141,10 @@ 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();
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,79 @@
|
||||||
|
/*
|
||||||
|
* 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 com.amazonaws.services.kinesis.leases.impl;
|
||||||
|
|
||||||
|
import com.amazonaws.services.kinesis.model.HashKeyRange;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonAutoDetect;
|
||||||
|
import lombok.NonNull;
|
||||||
|
import lombok.Value;
|
||||||
|
import lombok.experimental.Accessors;
|
||||||
|
import org.apache.commons.lang3.Validate;
|
||||||
|
|
||||||
|
import java.math.BigInteger;
|
||||||
|
|
||||||
|
@Value
|
||||||
|
@Accessors(fluent = true)
|
||||||
|
@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY)
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.getStartingHashKey(), hashKeyRange.getEndingHashKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -30,6 +30,8 @@ public class KinesisClientLease extends Lease {
|
||||||
private ExtendedSequenceNumber pendingCheckpoint;
|
private ExtendedSequenceNumber pendingCheckpoint;
|
||||||
private Long ownerSwitchesSinceCheckpoint = 0L;
|
private Long ownerSwitchesSinceCheckpoint = 0L;
|
||||||
private Set<String> parentShardIds = new HashSet<String>();
|
private Set<String> parentShardIds = new HashSet<String>();
|
||||||
|
private Set<String> childShardIds = new HashSet<String>();
|
||||||
|
private HashKeyRangeForLease hashKeyRangeForLease;
|
||||||
|
|
||||||
public KinesisClientLease() {
|
public KinesisClientLease() {
|
||||||
|
|
||||||
|
|
@ -41,17 +43,22 @@ public class KinesisClientLease extends Lease {
|
||||||
this.pendingCheckpoint = other.getPendingCheckpoint();
|
this.pendingCheckpoint = other.getPendingCheckpoint();
|
||||||
this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint();
|
this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint();
|
||||||
this.parentShardIds.addAll(other.getParentShardIds());
|
this.parentShardIds.addAll(other.getParentShardIds());
|
||||||
|
this.childShardIds = other.getChildShardIds();
|
||||||
|
this.hashKeyRangeForLease = other.getHashKeyRange();
|
||||||
}
|
}
|
||||||
|
|
||||||
KinesisClientLease(String leaseKey, String leaseOwner, Long leaseCounter, UUID concurrencyToken,
|
KinesisClientLease(String leaseKey, String leaseOwner, Long leaseCounter, UUID concurrencyToken,
|
||||||
Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint,
|
Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint,
|
||||||
Long ownerSwitchesSinceCheckpoint, Set<String> parentShardIds) {
|
Long ownerSwitchesSinceCheckpoint, Set<String> parentShardIds, Set<String> childShardIds,
|
||||||
|
HashKeyRangeForLease hashKeyRangeForLease) {
|
||||||
super(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos);
|
super(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos);
|
||||||
|
|
||||||
this.checkpoint = checkpoint;
|
this.checkpoint = checkpoint;
|
||||||
this.pendingCheckpoint = pendingCheckpoint;
|
this.pendingCheckpoint = pendingCheckpoint;
|
||||||
this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint;
|
this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint;
|
||||||
this.parentShardIds.addAll(parentShardIds);
|
this.parentShardIds.addAll(parentShardIds);
|
||||||
|
this.childShardIds.addAll(childShardIds);
|
||||||
|
this.hashKeyRangeForLease = hashKeyRangeForLease;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -100,6 +107,20 @@ public class KinesisClientLease extends Lease {
|
||||||
return new HashSet<String>(parentShardIds);
|
return new HashSet<String>(parentShardIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return shardIds that are the children of this lease. Used for resharding.
|
||||||
|
*/
|
||||||
|
public Set<String> getChildShardIds() {
|
||||||
|
return new HashSet<String>(childShardIds);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return hash key range that this lease covers.
|
||||||
|
*/
|
||||||
|
public HashKeyRangeForLease getHashKeyRange() {
|
||||||
|
return hashKeyRangeForLease;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets checkpoint.
|
* Sets checkpoint.
|
||||||
*
|
*
|
||||||
|
|
@ -142,6 +163,29 @@ public class KinesisClientLease extends Lease {
|
||||||
this.parentShardIds.clear();
|
this.parentShardIds.clear();
|
||||||
this.parentShardIds.addAll(parentShardIds);
|
this.parentShardIds.addAll(parentShardIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets childShardIds.
|
||||||
|
*
|
||||||
|
* @param childShardIds may not be null
|
||||||
|
*/
|
||||||
|
public void setChildShardIds(Collection<String> childShardIds) {
|
||||||
|
verifyNotNull(childShardIds, "childShardIds should not be null");
|
||||||
|
|
||||||
|
this.childShardIds.clear();
|
||||||
|
this.childShardIds.addAll(childShardIds);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets hashKeyRangeForLease.
|
||||||
|
*
|
||||||
|
* @param hashKeyRangeForLease may not be null
|
||||||
|
*/
|
||||||
|
public void setHashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) {
|
||||||
|
verifyNotNull(hashKeyRangeForLease, "hashKeyRangeForLease should not be null");
|
||||||
|
|
||||||
|
this.hashKeyRangeForLease = hashKeyRangeForLease;
|
||||||
|
}
|
||||||
|
|
||||||
private void verifyNotNull(Object object, String message) {
|
private void verifyNotNull(Object object, String message) {
|
||||||
if (object == null) {
|
if (object == null) {
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@
|
||||||
package com.amazonaws.services.kinesis.leases.impl;
|
package com.amazonaws.services.kinesis.leases.impl;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import com.amazonaws.services.dynamodbv2.model.AttributeAction;
|
import com.amazonaws.services.dynamodbv2.model.AttributeAction;
|
||||||
|
|
@ -26,8 +27,11 @@ import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
|
||||||
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
|
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
|
||||||
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseSerializer;
|
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseSerializer;
|
||||||
import com.amazonaws.services.kinesis.leases.util.DynamoUtils;
|
import com.amazonaws.services.kinesis.leases.util.DynamoUtils;
|
||||||
|
import com.amazonaws.util.CollectionUtils;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
|
|
||||||
|
import static com.amazonaws.services.kinesis.leases.impl.UpdateField.HASH_KEY_RANGE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An implementation of ILeaseSerializer for KinesisClientLease objects.
|
* An implementation of ILeaseSerializer for KinesisClientLease objects.
|
||||||
*/
|
*/
|
||||||
|
|
@ -39,6 +43,9 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer<KinesisCli
|
||||||
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";
|
||||||
public final String PARENT_SHARD_ID_KEY = "parentShardId";
|
public 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";
|
||||||
|
|
||||||
private final LeaseSerializer baseSerializer = new LeaseSerializer(KinesisClientLease.class);
|
private final LeaseSerializer baseSerializer = new LeaseSerializer(KinesisClientLease.class);
|
||||||
|
|
||||||
|
|
@ -155,6 +162,28 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer<KinesisCli
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(KinesisClientLease lease,
|
||||||
|
UpdateField updateField) {
|
||||||
|
Map<String, AttributeValueUpdate> result = new HashMap<>();
|
||||||
|
|
||||||
|
switch (updateField) {
|
||||||
|
case CHILD_SHARDS:
|
||||||
|
// TODO: Implement update fields for child shards
|
||||||
|
break;
|
||||||
|
case HASH_KEY_RANGE:
|
||||||
|
if (lease.getHashKeyRange() != null) {
|
||||||
|
result.put(STARTING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(
|
||||||
|
lease.getHashKeyRange().serializedStartingHashKey()), AttributeAction.PUT));
|
||||||
|
result.put(ENDING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(
|
||||||
|
lease.getHashKeyRange().serializedEndingHashKey()), AttributeAction.PUT));
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<KeySchemaElement> getKeySchema() {
|
public Collection<KeySchemaElement> getKeySchema() {
|
||||||
return baseSerializer.getKeySchema();
|
return baseSerializer.getKeySchema();
|
||||||
|
|
|
||||||
|
|
@ -590,6 +590,35 @@ public class LeaseManager<T extends Lease> implements ILeaseManager<T> {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void updateLeaseWithMetaInfo(T lease, UpdateField updateField)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
verifyNotNull(lease, "lease cannot be null");
|
||||||
|
verifyNotNull(updateField, "updateField cannot be null");
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Updating lease " + lease + " for field " + updateField);
|
||||||
|
}
|
||||||
|
|
||||||
|
UpdateItemRequest request = new UpdateItemRequest();
|
||||||
|
request.setTableName(table);
|
||||||
|
request.setKey(serializer.getDynamoHashKey(lease));
|
||||||
|
request.setExpected(serializer.getDynamoLeaseCounterExpectation(lease));
|
||||||
|
|
||||||
|
Map<String, AttributeValueUpdate> updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField);
|
||||||
|
updates.putAll(serializer.getDynamoUpdateLeaseUpdate(lease));
|
||||||
|
request.setAttributeUpdates(updates);
|
||||||
|
|
||||||
|
try {
|
||||||
|
dynamoDBClient.updateItem(request);
|
||||||
|
} catch (AmazonClientException e) {
|
||||||
|
throw convertAndRethrowExceptions("update", lease.getLeaseKey(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* This method contains boilerplate exception handling - it throws or returns something to be thrown. The
|
* This method contains boilerplate exception handling - it throws or returns something to be thrown. The
|
||||||
* inconsistency there exists to satisfy the compiler when this method is used at the end of non-void methods.
|
* inconsistency there exists to satisfy the compiler when this method is used at the end of non-void methods.
|
||||||
|
|
|
||||||
|
|
@ -177,6 +177,12 @@ public class LeaseSerializer implements ILeaseSerializer<Lease> {
|
||||||
return new HashMap<String, AttributeValueUpdate>();
|
return new HashMap<String, AttributeValueUpdate>();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) {
|
||||||
|
// There is no application-specific data in Lease - just return a map that increments the counter.
|
||||||
|
return new HashMap<String, AttributeValueUpdate>();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<KeySchemaElement> getKeySchema() {
|
public Collection<KeySchemaElement> getKeySchema() {
|
||||||
List<KeySchemaElement> keySchema = new ArrayList<KeySchemaElement>();
|
List<KeySchemaElement> keySchema = new ArrayList<KeySchemaElement>();
|
||||||
|
|
|
||||||
|
|
@ -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 com.amazonaws.services.kinesis.leases.impl;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
}
|
||||||
|
|
@ -20,6 +20,7 @@ import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
|
||||||
import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
|
import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
|
||||||
import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
|
import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
|
||||||
import com.amazonaws.services.kinesis.leases.impl.Lease;
|
import com.amazonaws.services.kinesis.leases.impl.Lease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.UpdateField;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Supports basic CRUD operations for Leases.
|
* Supports basic CRUD operations for Leases.
|
||||||
|
|
@ -180,6 +181,19 @@ public interface ILeaseManager<T extends Lease> {
|
||||||
public boolean updateLease(T lease)
|
public boolean updateLease(T 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.
|
||||||
|
**
|
||||||
|
* @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(T lease, UpdateField updateField)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
throw new UnsupportedOperationException("updateLeaseWithMetaInfo 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.
|
||||||
*
|
*
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import com.amazonaws.services.dynamodbv2.model.AttributeValueUpdate;
|
||||||
import com.amazonaws.services.dynamodbv2.model.ExpectedAttributeValue;
|
import com.amazonaws.services.dynamodbv2.model.ExpectedAttributeValue;
|
||||||
import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
|
import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
|
||||||
import com.amazonaws.services.kinesis.leases.impl.Lease;
|
import com.amazonaws.services.kinesis.leases.impl.Lease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.UpdateField;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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.
|
||||||
|
|
@ -104,6 +105,15 @@ public interface ILeaseSerializer<T extends Lease> {
|
||||||
*/
|
*/
|
||||||
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(T lease);
|
public Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(T lease);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param lease
|
||||||
|
* @param updateField
|
||||||
|
* @return the attribute value map that updates application-specific data for a lease
|
||||||
|
*/
|
||||||
|
default Map<String, AttributeValueUpdate> getDynamoUpdateLeaseUpdate(T 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
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.UpdateField;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
|
|
@ -53,6 +54,7 @@ class ExceptionThrowingLeaseManager implements ILeaseManager<KinesisClientLease>
|
||||||
DELETELEASE(9),
|
DELETELEASE(9),
|
||||||
DELETEALL(10),
|
DELETEALL(10),
|
||||||
UPDATELEASE(11),
|
UPDATELEASE(11),
|
||||||
|
UPDATELEASEWITHMETAINFO(12),
|
||||||
NONE(Integer.MIN_VALUE);
|
NONE(Integer.MIN_VALUE);
|
||||||
|
|
||||||
private Integer index;
|
private Integer index;
|
||||||
|
|
@ -197,6 +199,14 @@ class ExceptionThrowingLeaseManager implements ILeaseManager<KinesisClientLease>
|
||||||
return leaseManager.updateLease(lease);
|
return leaseManager.updateLease(lease);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateLeaseWithMetaInfo(KinesisClientLease lease, UpdateField updateField)
|
||||||
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
throwExceptions("updateLeaseWithMetaInfo", ExceptionThrowingLeaseManagerMethods.UPDATELEASEWITHMETAINFO);
|
||||||
|
|
||||||
|
leaseManager.updateLeaseWithMetaInfo(lease, updateField);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KinesisClientLease getLease(String shardId)
|
public KinesisClientLease getLease(String shardId)
|
||||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,613 @@
|
||||||
|
/*
|
||||||
|
* 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
||||||
|
|
||||||
|
import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
|
||||||
|
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
|
||||||
|
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
|
||||||
|
import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
|
||||||
|
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
|
||||||
|
import com.amazonaws.services.kinesis.model.HashKeyRange;
|
||||||
|
import com.amazonaws.services.kinesis.model.Shard;
|
||||||
|
import com.amazonaws.util.CollectionUtils;
|
||||||
|
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 java.math.BigInteger;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY;
|
||||||
|
import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MAX_HASH_KEY;
|
||||||
|
import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MIN_HASH_KEY;
|
||||||
|
import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.deserialize;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@RunWith(MockitoJUnitRunner.class)
|
||||||
|
public class PeriodicShardSyncManagerTest {
|
||||||
|
|
||||||
|
private static final String WORKER_ID = "workerId";
|
||||||
|
|
||||||
|
/** Manager for PERIODIC shard sync strategy */
|
||||||
|
private PeriodicShardSyncManager periodicShardSyncManager;
|
||||||
|
|
||||||
|
/** Manager for SHARD_END shard sync strategy */
|
||||||
|
private PeriodicShardSyncManager auditorPeriodicShardSyncManager;
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private LeaderDecider leaderDecider;
|
||||||
|
@Mock
|
||||||
|
private ShardSyncTask shardSyncTask;
|
||||||
|
@Mock
|
||||||
|
private ILeaseManager<KinesisClientLease> leaseManager;
|
||||||
|
@Mock
|
||||||
|
private IKinesisProxy kinesisProxy;
|
||||||
|
|
||||||
|
private IMetricsFactory metricsFactory = new NullMetricsFactory();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
periodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask,
|
||||||
|
metricsFactory, leaseManager, kinesisProxy, false);
|
||||||
|
auditorPeriodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask,
|
||||||
|
metricsFactory, leaseManager, kinesisProxy, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForFailureWhenHashRangesAreIncomplete() {
|
||||||
|
List<KinesisClientLease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertTrue(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenHashRangesAreComplete() {
|
||||||
|
List<KinesisClientLease> hashRanges = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenUnsortedHashRangesAreComplete() {
|
||||||
|
List<KinesisClientLease> hashRanges = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
add(deserialize("6", "23"));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() {
|
||||||
|
List<KinesisClientLease> hashRanges = 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()));
|
||||||
|
add(deserialize("24", "45"));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Assert.assertFalse(PeriodicShardSyncManager
|
||||||
|
.checkForHoleInHashKeyRanges(hashRanges).isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() throws Exception {
|
||||||
|
when(leaseManager.listLeases()).thenReturn(null);
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() throws Exception {
|
||||||
|
when(leaseManager.listLeases()).thenReturn(Collections.emptyList());
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsNotReached() throws Exception {
|
||||||
|
List<KinesisClientLease> leases = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsReached() throws Exception {
|
||||||
|
List<KinesisClientLease> leases = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenHoleIsDueToShardEnd() throws Exception {
|
||||||
|
List<KinesisClientLease> leases = new ArrayList<HashKeyRangeForLease>() {{
|
||||||
|
add(deserialize(MIN_HASH_KEY.toString(), "1"));
|
||||||
|
add(deserialize("2", "3"));
|
||||||
|
add(deserialize("6", "23")); // Introducing hole here through SHARD_END checkpoint
|
||||||
|
add(deserialize("4", "23"));
|
||||||
|
add(deserialize("24", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
if (lease.getHashKeyRange().startingHashKey().toString().equals("4")) {
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
} else {
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
}
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenNoLeasesAreUsedDueToShardEnd() throws Exception {
|
||||||
|
List<KinesisClientLease> leases = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenHoleShifts() throws Exception {
|
||||||
|
List<KinesisClientLease> leases1 = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases1);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
List<KinesisClientLease> leases2 = 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("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Resetting the holes
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases2);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfShardSyncIsInitiatedWhenHoleShiftsMoreThanOnce() throws Exception {
|
||||||
|
List<KinesisClientLease> leases1 = 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 -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases1);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
List<KinesisClientLease> leases2 = 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("25", MAX_HASH_KEY.toString()));
|
||||||
|
}}.stream().map(hashKeyRangeForLease -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setHashKeyRange(hashKeyRangeForLease);
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Resetting the holes
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases2);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
// Resetting the holes again
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases1);
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSync() throws Exception {
|
||||||
|
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(hashKeyRange -> new Shard()
|
||||||
|
.withShardId("shard-" + ++shardCounter[0])
|
||||||
|
.withHashKeyRange(new HashKeyRange()
|
||||||
|
.withStartingHashKey(hashKeyRange.serializedStartingHashKey())
|
||||||
|
.withEndingHashKey(hashKeyRange.serializedEndingHashKey())))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(kinesisProxy.getShardList()).thenReturn(kinesisShards);
|
||||||
|
|
||||||
|
final int[] leaseCounter = { 0 };
|
||||||
|
List<KinesisClientLease> leases = hashKeyRangeForLeases.stream()
|
||||||
|
.map(hashKeyRange -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setLeaseKey("shard-" + ++leaseCounter[0]);
|
||||||
|
// Setting the hash range only for the last two leases
|
||||||
|
if (leaseCounter[0] >= 3) {
|
||||||
|
lease.setHashKeyRange(hashKeyRange);
|
||||||
|
}
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
|
||||||
|
// Assert that SHARD_END shard sync should never trigger, but PERIODIC shard sync should always trigger
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
|
||||||
|
// Assert that all the leases now have hash ranges set
|
||||||
|
for (KinesisClientLease lease : leases) {
|
||||||
|
Assert.assertNotNull(lease.getHashKeyRange());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSyncInHoleScenario() throws Exception {
|
||||||
|
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(hashKeyRange -> new Shard()
|
||||||
|
.withShardId("shard-" + ++shardCounter[0])
|
||||||
|
.withHashKeyRange(new HashKeyRange()
|
||||||
|
.withStartingHashKey(hashKeyRange.serializedStartingHashKey())
|
||||||
|
.withEndingHashKey(hashKeyRange.serializedEndingHashKey())))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(kinesisProxy.getShardList()).thenReturn(kinesisShards);
|
||||||
|
|
||||||
|
final int[] leaseCounter = { 0 };
|
||||||
|
List<KinesisClientLease> leases = hashKeyRangeForLeases.stream()
|
||||||
|
.map(hashKeyRange -> {
|
||||||
|
KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
lease.setLeaseKey("shard-" + ++leaseCounter[0]);
|
||||||
|
// Setting the hash range only for the last two leases
|
||||||
|
if (leaseCounter[0] >= 3) {
|
||||||
|
lease.setHashKeyRange(hashKeyRange);
|
||||||
|
}
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
return lease;
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
when(leaseManager.listLeases()).thenReturn(leases);
|
||||||
|
|
||||||
|
// Assert that shard sync should trigger after breaching threshold
|
||||||
|
for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) {
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
}
|
||||||
|
Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync());
|
||||||
|
|
||||||
|
// Assert that all the leases now have hash ranges set
|
||||||
|
for (KinesisClientLease lease : leases) {
|
||||||
|
Assert.assertNotNull(lease.getHashKeyRange());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidSplitHierarchyTreeTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<KinesisClientLease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.SPLIT, maxInitialLeaseCount, false);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidMergeHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<KinesisClientLease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, true);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFor1000DifferentValidReshardHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() {
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int maxInitialLeaseCount = 100;
|
||||||
|
List<KinesisClientLease> leases = generateInitialLeases(maxInitialLeaseCount);
|
||||||
|
reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, true);
|
||||||
|
Collections.shuffle(leases);
|
||||||
|
Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(leases).isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<KinesisClientLease> generateInitialLeases(int initialShardCount) {
|
||||||
|
long hashRangeInternalMax = 10000000;
|
||||||
|
List<KinesisClientLease> initialLeases = new ArrayList<>();
|
||||||
|
long leaseStartKey = 0;
|
||||||
|
for (int i = 1; i <= initialShardCount; i++) {
|
||||||
|
final KinesisClientLease lease = new KinesisClientLease();
|
||||||
|
long leaseEndKey;
|
||||||
|
if (i != initialShardCount) {
|
||||||
|
leaseEndKey = (hashRangeInternalMax / initialShardCount) * i;
|
||||||
|
lease.setHashKeyRange(deserialize(leaseStartKey + "", leaseEndKey + ""));
|
||||||
|
} else {
|
||||||
|
leaseEndKey = 0;
|
||||||
|
lease.setHashKeyRange(deserialize(leaseStartKey + "", MAX_HASH_KEY.toString()));
|
||||||
|
}
|
||||||
|
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
lease.setLeaseKey("shard-" + i);
|
||||||
|
initialLeases.add(lease);
|
||||||
|
leaseStartKey = leaseEndKey + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return initialLeases;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reshard(List<KinesisClientLease> 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<KinesisClientLease> initialLeases, int leaseCounter, boolean shouldKeepSomeParentsInProgress) {
|
||||||
|
List<KinesisClientLease> leasesEligibleForMerge = initialLeases.stream()
|
||||||
|
.filter(l -> CollectionUtils.isNullOrEmpty(l.getChildShardIds())).collect(Collectors.toList());
|
||||||
|
|
||||||
|
int leasesToMerge = (int) ((leasesEligibleForMerge.size() - 1) / 2.0 * Math.random());
|
||||||
|
for (int i = 0; i < leasesToMerge; i += 2) {
|
||||||
|
KinesisClientLease parent1 = leasesEligibleForMerge.get(i);
|
||||||
|
KinesisClientLease parent2 = leasesEligibleForMerge.get(i + 1);
|
||||||
|
if (parent2.getHashKeyRange().startingHashKey()
|
||||||
|
.subtract(parent1.getHashKeyRange().endingHashKey()).equals(BigInteger.ONE)) {
|
||||||
|
parent1.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
|
||||||
|
if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) {
|
||||||
|
parent2.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
}
|
||||||
|
|
||||||
|
KinesisClientLease child = new KinesisClientLease();
|
||||||
|
child.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child.setLeaseKey("shard-" + ++leaseCounter);
|
||||||
|
child.setHashKeyRange(new HashKeyRangeForLease(parent1.getHashKeyRange().startingHashKey(),
|
||||||
|
parent2.getHashKeyRange().endingHashKey()));
|
||||||
|
parent1.setChildShardIds(Collections.singletonList(child.getLeaseKey()));
|
||||||
|
parent2.setChildShardIds(Collections.singletonList(child.getLeaseKey()));
|
||||||
|
child.setParentShardIds(Sets.newHashSet(parent1.getLeaseKey(), parent2.getLeaseKey()));
|
||||||
|
|
||||||
|
initialLeases.add(child);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return leaseCounter;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int split(List<KinesisClientLease> initialLeases, int leaseCounter) {
|
||||||
|
List<KinesisClientLease> leasesEligibleForSplit = initialLeases.stream()
|
||||||
|
.filter(l -> CollectionUtils.isNullOrEmpty(l.getChildShardIds())).collect(Collectors.toList());
|
||||||
|
|
||||||
|
int leasesToSplit = (int) (leasesEligibleForSplit.size() * Math.random());
|
||||||
|
for (int i = 0; i < leasesToSplit; i++) {
|
||||||
|
KinesisClientLease parent = leasesEligibleForSplit.get(i);
|
||||||
|
parent.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
|
||||||
|
|
||||||
|
KinesisClientLease child1 = new KinesisClientLease();
|
||||||
|
child1.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child1.setHashKeyRange(new HashKeyRangeForLease(parent.getHashKeyRange().startingHashKey(),
|
||||||
|
parent.getHashKeyRange().startingHashKey().add(parent.getHashKeyRange().endingHashKey())
|
||||||
|
.divide(new BigInteger("2"))));
|
||||||
|
child1.setLeaseKey("shard-" + ++leaseCounter);
|
||||||
|
|
||||||
|
KinesisClientLease child2 = new KinesisClientLease();
|
||||||
|
child2.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
|
||||||
|
child2.setHashKeyRange(new HashKeyRangeForLease(parent.getHashKeyRange().startingHashKey()
|
||||||
|
.add(parent.getHashKeyRange().endingHashKey()).divide(new BigInteger("2")).add(BigInteger.ONE),
|
||||||
|
parent.getHashKeyRange().endingHashKey()));
|
||||||
|
child2.setLeaseKey("shard-" + ++leaseCounter);
|
||||||
|
|
||||||
|
child1.setParentShardIds(Sets.newHashSet(parent.getLeaseKey()));
|
||||||
|
child2.setParentShardIds(Sets.newHashSet(parent.getLeaseKey()));
|
||||||
|
parent.setChildShardIds(Lists.newArrayList(child1.getLeaseKey(), child2.getLeaseKey()));
|
||||||
|
|
||||||
|
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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -174,6 +174,8 @@ public class WorkerTest {
|
||||||
@Mock
|
@Mock
|
||||||
private IKinesisProxy proxy;
|
private IKinesisProxy proxy;
|
||||||
@Mock
|
@Mock
|
||||||
|
private StreamConfig streamConfig;
|
||||||
|
@Mock
|
||||||
private WorkerThreadPoolExecutor executorService;
|
private WorkerThreadPoolExecutor executorService;
|
||||||
@Mock
|
@Mock
|
||||||
private WorkerCWMetricsFactory cwMetricsFactory;
|
private WorkerCWMetricsFactory cwMetricsFactory;
|
||||||
|
|
@ -200,6 +202,8 @@ public class WorkerTest {
|
||||||
config.withMaxInitializationAttempts(1);
|
config.withMaxInitializationAttempts(1);
|
||||||
recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory());
|
recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory());
|
||||||
when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory);
|
when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory);
|
||||||
|
when(leaseCoordinator.getLeaseManager()).thenReturn(mock(ILeaseManager.class));
|
||||||
|
when(streamConfig.getStreamProxy()).thenReturn(kinesisProxy);
|
||||||
}
|
}
|
||||||
|
|
||||||
// CHECKSTYLE:IGNORE AnonInnerLengthCheck FOR NEXT 50 LINES
|
// CHECKSTYLE:IGNORE AnonInnerLengthCheck FOR NEXT 50 LINES
|
||||||
|
|
@ -257,7 +261,6 @@ public class WorkerTest {
|
||||||
final String stageName = "testStageName";
|
final String stageName = "testStageName";
|
||||||
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
||||||
config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID);
|
config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID);
|
||||||
IKinesisProxy proxy = null;
|
|
||||||
ICheckpoint checkpoint = null;
|
ICheckpoint checkpoint = null;
|
||||||
int maxRecords = 1;
|
int maxRecords = 1;
|
||||||
int idleTimeInMilliseconds = 1000;
|
int idleTimeInMilliseconds = 1000;
|
||||||
|
|
@ -306,7 +309,6 @@ public class WorkerTest {
|
||||||
public void testWorkerLoopWithCheckpoint() {
|
public void testWorkerLoopWithCheckpoint() {
|
||||||
final String stageName = "testStageName";
|
final String stageName = "testStageName";
|
||||||
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
||||||
IKinesisProxy proxy = null;
|
|
||||||
ICheckpoint checkpoint = null;
|
ICheckpoint checkpoint = null;
|
||||||
int maxRecords = 1;
|
int maxRecords = 1;
|
||||||
int idleTimeInMilliseconds = 1000;
|
int idleTimeInMilliseconds = 1000;
|
||||||
|
|
@ -376,7 +378,6 @@ public class WorkerTest {
|
||||||
final String stageName = "testStageName";
|
final String stageName = "testStageName";
|
||||||
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2;
|
||||||
config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID);
|
config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID);
|
||||||
IKinesisProxy proxy = null;
|
|
||||||
ICheckpoint checkpoint = null;
|
ICheckpoint checkpoint = null;
|
||||||
int maxRecords = 1;
|
int maxRecords = 1;
|
||||||
int idleTimeInMilliseconds = 1000;
|
int idleTimeInMilliseconds = 1000;
|
||||||
|
|
@ -866,7 +867,6 @@ public class WorkerTest {
|
||||||
|
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -954,7 +954,6 @@ public class WorkerTest {
|
||||||
public void testShutdownCallableNotAllowedTwice() throws Exception {
|
public void testShutdownCallableNotAllowedTwice() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -1019,7 +1018,6 @@ public class WorkerTest {
|
||||||
public void testGracefulShutdownSingleFuture() throws Exception {
|
public void testGracefulShutdownSingleFuture() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -1107,7 +1105,6 @@ public class WorkerTest {
|
||||||
|
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -1190,7 +1187,6 @@ public class WorkerTest {
|
||||||
when(completedLease.getParentShardIds()).thenReturn(Collections.singleton(parentShardId));
|
when(completedLease.getParentShardIds()).thenReturn(Collections.singleton(parentShardId));
|
||||||
when(completedLease.getCheckpoint()).thenReturn(ExtendedSequenceNumber.SHARD_END);
|
when(completedLease.getCheckpoint()).thenReturn(ExtendedSequenceNumber.SHARD_END);
|
||||||
when(completedLease.getConcurrencyToken()).thenReturn(UUID.randomUUID());
|
when(completedLease.getConcurrencyToken()).thenReturn(UUID.randomUUID());
|
||||||
final StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
final IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
final IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
final List<KinesisClientLease> leases = Collections.singletonList(completedLease);
|
final List<KinesisClientLease> leases = Collections.singletonList(completedLease);
|
||||||
final List<ShardInfo> currentAssignments = new ArrayList<>();
|
final List<ShardInfo> currentAssignments = new ArrayList<>();
|
||||||
|
|
@ -1238,7 +1234,6 @@ public class WorkerTest {
|
||||||
public void testRequestShutdownWithLostLease() throws Exception {
|
public void testRequestShutdownWithLostLease() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -1351,7 +1346,6 @@ public class WorkerTest {
|
||||||
public void testRequestShutdownWithAllLeasesLost() throws Exception {
|
public void testRequestShutdownWithAllLeasesLost() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -1469,7 +1463,6 @@ public class WorkerTest {
|
||||||
public void testLeaseCancelledAfterShutdownRequest() throws Exception {
|
public void testLeaseCancelledAfterShutdownRequest() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
@ -1553,7 +1546,6 @@ public class WorkerTest {
|
||||||
public void testEndOfShardAfterShutdownRequest() throws Exception {
|
public void testEndOfShardAfterShutdownRequest() throws Exception {
|
||||||
|
|
||||||
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
|
||||||
StreamConfig streamConfig = mock(StreamConfig.class);
|
|
||||||
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
IMetricsFactory metricsFactory = mock(IMetricsFactory.class);
|
||||||
|
|
||||||
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L);
|
||||||
|
|
|
||||||
|
|
@ -30,6 +30,8 @@ public class KinesisClientLeaseBuilder {
|
||||||
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 HashKeyRangeForLease hashKeyRangeForLease;
|
||||||
|
|
||||||
public KinesisClientLeaseBuilder withLeaseKey(String leaseKey) {
|
public KinesisClientLeaseBuilder withLeaseKey(String leaseKey) {
|
||||||
this.leaseKey = leaseKey;
|
this.leaseKey = leaseKey;
|
||||||
|
|
@ -76,8 +78,19 @@ public class KinesisClientLeaseBuilder {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public KinesisClientLeaseBuilder withChildShardIds(Set<String> childShardIds) {
|
||||||
|
this.childShardIds = childShardIds;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public KinesisClientLeaseBuilder withHashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) {
|
||||||
|
this.hashKeyRangeForLease = hashKeyRangeForLease;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public KinesisClientLease build() {
|
public KinesisClientLease build() {
|
||||||
return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos,
|
return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos,
|
||||||
checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds);
|
checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds,
|
||||||
|
hashKeyRangeForLease);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Loading…
Reference in a new issue