Fix Kinesis resharding issues (#8644)

* Fix Kinesis resharding issues

* PR comments

* Adjust metadata error message

* Remove unused method

* Use sha1 for shard id hashing

* Add metadata sanity check, add comment

* Only use shard ID hashing for group mapping

* Style fix

* Fix unused import

* update comment

* Fix teamcity inspection
This commit is contained in:
Jonathan Wei 2019-10-10 00:16:44 -07:00 committed by Clint Wylie
parent 96d8523ecb
commit 0c387c1d47
5 changed files with 1224 additions and 25 deletions

View File

@ -24,6 +24,9 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableMap;
import com.google.common.hash.HashCode;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import org.apache.druid.common.aws.AWSCredentialsConfig;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.task.Task;
@ -44,6 +47,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
@ -53,9 +57,13 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervi
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.joda.time.DateTime;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -74,6 +82,10 @@ import java.util.concurrent.ScheduledExecutorService;
*/
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
{
private static final HashFunction HASH_FUNCTION = Hashing.sha1();
private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class);
public static final TypeReference<TreeMap<Integer, Map<String, String>>> CHECKPOINTS_TYPE_REF =
new TypeReference<TreeMap<Integer, Map<String, String>>>()
{
@ -212,6 +224,9 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
// not yet implemented, see issue #6739
}
/**
* We hash the shard ID string, and then use the first four bytes of the hash as an int % task count
*/
@Override
protected int getTaskGroupIdForPartition(String partitionId)
{
@ -219,7 +234,13 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
partitionIds.add(partitionId);
}
return partitionIds.indexOf(partitionId) % spec.getIoConfig().getTaskCount();
return Math.abs(getHashIntFromShardId(partitionId) % spec.getIoConfig().getTaskCount());
}
private int getHashIntFromShardId(String shardId)
{
HashCode hashCode = HASH_FUNCTION.hashString(shardId, StandardCharsets.UTF_8);
return hashCode.asInt();
}
@Override
@ -314,4 +335,79 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
{
return true;
}
@Override
protected Map<String, OrderedSequenceNumber<String>> filterExpiredPartitionsFromStartingOffsets(
Map<String, OrderedSequenceNumber<String>> startingOffsets
)
{
Map<String, OrderedSequenceNumber<String>> filteredOffsets = new HashMap<>();
for (Map.Entry<String, OrderedSequenceNumber<String>> entry : startingOffsets.entrySet()) {
if (!entry.getValue().get().equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)) {
filteredOffsets.put(entry.getKey(), entry.getValue());
} else {
log.debug("Excluding shard[%s] because it has reached EOS.", entry.getKey());
}
}
return filteredOffsets;
}
@Override
protected boolean supportsPartitionExpiration()
{
return true;
}
@Override
protected KinesisDataSourceMetadata createDataSourceMetadataWithoutExpiredPartitions(
SeekableStreamDataSourceMetadata<String, String> currentMetadata,
Set<String> expiredPartitionIds
)
{
log.info("Cleaning up dead shards: " + expiredPartitionIds);
final KinesisDataSourceMetadata dataSourceMetadata = (KinesisDataSourceMetadata) currentMetadata;
SeekableStreamSequenceNumbers<String, String> old = dataSourceMetadata.getSeekableStreamSequenceNumbers();
Map<String, String> oldPartitionSequenceNumberMap = old.getPartitionSequenceNumberMap();
Map<String, String> newPartitionSequenceNumberMap = new HashMap<>();
for (Map.Entry<String, String> entry : oldPartitionSequenceNumberMap.entrySet()) {
if (!expiredPartitionIds.contains(entry.getKey())) {
newPartitionSequenceNumberMap.put(entry.getKey(), entry.getValue());
}
}
Set<String> oldExclusiveStartPartitions;
Set<String> newExclusiveStartPartitions = null;
if (old instanceof SeekableStreamStartSequenceNumbers) {
newExclusiveStartPartitions = new HashSet<>();
oldExclusiveStartPartitions = ((SeekableStreamStartSequenceNumbers<String, String>) old).getExclusivePartitions();
for (String partitionId : oldExclusiveStartPartitions) {
if (!expiredPartitionIds.contains(partitionId)) {
newExclusiveStartPartitions.add(partitionId);
}
}
}
SeekableStreamSequenceNumbers<String, String> newSequences;
if (old instanceof SeekableStreamStartSequenceNumbers) {
newSequences = new SeekableStreamStartSequenceNumbers<String, String>(
old.getStream(),
null,
newPartitionSequenceNumberMap,
null,
newExclusiveStartPartitions
);
} else {
newSequences = new SeekableStreamEndSequenceNumbers<String, String>(
old.getStream(),
null,
newPartitionSequenceNumberMap,
null
);
}
return new KinesisDataSourceMetadata(newSequences);
}
}

View File

