You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/04/22 05:54:09 UTC

[GitHub] [druid] jihoonson commented on a change in pull request #9724: Add integration tests for kafka ingestion

jihoonson commented on a change in pull request #9724:
URL: https://github.com/apache/druid/pull/9724#discussion_r412655617



##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java
##########
@@ -25,5 +25,11 @@
 
   void shutdown();

Review comment:
       This method is not being called, but seems like it should be called in `AbstractStreamIndexingTest.doMethodTeardown()`.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java
##########
@@ -25,5 +25,11 @@
 
   void shutdown();
 
-  void flush();
+  void flush() throws Exception;

Review comment:
       Can you add javadoc for this class? 

##########
File path: integration-tests/README.md
##########
@@ -303,3 +314,13 @@ This will tell the test framework that the test class needs to be constructed us
 2) FromFileTestQueryHelper - reads queries with expected results from file and executes them and verifies the results using ResultVerifier
 
 Refer ITIndexerTest as an example on how to use dependency Injection
+
+### Running test methods in parallel
+By default, test methods in a test class will be run in sequential order one at a time. Test methods for a given test 
+class can be set to run in parallel (multiple test methods of the given class running at the same time) by excluding
+the given class/package from the "AllSerializedTests" test tag section and including it in the "AllParallelizedTests" 

Review comment:
       Does `AllParallelizedTests` parallelize tests in a single class? Or can it parallelize across classes? Probably worth mentioning here.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);
+  // Since this integration test can terminates or be killed un-expectedly, this tag is added to all streams created
+  // to help make stream clean up easier. (Normally, streams should be cleanup automattically by the teardown method)
+  // The value to this tag is a timestamp that can be used by a lambda function to remove unused stream.
+  private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after";
+  private static final int STREAM_SHARD_COUNT = 2;
+  private static final long WAIT_TIME_MILLIS = 3 * 60 * 1000L;
+  private static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json";
+  private static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/stream_supervisor_spec_input_format.json";
+  private static final String QUERIES_FILE = "/indexer/stream_index_queries.json";
+  private static final long CYCLE_PADDING_MS = 100;
+
+  @Inject
+  private DruidClusterAdminClient druidClusterAdminClient;
+
+  private StreamAdminClient streamAdminClient;
+  private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator;
+
+  abstract StreamAdminClient getStreamAdminClient() throws Exception;
+  abstract StreamEventWriter getStreamEventWriter() throws Exception;

Review comment:
       nit: probably better to name `createStreamAdminClient()` since it creates an instance rather than returning an existing one.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.testing.utils;
+
+import java.util.Map;
+
+public interface StreamAdminClient

Review comment:
       Can you add javadoc? It would be nice to explain where this class is used, how it is used, what is the contracts for implementations and so on.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.testing.utils;
