mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
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:
parent
96d8523ecb
commit
0c387c1d47
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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 {
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
*
|
||||
|
Loading…
x
Reference in New Issue
Block a user