@ -2596,6 +2596,92 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertFalse(sequenceMetadata.isCheckpointed());
}
/**
* Tests handling of a closed shard. The task is initially given an unlimited end sequence number and
* eventually gets an EOS marker which causes it to stop reading.
*/
@Test(timeout = 120_000L)
public void testEndOfShard() throws Exception
{
recordSupplier.assign(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(recordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
List<OrderedPartitionableRecord<String, String>> eosRecord = ImmutableList.of(
new OrderedPartitionableRecord<>(STREAM, SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER, null)
);
EasyMock.expect(recordSupplier.poll(EasyMock.anyLong()))
.andReturn(records.subList(2, 5)).once()
.andReturn(eosRecord).once();
recordSupplier.close();
EasyMock.expectLastCall().once();
replayAll();
final KinesisIndexTask task = createTask(
null,
new KinesisIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()
),
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER)
),
true,
null,
null,
"awsEndpoint",
null,
null,
null,
null,
false
)
);
final ListenableFuture<TaskStatus> future = runTask(task);
// Wait for task to exit
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
verifyAll();
// Check metrics
Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); // EOS marker
// Check published metadata
SegmentDescriptor desc1 = sd("2010/P1D", 0);
SegmentDescriptor desc2 = sd("2011/P1D", 0);
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER)
)
),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
private ListenableFuture<TaskStatus> runTask(final Task task)
{
try {

View File

@ -117,10 +117,14 @@ public class KinesisSupervisorTest extends EasyMockSupport
private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S");
private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S");
private static final String STREAM = "stream";
private static final String SHARD_ID1 = "1";
private static final String SHARD_ID0 = "0";
private static final StreamPartition<String> SHARD1_PARTITION = StreamPartition.of(STREAM, SHARD_ID1);
private static final String SHARD_ID0 = "shardId-000000000000";
private static final String SHARD_ID1 = "shardId-000000000001";
private static final String SHARD_ID2 = "shardId-000000000002";
private static final String SHARD_ID3 = "shardId-000000000003";
private static final StreamPartition<String> SHARD0_PARTITION = StreamPartition.of(STREAM, SHARD_ID0);
private static final StreamPartition<String> SHARD1_PARTITION = StreamPartition.of(STREAM, SHARD_ID1);
private static final StreamPartition<String> SHARD2_PARTITION = StreamPartition.of(STREAM, SHARD_ID2);
private static final StreamPartition<String> SHARD3_PARTITION = StreamPartition.of(STREAM, SHARD_ID3);
private static DataSchema dataSchema;
private KinesisRecordSupplier supervisorRecordSupplier;
@ -271,7 +275,6 @@ public class KinesisSupervisorTest extends EasyMockSupport
);
}
@Test
public void testMultiTask() throws Exception
{
@ -1283,7 +1286,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kinesisIndexTask.getTuningConfig());
KinesisIndexTaskIOConfig taskConfig = kinesisIndexTask.getIOConfig();
Assert.assertEquals("sequenceName-1", taskConfig.getBaseSequenceName());
Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction());
Assert.assertEquals(STREAM, taskConfig.getStartSequenceNumbers().getStream());
@ -1297,7 +1300,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
);
// start sequenceNumbers should be exclusive for the second batch of tasks
Assert.assertEquals(
ImmutableSet.of("0", "1"),
ImmutableSet.of(SHARD_ID0, SHARD_ID1),
((KinesisIndexTask) task).getIOConfig().getStartSequenceNumbers().getExclusivePartitions()
);
}
@ -2473,9 +2476,9 @@ public class KinesisSupervisorTest extends EasyMockSupport
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
.andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of("1", "100", "2", "200"))
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, "200"))
)
).times(4);
).times(3);
// getOffsetFromStorageForPartition() throws an exception when the offsets are automatically reset.
// Since getOffsetFromStorageForPartition() is called per partition, all partitions can't be reset at the same time.
// Instead, subsequent partitions will be reset in the following supervisor runs.
@ -2484,7 +2487,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
DATASOURCE,
new KinesisDataSourceMetadata(
// Only one partition is reset in a single supervisor run.
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of("2", "200"))
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID2, "200"))
)
)
).andReturn(true);
@ -3809,6 +3812,838 @@ public class KinesisSupervisorTest extends EasyMockSupport
verifyAll();
}
@Test
public void testShardSplit() throws Exception
{
supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null);
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
List<Task> phaseOneTasks = testShardSplitPhaseOne();
List<Task> phaseTwoTasks = testShardSplitPhaseTwo(phaseOneTasks);
testShardSplitPhaseThree(phaseTwoTasks);
}
private List<Task> testShardSplitPhaseOne() throws Exception
{
supervisorRecordSupplier.assign(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID0))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD0_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(EasyMock.anyObject())).andReturn("100").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> captured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(1);
replayAll();
supervisor.start();
supervisor.runInternal();
verifyAll();
List<Task> tasks = captured.getValues();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpoints1 = new TreeMap<>();
checkpoints1.put(0, ImmutableMap.of(
SHARD_ID0,
"0"
));
// there would be 4 tasks, 2 for each task group
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpoints1))
.times(1);
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(tasks).anyTimes();
for (Task task : tasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
return tasks;
}
/**
* Test task creation after a shard split with a closed shard
*
* @param phaseOneTasks List of tasks from the initial phase where only one shard was present
*/
private List<Task> testShardSplitPhaseTwo(List<Task> phaseOneTasks) throws Exception
{
EasyMock.reset(indexerMetadataStorageCoordinator);
EasyMock.reset(taskStorage);
EasyMock.reset(taskQueue);
EasyMock.reset(taskClient);
EasyMock.reset(taskMaster);
EasyMock.reset(taskRunner);
EasyMock.reset(supervisorRecordSupplier);
// first task ran, its shard 0 has reached EOS
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(SHARD_ID0, KinesisSequenceNumber.END_OF_SHARD_MARKER)
)
)
).anyTimes();
// Normally the split would result in 0 -> 1,2, but we use shard ID 3 instead to ensure that each
// task group gets one shard after the split (due to hashing behavior)
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID0, SHARD_ID1, SHARD_ID3))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD0_PARTITION, SHARD1_PARTITION, SHARD3_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID0)))
.andReturn(KinesisSequenceNumber.END_OF_SHARD_MARKER).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID1)))
.andReturn("100").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID3)))
.andReturn("100").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> postSplitCaptured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
Task successfulTask = phaseOneTasks.get(0);
EasyMock.expect(taskStorage.getStatus(successfulTask.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask.getId())).andReturn(Optional.of(successfulTask)).anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(postSplitCaptured))).andReturn(true).times(2);
replayAll();
supervisor.runInternal();
verifyAll();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpointsGroup0 = new TreeMap<>();
checkpointsGroup0.put(0, ImmutableMap.of(
SHARD_ID1, "0",
SHARD_ID0, KinesisSequenceNumber.END_OF_SHARD_MARKER
));
TreeMap<Integer, Map<String, String>> checkpointsGroup1 = new TreeMap<>();
checkpointsGroup1.put(1, ImmutableMap.of(
SHARD_ID3, "0"
));
// there would be 2 tasks, 1 for each task group
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup0))
.times(1);
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup1))
.times(1);
List<Task> postSplitTasks = postSplitCaptured.getValues();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(postSplitTasks).anyTimes();
for (Task task : postSplitTasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
// Check that shardId-000000000000 which has hit EOS is not included in the sequences sent to the task for group 0
SeekableStreamStartSequenceNumbers<String, String> group0ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID1, "0"
),
ImmutableSet.of()
);
SeekableStreamEndSequenceNumbers<String, String> group0ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
SeekableStreamStartSequenceNumbers<String, String> group1ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID3, "0"
),
ImmutableSet.of()
);
SeekableStreamEndSequenceNumbers<String, String> group1ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID3, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
Assert.assertEquals(2, postSplitTasks.size());
KinesisIndexTaskIOConfig group0Config = ((KinesisIndexTask) postSplitTasks.get(0)).getIOConfig();
KinesisIndexTaskIOConfig group1Config = ((KinesisIndexTask) postSplitTasks.get(1)).getIOConfig();
Assert.assertEquals((Integer) 0, group0Config.getTaskGroupId());
Assert.assertEquals((Integer) 1, group1Config.getTaskGroupId());
Assert.assertEquals(group0ExpectedStartSequenceNumbers, group0Config.getStartSequenceNumbers());
Assert.assertEquals(group0ExpectedEndSequenceNumbers, group0Config.getEndSequenceNumbers());
Assert.assertEquals(group1ExpectedStartSequenceNumbers, group1Config.getStartSequenceNumbers());
Assert.assertEquals(group1ExpectedEndSequenceNumbers, group1Config.getEndSequenceNumbers());
return postSplitTasks;
}
/**
* Test task creation after a shard split with a closed shard, with the closed shards expiring and no longer
* being returned from record supplier.
*
* @param phaseTwoTasks List of tasks from the second phase where closed but not expired shards were present.
*/
private void testShardSplitPhaseThree(List<Task> phaseTwoTasks) throws Exception
{
EasyMock.reset(indexerMetadataStorageCoordinator);
EasyMock.reset(taskStorage);
EasyMock.reset(taskQueue);
EasyMock.reset(taskClient);
EasyMock.reset(taskMaster);
EasyMock.reset(taskRunner);
EasyMock.reset(supervisorRecordSupplier);
// second set of tasks ran, shard 0 has expired, but shard 1 and 2 have data
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(
SHARD_ID0, KinesisSequenceNumber.END_OF_SHARD_MARKER,
SHARD_ID1, "100",
SHARD_ID3, "100"
)
)
)
).anyTimes();
EasyMock.expect(
indexerMetadataStorageCoordinator.resetDataSourceMetadata(
DATASOURCE,
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(
SHARD_ID1, "100",
SHARD_ID3, "100"
)
)
)
)
).andReturn(true).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID1, SHARD_ID3))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD1_PARTITION, SHARD3_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID1)))
.andReturn("200").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID3)))
.andReturn("200").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> postSplitCaptured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
Task successfulTask0 = phaseTwoTasks.get(0);
EasyMock.expect(taskStorage.getStatus(successfulTask0.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask0.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask0.getId())).andReturn(Optional.of(successfulTask0)).anyTimes();
Task successfulTask1 = phaseTwoTasks.get(1);
EasyMock.expect(taskStorage.getStatus(successfulTask1.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask1.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask1.getId())).andReturn(Optional.of(successfulTask1)).anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(postSplitCaptured))).andReturn(true).times(2);
replayAll();
supervisor.runInternal();
verifyAll();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpointsGroup0 = new TreeMap<>();
checkpointsGroup0.put(0, ImmutableMap.of(
SHARD_ID1, "100"
));
TreeMap<Integer, Map<String, String>> checkpointsGroup1 = new TreeMap<>();
checkpointsGroup1.put(1, ImmutableMap.of(
SHARD_ID3, "100"
));
// there would be 2 tasks, 1 for each task group
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup0))
.times(1);
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup1))
.times(1);
List<Task> postSplitTasks = postSplitCaptured.getValues();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(postSplitTasks).anyTimes();
for (Task task : postSplitTasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
// Check that shardId-000000000000 which has hit EOS is not included in the sequences sent to the task for group 0
SeekableStreamStartSequenceNumbers<String, String> group0ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID1, "100"
),
ImmutableSet.of(SHARD_ID1)
);
SeekableStreamEndSequenceNumbers<String, String> group0ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
SeekableStreamStartSequenceNumbers<String, String> group1ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID3, "100"
),
ImmutableSet.of(SHARD_ID3)
);
SeekableStreamEndSequenceNumbers<String, String> group1ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID3, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
Assert.assertEquals(2, postSplitTasks.size());
KinesisIndexTaskIOConfig group0Config = ((KinesisIndexTask) postSplitTasks.get(0)).getIOConfig();
KinesisIndexTaskIOConfig group1Config = ((KinesisIndexTask) postSplitTasks.get(1)).getIOConfig();
Assert.assertEquals((Integer) 0, group0Config.getTaskGroupId());
Assert.assertEquals((Integer) 1, group1Config.getTaskGroupId());
Assert.assertEquals(group0ExpectedStartSequenceNumbers, group0Config.getStartSequenceNumbers());
Assert.assertEquals(group0ExpectedEndSequenceNumbers, group0Config.getEndSequenceNumbers());
Assert.assertEquals(group1ExpectedStartSequenceNumbers, group1Config.getStartSequenceNumbers());
Assert.assertEquals(group1ExpectedEndSequenceNumbers, group1Config.getEndSequenceNumbers());
Map<Integer, Map<String, String>> expectedPartitionGroups = ImmutableMap.of(
0, ImmutableMap.of(SHARD_ID1, "-1"),
1, ImmutableMap.of(SHARD_ID3, "-1")
);
Assert.assertEquals(expectedPartitionGroups, supervisor.getPartitionGroups());
}
@Test
public void testShardMerge() throws Exception
{
supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null);
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
List<Task> phaseOneTasks = testShardMergePhaseOne();
List<Task> phaseTwoTasks = testShardMergePhaseTwo(phaseOneTasks);
testShardMergePhaseThree(phaseTwoTasks);
}
private List<Task> testShardMergePhaseOne() throws Exception
{
supervisorRecordSupplier.assign(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID0, SHARD_ID1))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD0_PARTITION, SHARD1_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(EasyMock.anyObject())).andReturn("100").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> captured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2);
replayAll();
supervisor.start();
supervisor.runInternal();
verifyAll();
List<Task> tasks = captured.getValues();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpoints0 = new TreeMap<>();
checkpoints0.put(0, ImmutableMap.of(
SHARD_ID1,
"0"
));
TreeMap<Integer, Map<String, String>> checkpoints1 = new TreeMap<>();
checkpoints1.put(0, ImmutableMap.of(
SHARD_ID0,
"0"
));
// there would be 2 tasks, 1 for each task group
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpoints0))
.times(1);
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpoints1))
.times(1);
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(tasks).anyTimes();
for (Task task : tasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
return tasks;
}
/**
* Test task creation after a shard split with a closed shard
*
* @param phaseOneTasks List of tasks from the initial phase where only one shard was present
*/
private List<Task> testShardMergePhaseTwo(List<Task> phaseOneTasks) throws Exception
{
EasyMock.reset(indexerMetadataStorageCoordinator);
EasyMock.reset(taskStorage);
EasyMock.reset(taskQueue);
EasyMock.reset(taskClient);
EasyMock.reset(taskMaster);
EasyMock.reset(taskRunner);
EasyMock.reset(supervisorRecordSupplier);
// first tasks ran, both shard 0 and shard 1 have reached EOS, merged into shard 2
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(
SHARD_ID0, KinesisSequenceNumber.END_OF_SHARD_MARKER,
SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER
)
)
)
).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID0, SHARD_ID1, SHARD_ID2))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD0_PARTITION, SHARD1_PARTITION, SHARD2_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID0)))
.andReturn(KinesisSequenceNumber.END_OF_SHARD_MARKER).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID1)))
.andReturn(KinesisSequenceNumber.END_OF_SHARD_MARKER).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID2)))
.andReturn("100").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> postMergeCaptured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
Task successfulTask0 = phaseOneTasks.get(0);
Task successfulTask1 = phaseOneTasks.get(1);
EasyMock.expect(taskStorage.getStatus(successfulTask0.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask0.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask0.getId())).andReturn(Optional.of(successfulTask0)).anyTimes();
EasyMock.expect(taskStorage.getStatus(successfulTask1.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask1.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask1.getId())).andReturn(Optional.of(successfulTask1)).anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(postMergeCaptured))).andReturn(true).times(1);
replayAll();
supervisor.runInternal();
verifyAll();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpointsGroup0 = new TreeMap<>();
checkpointsGroup0.put(0, ImmutableMap.of(
SHARD_ID2, "0",
SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER
));
// there would be 1 tasks, 1 for each task group, but task group 1 only has closed shards, so no task is created
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup0))
.times(1);
List<Task> postMergeTasks = postMergeCaptured.getValues();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(postMergeTasks).anyTimes();
for (Task task : postMergeTasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
// Check that shardId-000000000000 which has hit EOS is not included in the sequences sent to the task for group 0
SeekableStreamStartSequenceNumbers<String, String> group0ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID2, "0"
),
ImmutableSet.of()
);
SeekableStreamEndSequenceNumbers<String, String> group0ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID2, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
Assert.assertEquals(1, postMergeTasks.size());
KinesisIndexTaskIOConfig group0Config = ((KinesisIndexTask) postMergeTasks.get(0)).getIOConfig();
Assert.assertEquals((Integer) 0, group0Config.getTaskGroupId());
Assert.assertEquals(group0ExpectedStartSequenceNumbers, group0Config.getStartSequenceNumbers());
Assert.assertEquals(group0ExpectedEndSequenceNumbers, group0Config.getEndSequenceNumbers());
return postMergeTasks;
}
/**
* Test task creation after a shard merge with two closed shards and one open shard, with the closed shards
* expiring and no longer being returned from record supplier.
*
* @param phaseTwoTasks List of tasks from the second phase where closed but not expired shards were present.
*/
private void testShardMergePhaseThree(List<Task> phaseTwoTasks) throws Exception
{
EasyMock.reset(indexerMetadataStorageCoordinator);
EasyMock.reset(taskStorage);
EasyMock.reset(taskQueue);
EasyMock.reset(taskClient);
EasyMock.reset(taskMaster);
EasyMock.reset(taskRunner);
EasyMock.reset(supervisorRecordSupplier);
// second set of tasks ran, shard 0 has expired, but shard 1 and 2 have data
EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(
SHARD_ID0, KinesisSequenceNumber.END_OF_SHARD_MARKER,
SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER,
SHARD_ID2, "100"
)
)
)
).anyTimes();
EasyMock.expect(
indexerMetadataStorageCoordinator.resetDataSourceMetadata(
DATASOURCE,
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<String, String>(
STREAM,
ImmutableMap.of(
SHARD_ID2, "100"
)
)
)
)
).andReturn(true).anyTimes();
EasyMock.expect(supervisorRecordSupplier.getPartitionIds(STREAM))
.andReturn(ImmutableSet.of(SHARD_ID2))
.anyTimes();
EasyMock.expect(supervisorRecordSupplier.getAssignment())
.andReturn(ImmutableSet.of(SHARD2_PARTITION))
.anyTimes();
supervisorRecordSupplier.seekToLatest(EasyMock.anyObject());
EasyMock.expectLastCall().anyTimes();
EasyMock.expect(supervisorRecordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes();
EasyMock.expect(supervisorRecordSupplier.getLatestSequenceNumber(new StreamPartition<>(STREAM, SHARD_ID2)))
.andReturn("200").anyTimes();
supervisorRecordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
EasyMock.expectLastCall().anyTimes();
Capture<Task> postSplitCaptured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
Task successfulTask0 = phaseTwoTasks.get(0);
EasyMock.expect(taskStorage.getStatus(successfulTask0.getId()))
.andReturn(Optional.of(TaskStatus.success(successfulTask0.getId())));
EasyMock.expect(taskStorage.getTask(successfulTask0.getId())).andReturn(Optional.of(successfulTask0)).anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(postSplitCaptured))).andReturn(true).times(1);
replayAll();
supervisor.runInternal();
verifyAll();
EasyMock.reset(taskStorage);
EasyMock.reset(taskClient);
EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED))
.anyTimes();
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
TreeMap<Integer, Map<String, String>> checkpointsGroup0 = new TreeMap<>();
checkpointsGroup0.put(0, ImmutableMap.of(
SHARD_ID2, "100"
));
// there would be 1 task, only task group 0 has a shard
EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpointsGroup0))
.times(1);
List<Task> postSplitTasks = postSplitCaptured.getValues();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(postSplitTasks).anyTimes();
for (Task task : postSplitTasks) {
EasyMock.expect(taskStorage.getStatus(task.getId()))
.andReturn(Optional.of(TaskStatus.running(task.getId())))
.anyTimes();
EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
}
EasyMock.replay(taskStorage);
EasyMock.replay(taskClient);
supervisor.runInternal();
verifyAll();
// Check that shardId-000000000000 which has hit EOS is not included in the sequences sent to the task for group 0
SeekableStreamStartSequenceNumbers<String, String> group0ExpectedStartSequenceNumbers =
new SeekableStreamStartSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID2, "100"
),
ImmutableSet.of(SHARD_ID2)
);
SeekableStreamEndSequenceNumbers<String, String> group0ExpectedEndSequenceNumbers =
new SeekableStreamEndSequenceNumbers<>(
STREAM,
ImmutableMap.of(
SHARD_ID2, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
)
);
Assert.assertEquals(1, postSplitTasks.size());
KinesisIndexTaskIOConfig group0Config = ((KinesisIndexTask) postSplitTasks.get(0)).getIOConfig();
Assert.assertEquals((Integer) 0, group0Config.getTaskGroupId());
Assert.assertEquals(group0ExpectedStartSequenceNumbers, group0Config.getStartSequenceNumbers());
Assert.assertEquals(group0ExpectedEndSequenceNumbers, group0Config.getEndSequenceNumbers());
Map<Integer, Map<String, String>> expectedPartitionGroups = ImmutableMap.of(
0, ImmutableMap.of(SHARD_ID2, "-1"),
1, ImmutableMap.of()
);
Assert.assertEquals(expectedPartitionGroups, supervisor.getPartitionGroups());
}
private TestableKinesisSupervisor getTestableSupervisor(
int replicas,
int taskCount,
@ -4402,6 +5237,9 @@ public class KinesisSupervisorTest extends EasyMockSupport
SeekableStreamIndexTaskTuningConfig tuningConfig
)
{
if (startPartitions.isEmpty()) {
return StringUtils.format("sequenceName-NOT-USED");
}
final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next());
return StringUtils.format("sequenceName-%d", groupId);
}