+
+import java.util.Map;
+
+public interface StreamAdminClient
+{
+  void createStream(String streamName, int partitionCount, Map<String, String> tags) throws Exception;
+
+  void deleteStream(String streamName) throws Exception;
+
+  void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception;

Review comment:
       SeekableStream series use the word "partition" instead of "shard". It would be nice to use a consistent name.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.testing.utils;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+public class KafkaAdminClient implements StreamAdminClient
+{
+  AdminClient adminClient;

Review comment:
       nit: can be private final.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java
##########
@@ -251,6 +251,9 @@ private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffset
                                                                                  null);
     compactionResource.submitCompactionConfig(compactionConfig);
 
+    // Wait for compaction config to persist
+    Thread.sleep(2000);

Review comment:
       nit: it would be better to check the compaction config is set by calling the compaction config API.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.testing.utils;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+public class KafkaAdminClient implements StreamAdminClient
+{
+  AdminClient adminClient;
+
+  public KafkaAdminClient(String kafkaInternalHost)
+  {
+    Properties config = new Properties();
+    config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost);
+    adminClient = AdminClient.create(config);
+  }
+
+  @Override
+  public void createStream(String streamName, int partitionCount, Map<String, String> tags) throws Exception
+  {
+    final short replicationFactor = 1;
+    final NewTopic newTopic = new NewTopic(streamName, partitionCount, replicationFactor);
+    final CreateTopicsResult createTopicsResult = adminClient.createTopics(Collections.singleton(newTopic));
+    // Wait for create topic to compelte
+    createTopicsResult.values().get(streamName).get();
+  }
+
+  @Override
+  public void deleteStream(String streamName) throws Exception
+  {
+    DeleteTopicsResult deleteTopicsResult = adminClient.deleteTopics(ImmutableList.of(streamName));
+    deleteTopicsResult.values().get(streamName).get();
+  }
+
+  /**
+   * This method can only increase the partition count of {@param streamName} to have a final partition
+   * count of {@param newPartitionCount}
+   * If {@param blocksUntilStarted} is set to true, then this method will blocks until the partitioning
+   * started (but not nessesary finished), otherwise, the method will returns right after issue the reshard command
+   */
+  @Override
+  public void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception
+  {
+    Map<String, NewPartitions> counts = new HashMap<>();
+    counts.put(streamName, NewPartitions.increaseTo(newPartitionCount));
+    CreatePartitionsResult createPartitionsResult = adminClient.createPartitions(counts);
+    if (blocksUntilStarted) {
+      createPartitionsResult.values().get(streamName).get();
+
+    }
+  }
+
+  @Override
+  public boolean isStreamActive(String streamName)
+  {
+    return true;

Review comment:
       What does "active stream" mean for Kafka?

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java
##########
@@ -57,7 +57,7 @@
   protected TestQueryHelper queryHelper;
 
   @Inject
-  private IntegrationTestingConfig config;
+  public IntegrationTestingConfig config;

Review comment:
       Why not injecting it in Kafka/Kinesis tests?

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.tests.indexer;
+
+import org.apache.druid.indexing.kafka.KafkaConsumerConfigs;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testing.utils.KafkaAdminClient;
+import org.apache.druid.testing.utils.KafkaEventWriter;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Function;
+
+public abstract class AbstractKafkaIndexingServiceTest extends AbstractStreamIndexingTest
+{
+  public abstract boolean isKafkaWriterTransactionalEnabled();

Review comment:
       nit: can be protected.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.testing.utils;
+
+import org.apache.druid.indexer.TaskIdUtils;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Future;
+
+public class KafkaEventWriter implements StreamEventWriter
+{
+  private static final String TEST_PROPERTY_PREFIX = "kafka.test.property.";
+  private KafkaProducer<String, String> producer;
+  private boolean txnEnabled;
+  private List<Future<RecordMetadata>> pendingWriteRecords = new ArrayList<>();

Review comment:
       These 3 private variables can be final.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);

Review comment:
       nit: can be private.

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.testing.utils;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+public class KafkaAdminClient implements StreamAdminClient
+{
+  AdminClient adminClient;
+
+  public KafkaAdminClient(String kafkaInternalHost)
+  {
+    Properties config = new Properties();
+    config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost);

Review comment:
       nit: Intellij recommends using `setProperty()` instead of `put()`. I guess it's because the parameter type is more strict.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);
+  // Since this integration test can terminates or be killed un-expectedly, this tag is added to all streams created
+  // to help make stream clean up easier. (Normally, streams should be cleanup automattically by the teardown method)
+  // The value to this tag is a timestamp that can be used by a lambda function to remove unused stream.
+  private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after";
+  private static final int STREAM_SHARD_COUNT = 2;
+  private static final long WAIT_TIME_MILLIS = 3 * 60 * 1000L;
+  private static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json";
+  private static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/stream_supervisor_spec_input_format.json";
+  private static final String QUERIES_FILE = "/indexer/stream_index_queries.json";
+  private static final long CYCLE_PADDING_MS = 100;
+
+  @Inject
+  private DruidClusterAdminClient druidClusterAdminClient;
+
+  private StreamAdminClient streamAdminClient;
+  private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator;
+
+  abstract StreamAdminClient getStreamAdminClient() throws Exception;
+  abstract StreamEventWriter getStreamEventWriter() throws Exception;
+  abstract Function<String, String> generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName);
+  abstract Function<String, String> generateStreamQueryPropsTransform(String streamName, String fullDatasourceName);
+  public abstract String getTestNamePrefix();
+
+  protected void doBeforeClass() throws Exception
+  {
+    streamAdminClient = getStreamAdminClient();
+    wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS);
+  }
+
+  protected void doClassTeardown()
+  {
+    wikipediaStreamEventGenerator.shutdown();
+  }
+
+  protected void doTestIndexDataWithLegacyParserStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithInputFormatStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  void doTestIndexDataWithLosingCoordinator() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady());
+  }
+
+  void doTestIndexDataWithLosingOverlord() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady());
+  }
+
+  void doTestIndexDataWithLosingHistorical() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady());
+  }
+
+  protected void doTestIndexDataWithStartStopSupervisor() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating half of the data
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Suspend the supervisor
+      indexer.suspendSupervisor(generatedTestConfig.getSupervisorId());
+      // Start generating remainning half of the data
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Resume the supervisor
+      indexer.resumeSupervisor(generatedTestConfig.getSupervisorId());
+      // Verify supervisor is healthy after suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor can catch up with the stream
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithStreamReshardSplit() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2);
+  }
+
+  protected void doTestIndexDataWithStreamReshardMerge() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT / 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT / 2);
+  }
+
+  private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating one third of the data (before restarting)
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before restart
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Restart Druid process
+      LOG.info("Restarting Druid process");
+      restartRunnable.run();
+      LOG.info("Restarted Druid process");
+      // Start generating one third of the data (while restarting)
+      int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Wait for Druid process to be available
+      LOG.info("Waiting for Druid process to be available");
+      waitForReadyRunnable.run();
+      LOG.info("Druid process is now available");
+      // Start generating remainding data (after restarting)
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound));
+      // Verify supervisor is healthy
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor ingested all data
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  private void testIndexWithStreamReshardHelper(int newShardCount) throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating one third of the data (before resharding)
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before resahrding
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Reshard the supervisor by split from STREAM_SHARD_COUNT to newShardCount and waits until the resharding starts
+      streamAdminClient.updateShardCount(generatedTestConfig.getStreamName(), newShardCount, true);
+      // Start generating one third of the data (while resharding)
+      int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Wait for stream to finish resharding
+      ITRetryUtil.retryUntil(
+          () -> streamAdminClient.isStreamActive(generatedTestConfig.getStreamName()),
+          true,
+          10000,
+          30,
+          "Waiting for stream to finish resharding"
+      );
+      ITRetryUtil.retryUntil(
+          () -> streamAdminClient.verfiyShardCountUpdated(generatedTestConfig.getStreamName(), STREAM_SHARD_COUNT, newShardCount),
+          true,
+          10000,
+          30,
+          "Waiting for stream to finish resharding"
+      );
+      // Start generating remainding data (after resharding)
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound));
+      // Verify supervisor is healthy after resahrding
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor can catch up with the stream
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  private void verifyIngestedData(GeneratedTestConfig generatedTestConfig) throws Exception
+  {
+    // Wait for supervisor to consume events
+    LOG.info("Waiting for [%s] millis for stream indexing tasks to consume events", WAIT_TIME_MILLIS);
+    Thread.sleep(WAIT_TIME_MILLIS);
+    // Query data
+    final String querySpec = generatedTestConfig.getStreamQueryPropsTransform().apply(getResourceAsString(QUERIES_FILE));
+    // this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing
+    this.queryHelper.testQueriesFromString(querySpec, 2);
+    LOG.info("Shutting down supervisor");
+    indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId());
+    // wait for all indexing tasks to finish
+    LOG.info("Waiting for all indexing tasks to finish");
+    ITRetryUtil.retryUntilTrue(
+        () -> (indexer.getUncompletedTasksForDataSource(generatedTestConfig.getFullDatasourceName()).size() == 0),
+        "Waiting for Tasks Completion"
+    );
+    // wait for segments to be handed off
+    ITRetryUtil.retryUntil(
+        () -> coordinator.areSegmentsLoaded(generatedTestConfig.getFullDatasourceName()),
+        true,
+        10000,
+        30,
+        "Real-time generated segments loaded"
+    );
+
+    // this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4
+    this.queryHelper.testQueriesFromString(querySpec, 2);
+  }
+
+  long getSumOfEventSequence(int numEvents)
+  {
+    return (numEvents * (1 + numEvents)) / 2;
+  }
+
+  private void doMethodTeardown(GeneratedTestConfig generatedTestConfig, StreamEventWriter streamEventWriter)
+  {
+    try {
+      streamEventWriter.flush();
+    }
+    catch (Exception e) {
+      // Best effort cleanup as the writer may have already went Bye-Bye

Review comment:
       BTW, I don't think ignoring exceptions is a good idea in any case. Probably it should log at least.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.tests.indexer;
+
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.tests.TestNGGroup;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+@Test(groups = TestNGGroup.KINESIS_INDEX)
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITKinesisIndexingServiceSerializedTest extends AbstractKinesisIndexingServiceTest

Review comment:
       Similarly, can you add javadoc that explains what kind of tests should be done in sequential?

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);
+  // Since this integration test can terminates or be killed un-expectedly, this tag is added to all streams created
+  // to help make stream clean up easier. (Normally, streams should be cleanup automattically by the teardown method)
+  // The value to this tag is a timestamp that can be used by a lambda function to remove unused stream.
+  private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after";
+  private static final int STREAM_SHARD_COUNT = 2;
+  private static final long WAIT_TIME_MILLIS = 3 * 60 * 1000L;
+  private static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json";
+  private static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/stream_supervisor_spec_input_format.json";
+  private static final String QUERIES_FILE = "/indexer/stream_index_queries.json";
+  private static final long CYCLE_PADDING_MS = 100;
+
+  @Inject
+  private DruidClusterAdminClient druidClusterAdminClient;
+
+  private StreamAdminClient streamAdminClient;
+  private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator;
+
+  abstract StreamAdminClient getStreamAdminClient() throws Exception;
+  abstract StreamEventWriter getStreamEventWriter() throws Exception;
+  abstract Function<String, String> generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName);
+  abstract Function<String, String> generateStreamQueryPropsTransform(String streamName, String fullDatasourceName);
+  public abstract String getTestNamePrefix();
+
+  protected void doBeforeClass() throws Exception
+  {
+    streamAdminClient = getStreamAdminClient();
+    wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS);
+  }
+
+  protected void doClassTeardown()
+  {
+    wikipediaStreamEventGenerator.shutdown();
+  }
+
+  protected void doTestIndexDataWithLegacyParserStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);

