From 372dca4722a57d72877b61b08e3e7a0b2387b82a Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Mar 2019 17:59:19 -0800 Subject: [PATCH 01/17] Recompute hash in isTaskCurrent() and added tests --- .../kafka/KafkaIndexTaskTuningConfigTest.java | 97 ++ .../kafka/supervisor/KafkaSupervisorTest.java | 775 ++++++--- ...estModifiedKafkaIndexTaskTuningConfig.java | 110 ++ .../KinesisIndexTaskTuningConfigTest.java | 117 ++ .../supervisor/KinesisSupervisorTest.java | 1405 ++++++++++------- ...tModifiedKinesisIndexTaskTuningConfig.java | 130 ++ .../supervisor/SeekableStreamSupervisor.java | 52 +- .../segment/indexing/DataSchemaTest.java | 80 + .../indexing/TestModifiedDataSchema.java | 73 + 9 files changed, 1987 insertions(+), 852 deletions(-) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java create mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java create mode 100644 server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 14ecd4e90e82..aecd734ccc4d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.indexing.kafka.test.TestModifiedKafkaIndexTaskTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfig; @@ -30,6 +31,7 @@ import org.junit.Test; import java.io.File; +import java.io.IOException; public class KafkaIndexTaskTuningConfigTest { @@ -145,6 +147,101 @@ public void testConvert() Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); } + @Test + public void testSerdeWithModifiedTuningConfigAddedField() throws IOException + { + KafkaIndexTaskTuningConfig base = new KafkaIndexTaskTuningConfig( + 1, + null, + 2, + 10L, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + null, + null, + null, + true, + 42, + 42 + ); + + String serialized = mapper.writeValueAsString(base); + TestModifiedKafkaIndexTaskTuningConfig deserialized = + mapper.readValue(serialized, TestModifiedKafkaIndexTaskTuningConfig.class); + + Assert.assertEquals(null, deserialized.getExtra()); + Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); + Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); + Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); + Assert.assertEquals(base.getMaxTotalRows(), deserialized.getMaxTotalRows()); + Assert.assertEquals(base.getIntermediatePersistPeriod(), deserialized.getIntermediatePersistPeriod()); + Assert.assertEquals(base.getBasePersistDirectory(), deserialized.getBasePersistDirectory()); + Assert.assertEquals(base.getMaxPendingPersists(), deserialized.getMaxPendingPersists()); + Assert.assertEquals(base.getIndexSpec(), deserialized.getIndexSpec()); + Assert.assertEquals(base.getBuildV9Directly(), deserialized.getBuildV9Directly()); + Assert.assertEquals(base.isReportParseExceptions(), deserialized.isReportParseExceptions()); + Assert.assertEquals(base.getHandoffConditionTimeout(), deserialized.getHandoffConditionTimeout()); + Assert.assertEquals(base.isResetOffsetAutomatically(), deserialized.isResetOffsetAutomatically()); + Assert.assertEquals(base.getSegmentWriteOutMediumFactory(), deserialized.getSegmentWriteOutMediumFactory()); + Assert.assertEquals(base.getIntermediateHandoffPeriod(), deserialized.getIntermediateHandoffPeriod()); + Assert.assertEquals(base.isLogParseExceptions(), deserialized.isLogParseExceptions()); + Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); + Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); + } + + @Test + public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException + { + TestModifiedKafkaIndexTaskTuningConfig base = new TestModifiedKafkaIndexTaskTuningConfig( + 1, + null, + 2, + 10L, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + null, + null, + null, + true, + 42, + 42, + "extra string" + ); + + String serialized = mapper.writeValueAsString(base); + KafkaIndexTaskTuningConfig deserialized = + mapper.readValue(serialized, KafkaIndexTaskTuningConfig.class); + + Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); + Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); + Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); + Assert.assertEquals(base.getMaxTotalRows(), deserialized.getMaxTotalRows()); + Assert.assertEquals(base.getIntermediatePersistPeriod(), deserialized.getIntermediatePersistPeriod()); + Assert.assertEquals(base.getBasePersistDirectory(), deserialized.getBasePersistDirectory()); + Assert.assertEquals(base.getMaxPendingPersists(), deserialized.getMaxPendingPersists()); + Assert.assertEquals(base.getIndexSpec(), deserialized.getIndexSpec()); + Assert.assertEquals(base.getBuildV9Directly(), deserialized.getBuildV9Directly()); + Assert.assertEquals(base.isReportParseExceptions(), deserialized.isReportParseExceptions()); + Assert.assertEquals(base.getHandoffConditionTimeout(), deserialized.getHandoffConditionTimeout()); + Assert.assertEquals(base.isResetOffsetAutomatically(), deserialized.isResetOffsetAutomatically()); + Assert.assertEquals(base.getSegmentWriteOutMediumFactory(), deserialized.getSegmentWriteOutMediumFactory()); + Assert.assertEquals(base.getIntermediateHandoffPeriod(), deserialized.getIntermediateHandoffPeriod()); + Assert.assertEquals(base.isLogParseExceptions(), deserialized.isLogParseExceptions()); + Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); + Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); + } + + private static KafkaIndexTaskTuningConfig copy(KafkaIndexTaskTuningConfig config) { return new KafkaIndexTaskTuningConfig( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index f2db280c51eb..0046adf9f725 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -49,6 +49,7 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -60,6 +61,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; @@ -258,7 +260,7 @@ public static void tearDownClass() throws IOException @Test public void testNoInitialState() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(); @@ -304,7 +306,7 @@ public void testNoInitialState() throws Exception @Test public void testSkipOffsetGaps() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(); @@ -332,7 +334,7 @@ public void testSkipOffsetGaps() throws Exception @Test public void testMultiTask() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -378,7 +380,7 @@ public void testMultiTask() throws Exception @Test public void testReplicas() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -415,7 +417,7 @@ public void testReplicas() throws Exception @Test public void testLateMessageRejectionPeriod() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -454,7 +456,7 @@ public void testLateMessageRejectionPeriod() throws Exception @Test public void testEarlyMessageRejectionPeriod() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -496,7 +498,7 @@ public void testEarlyMessageRejectionPeriod() throws Exception */ public void testLatestOffset() throws Exception { - supervisor = getSupervisor(1, 1, false, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, false, "PT1H", null, null); addSomeEvents(1100); Capture captured = Capture.newInstance(); @@ -528,7 +530,7 @@ public void testLatestOffset() throws Exception */ public void testDatasourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(100); Capture captured = Capture.newInstance(); @@ -558,7 +560,7 @@ public void testDatasourceMetadata() throws Exception @Test(expected = ISE.class) public void testBadMetadataOffsets() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(1); expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); @@ -574,222 +576,10 @@ public void testBadMetadataOffsets() throws Exception supervisor.runInternal(); } - @Test - public void testKillIncompatibleTasks() throws Exception - { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - addSomeEvents(1); - - // unexpected # of partitions (kill) - Task id1 = createKafkaIndexTask( - "id1", - DATASOURCE, - 1, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)), - null, - null - ); - - // correct number of partitions and ranges (don't kill) - Task id2 = createKafkaIndexTask( - "id2", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)), - null, - null - ); - - // unexpected range on partition 2 (kill) - Task id3 = createKafkaIndexTask( - "id3", - DATASOURCE, - 1, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)), - null, - null - ); - - // different datasource (don't kill) - Task id4 = createKafkaIndexTask( - "id4", - "other-datasource", - 2, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)), - null, - null - ); - - // non KafkaIndexTask (don't kill) - Task id5 = new RealtimeIndexTask( - "id5", - null, - new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null, null), - null - ), - null - ); - - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( - new KafkaDataSourceMetadata( - null - ) - ).anyTimes(); - expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); - taskQueue.shutdown("id3", "Task [%s] failed to stop in a timely manner, killing task", "id3"); - - expect(taskQueue.add(anyObject(Task.class))).andReturn(true); - - TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); - - replayAll(); - - supervisor.start(); - supervisor.runInternal(); - verifyAll(); - } - - @Test - public void testKillBadPartitionAssignment() throws Exception - { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null); - addSomeEvents(1); - - Task id1 = createKafkaIndexTask( - "id1", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), - null, - null - ); - Task id2 = createKafkaIndexTask( - "id2", - DATASOURCE, - 1, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)), - null, - null - ); - Task id3 = createKafkaIndexTask( - "id3", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), - new SeekableStreamPartitions<>( - "topic", - ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) - ), - null, - null - ); - Task id4 = createKafkaIndexTask( - "id4", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), - null, - null - ); - Task id5 = createKafkaIndexTask( - "id5", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L, 2, 0L)), - new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), - null, - null - ); - - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( - new KafkaDataSourceMetadata( - null - ) - ).anyTimes(); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); - - TreeMap> checkpoints1 = new TreeMap<>(); - checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); - TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(1); - - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); - taskQueue.shutdown("id4", "Task [%s] failed to stop in a timely manner, killing task", "id4"); - taskQueue.shutdown("id5", "Task [%s] failed to stop in a timely manner, killing task", "id5"); - replayAll(); - - supervisor.start(); - supervisor.runInternal(); - verifyAll(); - } - @Test public void testRequeueTaskWhenFailed() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -878,7 +668,7 @@ public void testRequeueTaskWhenFailed() throws Exception @Test public void testRequeueAdoptedTaskWhenFailed() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); addSomeEvents(1); DateTime now = DateTimes.nowUtc(); @@ -979,7 +769,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception @Test public void testQueueNextTasksOnSuccess() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1081,7 +871,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1176,7 +966,7 @@ public void testDiscoverExistingPublishingTask() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(1); Task task = createKafkaIndexTask( @@ -1282,7 +1072,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(1); Task task = createKafkaIndexTask( @@ -1381,7 +1171,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); addSomeEvents(6); Task id1 = createKafkaIndexTask( @@ -1488,7 +1278,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception @Test public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1547,7 +1337,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1633,7 +1423,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1724,7 +1514,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception @Test(expected = IllegalStateException.class) public void testStopNotStarted() { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.stop(false); } @@ -1736,7 +1526,7 @@ public void testStop() taskRunner.unregisterListener(StringUtils.format("KafkaSupervisor-%s", DATASOURCE)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.start(); supervisor.stop(false); @@ -1750,7 +1540,7 @@ public void testStopGracefully() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -1862,7 +1652,7 @@ public void testResetNoTasks() throws Exception taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.start(); supervisor.runInternal(); verifyAll(); @@ -1879,7 +1669,7 @@ public void testResetNoTasks() throws Exception @Test public void testResetDataSourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -1935,7 +1725,7 @@ public void testResetDataSourceMetadata() throws Exception @Test public void testResetNoDataSourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -1968,7 +1758,7 @@ public void testResetRunningTasks() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -2066,7 +1856,7 @@ public void testResetRunningTasks() throws Exception public void testNoDataIngestionTasks() throws Exception { final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1S", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null); //not adding any events Task id1 = createKafkaIndexTask( "id1", @@ -2162,7 +1952,7 @@ public void testNoDataIngestionTasks() throws Exception public void testCheckpointForInactiveTaskGroup() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { - supervisor = getSupervisor(2, 1, true, "PT1S", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null); //not adding any events final Task id1 = createKafkaIndexTask( "id1", @@ -2265,7 +2055,7 @@ public void testCheckpointForInactiveTaskGroup() public void testCheckpointForUnknownTaskGroup() throws InterruptedException { - supervisor = getSupervisor(2, 1, true, "PT1S", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null); //not adding any events final Task id1 = createKafkaIndexTask( "id1", @@ -2344,7 +2134,7 @@ public void testCheckpointForUnknownTaskGroup() public void testCheckpointWithNullTaskGroupId() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { - supervisor = getSupervisor(1, 3, true, "PT1S", null, null); + supervisor = getTestableSupervisor(1, 3, true, "PT1S", null, null); //not adding any events final Task id1 = createKafkaIndexTask( "id1", @@ -2436,7 +2226,7 @@ public void testCheckpointWithNullTaskGroupId() @Test public void testSuspendedNoRunningTasks() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true, kafkaHost); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null, true, kafkaHost); addSomeEvents(1); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -2469,7 +2259,7 @@ public void testSuspendedRunningTasks() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null, true, kafkaHost); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null, true, kafkaHost); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -2577,7 +2367,7 @@ public void testResetSuspended() throws Exception taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true, kafkaHost); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null, true, kafkaHost); supervisor.start(); supervisor.runInternal(); verifyAll(); @@ -2594,7 +2384,7 @@ public void testResetSuspended() throws Exception public void testFailedInitializationAndRecovery() throws Exception { // Block the supervisor initialization with a bad hostname config, make sure this doesn't block the lifecycle - supervisor = getSupervisor( + supervisor = getTestableSupervisor( 1, 1, true, @@ -2682,7 +2472,7 @@ public void testFailedInitializationAndRecovery() throws Exception @Test public void testGetCurrentTotalStats() { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null, false, kafkaHost); + supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null, false, kafkaHost); supervisor.addTaskGroupToActivelyReadingTaskGroup( supervisor.getTaskGroupIdForPartition(0), ImmutableMap.of(0, 0L), @@ -2723,6 +2513,246 @@ public void testGetCurrentTotalStats() Assert.assertEquals(ImmutableMap.of("task2", ImmutableMap.of("prop2", "val2")), stats.get("1")); } + @Test + public void testDoNotKillCompatibleTasks() + throws Exception + { + // This supervisor always returns true for isTaskCurrent -> it should not kill its tasks + int numReplicas = 2; + supervisor = getTestableSupervisorCustomIsTaskCurrent( + numReplicas, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), + false, + true + ); + + addSomeEvents(1); + + Task task = createKafkaIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null, + null + ); + + List existingTasks = ImmutableList.of(task); + + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(numReplicas); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillIncompatibleTasks() + throws Exception + { + // This supervisor always returns false for isTaskCurrent -> it should kill its tasks + int numReplicas = 2; + supervisor = getTestableSupervisorCustomIsTaskCurrent( + numReplicas, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), + false, + false + ); + + addSomeEvents(1); + + Task task = createKafkaIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null, + null + ); + + List existingTasks = ImmutableList.of(task); + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + EasyMock.expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true).times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testIsTaskCurrent() + { + DateTime minMessageTime = DateTimes.nowUtc(); + DateTime maxMessageTime = DateTimes.nowUtc().plus(10000); + + KafkaSupervisor supervisor = getSupervisor( + 2, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), + false, + kafkaHost, + dataSchema, + tuningConfig + ); + + supervisor.addTaskGroupToActivelyReadingTaskGroup( + 42, + ImmutableMap.of(0, 0L, 2, 0L), + Optional.of(minMessageTime), + Optional.of(maxMessageTime), + ImmutableSet.of("id1", "id2", "id3", "id4"), + ImmutableSet.of() + ); + + DataSchema modifiedDataSchema = getDataSchema("some other datasource"); + + KafkaSupervisorTuningConfig modifiedTuningConfig = new KafkaSupervisorTuningConfig( + 42, // This is different + null, + 50000, + null, + new Period("P1Y"), + new File("/test"), + null, + null, + true, + false, + null, + null, + null, + numThreads, + TEST_CHAT_THREADS, + TEST_CHAT_RETRIES, + TEST_HTTP_TIMEOUT, + TEST_SHUTDOWN_TIMEOUT, + null, + null, + null, + null, + null + ); + + KafkaIndexTask taskFromStorage = createKafkaIndexTask( + "id1", + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + minMessageTime, + maxMessageTime, + dataSchema + ); + + KafkaIndexTask taskFromStorageMismatchedDataSchema = createKafkaIndexTask( + "id2", + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + minMessageTime, + maxMessageTime, + modifiedDataSchema + ); + + KafkaIndexTask taskFromStorageMismatchedTuningConfig = createKafkaIndexTask( + "id3", + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + minMessageTime, + maxMessageTime, + dataSchema, + modifiedTuningConfig + ); + + KafkaIndexTask taskFromStorageMismatchedPartitionsWithTaskGroup = createKafkaIndexTask( + "id4", + 0, + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 6L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + minMessageTime, + maxMessageTime, + dataSchema + ); + + EasyMock.expect(taskStorage.getTask("id1")) + .andReturn(Optional.of(taskFromStorage)) + .once(); + EasyMock.expect(taskStorage.getTask("id2")) + .andReturn(Optional.of(taskFromStorageMismatchedDataSchema)) + .once(); + EasyMock.expect(taskStorage.getTask("id3")) + .andReturn(Optional.of(taskFromStorageMismatchedTuningConfig)) + .once(); + EasyMock.expect(taskStorage.getTask("id4")) + .andReturn(Optional.of(taskFromStorageMismatchedPartitionsWithTaskGroup)) + .once(); + + replayAll(); + + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4")); + verifyAll(); + } private void addSomeEvents(int numEventsPerPartition) throws Exception { @@ -2748,7 +2778,7 @@ private void addSomeEvents(int numEventsPerPartition) throws Exception } } - private KafkaSupervisor getSupervisor( + private KafkaSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -2757,7 +2787,7 @@ private KafkaSupervisor getSupervisor( Period earlyMessageRejectionPeriod ) { - return getSupervisor( + return getTestableSupervisor( replicas, taskCount, useEarliestOffset, @@ -2769,7 +2799,7 @@ private KafkaSupervisor getSupervisor( ); } - private KafkaSupervisor getSupervisor( + private KafkaSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -2845,6 +2875,168 @@ public KafkaIndexTaskClient build( ); } + /** + * Use when you want to mock the return value of SeekableStreamSupervisor#isTaskCurrent() + */ + private KafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + boolean isTaskCurrentReturn + ) + { + Map consumerProperties = new HashMap<>(); + consumerProperties.put("myCustomKey", "myCustomValue"); + consumerProperties.put("bootstrap.servers", kafkaHost); + consumerProperties.put("isolation.level", "read_committed"); + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( + topic, + replicas, + taskCount, + new Period(duration), + consumerProperties, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod + ); + + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null + ) + { + @Override + public KafkaIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new TestableKafkaSupervisorWithCustomIsTaskCurrent( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KafkaSupervisorSpec( + dataSchema, + tuningConfig, + kafkaSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory + ), + rowIngestionMetersFactory, + isTaskCurrentReturn + ); + } + + /** + * Use when you don't want generateSequenceNumber overridden + */ + + private KafkaSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + String kafkaHost, + DataSchema dataSchema, + KafkaSupervisorTuningConfig tuningConfig + ) + { + Map consumerProperties = new HashMap<>(); + consumerProperties.put("myCustomKey", "myCustomValue"); + consumerProperties.put("bootstrap.servers", kafkaHost); + consumerProperties.put("isolation.level", "read_committed"); + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( + topic, + replicas, + taskCount, + new Period(duration), + consumerProperties, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod + ); + + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null + ) + { + @Override + public KafkaIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new KafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KafkaSupervisorSpec( + dataSchema, + tuningConfig, + kafkaSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory + ), + rowIngestionMetersFactory + ); + } + private static DataSchema getDataSchema(String dataSource) { List dimensions = new ArrayList<>(); @@ -2889,11 +3081,55 @@ private KafkaIndexTask createKafkaIndexTask( DateTime minimumMessageTime, DateTime maximumMessageTime ) + { + return createKafkaIndexTask( + id, + taskGroupId, + startPartitions, + endPartitions, + minimumMessageTime, + maximumMessageTime, + getDataSchema(dataSource) + ); + } + + private KafkaIndexTask createKafkaIndexTask( + String id, + int taskGroupId, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + DataSchema schema + ) + { + return createKafkaIndexTask( + id, + taskGroupId, + startPartitions, + endPartitions, + minimumMessageTime, + maximumMessageTime, + schema, + (KafkaIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig() + ); + } + + private KafkaIndexTask createKafkaIndexTask( + String id, + int taskGroupId, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + DataSchema schema, + KafkaIndexTaskTuningConfig tuningConfig + ) { return new KafkaIndexTask( id, null, - getDataSchema(dataSource), + schema, tuningConfig, new KafkaIndexTaskIOConfig( taskGroupId, @@ -2945,7 +3181,6 @@ public String getDataSource() { return dataSource; } - } private static class TestableKafkaSupervisor extends KafkaSupervisor @@ -2975,7 +3210,9 @@ public TestableKafkaSupervisor( protected String generateSequenceName( Map startPartitions, Optional minimumMessageTime, - Optional maximumMessageTime + Optional maximumMessageTime, + DataSchema dataSchema, + SeekableStreamIndexTaskTuningConfig tuningConfig ) { final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); @@ -2983,5 +3220,37 @@ protected String generateSequenceName( } } + private class TestableKafkaSupervisorWithCustomIsTaskCurrent extends TestableKafkaSupervisor + { + private boolean isTaskCurrentReturn; + public TestableKafkaSupervisorWithCustomIsTaskCurrent( + TaskStorage taskStorage, + TaskMaster taskMaster, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + KafkaIndexTaskClientFactory taskClientFactory, + ObjectMapper mapper, + KafkaSupervisorSpec spec, + RowIngestionMetersFactory rowIngestionMetersFactory, + boolean isTaskCurrentReturn + ) + { + super( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory + ); + this.isTaskCurrentReturn = isTaskCurrentReturn; + } + + @Override + public boolean isTaskCurrent(int taskGroupId, String taskId) + { + return isTaskCurrentReturn; + } + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java new file mode 100644 index 000000000000..eb8bee9246d9 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kafka.test; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; + +public class TestModifiedKafkaIndexTaskTuningConfig extends KafkaIndexTaskTuningConfig +{ + private final String extra; + + @JsonCreator + public TestModifiedKafkaIndexTaskTuningConfig( + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("extra") String extra + ) + { + super( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + this.extra = extra; + } + + public TestModifiedKafkaIndexTaskTuningConfig(KafkaIndexTaskTuningConfig base, String extra) + { + super( + base.getMaxRowsInMemory(), + base.getMaxBytesInMemory(), + base.getMaxRowsPerSegment(), + base.getMaxTotalRows(), + base.getIntermediatePersistPeriod(), + base.getBasePersistDirectory(), + base.getMaxPendingPersists(), + base.getIndexSpec(), + base.getBuildV9Directly(), + base.isReportParseExceptions(), + base.getHandoffConditionTimeout(), + base.isResetOffsetAutomatically(), + base.getSegmentWriteOutMediumFactory(), + base.getIntermediateHandoffPeriod(), + base.isLogParseExceptions(), + base.getMaxParseExceptions(), + base.getMaxSavedParseExceptions() + ); + this.extra = extra; + } + + @JsonProperty("extra") + public String getExtra() + { + return extra; + } +} \ No newline at end of file diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 2983324b0ce4..71d4b42e285f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; +import org.apache.druid.indexing.kinesis.test.TestModifiedKinesisIndexTaskTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfig; @@ -34,6 +35,7 @@ import org.junit.rules.ExpectedException; import java.io.File; +import java.io.IOException; public class KinesisIndexTaskTuningConfigTest { @@ -130,6 +132,121 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertFalse(config.isResetOffsetAutomatically()); } + @Test + public void testSerdeWithModifiedTuningConfigAddedField() throws IOException + { + KinesisIndexTaskTuningConfig base = new KinesisIndexTaskTuningConfig( + 1, + 3L, + 2, + 100L, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + true, + false, + 1000, + 1000, + 500, + null, + 42, + null, + false, + 500, + 500, + 6000, + new Period("P3D") + ); + + String serialized = mapper.writeValueAsString(base); + TestModifiedKinesisIndexTaskTuningConfig deserialized = + mapper.readValue(serialized, TestModifiedKinesisIndexTaskTuningConfig.class); + + Assert.assertEquals(null, deserialized.getExtra()); + Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); + Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); + Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); + Assert.assertEquals(base.getMaxTotalRows(), deserialized.getMaxTotalRows()); + Assert.assertEquals(base.getIntermediatePersistPeriod(), deserialized.getIntermediatePersistPeriod()); + Assert.assertEquals(base.getBasePersistDirectory(), deserialized.getBasePersistDirectory()); + Assert.assertEquals(base.getMaxPendingPersists(), deserialized.getMaxPendingPersists()); + Assert.assertEquals(base.getIndexSpec(), deserialized.getIndexSpec()); + Assert.assertEquals(base.getBuildV9Directly(), deserialized.getBuildV9Directly()); + Assert.assertEquals(base.isReportParseExceptions(), deserialized.isReportParseExceptions()); + Assert.assertEquals(base.getHandoffConditionTimeout(), deserialized.getHandoffConditionTimeout()); + Assert.assertEquals(base.isResetOffsetAutomatically(), deserialized.isResetOffsetAutomatically()); + Assert.assertEquals(base.getSegmentWriteOutMediumFactory(), deserialized.getSegmentWriteOutMediumFactory()); + Assert.assertEquals(base.getIntermediateHandoffPeriod(), deserialized.getIntermediateHandoffPeriod()); + Assert.assertEquals(base.isLogParseExceptions(), deserialized.isLogParseExceptions()); + Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); + Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); + Assert.assertEquals(base.getRecordBufferFullWait(), deserialized.getRecordBufferFullWait()); + Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); + Assert.assertEquals(base.getRecordBufferSize(), deserialized.getRecordBufferSize()); + Assert.assertEquals(base.getMaxRecordsPerPoll(), deserialized.getMaxRecordsPerPoll()); + } + + @Test + public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException, IOException + { + KinesisIndexTaskTuningConfig base = new KinesisIndexTaskTuningConfig( + 1, + 3L, + 2, + 100L, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + true, + false, + 1000, + 1000, + 500, + null, + 42, + null, + false, + 500, + 500, + 6000, + new Period("P3D") + ); + + String serialized = mapper.writeValueAsString(new TestModifiedKinesisIndexTaskTuningConfig(base, "loool")); + KinesisIndexTaskTuningConfig deserialized = + mapper.readValue(serialized, KinesisIndexTaskTuningConfig.class); + + Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); + Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); + Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); + Assert.assertEquals(base.getMaxTotalRows(), deserialized.getMaxTotalRows()); + Assert.assertEquals(base.getIntermediatePersistPeriod(), deserialized.getIntermediatePersistPeriod()); + Assert.assertEquals(base.getBasePersistDirectory(), deserialized.getBasePersistDirectory()); + Assert.assertEquals(base.getMaxPendingPersists(), deserialized.getMaxPendingPersists()); + Assert.assertEquals(base.getIndexSpec(), deserialized.getIndexSpec()); + Assert.assertEquals(base.getBuildV9Directly(), deserialized.getBuildV9Directly()); + Assert.assertEquals(base.isReportParseExceptions(), deserialized.isReportParseExceptions()); + Assert.assertEquals(base.getHandoffConditionTimeout(), deserialized.getHandoffConditionTimeout()); + Assert.assertEquals(base.isResetOffsetAutomatically(), deserialized.isResetOffsetAutomatically()); + Assert.assertEquals(base.getSegmentWriteOutMediumFactory(), deserialized.getSegmentWriteOutMediumFactory()); + Assert.assertEquals(base.getIntermediateHandoffPeriod(), deserialized.getIntermediateHandoffPeriod()); + Assert.assertEquals(base.isLogParseExceptions(), deserialized.isLogParseExceptions()); + Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); + Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); + Assert.assertEquals(base.getRecordBufferFullWait(), deserialized.getRecordBufferFullWait()); + Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); + Assert.assertEquals(base.getRecordBufferSize(), deserialized.getRecordBufferSize()); + Assert.assertEquals(base.getMaxRecordsPerPoll(), deserialized.getMaxRecordsPerPoll()); + } + @Test public void testResetOffsetAndSkipSequenceNotBothTrue() throws Exception { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 2ceadb527cd5..552cfa4a2816 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -39,13 +39,13 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -56,6 +56,7 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -66,13 +67,12 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -110,8 +110,6 @@ public class KinesisSupervisorTest extends EasyMockSupport { - - private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final String DATASOURCE = "testDS"; private static final int TEST_CHAT_THREADS = 3; @@ -207,7 +205,7 @@ public void tearDownTest() @Test public void testNoInitialState() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -272,7 +270,7 @@ public void testNoInitialState() throws Exception @Test public void testMultiTask() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -329,7 +327,7 @@ public void testMultiTask() throws Exception @Test public void testReplicas() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -403,7 +401,7 @@ public void testReplicas() throws Exception @Test public void testLateMessageRejectionPeriod() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -452,7 +450,7 @@ public void testLateMessageRejectionPeriod() throws Exception @Test public void testEarlyMessageRejectionPeriod() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -506,7 +504,7 @@ public void testEarlyMessageRejectionPeriod() throws Exception */ public void testDatasourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -559,7 +557,7 @@ public void testDatasourceMetadata() throws Exception @Test(expected = ISE.class) public void testBadMetadataOffsets() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -593,10 +591,9 @@ public void testBadMetadataOffsets() throws Exception } @Test - public void testKillIncompatibleTasks() throws Exception + public void testRequeueTaskWhenFailed() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -609,110 +606,12 @@ public void testKillIncompatibleTasks() throws Exception supervisorRecordSupplier.seek(anyObject(), anyString()); expectLastCall().anyTimes(); - // unexpected # of partitions (kill) - Task id1 = createKinesisIndexTask( - "id1", - DATASOURCE, - 1, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1" - )), - null, - null - ); - - // correct number of partitions and ranges (don't kill) - Task id2 = createKinesisIndexTask( - "id2", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0", - shardId1, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1", - shardId1, - "12" - )), - null, - null - ); - - // unexpected range on partition 2 (kill) - Task id3 = createKinesisIndexTask( - "id3", - DATASOURCE, - 1, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0", - shardId1, - "1" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1", - shardId1, - "11" - )), - null, - null - ); - - // different datasource (don't kill) - Task id4 = createKinesisIndexTask( - "id4", - "other-datasource", - 2, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0", - shardId1, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1", - shardId1, - "12" - )), - null, - null - ); - - // non KinesisIndexTask (don't kill) - Task id5 = new RealtimeIndexTask( - "id5", - null, - new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null, null), - null - ), - null - ); - - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + Capture 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); @@ -724,36 +623,86 @@ public void testKillIncompatibleTasks() throws Exception null ) ).anyTimes(); - EasyMock.expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - taskQueue.shutdown("id3", "Task [%s] failed to stop in a timely manner, killing task", "id3"); - - EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); - TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ImmutableMap.of( + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put( + 0, + ImmutableMap.of( + shardId1, + "0" + ) + ); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of( shardId0, - "0", - shardId1, "0" )); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .anyTimes(); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); - + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); + supervisor.start(); supervisor.runInternal(); verifyAll(); + + List tasks = captured.getValues(); + + // test that running the main loop again checks the status of the tasks that were created and does nothing if they + // are all still running + EasyMock.reset(taskStorage); + EasyMock.expect(taskStorage.getActiveTasks()).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); + + supervisor.runInternal(); + verifyAll(); + + // test that a task failing causes a new task to be re-queued with the same parameters + Capture aNewTaskCapture = Capture.newInstance(); + List imStillAlive = tasks.subList(0, 3); + KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + for (Task task : imStillAlive) { + 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.expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); + + supervisor.runInternal(); + verifyAll(); + + Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); + Assert.assertEquals( + iHaveFailed.getIOConfig().getBaseSequenceName(), + ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() + ); } @Test - public void testKillBadPartitionAssignment() throws Exception + public void testRequeueAdoptedTaskWhenFailed() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null); - + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -766,289 +715,31 @@ public void testKillBadPartitionAssignment() throws Exception supervisorRecordSupplier.seek(anyObject(), anyString()); expectLastCall().anyTimes(); + DateTime now = DateTimes.nowUtc(); + DateTime maxi = now.plusMinutes(60); Task id1 = createKinesisIndexTask( "id1", DATASOURCE, 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - "12" - )), - null, - null - ); - Task id2 = createKinesisIndexTask( - "id2", - DATASOURCE, - 1, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1" - )), - null, - null - ); - Task id3 = createKinesisIndexTask( - "id3", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, "0", - shardId1, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1", - shardId1, - "12" - )), - null, - null - ); - Task id4 = createKinesisIndexTask( - "id4", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "0" - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId0, - "1" - )), - null, - null - ); - Task id5 = createKinesisIndexTask( - "id5", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, "0" )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - "1" + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), - null, - null + now, + maxi ); - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + List existingTasks = ImmutableList.of(id1); - 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(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(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - EasyMock.expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - EasyMock.expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); - - TreeMap> checkpoints1 = new TreeMap<>(); - checkpoints1.put(0, ImmutableMap.of(shardId1, "0")); - TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(1); - - - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - taskQueue.shutdown("id4", "Task [%s] failed to stop in a timely manner, killing task", "id4"); - taskQueue.shutdown("id5", "Task [%s] failed to stop in a timely manner, killing task", "id5"); - replayAll(); - - supervisor.start(); - supervisor.runInternal(); - verifyAll(); - } - - @Test - public void testRequeueTaskWhenFailed() throws Exception - { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); - supervisorRecordSupplier.assign(anyObject()); - expectLastCall().anyTimes(); - expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); - expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) - .anyTimes(); - supervisorRecordSupplier.seekToLatest(anyObject()); - expectLastCall().anyTimes(); - expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); - expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); - supervisorRecordSupplier.seek(anyObject(), anyString()); - expectLastCall().anyTimes(); - - - Capture 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.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(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(4); - - TreeMap> checkpoints1 = new TreeMap<>(); - checkpoints1.put( - 0, - ImmutableMap.of( - shardId1, - "0" - ) - ); - TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of( - shardId0, - "0" - )); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .anyTimes(); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .anyTimes(); - - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - replayAll(); - - supervisor.start(); - supervisor.runInternal(); - verifyAll(); - - List tasks = captured.getValues(); - - // test that running the main loop again checks the status of the tasks that were created and does nothing if they - // are all still running - EasyMock.reset(taskStorage); - EasyMock.expect(taskStorage.getActiveTasks()).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); - - supervisor.runInternal(); - verifyAll(); - - // test that a task failing causes a new task to be re-queued with the same parameters - Capture aNewTaskCapture = Capture.newInstance(); - List imStillAlive = tasks.subList(0, 3); - KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); - EasyMock.reset(taskStorage); - EasyMock.reset(taskQueue); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); - for (Task task : imStillAlive) { - 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.expect(taskStorage.getStatus(iHaveFailed.getId())) - .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); - EasyMock.replay(taskStorage); - EasyMock.replay(taskQueue); - - supervisor.runInternal(); - verifyAll(); - - Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); - Assert.assertEquals( - iHaveFailed.getIOConfig().getBaseSequenceName(), - ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() - ); - } - - @Test - public void testRequeueAdoptedTaskWhenFailed() throws Exception - { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - supervisorRecordSupplier.assign(anyObject()); - expectLastCall().anyTimes(); - expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); - expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) - .anyTimes(); - supervisorRecordSupplier.seekToLatest(anyObject()); - expectLastCall().anyTimes(); - expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); - expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); - supervisorRecordSupplier.seek(anyObject(), anyString()); - expectLastCall().anyTimes(); - - DateTime now = DateTimes.nowUtc(); - DateTime maxi = now.plusMinutes(60); - Task id1 = createKinesisIndexTask( - "id1", - DATASOURCE, - 0, - new SeekableStreamPartitions<>("stream", ImmutableMap.of( - shardId1, - "0", - shardId0, - "0" - )), - new SeekableStreamPartitions<>("stream", ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - now, - maxi - ); - - List existingTasks = ImmutableList.of(id1); - - Capture captured = Capture.newInstance(); + Capture captured = Capture.newInstance(); 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(); @@ -1143,7 +834,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception @Test public void testQueueNextTasksOnSuccess() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -1264,7 +955,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -1397,7 +1088,7 @@ public void testDiscoverExistingPublishingTask() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -1548,7 +1239,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -1689,7 +1380,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -1859,7 +1550,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception @Test public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -1938,7 +1629,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -2043,7 +1734,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234, -1); - supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + supervisor = getTestableSupervisor(2, 2, true, "PT1M", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -2159,7 +1850,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception @Test(expected = IllegalStateException.class) public void testStopNotStarted() { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.stop(false); } @@ -2174,7 +1865,7 @@ public void testStop() taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.start(); supervisor.stop(false); @@ -2188,7 +1879,7 @@ public void testStopGracefully() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -2357,7 +2048,7 @@ public void testResetNoTasks() throws Exception taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); supervisor.start(); @@ -2377,7 +2068,7 @@ public void testResetNoTasks() throws Exception public void testResetDataSourceMetadata() throws Exception { expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); 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(); @@ -2445,7 +2136,7 @@ public void testResetDataSourceMetadata() throws Exception public void testResetNoDataSourceMetadata() throws Exception { expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); 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(); @@ -2481,7 +2172,7 @@ public void testResetRunningTasks() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); @@ -2639,7 +2330,7 @@ public void testResetRunningTasks() throws Exception public void testNoDataIngestionTasks() throws Exception { final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); //not adding any events Task id1 = createKinesisIndexTask( @@ -2778,7 +2469,7 @@ public void testNoDataIngestionTasks() throws Exception public void testCheckpointForInactiveTaskGroup() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { - supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); + supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events final Task id1; id1 = createKinesisIndexTask( @@ -2931,7 +2622,7 @@ public void testCheckpointForInactiveTaskGroup() public void testCheckpointForUnknownTaskGroup() throws InterruptedException { - supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); + supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null, false); supervisorRecordSupplier.assign(anyObject()); @@ -3056,7 +2747,7 @@ public void testCheckpointForUnknownTaskGroup() public void testCheckpointWithNullTaskGroupId() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { - supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); + supervisor = getTestableSupervisor(1, 3, true, "PT1S", null, null, false); //not adding any events final Task id1 = createKinesisIndexTask( "id1", @@ -3169,7 +2860,7 @@ public void testCheckpointWithNullTaskGroupId() @Test public void testSuspendedNoRunningTasks() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null, true); expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -3202,7 +2893,7 @@ public void testSuspendedRunningTasks() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final DateTime startTime = DateTimes.nowUtc(); - supervisor = getSupervisor(2, 1, true, "PT1H", null, null, true); + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null, true); supervisorRecordSupplier.assign(anyObject()); expectLastCall().anyTimes(); @@ -3365,7 +3056,7 @@ public void testResetSuspended() throws Exception taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null, true); supervisor.start(); supervisor.runInternal(); verifyAll(); @@ -3382,7 +3073,7 @@ public void testResetSuspended() throws Exception @Test public void testGetCurrentTotalStats() { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null, false); + supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null, false); supervisor.addTaskGroupToActivelyReadingTaskGroup( supervisor.getTaskGroupIdForPartition("0"), ImmutableMap.of("0", "0"), @@ -3423,68 +3114,552 @@ public void testGetCurrentTotalStats() Assert.assertEquals(ImmutableMap.of("task2", ImmutableMap.of("prop2", "val2")), stats.get("1")); } - private KinesisSupervisor getSupervisor( - int replicas, - int taskCount, - boolean useEarliestOffset, - String duration, - Period lateMessageRejectionPeriod, - Period earlyMessageRejectionPeriod - ) + @Test + public void testDoNotKillCompatibleTasks() + throws InterruptedException, EntryExistsException, ExecutionException, TimeoutException, JsonProcessingException { - return getSupervisor( - replicas, - taskCount, - useEarliestOffset, - duration, - lateMessageRejectionPeriod, - earlyMessageRejectionPeriod, + // This supervisor always returns true for isTaskCurrent -> it should not kill its tasks + int numReplicas = 2; + supervisor = getTestableSupervisorCustomIsTaskCurrent( + numReplicas, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), false, - null, - null + 42, + 1000, + true ); - } - private KinesisSupervisor getSupervisor( - int replicas, - int taskCount, - boolean useEarliestOffset, - String duration, - Period lateMessageRejectionPeriod, - Period earlyMessageRejectionPeriod, - boolean suspended, - Integer recordsPerFetch, - Integer fetchDelayMillis + supervisorRecordSupplier.assign(EasyMock.anyObject()); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(supervisorRecordSupplier.getPartitionIds(stream)) + .andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + EasyMock.expect(supervisorRecordSupplier.getAssignment()) + .andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .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(); - ) - { - KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( - stream, - "awsEndpoint", - null, - replicas, - taskCount, - new Period(duration), - new Period("P1D"), - new Period("PT30S"), - useEarliestOffset, - new Period("PT30M"), - lateMessageRejectionPeriod, - earlyMessageRejectionPeriod, - recordsPerFetch, - fetchDelayMillis, - null, + Task task = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId0, + "0", + shardId1, + "0" + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId0, + "1", + shardId1, + "12" + )), null, - false + null ); - KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( - null, - null - ) - { - @Override - public KinesisIndexTaskClient build( + List existingTasks = ImmutableList.of(task); + + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(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(); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId0, + "0", + shardId1, + "0" + )); + + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(numReplicas); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillIncompatibleTasks() + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, EntryExistsException + { + // This supervisor always returns false for isTaskCurrent -> it should kill its tasks + int numReplicas = 2; + supervisor = getTestableSupervisorCustomIsTaskCurrent( + numReplicas, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), + false, + 42, + 1000, + false + ); + supervisorRecordSupplier.assign(EasyMock.anyObject()); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(supervisorRecordSupplier.getPartitionIds(stream)) + .andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + EasyMock.expect(supervisorRecordSupplier.getAssignment()) + .andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .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(); + + Task task = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId0, + "0", + shardId1, + "0" + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId0, + "1", + shardId1, + "12" + )), + null, + null + ); + + List existingTasks = ImmutableList.of(task); + + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(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(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true).times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testIsTaskCurrent() + { + DateTime minMessageTime = DateTimes.nowUtc(); + DateTime maxMessageTime = DateTimes.nowUtc().plus(10000); + + KinesisSupervisor supervisor = getSupervisor( + 1, + 1, + true, + "PT1H", + new Period("P1D"), + new Period("P1D"), + false, + 42, + 42, + dataSchema, + tuningConfig + ); + + supervisor.addTaskGroupToActivelyReadingTaskGroup( + 42, + ImmutableMap.of(shardId1, "3"), + Optional.of(minMessageTime), + Optional.of(maxMessageTime), + ImmutableSet.of("id1", "id2", "id3", "id4"), + ImmutableSet.of() + ); + + DataSchema modifiedDataSchema = getDataSchema("some other datasource"); + + KinesisSupervisorTuningConfig modifiedTuningConfig = new KinesisSupervisorTuningConfig( + 1000, + null, + 50000, + null, + new Period("P1Y"), + new File("/test"), + null, + null, + true, + false, + null, + null, + null, + null, + numThreads, + TEST_CHAT_THREADS, + TEST_CHAT_RETRIES, + TEST_HTTP_TIMEOUT, + TEST_SHUTDOWN_TIMEOUT, + null, + null, + null, + 5000, + null, + null, + null, + null, + 42, // This property is different from tuningConfig + null + ); + + KinesisIndexTask taskFromStorage = createKinesisIndexTask( + "id1", + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "3" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + minMessageTime, + maxMessageTime, + dataSchema + ); + + KinesisIndexTask taskFromStorageMismatchedDataSchema = createKinesisIndexTask( + "id2", + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "3" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + minMessageTime, + maxMessageTime, + modifiedDataSchema + ); + + KinesisIndexTask taskFromStorageMismatchedTuningConfig = createKinesisIndexTask( + "id3", + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "3" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + minMessageTime, + maxMessageTime, + dataSchema, + modifiedTuningConfig + ); + + KinesisIndexTask taskFromStorageMismatchedPartitionsWithTaskGroup = createKinesisIndexTask( + "id4", + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "4" // this is the mismatch + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + minMessageTime, + maxMessageTime, + dataSchema + ); + + EasyMock.expect(taskStorage.getTask("id1")) + .andReturn(Optional.of(taskFromStorage)) + .once(); + EasyMock.expect(taskStorage.getTask("id2")) + .andReturn(Optional.of(taskFromStorageMismatchedDataSchema)) + .once(); + EasyMock.expect(taskStorage.getTask("id3")) + .andReturn(Optional.of(taskFromStorageMismatchedTuningConfig)) + .once(); + EasyMock.expect(taskStorage.getTask("id4")) + .andReturn(Optional.of(taskFromStorageMismatchedPartitionsWithTaskGroup)) + .once(); + + replayAll(); + + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4")); + verifyAll(); + } + + private KinesisSupervisor getTestableSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + "awsEndpoint", + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + null, + null, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + KinesisSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + null + ), + rowIngestionMetersFactory + ); + } + + private KinesisSupervisor getTestableSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod + ) + { + return getTestableSupervisor( + replicas, + taskCount, + useEarliestOffset, + duration, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + false, + null, + null + ); + } + + private KinesisSupervisor getTestableSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + Integer recordsPerFetch, + Integer fetchDelayMillis + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + "awsEndpoint", + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + recordsPerFetch, + fetchDelayMillis, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + KinesisSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + null + ), + rowIngestionMetersFactory + ); + } + + /** + * Use when you want to mock the return value of SeekableStreamSupervisor#isTaskCurrent() + */ + private KinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + Integer recordsPerFetch, + Integer fetchDelayMillis, + boolean isTaskCurrentReturn + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + "awsEndpoint", + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + recordsPerFetch, + fetchDelayMillis, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( TaskInfoProvider taskInfoProvider, String dataSource, int numThreads, @@ -3499,7 +3674,7 @@ public KinesisIndexTaskClient build( } }; - return new TestableKinesisSupervisor( + return new TestableKinesisSupervisorWithCustomIsTaskCurrent( taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -3521,7 +3696,93 @@ public KinesisIndexTaskClient build( rowIngestionMetersFactory, null ), - rowIngestionMetersFactory + rowIngestionMetersFactory, + isTaskCurrentReturn + ); + } + + /** + * Use for tests where you don't want generateSequenceName to be overridden out + */ + private KinesisSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + Integer recordsPerFetch, + Integer fetchDelayMillis, + DataSchema dataSchema, + KinesisSupervisorTuningConfig tuningConfig + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + "awsEndpoint", + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + recordsPerFetch, + fetchDelayMillis, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new KinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + KinesisSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + null + ), + rowIngestionMetersFactory, + null ); } @@ -3560,46 +3821,65 @@ private static DataSchema getDataSchema(String dataSource) ); } - private static List jb( - String timestamp, - String dim1, - String dim2, - String dimLong, - String dimFloat, - String met1 + + private KinesisIndexTask createKinesisIndexTask( + String id, + String dataSource, + int taskGroupId, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, + DateTime minimumMessageTime, + DateTime maximumMessageTime ) { - try { - return Collections.singletonList(new ObjectMapper().writeValueAsBytes( - ImmutableMap.builder() - .put("timestamp", timestamp) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - )); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + return createKinesisIndexTask( + id, + taskGroupId, + startPartitions, + endPartitions, + minimumMessageTime, + maximumMessageTime, + getDataSchema(dataSource) + ); } private KinesisIndexTask createKinesisIndexTask( String id, - String dataSource, int taskGroupId, SeekableStreamPartitions startPartitions, SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, - DateTime maximumMessageTime + DateTime maximumMessageTime, + DataSchema dataSchema + ) + { + return createKinesisIndexTask( + id, + taskGroupId, + startPartitions, + endPartitions, + minimumMessageTime, + maximumMessageTime, + dataSchema, + (KinesisIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig() + ); + } + + private KinesisIndexTask createKinesisIndexTask( + String id, + int taskGroupId, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + DataSchema dataSchema, + KinesisIndexTaskTuningConfig tuningConfig ) { return new KinesisIndexTask( id, null, - getDataSchema(dataSource), + dataSchema, tuningConfig, new KinesisIndexTaskIOConfig( null, @@ -3625,83 +3905,6 @@ private KinesisIndexTask createKinesisIndexTask( ); } - private KinesisSupervisor getSupervisor( - int replicas, - int taskCount, - boolean useEarliestOffset, - String duration, - Period lateMessageRejectionPeriod, - Period earlyMessageRejectionPeriod, - boolean suspended - ) - { - KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( - stream, - "awsEndpoint", - null, - replicas, - taskCount, - new Period(duration), - new Period("P1D"), - new Period("PT30S"), - useEarliestOffset, - new Period("PT30M"), - lateMessageRejectionPeriod, - earlyMessageRejectionPeriod, - null, - null, - null, - null, - false - ); - - KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( - null, - null - ) - { - @Override - public KinesisIndexTaskClient build( - TaskInfoProvider taskInfoProvider, - String dataSource, - int numThreads, - Duration httpTimeout, - long numRetries - ) - { - Assert.assertEquals(TEST_CHAT_THREADS, numThreads); - Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); - Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); - return taskClient; - } - }; - - return new TestableKinesisSupervisor( - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - taskClientFactory, - objectMapper, - new KinesisSupervisorSpec( - dataSchema, - tuningConfig, - KinesisSupervisorIOConfig, - null, - suspended, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - taskClientFactory, - objectMapper, - new NoopServiceEmitter(), - new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory, - null - ), - rowIngestionMetersFactory - ); - } - private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem { private final String taskType; @@ -3767,7 +3970,9 @@ public TestableKinesisSupervisor( protected String generateSequenceName( Map startPartitions, Optional minimumMessageTime, - Optional maximumMessageTime + Optional maximumMessageTime, + DataSchema dataSchema, + SeekableStreamIndexTaskTuningConfig tuningConfig ) { final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); @@ -3779,7 +3984,39 @@ protected RecordSupplier setupRecordSupplier() { return supervisorRecordSupplier; } + } + + private class TestableKinesisSupervisorWithCustomIsTaskCurrent extends TestableKinesisSupervisor + { + private boolean isTaskCurrentReturn; + public TestableKinesisSupervisorWithCustomIsTaskCurrent( + TaskStorage taskStorage, + TaskMaster taskMaster, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + KinesisIndexTaskClientFactory taskClientFactory, + ObjectMapper mapper, + KinesisSupervisorSpec spec, + RowIngestionMetersFactory rowIngestionMetersFactory, + boolean isTaskCurrentReturn + ) + { + super( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory + ); + this.isTaskCurrentReturn = isTaskCurrentReturn; + } + @Override + public boolean isTaskCurrent(int taskGroupId, String taskId) + { + return isTaskCurrentReturn; + } } -} +} \ No newline at end of file diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java new file mode 100644 index 000000000000..d331ffb66540 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.test; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; + +public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTuningConfig +{ + private final String extra; + + @JsonCreator + public TestModifiedKinesisIndexTaskTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, + @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, + @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, + @JsonProperty("fetchSequenceNumberTimeout") Integer fetchSequenceNumberTimeout, + @JsonProperty("fetchThreads") Integer fetchThreads, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("extra") String extra + ) + { + super( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + skipSequenceNumberAvailabilityCheck, + recordBufferSize, + recordBufferOfferTimeout, + recordBufferFullWait, + fetchSequenceNumberTimeout, + fetchThreads, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxRecordsPerPoll, + intermediateHandoffPeriod + ); + this.extra = extra; + } + + public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig base, String extra) + { + super( + base.getMaxRowsInMemory(), + base.getMaxBytesInMemory(), + base.getMaxRowsPerSegment(), + base.getMaxTotalRows(), + base.getIntermediatePersistPeriod(), + base.getBasePersistDirectory(), + base.getMaxPendingPersists(), + base.getIndexSpec(), + base.getBuildV9Directly(), + base.isReportParseExceptions(), + base.getHandoffConditionTimeout(), + base.isResetOffsetAutomatically(), + base.isSkipSequenceNumberAvailabilityCheck(), + base.getRecordBufferSize(), + base.getRecordBufferOfferTimeout(), + base.getRecordBufferFullWait(), + base.getFetchSequenceNumberTimeout(), + base.getFetchThreads(), + base.getSegmentWriteOutMediumFactory(), + base.isLogParseExceptions(), + base.getMaxParseExceptions(), + base.getMaxSavedParseExceptions(), + base.getMaxRecordsPerPoll(), + base.getIntermediateHandoffPeriod() + ); + this.extra = extra; + } + + @JsonProperty("extra") + public String getExtra() + { + return extra; + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 4c6509d31f82..98978236eb0b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -74,6 +74,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.segment.indexing.DataSchema; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -176,7 +177,13 @@ private class TaskGroup this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null ? exclusiveStartSequenceNumberPartitions : new HashSet<>(); - this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); + this.baseSequenceName = generateSequenceName( + startingSequences, + minimumMessageTime, + maximumMessageTime, + spec.getDataSchema(), + taskTuningConfig + ); } int addNewCheckpoint(Map checkpoint) @@ -1629,7 +1636,8 @@ public Void apply(@Nullable Boolean result) ); } - private boolean isTaskCurrent(int taskGroupId, String taskId) + @VisibleForTesting + public boolean isTaskCurrent(int taskGroupId, String taskId) { Optional taskOptional = taskStorage.getTask(taskId); if (!taskOptional.isPresent() || !doesTaskTypeMatchSupervisor(taskOptional.get())) { @@ -1640,19 +1648,31 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional .get(); - String taskSequenceName = task.getIOConfig().getBaseSequenceName(); + String taskSequenceName = generateSequenceName( + task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap(), + task.getIOConfig().getMinimumMessageTime(), + task.getIOConfig().getMaximumMessageTime(), + task.getDataSchema(), + task.getTuningConfig() + ); if (activelyReadingTaskGroups.get(taskGroupId) != null) { - return Preconditions - .checkNotNull(activelyReadingTaskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) - .baseSequenceName - .equals(taskSequenceName); + TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); + return generateSequenceName( + taskGroup.startingSequences, + taskGroup.minimumMessageTime, + taskGroup.maximumMessageTime, + spec.getDataSchema(), + taskTuningConfig + ).equals(taskSequenceName); } else { return generateSequenceName( task.getIOConfig() .getStartPartitions() .getPartitionSequenceNumberMap(), task.getIOConfig().getMinimumMessageTime(), - task.getIOConfig().getMaximumMessageTime() + task.getIOConfig().getMaximumMessageTime(), + spec.getDataSchema(), + taskTuningConfig ).equals(taskSequenceName); } } @@ -1661,7 +1681,9 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) protected String generateSequenceName( Map startPartitions, Optional minimumMessageTime, - Optional maximumMessageTime + Optional maximumMessageTime, + DataSchema dataSchema, + SeekableStreamIndexTaskTuningConfig tuningConfig ) { StringBuilder sb = new StringBuilder(); @@ -1674,17 +1696,17 @@ protected String generateSequenceName( String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); - String dataSchema, tuningConfig; + String dataSchemaStr, tuningConfigStr; try { - dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); - tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); + dataSchemaStr = sortingMapper.writeValueAsString(dataSchema); + tuningConfigStr = sortingMapper.writeValueAsString(tuningConfig); } catch (JsonProcessingException e) { throw Throwables.propagate(e); } - String hashCode = DigestUtils.sha1Hex(dataSchema - + tuningConfig + String hashCode = DigestUtils.sha1Hex(dataSchemaStr + + tuningConfigStr + partitionOffsetStr + minMsgTimeStr + maxMsgTimeStr) @@ -2690,7 +2712,7 @@ protected abstract List parser = jsonMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("time", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimB", "dimA")), null, null), + null, + null + ), + null + ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + DataSchema originalSchema = new DataSchema( + "test", + parser, + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + jsonMapper + ); + + String serialized = jsonMapper.writeValueAsString(originalSchema); + TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class); + + Assert.assertEquals(null, deserialized.getExtra()); + Assert.assertEquals(originalSchema.getDataSource(), deserialized.getDataSource()); + Assert.assertEquals(originalSchema.getGranularitySpec(), deserialized.getGranularitySpec()); + Assert.assertEquals(originalSchema.getParser().getParseSpec(), deserialized.getParser().getParseSpec()); + Assert.assertArrayEquals(originalSchema.getAggregators(), deserialized.getAggregators()); + Assert.assertEquals(originalSchema.getTransformSpec(), deserialized.getTransformSpec()); + Assert.assertEquals(originalSchema.getParserMap(), deserialized.getParserMap()); + } + + @Test + public void testSerdeWithUpdatedDataSchemaRemovedField() throws IOException + { + Map parser = jsonMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("time", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimB", "dimA")), null, null), + null, + null + ), + null + ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + TestModifiedDataSchema originalSchema = new TestModifiedDataSchema( + "test", + parser, + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + jsonMapper, + "some arbitrary string" + ); + + String serialized = jsonMapper.writeValueAsString(originalSchema); + DataSchema deserialized = jsonMapper.readValue(serialized, DataSchema.class); + + Assert.assertEquals(originalSchema.getDataSource(), deserialized.getDataSource()); + Assert.assertEquals(originalSchema.getGranularitySpec(), deserialized.getGranularitySpec()); + Assert.assertEquals(originalSchema.getParser().getParseSpec(), deserialized.getParser().getParseSpec()); + Assert.assertArrayEquals(originalSchema.getAggregators(), deserialized.getAggregators()); + Assert.assertEquals(originalSchema.getTransformSpec(), deserialized.getTransformSpec()); + Assert.assertEquals(originalSchema.getParserMap(), deserialized.getParserMap()); + } } diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java new file mode 100644 index 000000000000..37e214554b11 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.indexing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.transform.TransformSpec; + +import java.util.Map; + +public class TestModifiedDataSchema extends DataSchema +{ + private final String extra; + + @JsonCreator + public TestModifiedDataSchema( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("parser") Map parser, + @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, + @JsonProperty("granularitySpec") GranularitySpec granularitySpec, + @JsonProperty("transformSpec") TransformSpec transformSpec, + @JacksonInject ObjectMapper jsonMapper, + @JsonProperty("extra") String extra + ) + { + super(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); + this.extra = extra; + } + + public TestModifiedDataSchema( + DataSchema base, + String extra + ) + { + super( + base.getDataSource(), + base.getParserMap(), + base.getAggregators(), + base.getGranularitySpec(), + base.getTransformSpec(), + new DefaultObjectMapper() + ); + this.extra = extra; + } + + @JsonProperty("extra") + public String getExtra() + { + return extra; + } +} \ No newline at end of file From 44338fd9607627f2dae3202dfe07b05c4f477ad3 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Mar 2019 18:18:49 -0800 Subject: [PATCH 02/17] Fixed checkstyle stuff --- .../kafka/KafkaIndexTaskTuningConfigTest.java | 1 - .../kafka/supervisor/KafkaSupervisorTest.java | 3 --- ...estModifiedKafkaIndexTaskTuningConfig.java | 26 +------------------ .../KinesisIndexTaskTuningConfigTest.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 3 +-- ...tModifiedKinesisIndexTaskTuningConfig.java | 2 +- .../segment/indexing/DataSchemaTest.java | 1 - .../indexing/TestModifiedDataSchema.java | 19 +------------- 8 files changed, 5 insertions(+), 52 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index aecd734ccc4d..63b0e985e0b7 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -241,7 +241,6 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); } - private static KafkaIndexTaskTuningConfig copy(KafkaIndexTaskTuningConfig config) { return new KafkaIndexTaskTuningConfig( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 0046adf9f725..5386c235ee9f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -42,7 +42,6 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; import org.apache.druid.indexing.kafka.KafkaIndexTask; @@ -74,9 +73,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java index eb8bee9246d9..f32f34ffa7c2 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java @@ -78,33 +78,9 @@ public TestModifiedKafkaIndexTaskTuningConfig( this.extra = extra; } - public TestModifiedKafkaIndexTaskTuningConfig(KafkaIndexTaskTuningConfig base, String extra) - { - super( - base.getMaxRowsInMemory(), - base.getMaxBytesInMemory(), - base.getMaxRowsPerSegment(), - base.getMaxTotalRows(), - base.getIntermediatePersistPeriod(), - base.getBasePersistDirectory(), - base.getMaxPendingPersists(), - base.getIndexSpec(), - base.getBuildV9Directly(), - base.isReportParseExceptions(), - base.getHandoffConditionTimeout(), - base.isResetOffsetAutomatically(), - base.getSegmentWriteOutMediumFactory(), - base.getIntermediateHandoffPeriod(), - base.isLogParseExceptions(), - base.getMaxParseExceptions(), - base.getMaxSavedParseExceptions() - ); - this.extra = extra; - } - @JsonProperty("extra") public String getExtra() { return extra; } -} \ No newline at end of file +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 71d4b42e285f..57ffebcb676e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -191,7 +191,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException } @Test - public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException, IOException + public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException { KinesisIndexTaskTuningConfig base = new KinesisIndexTaskTuningConfig( 1, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 552cfa4a2816..f09af3ee3bd1 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -4019,4 +4018,4 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) return isTaskCurrentReturn; } } -} \ No newline at end of file +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index d331ffb66540..aac8541a20e7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -127,4 +127,4 @@ public String getExtra() { return extra; } -} \ No newline at end of file +} diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 96cfa097973d..58a2c4bd39e8 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.indexing; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java index 37e214554b11..ca030fe875c5 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.transform.TransformSpec; @@ -49,25 +48,9 @@ public TestModifiedDataSchema( this.extra = extra; } - public TestModifiedDataSchema( - DataSchema base, - String extra - ) - { - super( - base.getDataSource(), - base.getParserMap(), - base.getAggregators(), - base.getGranularitySpec(), - base.getTransformSpec(), - new DefaultObjectMapper() - ); - this.extra = extra; - } - @JsonProperty("extra") public String getExtra() { return extra; } -} \ No newline at end of file +} From 8ec514bc830de3ecbb901b882c5990e049b3c6c7 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 8 Mar 2019 14:13:47 -0800 Subject: [PATCH 03/17] Fixed failing tests --- .../kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java | 2 ++ .../kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java index f32f34ffa7c2..3cc124f6afd2 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -29,6 +30,7 @@ import javax.annotation.Nullable; import java.io.File; +@JsonTypeName("KafkaTuningConfig") public class TestModifiedKafkaIndexTaskTuningConfig extends KafkaIndexTaskTuningConfig { private final String extra; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index aac8541a20e7..2485e977d60f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -29,6 +30,7 @@ import javax.annotation.Nullable; import java.io.File; +@JsonTypeName("KinesisTuningConfig") public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTuningConfig { private final String extra; From 88fe807d2b2f8d45404b0920f52d97d400533eb7 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Mar 2019 11:14:17 -0700 Subject: [PATCH 04/17] Make TestableKafkaSupervisorWithCustomIsTaskCurrent static --- .../druid/indexing/kafka/supervisor/KafkaSupervisorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 5386c235ee9f..a612bd783f40 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -3217,7 +3217,7 @@ protected String generateSequenceName( } } - private class TestableKafkaSupervisorWithCustomIsTaskCurrent extends TestableKafkaSupervisor + private static class TestableKafkaSupervisorWithCustomIsTaskCurrent extends TestableKafkaSupervisor { private boolean isTaskCurrentReturn; From 7febbef75bb6aa6a52a9c2163b446b6f7e32148d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 15 Mar 2019 14:11:14 -0700 Subject: [PATCH 05/17] Add doc --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 98978236eb0b..9620fa94b20b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1648,6 +1648,10 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional .get(); + // We recompute the sequence name hash for the supervisor's own configuration and compare this to the hash created + // by rehashing the task's sequence name using the most up-to-date class definitions of tuning config and + // data schema. Recomputing both ensures that forwards-compatible tasks won't be killed (which would occur + // if the hash generated using the old class definitions was used). String taskSequenceName = generateSequenceName( task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap(), task.getIOConfig().getMinimumMessageTime(), From c2efb2584df8c807fb247c4caa557436da633b5e Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 15 Mar 2019 16:09:29 -0700 Subject: [PATCH 06/17] baseSequenceName change --- .../supervisor/SeekableStreamSupervisor.java | 41 +++++++++++++++---- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 9620fa94b20b..1a90d054095f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -168,6 +168,31 @@ private class TaskGroup Optional maximumMessageTime, Set exclusiveStartSequenceNumberPartitions ) + { + this( + groupId, + startingSequences, + minimumMessageTime, + maximumMessageTime, + exclusiveStartSequenceNumberPartitions, + generateSequenceName( + startingSequences, + minimumMessageTime, + maximumMessageTime, + spec.getDataSchema(), + taskTuningConfig + ) + ); + } + + TaskGroup( + int groupId, + ImmutableMap startingSequences, + Optional minimumMessageTime, + Optional maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + String baseSequenceName + ) { this.groupId = groupId; this.startingSequences = startingSequences; @@ -177,13 +202,7 @@ private class TaskGroup this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null ? exclusiveStartSequenceNumberPartitions : new HashSet<>(); - this.baseSequenceName = generateSequenceName( - startingSequences, - minimumMessageTime, - maximumMessageTime, - spec.getDataSchema(), - taskTuningConfig - ); + this.baseSequenceName = baseSequenceName; } int addNewCheckpoint(Map checkpoint) @@ -1661,6 +1680,14 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) ); if (activelyReadingTaskGroups.get(taskGroupId) != null) { TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); + activelyReadingTaskGroups.put(taskGroupId, new TaskGroup( + taskGroup.groupId, + taskGroup.startingSequences, + taskGroup.minimumMessageTime, + taskGroup.maximumMessageTime, + taskGroup.exclusiveStartSequenceNumberPartitions, + task.getIOConfig().getBaseSequenceName() + )); return generateSequenceName( taskGroup.startingSequences, taskGroup.minimumMessageTime, From 076d7da9f3ca5b9b568243dd9c788b2adc08b71f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 13:03:56 -0700 Subject: [PATCH 07/17] Added comment --- .../supervisor/SeekableStreamSupervisor.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 1a90d054095f..54f300a66f85 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1664,8 +1664,8 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) } @SuppressWarnings("unchecked") - SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional - .get(); + SeekableStreamIndexTask task = + (SeekableStreamIndexTask) taskOptional.get(); // We recompute the sequence name hash for the supervisor's own configuration and compare this to the hash created // by rehashing the task's sequence name using the most up-to-date class definitions of tuning config and @@ -1678,8 +1678,11 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) task.getDataSchema(), task.getTuningConfig() ); + if (activelyReadingTaskGroups.get(taskGroupId) != null) { TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); + // We reassign the task's original base sequence name (from the existing task) to the task group so that the + // replica segment allocations are the same. activelyReadingTaskGroups.put(taskGroupId, new TaskGroup( taskGroup.groupId, taskGroup.startingSequences, From 684ba8062d0845b77e4d7c723238a3a9639c72a9 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 17:45:52 -0700 Subject: [PATCH 08/17] WIP --- .../supervisor/SeekableStreamSupervisor.java | 63 ++++++++++++------- 1 file changed, 42 insertions(+), 21 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 54f300a66f85..108c41497707 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -83,6 +83,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -453,6 +454,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // Map<{group RandomIdUtils}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); + // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. @@ -1364,6 +1366,8 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) taskGroupId, k -> { log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); + // We reassign the task's original base sequence name (from the existing task) to the + // task group so that the replica segment allocations are the same. return new TaskGroup( taskGroupId, ImmutableMap.copyOf( @@ -1373,7 +1377,8 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) ), seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), - seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions() + seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions(), + seekableStreamIndexTask.getIOConfig().getBaseSequenceName() ); } ); @@ -1386,6 +1391,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) taskId ); } + verifySameSequenceNameForAllTasksInGroup(taskGroupId); } } return true; @@ -1403,7 +1409,6 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) } } - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int i = 0; i < results.size(); i++) { if (results.get(i) == null) { @@ -1448,10 +1453,11 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final List taskIds = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( - taskId, - true - ); + final ListenableFuture>> checkpointsFuture = + taskClient.getCheckpointsAsync( + taskId, + true + ); futures.add(checkpointsFuture); taskIds.add(taskId); } @@ -1636,6 +1642,33 @@ private void addDiscoveredTaskToPendingCompletionTaskGroups( taskGroupList.add(newTaskGroup); } + private void verifySameSequenceNameForAllTasksInGroup(int groupId) + { + String taskGroupSequenceName = activelyReadingTaskGroups.get(groupId).baseSequenceName; + boolean allSequenceNamesMatch = + activelyReadingTaskGroups.get(groupId) + .tasks + .keySet() + .stream() + .map(x -> { + Optional taskOptional = taskStorage.getTask(x); + if (!taskOptional.isPresent() || !doesTaskTypeMatchSupervisor(taskOptional.get())) { + return false; + } + @SuppressWarnings("unchecked") + SeekableStreamIndexTask task = + (SeekableStreamIndexTask) taskOptional.get(); + return task.getIOConfig().getBaseSequenceName(); + }) + .allMatch(taskSeqName -> taskSeqName == taskGroupSequenceName); + if (!allSequenceNamesMatch) { + throw new ISE( + "Base sequence names do not match for the tasks in the task group with ID [%s]", + groupId + ); + } + } + private ListenableFuture stopTask(final String id, final boolean publish) { return Futures.transform( @@ -1681,16 +1714,6 @@ public boolean isTaskCurrent(int taskGroupId, String taskId) if (activelyReadingTaskGroups.get(taskGroupId) != null) { TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); - // We reassign the task's original base sequence name (from the existing task) to the task group so that the - // replica segment allocations are the same. - activelyReadingTaskGroups.put(taskGroupId, new TaskGroup( - taskGroup.groupId, - taskGroup.startingSequences, - taskGroup.minimumMessageTime, - taskGroup.maximumMessageTime, - taskGroup.exclusiveStartSequenceNumberPartitions, - task.getIOConfig().getBaseSequenceName() - )); return generateSequenceName( taskGroup.startingSequences, taskGroup.minimumMessageTime, @@ -2336,7 +2359,6 @@ private void createNewTasks() exclusiveStartSequenceNumberPartitions ) ); - } } @@ -2517,10 +2539,10 @@ private void createTasksForGroup(int groupId, int replicas) for (PartitionIdType partition : startPartitions.keySet()) { endPartitions.put(partition, getEndOfPartitionMarker()); } - Set exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + Set exclusiveStartSequenceNumberPartitions = group.exclusiveStartSequenceNumberPartitions; - DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull(); - DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull(); + DateTime minimumMessageTime = group.minimumMessageTime.orNull(); + DateTime maximumMessageTime = group.maximumMessageTime.orNull(); SeekableStreamIndexTaskIOConfig newIoConfig = createTaskIoConfig( groupId, @@ -2533,7 +2555,6 @@ private void createTasksForGroup(int groupId, int replicas) ioConfig ); - List> taskList = createIndexTasks( replicas, group.baseSequenceName, From 3b88163625bb1c4f825cc87db0d7a8d67c149b74 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 18:13:45 -0700 Subject: [PATCH 09/17] Fixed imports --- .../supervisor/SeekableStreamSupervisor.java | 200 +++++++++--------- 1 file changed, 97 insertions(+), 103 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 108c41497707..e6def57f4b00 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -83,7 +83,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -1285,73 +1284,54 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti futureTaskIds.add(taskId); futures.add( Futures.transform( - taskClient.getStatusAsync(taskId), new Function() - { - @Override - public Boolean apply(SeekableStreamIndexTaskRunner.Status status) - { - try { - log.debug("Task [%s], status [%s]", taskId, status); - if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) { - seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet() - .forEach( - partition -> addDiscoveredTaskToPendingCompletionTaskGroups( - getTaskGroupIdForPartition(partition), - taskId, - seekableStreamIndexTask.getIOConfig() + taskClient.getStatusAsync(taskId), + (Function) status -> { + try { + log.debug("Task [%s], status [%s]", taskId, status); + if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) { + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet() + .forEach( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition(partition), + taskId, + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + )); + + // update partitionGroups with the publishing task's sequences (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskEndOffsets = taskClient.getEndOffsets( + taskId); + + for (Entry entry : publishingTaskEndOffsets.entrySet()) { + PartitionIdType partition = entry.getKey(); + SequenceOffsetType sequence = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + getTaskGroupIdForPartition(partition) + ); + + boolean succeeded; + do { + succeeded = true; + SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); + if (previousOffset != null + && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(sequence))) < 0) { + succeeded = partitionOffsets.replace(partition, previousOffset, sequence); + } + } while (!succeeded); + } + } else { + for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getPartitionSequenceNumberMap() - )); - - // update partitionGroups with the publishing task's sequences (if they are greater than what is - // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets( - taskId); - - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - PartitionIdType partition = entry.getKey(); - SequenceOffsetType sequence = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( - getTaskGroupIdForPartition(partition) - ); - - boolean succeeded; - do { - succeeded = true; - SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); - if (previousOffset != null - && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(sequence))) < 0) { - succeeded = partitionOffsets.replace(partition, previousOffset, sequence); - } - } while (!succeeded); - } - } else { - for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn( - "Stopping task [%s] which does not match the expected partition allocation", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } - } - // make sure the task's io and tuning configs match with the supervisor config - // if it is current then only create corresponding taskGroup if it does not exist - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info( - "Stopping task [%s] which does not match the expected parameters and ingestion spec", + .keySet()) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", taskId ); try { @@ -1361,47 +1341,61 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) log.warn(e, "Exception while stopping task"); } return false; - } else { - final TaskGroup taskGroup = activelyReadingTaskGroups.computeIfAbsent( - taskGroupId, - k -> { - log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); - // We reassign the task's original base sequence name (from the existing task) to the - // task group so that the replica segment allocations are the same. - return new TaskGroup( - taskGroupId, - ImmutableMap.copyOf( - seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - ), - seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), - seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), - seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions(), - seekableStreamIndexTask.getIOConfig().getBaseSequenceName() - ); - } + } + } + // make sure the task's io and tuning configs match with the supervisor config + // if it is current then only create corresponding taskGroup if it does not exist + if (!isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } else { + final TaskGroup taskGroup1 = activelyReadingTaskGroups.computeIfAbsent( + taskGroupId, + k -> { + log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); + // We reassign the task's original base sequence name (from the existing task) to the + // task group so that the replica segment allocations are the same. + return new TaskGroup( + taskGroupId, + ImmutableMap.copyOf( + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + ), + seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), + seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), + seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions(), + seekableStreamIndexTask.getIOConfig().getBaseSequenceName() + ); + } + ); + taskGroupsToVerify.put(taskGroupId, taskGroup1); + final TaskData prevTaskData = taskGroup1.tasks.putIfAbsent(taskId, new TaskData()); + if (prevTaskData != null) { + throw new ISE( + "WTH? a taskGroup[%s] already exists for new task[%s]", + prevTaskData, + taskId ); - taskGroupsToVerify.put(taskGroupId, taskGroup); - final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); - if (prevTaskData != null) { - throw new ISE( - "WTH? a taskGroup[%s] already exists for new task[%s]", - prevTaskData, - taskId - ); - } - verifySameSequenceNameForAllTasksInGroup(taskGroupId); } + verifySameSequenceNameForAllTasksInGroup(taskGroupId); } - return true; - } - catch (Throwable t) { - log.error(t, "Something bad while discovering task [%s]", taskId); - return null; } + return true; + } + catch (Throwable t) { + log.error(t, "Something bad while discovering task [%s]", taskId); + return null; } - }, workerExec ) ); @@ -1660,7 +1654,7 @@ private void verifySameSequenceNameForAllTasksInGroup(int groupId) (SeekableStreamIndexTask) taskOptional.get(); return task.getIOConfig().getBaseSequenceName(); }) - .allMatch(taskSeqName -> taskSeqName == taskGroupSequenceName); + .allMatch(taskSeqName -> taskSeqName.equals(taskGroupSequenceName)); if (!allSequenceNamesMatch) { throw new ISE( "Base sequence names do not match for the tasks in the task group with ID [%s]", From 06a8b0bf999e4e665a22317ea1a304d398e3c9cc Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 18:15:46 -0700 Subject: [PATCH 10/17] Undid lambda change for diff sake --- .../supervisor/SeekableStreamSupervisor.java | 202 +++++++++--------- 1 file changed, 106 insertions(+), 96 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index e6def57f4b00..a8430d4b7c4f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1285,116 +1285,126 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti futures.add( Futures.transform( taskClient.getStatusAsync(taskId), - (Function) status -> { - try { - log.debug("Task [%s], status [%s]", taskId, status); - if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) { - seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet() - .forEach( - partition -> addDiscoveredTaskToPendingCompletionTaskGroups( - getTaskGroupIdForPartition(partition), - taskId, - seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - )); - - // update partitionGroups with the publishing task's sequences (if they are greater than what is - // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets( - taskId); - - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - PartitionIdType partition = entry.getKey(); - SequenceOffsetType sequence = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( - getTaskGroupIdForPartition(partition) - ); - - boolean succeeded; - do { - succeeded = true; - SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); - if (previousOffset != null - && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(sequence))) < 0) { - succeeded = partitionOffsets.replace(partition, previousOffset, sequence); - } - } while (!succeeded); - } - } else { - for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig() + new Function() + { + @Override + public Boolean apply(SeekableStreamIndexTaskRunner.Status status) + { + try { + log.debug("Task [%s], status [%s]", taskId, status); + if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) { + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet() + .forEach( + partition -> SeekableStreamSupervisor.this.addDiscoveredTaskToPendingCompletionTaskGroups( + SeekableStreamSupervisor.this.getTaskGroupIdForPartition( + partition), + taskId, + seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getPartitionSequenceNumberMap() - .keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn( - "Stopping task [%s] which does not match the expected partition allocation", + )); + + // update partitionGroups with the publishing task's sequences (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskEndOffsets = taskClient.getEndOffsets( + taskId); + + for (Entry entry : publishingTaskEndOffsets.entrySet()) { + PartitionIdType partition = entry.getKey(); + SequenceOffsetType sequence = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + SeekableStreamSupervisor.this.getTaskGroupIdForPartition(partition) + ); + + boolean succeeded; + do { + succeeded = true; + SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); + if (previousOffset != null + && (SeekableStreamSupervisor.this.makeSequenceNumber(previousOffset) + .compareTo(SeekableStreamSupervisor.this.makeSequenceNumber( + sequence))) < 0) { + succeeded = partitionOffsets.replace(partition, previousOffset, sequence); + } + } while (!succeeded); + } + } else { + for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet()) { + if (!taskGroupId.equals(SeekableStreamSupervisor.this.getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", + taskId + ); + try { + SeekableStreamSupervisor.this.stopTask(taskId, false) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } + } + // make sure the task's io and tuning configs match with the supervisor config + // if it is current then only create corresponding taskGroup if it does not exist + if (!SeekableStreamSupervisor.this.isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", taskId ); try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + SeekableStreamSupervisor.this.stopTask(taskId, false) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { log.warn(e, "Exception while stopping task"); } return false; - } - } - // make sure the task's io and tuning configs match with the supervisor config - // if it is current then only create corresponding taskGroup if it does not exist - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info( - "Stopping task [%s] which does not match the expected parameters and ingestion spec", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } else { - final TaskGroup taskGroup1 = activelyReadingTaskGroups.computeIfAbsent( - taskGroupId, - k -> { - log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); - // We reassign the task's original base sequence name (from the existing task) to the - // task group so that the replica segment allocations are the same. - return new TaskGroup( - taskGroupId, - ImmutableMap.copyOf( - seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - ), - seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), - seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), - seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions(), - seekableStreamIndexTask.getIOConfig().getBaseSequenceName() - ); - } - ); - taskGroupsToVerify.put(taskGroupId, taskGroup1); - final TaskData prevTaskData = taskGroup1.tasks.putIfAbsent(taskId, new TaskData()); - if (prevTaskData != null) { - throw new ISE( - "WTH? a taskGroup[%s] already exists for new task[%s]", - prevTaskData, - taskId + } else { + final TaskGroup taskGroup1 = activelyReadingTaskGroups.computeIfAbsent( + taskGroupId, + k -> { + log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); + // We reassign the task's original base sequence name (from the existing task) to the + // task group so that the replica segment allocations are the same. + return new TaskGroup( + taskGroupId, + ImmutableMap.copyOf( + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + ), + seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), + seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), + seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions(), + seekableStreamIndexTask.getIOConfig().getBaseSequenceName() + ); + } ); + taskGroupsToVerify.put(taskGroupId, taskGroup1); + final TaskData prevTaskData = taskGroup1.tasks.putIfAbsent(taskId, new TaskData()); + if (prevTaskData != null) { + throw new ISE( + "WTH? a taskGroup[%s] already exists for new task[%s]", + prevTaskData, + taskId + ); + } + SeekableStreamSupervisor.this.verifySameSequenceNameForAllTasksInGroup(taskGroupId); } - verifySameSequenceNameForAllTasksInGroup(taskGroupId); } + return true; + } + catch (Throwable t) { + log.error(t, "Something bad while discovering task [%s]", taskId); + return null; } - return true; - } - catch (Throwable t) { - log.error(t, "Something bad while discovering task [%s]", taskId); - return null; } }, workerExec ) From 3a77c8c3ff5d2ef11888c06cb193916bf72969b7 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 18:16:40 -0700 Subject: [PATCH 11/17] Cleanup --- .../supervisor/SeekableStreamSupervisor.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index a8430d4b7c4f..e978970298a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1298,8 +1298,8 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) .getPartitionSequenceNumberMap() .keySet() .forEach( - partition -> SeekableStreamSupervisor.this.addDiscoveredTaskToPendingCompletionTaskGroups( - SeekableStreamSupervisor.this.getTaskGroupIdForPartition( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition( partition), taskId, seekableStreamIndexTask.getIOConfig() @@ -1316,7 +1316,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) PartitionIdType partition = entry.getKey(); SequenceOffsetType sequence = entry.getValue(); ConcurrentHashMap partitionOffsets = partitionGroups.get( - SeekableStreamSupervisor.this.getTaskGroupIdForPartition(partition) + getTaskGroupIdForPartition(partition) ); boolean succeeded; @@ -1324,8 +1324,8 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) succeeded = true; SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); if (previousOffset != null - && (SeekableStreamSupervisor.this.makeSequenceNumber(previousOffset) - .compareTo(SeekableStreamSupervisor.this.makeSequenceNumber( + && (makeSequenceNumber(previousOffset) + .compareTo(makeSequenceNumber( sequence))) < 0) { succeeded = partitionOffsets.replace(partition, previousOffset, sequence); } @@ -1336,13 +1336,13 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) .getStartPartitions() .getPartitionSequenceNumberMap() .keySet()) { - if (!taskGroupId.equals(SeekableStreamSupervisor.this.getTaskGroupIdForPartition(partition))) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { log.warn( "Stopping task [%s] which does not match the expected partition allocation", taskId ); try { - SeekableStreamSupervisor.this.stopTask(taskId, false) + stopTask(taskId, false) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { @@ -1353,13 +1353,13 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) } // make sure the task's io and tuning configs match with the supervisor config // if it is current then only create corresponding taskGroup if it does not exist - if (!SeekableStreamSupervisor.this.isTaskCurrent(taskGroupId, taskId)) { + if (!isTaskCurrent(taskGroupId, taskId)) { log.info( "Stopping task [%s] which does not match the expected parameters and ingestion spec", taskId ); try { - SeekableStreamSupervisor.this.stopTask(taskId, false) + stopTask(taskId, false) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { @@ -1396,7 +1396,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) taskId ); } - SeekableStreamSupervisor.this.verifySameSequenceNameForAllTasksInGroup(taskGroupId); + verifySameSequenceNameForAllTasksInGroup(taskGroupId); } } return true; From 2cab52fb978c46514db67b27a9105a7e2a94f7f2 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 18:18:17 -0700 Subject: [PATCH 12/17] Added comment --- .../supervisor/SeekableStreamSupervisor.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index e978970298a3..bb2200248660 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1367,7 +1367,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) } return false; } else { - final TaskGroup taskGroup1 = activelyReadingTaskGroups.computeIfAbsent( + final TaskGroup taskGroup = activelyReadingTaskGroups.computeIfAbsent( taskGroupId, k -> { log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); @@ -1387,8 +1387,8 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) ); } ); - taskGroupsToVerify.put(taskGroupId, taskGroup1); - final TaskData prevTaskData = taskGroup1.tasks.putIfAbsent(taskId, new TaskData()); + taskGroupsToVerify.put(taskGroupId, taskGroup); + final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); if (prevTaskData != null) { throw new ISE( "WTH? a taskGroup[%s] already exists for new task[%s]", @@ -1645,7 +1645,7 @@ private void addDiscoveredTaskToPendingCompletionTaskGroups( taskGroupList.add(newTaskGroup); } - + // Sanity check to ensure that tasks have the same sequence name as their task group private void verifySameSequenceNameForAllTasksInGroup(int groupId) { String taskGroupSequenceName = activelyReadingTaskGroups.get(groupId).baseSequenceName; From fc1fcec90a51e8d8d37a294f84b1b73043630bb3 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 8 Apr 2019 11:37:47 -0700 Subject: [PATCH 13/17] Reinsert Kafka tests --- .../kafka/supervisor/KafkaSupervisorTest.java | 172 +++++++++++++++++- 1 file changed, 170 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 1e5a5c6e4262..a688ff5fd543 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; @@ -75,7 +76,9 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -139,7 +142,6 @@ public class KafkaSupervisorTest extends EasyMockSupport private static int topicPostfix; private static ZkUtils zkUtils; - private final int numThreads; private KafkaSupervisor supervisor; @@ -637,6 +639,172 @@ public void testBadMetadataOffsets() throws Exception supervisor.runInternal(); } + @Test + public void testKillIncompatibleTasksMismatchedDatasourceAndType() throws Exception + { + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); + addSomeEvents(1); + + // different datasource (don't kill) + Task id1 = createKafkaIndexTask( + "id1", + "other-datasource", + 2, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, 10L)), + null, + null + ); + + // non KafkaIndexTask (don't kill) + Task id2 = new RealtimeIndexTask( + "id2", + null, + new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ), + null + ); + + List existingTasks = ImmutableList.of(id1, id2); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + + expect(taskQueue.add(anyObject(Task.class))).andReturn(true).anyTimes(); + + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillBadPartitionAssignment() throws Exception + { + supervisor = getTestableSupervisor(1, 2, true, "PT1H", null, null); + addSomeEvents(1); + + Task id1 = createKafkaIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 2, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null, + null + ); + Task id2 = createKafkaIndexTask( + "id2", + DATASOURCE, + 1, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(1, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)), + null, + null + ); + Task id3 = createKafkaIndexTask( + "id3", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), + null, + null + ); + Task id4 = createKafkaIndexTask( + "id4", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), + null, + null + ); + Task id5 = createKafkaIndexTask( + "id5", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(1, 0L, 2, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null, + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(1); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskQueue.shutdown("id4", "Task [%s] failed to stop in a timely manner, killing task", "id4"); + taskQueue.shutdown("id5", "Task [%s] failed to stop in a timely manner, killing task", "id5"); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test public void testRequeueTaskWhenFailed() throws Exception { @@ -2727,7 +2895,7 @@ public void testDoNotKillCompatibleTasks() } @Test - public void testKillIncompatibleTasks() + public void testKillIncompatibleNonCurrentTasks() throws Exception { // This supervisor always returns false for isTaskCurrent -> it should kill its tasks From fbd28737649b74229fd20c13d72bed5fcf461709 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 8 Apr 2019 11:58:19 -0700 Subject: [PATCH 14/17] Readded kinesis test --- .../kafka/supervisor/KafkaSupervisorTest.java | 4 +- .../supervisor/KinesisSupervisorTest.java | 74 +++++++++++++++++++ 2 files changed, 76 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index a688ff5fd543..b6e7b3406771 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -640,7 +640,7 @@ public void testBadMetadataOffsets() throws Exception } @Test - public void testKillIncompatibleTasksMismatchedDatasourceAndType() throws Exception + public void testDontKillTasksWithMismatchedDatasourceAndType() throws Exception { supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); addSomeEvents(1); @@ -2895,7 +2895,7 @@ public void testDoNotKillCompatibleTasks() } @Test - public void testKillIncompatibleNonCurrentTasks() + public void testKillIncompatibleTasks() throws Exception { // This supervisor always returns false for isTaskCurrent -> it should kill its tasks diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 6d4874550f0c..a9a18befbb4a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; import org.apache.druid.indexing.kinesis.KinesisIndexTask; @@ -73,7 +74,9 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -589,6 +592,77 @@ public void testBadMetadataOffsets() throws Exception supervisor.runInternal(); } + @Test + public void testDontKillTasksWithMismatchedDatasourceAndType() throws Exception + { + supervisor = getTestableSupervisor(2, 1, true, "PT1H", null, null); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + // different datasource (don't kill) + Task id1 = createKinesisIndexTask( + "id1", + "other-datasource", + 2, + new SeekableStreamStartSequenceNumbers<>( + stream, + ImmutableMap.of(shardId0, "0", shardId1, "0"), + ImmutableSet.of(shardId0, shardId1) + ), + new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "12")), + null, + null + ); + + // non KinesisIndexTask (don't kill) + Task id2 = new RealtimeIndexTask( + "id2", + null, + new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ), + null + ); + + List existingTasks = ImmutableList.of(id1, id2); + + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(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(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true).times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + @Test public void testRequeueTaskWhenFailed() throws Exception { From d0e253acddcc1c3e1821a92a5d32adaa302bec2f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 9 Apr 2019 16:54:44 -0700 Subject: [PATCH 15/17] Readd bad partition assignment in kinesis supervisor test --- .../supervisor/KinesisSupervisorTest.java | 117 ++++++++++++++++++ 1 file changed, 117 insertions(+) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index a9a18befbb4a..5ba810eba025 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -663,6 +663,123 @@ public void testDontKillTasksWithMismatchedDatasourceAndType() throws Exception verifyAll(); } + @Test + public void testKillBadPartitionAssignment() throws Exception + { + supervisor = getTestableSupervisor( + 1, + 2, + true, + "PT1H", + null, + null + ); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "12")), + null, + null + ); + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 1, + new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of(shardId0)), + new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")), + null, + null + ); + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>( + stream, + ImmutableMap.of(shardId0, "0", shardId1, "0"), ImmutableSet.of(shardId0, shardId1) + ), + new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "12")), + null, + null + ); + Task id4 = createKinesisIndexTask( + "id4", + DATASOURCE, + 0, + new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of(shardId0)), + new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")), + null, + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4); + + 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.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(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(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(shardId1, "0")); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); + TreeMap> checkpoints4 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id4"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints4)) + .times(1); + + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + taskQueue.shutdown("id4", "Task [%s] failed to return status, killing task", "id4"); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + @Test public void testRequeueTaskWhenFailed() throws Exception { From 991fbea2217b5e8d53803c80faa969d7d8b489da Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 9 Apr 2019 18:08:13 -0700 Subject: [PATCH 16/17] Nit --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index c8265ec39e74..032ab7c5ccff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -452,7 +452,6 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // Map<{group RandomIdUtils}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); - // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. From 814fc5257decc77feba68c1177aa8149992e5804 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 10 Apr 2019 15:17:37 -0700 Subject: [PATCH 17/17] Misnamed var --- .../indexing/kinesis/supervisor/KinesisSupervisorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9ef2c8fe2e4a..cc5ae9ee0704 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -759,7 +759,7 @@ public void testKillBadPartitionAssignment() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); TreeMap> checkpoints4 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); + checkpoints4.put(0, ImmutableMap.of(shardId0, "0")); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(1);