View File

@ -768,10 +768,14 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
status = Status.PUBLISHING;
}
for (int i = 0; i < sequences.size(); i++) {
final SequenceMetadata<PartitionIdType, SequenceOffsetType> sequenceMetadata = sequences.get(i);
// We need to copy sequences here, because the success callback in publishAndRegisterHandoff removes items from
// the sequence list. If a publish finishes before we finish iterating through the sequence list, we can
// end up skipping some sequences.
List<SequenceMetadata<PartitionIdType, SequenceOffsetType>> sequencesSnapshot = new ArrayList<>(sequences);
for (int i = 0; i < sequencesSnapshot.size(); i++) {
final SequenceMetadata<PartitionIdType, SequenceOffsetType> sequenceMetadata = sequencesSnapshot.get(i);
if (!publishingSequences.contains(sequenceMetadata.getSequenceName())) {
final boolean isLast = i == (sequences.size() - 1);
final boolean isLast = i == (sequencesSnapshot.size() - 1);
if (isLast) {
// Shorten endOffsets of the last sequence to match currOffsets.
sequenceMetadata.setEndOffsets(currOffsets);

View File

@ -155,6 +155,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
// same sequences, even if the values in [partitionGroups] has been changed.
final ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences;
// We don't include closed partitions in the starting offsets. However, we keep the full unfiltered map of
// partitions, only used for generating the sequence name, to avoid ambiguity in sequence names if mulitple
// task groups have nothing but closed partitions in their assignments.
final ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName;
final ConcurrentHashMap<String, TaskData> tasks = new ConcurrentHashMap<>();
final Optional<DateTime> minimumMessageTime;
final Optional<DateTime> maximumMessageTime;
@ -166,6 +171,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
TaskGroup(
int groupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
@Nullable ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName,
Optional<DateTime> minimumMessageTime,
Optional<DateTime> maximumMessageTime,
@Nullable Set<PartitionIdType> exclusiveStartSequenceNumberPartitions
@ -174,11 +180,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
this(
groupId,
startingSequences,
unfilteredStartingSequencesForSequenceName,
minimumMessageTime,
maximumMessageTime,
exclusiveStartSequenceNumberPartitions,
generateSequenceName(
startingSequences,
unfilteredStartingSequencesForSequenceName == null
? startingSequences
: unfilteredStartingSequencesForSequenceName,
minimumMessageTime,
maximumMessageTime,
spec.getDataSchema(),
@ -190,6 +199,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
TaskGroup(
int groupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
@Nullable ImmutableMap<PartitionIdType, SequenceOffsetType> unfilteredStartingSequencesForSequenceName,
Optional<DateTime> minimumMessageTime,
Optional<DateTime> maximumMessageTime,
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions,
@ -198,6 +208,9 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
{
this.groupId = groupId;
this.startingSequences = startingSequences;
this.unfilteredStartingSequencesForSequenceName = unfilteredStartingSequencesForSequenceName == null
? startingSequences
: unfilteredStartingSequencesForSequenceName;
this.minimumMessageTime = minimumMessageTime;
this.maximumMessageTime = maximumMessageTime;
this.checkpointSequences.put(0, startingSequences);
@ -278,7 +291,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
}
}
private class RunNotice implements Notice
{
@Override
@ -467,21 +479,21 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
// start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task
// failures during publishing.
// Map<{group RandomIdUtils}, Map<{partition RandomIdUtils}, {startingOffset}>>
private final ConcurrentHashMap<Integer, ConcurrentHashMap<PartitionIdType, SequenceOffsetType>> partitionGroups = new ConcurrentHashMap<>();
protected final ConcurrentHashMap<Integer, ConcurrentHashMap<PartitionIdType, SequenceOffsetType>> partitionGroups = new ConcurrentHashMap<>();
protected final ObjectMapper sortingMapper;
protected final List<PartitionIdType> partitionIds = new CopyOnWriteArrayList<>();
protected final SeekableStreamSupervisorStateManager stateManager;
protected volatile DateTime sequenceLastUpdated;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
protected final String dataSource;
private final Set<PartitionIdType> subsequentlyDiscoveredPartitions = new HashSet<>();
private final TaskStorage taskStorage;
private final TaskMaster taskMaster;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private final SeekableStreamIndexTaskClient<PartitionIdType, SequenceOffsetType> taskClient;
private final SeekableStreamSupervisorSpec spec;
private final String dataSource;
private final SeekableStreamSupervisorIOConfig ioConfig;
private final SeekableStreamSupervisorTuningConfig tuningConfig;
private final SeekableStreamIndexTaskTuningConfig taskTuningConfig;
@ -1000,6 +1012,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
TaskGroup group = new TaskGroup(
taskGroupId,
partitionOffsets,
null,
minMsgTime,
maxMsgTime,
exclusiveStartingSequencePartitions
@ -1026,6 +1039,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
TaskGroup group = new TaskGroup(
taskGroupId,
partitionOffsets,
null,
minMsgTime,
maxMsgTime,
exclusiveStartingSequencePartitions
@ -1428,6 +1442,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
.getStartSequenceNumbers()
.getPartitionSequenceNumberMap()
),
null,
seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(),
seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(),
seekableStreamIndexTask.getIOConfig()
@ -1682,10 +1697,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId);
// reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot
// change to a state where it will read any more events
// change to a state where it will read any more events.
// This is a discovered task, so it would not have been assigned closed partitions initially.
TaskGroup newTaskGroup = new TaskGroup(
groupId,
ImmutableMap.copyOf(startingPartitions),
null,
Optional.absent(),
Optional.absent(),
null
@ -1804,7 +1821,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
for (Entry<PartitionIdType, SequenceOffsetType> entry : startPartitions.entrySet()) {
sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString()));
}
String partitionOffsetStr = sb.toString().substring(1);
String partitionOffsetStr = startPartitions.size() == 0 ? "" : sb.toString().substring(1);
String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : "");
String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : "");
@ -1830,6 +1847,45 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
protected abstract String baseTaskName();
protected boolean supportsPartitionExpiration()
{
return false;
}
/**
*
* Some seekable stream systems such as Kinesis allow partitions to expire. When this occurs, the supervisor should
* remove the expired partitions from saved metadata and from the partition groups stored in memory.
*
* @param currentMetadata The current DataSourceMetadata from metadata storage
* @param expiredPartitionIds The set of expired partition IDs.
* @return currentMetadata but with any expired partitions removed.
*/
protected SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetadataWithoutExpiredPartitions(
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> currentMetadata,
Set<PartitionIdType> expiredPartitionIds
)
{
throw new UnsupportedOperationException("This supervisor type does not support partition expiration.");
}
/**
* Removes a set of expired partition IDs from partitionIds and partitionGroups. This is called after
* successfully removing expired partitions from metadata, for supervisor types that support partition expiration.
*
* @param expiredPartitionIds Set of expired partition IDs.
*/
private void removeExpiredPartitionsFromMemory(Set<PartitionIdType> expiredPartitionIds)
{
partitionIds.removeAll(expiredPartitionIds);
for (ConcurrentHashMap<PartitionIdType, SequenceOffsetType> partitionGroup : partitionGroups.values()) {
for (PartitionIdType expiredShard : expiredPartitionIds) {
partitionGroup.remove(expiredShard);
}
}
}
private boolean updatePartitionDataFromStream()
{
Set<PartitionIdType> partitionIds;
@ -1890,9 +1946,86 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
}
}
if (supportsPartitionExpiration()) {
// Look for expired shards and remove them from metadata storage and the partition groups
Set<PartitionIdType> expiredPartitions = new HashSet<>();
for (PartitionIdType partitionTd : closedPartitions) {
if (!partitionIds.contains(partitionTd)) {
expiredPartitions.add(partitionTd);
}
}
if (expiredPartitions.size() > 0) {
@SuppressWarnings("unchecked")
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> currentMetadata =
(SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) indexerMetadataStorageCoordinator.getDataSourceMetadata(
dataSource);
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> cleanedMetadata =
createDataSourceMetadataWithoutExpiredPartitions(currentMetadata, expiredPartitions);
validateMetadataPartitionExpiration(currentMetadata, cleanedMetadata);
try {
boolean success = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, cleanedMetadata);
if (success) {
removeExpiredPartitionsFromMemory(expiredPartitions);
} else {
log.error("Failed to update datasource metadata[%s] with expired partitions removed", cleanedMetadata);
}
}
catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}
return true;
}
/**
* Perform a sanity check on the datasource metadata returned by
* {@link #createDataSourceMetadataWithoutExpiredPartitions}.
*
* Specifically, we check that the cleaned metadata's partitions are a subset of the original metadata's partitions,
* and that none of the offsets for the non-expired partitions have changed.
*
* @param oldMetadata metadata containing expired partitions.
* @param cleanedMetadata new metadata without expired partitions, generated by the subclass
*/
private void validateMetadataPartitionExpiration(
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> oldMetadata,
SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> cleanedMetadata
)
{
Map<PartitionIdType, SequenceOffsetType> oldPartitionSeqNos = oldMetadata.getSeekableStreamSequenceNumbers()
.getPartitionSequenceNumberMap();
Map<PartitionIdType, SequenceOffsetType> cleanedPartitionSeqNos = cleanedMetadata.getSeekableStreamSequenceNumbers()
.getPartitionSequenceNumberMap();
for (Entry<PartitionIdType, SequenceOffsetType> cleanedPartitionSeqNo : cleanedPartitionSeqNos.entrySet()) {
if (!oldPartitionSeqNos.containsKey(cleanedPartitionSeqNo.getKey())) {
// cleaning the expired partitions added a partition somehow
throw new IAE(
"Cleaned partition map [%s] contains unexpected partition ID [%s], original partition map: [%s]",
cleanedPartitionSeqNos,
cleanedPartitionSeqNo.getKey(),
oldPartitionSeqNos
);
}
SequenceOffsetType oldOffset = oldPartitionSeqNos.get(cleanedPartitionSeqNo.getKey());
if (!oldOffset.equals(cleanedPartitionSeqNo.getValue())) {
throw new IAE(
"Cleaned partition map [%s] has offset mismatch for partition ID [%s], original partition map: [%s]",
cleanedPartitionSeqNos,
cleanedPartitionSeqNo.getKey(),
oldPartitionSeqNos
);
}
}
}
private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException
{
final List<ListenableFuture<Boolean>> futures = new ArrayList<>();
@ -2369,6 +2502,20 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
}
/**
* If the seekable stream system supported by this supervisor allows for partition expiration, expired partitions
* should be removed from the starting offsets sent to the tasks.
*
* @param startingOffsets
* @return
*/
protected Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> filterExpiredPartitionsFromStartingOffsets(
Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets
)
{
return startingOffsets;
}
private void createNewTasks()
throws JsonProcessingException
{
@ -2393,10 +2540,16 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get())
) : Optional.absent());
final Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets =
final Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> unfilteredStartingOffsets =
generateStartingSequencesForPartitionGroup(groupId);
final Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets;
if (supportsPartitionExpiration()) {
startingOffsets = filterExpiredPartitionsFromStartingOffsets(unfilteredStartingOffsets);
} else {
startingOffsets = unfilteredStartingOffsets;
}
ImmutableMap<PartitionIdType, SequenceOffsetType> simpleStartingOffsets = startingOffsets
.entrySet()
.stream()
@ -2407,6 +2560,21 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
)
);
ImmutableMap<PartitionIdType, SequenceOffsetType> simpleUnfilteredStartingOffsets;
if (supportsPartitionExpiration()) {
simpleUnfilteredStartingOffsets = unfilteredStartingOffsets
.entrySet()
.stream()
.filter(x -> x.getValue().get() != null)
.collect(
Collectors.collectingAndThen(
Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf
)
);
} else {
simpleUnfilteredStartingOffsets = simpleStartingOffsets;
}
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence
? Collections.emptySet()
: startingOffsets
@ -2422,6 +2590,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
new TaskGroup(
groupId,
simpleStartingOffsets,
simpleUnfilteredStartingOffsets,
minimumMessageTime,
maximumMessageTime,
exclusiveStartSequenceNumberPartitions
@ -2436,8 +2605,9 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
TaskGroup taskGroup = entry.getValue();
Integer groupId = entry.getKey();
if (taskGroup.startingSequences == null || taskGroup.startingSequences
.values().stream().allMatch(x -> x == null || isEndOfShard(x))) {
if (taskGroup.startingSequences == null ||
taskGroup.startingSequences.size() == 0 ||
taskGroup.startingSequences.values().stream().allMatch(x -> x == null || isEndOfShard(x))) {
log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId);
continue;
}
@ -2457,7 +2627,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
// just created. This is mainly for the benefit of the status API in situations where the run period is lengthy.
scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS);
}
}
private void addNotice(Notice notice)
@ -2818,6 +2987,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return contexts;
}
@VisibleForTesting
public ConcurrentHashMap<Integer, ConcurrentHashMap<PartitionIdType, SequenceOffsetType>> getPartitionGroups()
{
return partitionGroups;
}
/**
* creates a specific task IOConfig instance for Kafka/Kinesis
*