Review comment:
       Probably better to call `run()` or `generate()` rather than `start()` since the generator is not something you can start/stop, but generates all events in `start()`.

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.tests.parallelized;
+
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+@Test(groups = TestNGGroup.KAFKA_INDEX)
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITKafkaIndexingServiceNonTransactionalParallelizedTest extends AbstractKafkaIndexingServiceTest

Review comment:
       Can you add javadoc that explains what kind of tests should be done in parallel?

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);
+  // Since this integration test can terminates or be killed un-expectedly, this tag is added to all streams created
+  // to help make stream clean up easier. (Normally, streams should be cleanup automattically by the teardown method)
+  // The value to this tag is a timestamp that can be used by a lambda function to remove unused stream.
+  private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after";
+  private static final int STREAM_SHARD_COUNT = 2;
+  private static final long WAIT_TIME_MILLIS = 3 * 60 * 1000L;
+  private static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json";
+  private static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/stream_supervisor_spec_input_format.json";
+  private static final String QUERIES_FILE = "/indexer/stream_index_queries.json";
+  private static final long CYCLE_PADDING_MS = 100;
+
+  @Inject
+  private DruidClusterAdminClient druidClusterAdminClient;
+
+  private StreamAdminClient streamAdminClient;
+  private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator;
+
+  abstract StreamAdminClient getStreamAdminClient() throws Exception;
+  abstract StreamEventWriter getStreamEventWriter() throws Exception;
+  abstract Function<String, String> generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName);
+  abstract Function<String, String> generateStreamQueryPropsTransform(String streamName, String fullDatasourceName);
+  public abstract String getTestNamePrefix();
+
+  protected void doBeforeClass() throws Exception
+  {
+    streamAdminClient = getStreamAdminClient();
+    wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS);
+  }
+
+  protected void doClassTeardown()
+  {
+    wikipediaStreamEventGenerator.shutdown();
+  }
+
+  protected void doTestIndexDataWithLegacyParserStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithInputFormatStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  void doTestIndexDataWithLosingCoordinator() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady());
+  }
+
+  void doTestIndexDataWithLosingOverlord() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady());
+  }
+
+  void doTestIndexDataWithLosingHistorical() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady());
+  }
+
+  protected void doTestIndexDataWithStartStopSupervisor() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating half of the data
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Suspend the supervisor
+      indexer.suspendSupervisor(generatedTestConfig.getSupervisorId());
+      // Start generating remainning half of the data
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Resume the supervisor
+      indexer.resumeSupervisor(generatedTestConfig.getSupervisorId());
+      // Verify supervisor is healthy after suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor can catch up with the stream
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithStreamReshardSplit() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2);
+  }
+
+  protected void doTestIndexDataWithStreamReshardMerge() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT / 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT / 2);
+  }
+
+  private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating one third of the data (before restarting)
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before restart
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Restart Druid process
+      LOG.info("Restarting Druid process");
+      restartRunnable.run();
+      LOG.info("Restarted Druid process");
+      // Start generating one third of the data (while restarting)
+      int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Wait for Druid process to be available
+      LOG.info("Waiting for Druid process to be available");
+      waitForReadyRunnable.run();
+      LOG.info("Druid process is now available");
+      // Start generating remainding data (after restarting)

Review comment:
       remainding -> remaining?

##########
File path: integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java
##########
@@ -0,0 +1,439 @@
+/*
+ * 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.tests.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.utils.DruidClusterAdminClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.StreamAdminClient;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
+{
+  static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0);
+  // format for the querying interval
+  static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
+  // format for the expected timestamp in a query response
+  static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
+  static final int EVENTS_PER_SECOND = 6;
+  static final int TOTAL_NUMBER_OF_SECOND = 10;
+  static final Logger LOG = new Logger(AbstractStreamIndexingTest.class);
+  // Since this integration test can terminates or be killed un-expectedly, this tag is added to all streams created
+  // to help make stream clean up easier. (Normally, streams should be cleanup automattically by the teardown method)
+  // The value to this tag is a timestamp that can be used by a lambda function to remove unused stream.
+  private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after";
+  private static final int STREAM_SHARD_COUNT = 2;
+  private static final long WAIT_TIME_MILLIS = 3 * 60 * 1000L;
+  private static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json";
+  private static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/stream_supervisor_spec_input_format.json";
+  private static final String QUERIES_FILE = "/indexer/stream_index_queries.json";
+  private static final long CYCLE_PADDING_MS = 100;
+
+  @Inject
+  private DruidClusterAdminClient druidClusterAdminClient;
+
+  private StreamAdminClient streamAdminClient;
+  private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator;
+
+  abstract StreamAdminClient getStreamAdminClient() throws Exception;
+  abstract StreamEventWriter getStreamEventWriter() throws Exception;
+  abstract Function<String, String> generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName);
+  abstract Function<String, String> generateStreamQueryPropsTransform(String streamName, String fullDatasourceName);
+  public abstract String getTestNamePrefix();
+
+  protected void doBeforeClass() throws Exception
+  {
+    streamAdminClient = getStreamAdminClient();
+    wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS);
+  }
+
+  protected void doClassTeardown()
+  {
+    wikipediaStreamEventGenerator.shutdown();
+  }
+
+  protected void doTestIndexDataWithLegacyParserStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithInputFormatStableState() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start data generator
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME);
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  void doTestIndexDataWithLosingCoordinator() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady());
+  }
+
+  void doTestIndexDataWithLosingOverlord() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady());
+  }
+
+  void doTestIndexDataWithLosingHistorical() throws Exception
+  {
+    testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady());
+  }
+
+  protected void doTestIndexDataWithStartStopSupervisor() throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating half of the data
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Suspend the supervisor
+      indexer.suspendSupervisor(generatedTestConfig.getSupervisorId());
+      // Start generating remainning half of the data
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Resume the supervisor
+      indexer.resumeSupervisor(generatedTestConfig.getSupervisorId());
+      // Verify supervisor is healthy after suspension
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor can catch up with the stream
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  protected void doTestIndexDataWithStreamReshardSplit() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2);
+  }
+
+  protected void doTestIndexDataWithStreamReshardMerge() throws Exception
+  {
+    // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT / 2
+    testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT / 2);
+  }
+
+  private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating one third of the data (before restarting)
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before restart
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Restart Druid process
+      LOG.info("Restarting Druid process");
+      restartRunnable.run();
+      LOG.info("Restarted Druid process");
+      // Start generating one third of the data (while restarting)
+      int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Wait for Druid process to be available
+      LOG.info("Waiting for Druid process to be available");
+      waitForReadyRunnable.run();
+      LOG.info("Druid process is now available");
+      // Start generating remainding data (after restarting)
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound));
+      // Verify supervisor is healthy
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor ingested all data
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  private void testIndexWithStreamReshardHelper(int newShardCount) throws Exception
+  {
+    StreamEventWriter streamEventWriter = getStreamEventWriter();
+    final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig();
+    try (
+        final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName())
+    ) {
+      final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT));
+      LOG.info("supervisorSpec: [%s]\n", taskSpec);
+      // Start supervisor
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("Submitted supervisor");
+      // Start generating one third of the data (before resharding)
+      int secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND;
+      int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME);
+      // Verify supervisor is healthy before resahrding
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Reshard the supervisor by split from STREAM_SHARD_COUNT to newShardCount and waits until the resharding starts
+      streamAdminClient.updateShardCount(generatedTestConfig.getStreamName(), newShardCount, true);
+      // Start generating one third of the data (while resharding)
+      int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3;
+      secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound;
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound));
+      // Wait for stream to finish resharding
+      ITRetryUtil.retryUntil(
+          () -> streamAdminClient.isStreamActive(generatedTestConfig.getStreamName()),
+          true,
+          10000,
+          30,
+          "Waiting for stream to finish resharding"
+      );
+      ITRetryUtil.retryUntil(
+          () -> streamAdminClient.verfiyShardCountUpdated(generatedTestConfig.getStreamName(), STREAM_SHARD_COUNT, newShardCount),
+          true,
+          10000,
+          30,
+          "Waiting for stream to finish resharding"
+      );
+      // Start generating remainding data (after resharding)
+      wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound));
+      // Verify supervisor is healthy after resahrding
+      ITRetryUtil.retryUntil(
+          () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())),
+          true,
+          10000,
+          30,
+          "Waiting for supervisor to be healthy"
+      );
+      // Verify that supervisor can catch up with the stream
+      verifyIngestedData(generatedTestConfig);
+    }
+    finally {
+      doMethodTeardown(generatedTestConfig, streamEventWriter);
+    }
+  }
+
+  private void verifyIngestedData(GeneratedTestConfig generatedTestConfig) throws Exception
+  {
+    // Wait for supervisor to consume events
+    LOG.info("Waiting for [%s] millis for stream indexing tasks to consume events", WAIT_TIME_MILLIS);
+    Thread.sleep(WAIT_TIME_MILLIS);
+    // Query data
+    final String querySpec = generatedTestConfig.getStreamQueryPropsTransform().apply(getResourceAsString(QUERIES_FILE));
+    // this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing
+    this.queryHelper.testQueriesFromString(querySpec, 2);
+    LOG.info("Shutting down supervisor");
+    indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId());
+    // wait for all indexing tasks to finish
+    LOG.info("Waiting for all indexing tasks to finish");
+    ITRetryUtil.retryUntilTrue(
+        () -> (indexer.getUncompletedTasksForDataSource(generatedTestConfig.getFullDatasourceName()).size() == 0),
+        "Waiting for Tasks Completion"
+    );
+    // wait for segments to be handed off
+    ITRetryUtil.retryUntil(
+        () -> coordinator.areSegmentsLoaded(generatedTestConfig.getFullDatasourceName()),
+        true,
+        10000,
+        30,
+        "Real-time generated segments loaded"
+    );
+
+    // this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4
+    this.queryHelper.testQueriesFromString(querySpec, 2);
+  }
+
+  long getSumOfEventSequence(int numEvents)
+  {
+    return (numEvents * (1 + numEvents)) / 2;
+  }
+
+  private void doMethodTeardown(GeneratedTestConfig generatedTestConfig, StreamEventWriter streamEventWriter)
+  {
+    try {
+      streamEventWriter.flush();
+    }
+    catch (Exception e) {
+      // Best effort cleanup as the writer may have already went Bye-Bye

Review comment:
       "may have already gone"?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org