From fe95b67954aa9233a0108e48dc33d475a41f34dd Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 16 Apr 2020 20:51:42 -1000 Subject: [PATCH 01/17] add kafka admin and kafka writer --- integration-tests/README.md | 13 +- integration-tests/pom.xml | 21 +- integration-tests/run_cluster.sh | 6 + .../druid/testing/utils/KafkaAdminClient.java | 98 +++++++ .../druid/testing/utils/KafkaEventWriter.java | 116 ++++++++ .../testing/utils/KinesisAdminClient.java | 7 +- .../testing/utils/KinesisEventWriter.java | 9 + .../testing/utils/StreamAdminClient.java | 35 +++ .../testing/utils/StreamEventWriter.java | 8 +- .../indexer/AbstractKafkaIndexerTest.java | 259 ------------------ integration-tests/stop_cluster.sh | 6 + 11 files changed, 300 insertions(+), 278 deletions(-) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java diff --git a/integration-tests/README.md b/integration-tests/README.md index baadd3a7bd1a..f13ac9fd3015 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -68,7 +68,18 @@ can either be 8 or 11. Druid's configuration (using Docker) can be overrided by providing -Doverride.config.path=. The file must contain one property per line, the key must start with `druid_` and the format should be snake case. -## Debugging Druid while running tests +## Tips & tricks for debugging and developing integration tests + +### Useful mvn command flags + +- -Dskip.start.docker=true to skip starting docker containers. This can save ~6 minutes by skipping building and bringing +up the docker containers (Druid, Kafka, Hadoop, MYSQL, zookeeper, etc). Please make sure that you actually do have +these containers already running if using this flag. Additionally, please make sure that the running containers +are in the same state that the setup script (run_cluster.sh) would have brought it up in. +- -Dskip.stop.docker=true to skip stopping and teardowning down the docker containers. This can be useful in further +debugging after the integration tests have finish running. + +### Debugging Druid while running tests For your convenience, Druid processes running inside Docker have debugging enabled and the following ports have been made available to attach your remote debugger (such as via IntelliJ IDEA's Remote Configuration): diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 885419623d32..ede1de0abfaa 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -304,21 +304,6 @@ easymock test - - org.apache.kafka - kafka_2.12 - ${apache.kafka.version} - - - log4j - log4j - - - slf4j-log4j12 - org.slf4j - - - @@ -367,6 +352,8 @@ integration-tests false + false + false @@ -385,6 +372,7 @@ ${start.hadoop.docker} + ${skip.start.docker} ${jvm.runtime} ${groups} ${override.config.path} @@ -400,6 +388,9 @@ post-integration-test + + ${skip.stop.docker} + ${project.basedir}/stop_cluster.sh diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh index 122cf6a4d805..ba07e36d8f7f 100755 --- a/integration-tests/run_cluster.sh +++ b/integration-tests/run_cluster.sh @@ -14,6 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. +# Skip starting docker if flag set (For use during development) +if [ -n "$DRUID_INTEGRATION_TEST_SKIP_START_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_SKIP_START_DOCKER" == true ] + then + exit 0 + fi + # Cleanup old images/containers { for node in druid-historical druid-coordinator druid-overlord druid-router druid-router-permissive-tls druid-router-no-client-auth-tls druid-router-custom-check-tls druid-broker druid-middlemanager druid-zookeeper-kafka druid-metadata-storage druid-it-hadoop; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java new file mode 100644 index 000000000000..dc0cf5213dae --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java @@ -0,0 +1,98 @@ +/* + * 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) throws Exception + { + Properties config = new Properties(); + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost); + adminClient = AdminClient.create(config); + } + + @Override + public void createStream(String streamName, int partitionCount, Map tags) throws Exception + { + final short replicationFactor = 1; + Map configs = new HashMap<>(); + 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 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) throws Exception + { + return true; + } + + @Override + public int getStreamShardCount(String streamName) throws Exception + { + DescribeTopicsResult result = adminClient.describeTopics(ImmutableList.of(streamName)); + TopicDescription topicDescription = result.values().get(streamName).get(); + return topicDescription.partitions().size(); + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java new file mode 100644 index 000000000000..eb2409bcb9d2 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java @@ -0,0 +1,116 @@ +/* + * 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 producer; + private boolean txnEnabled; + private List> pendingWriteRecords = new ArrayList<>(); + + public KafkaEventWriter(IntegrationTestingConfig config, boolean txnEnabled) + { + Properties properties = new Properties(); + addFilteredProperties(config, properties); + properties.setProperty("bootstrap.servers", config.getKafkaHost()); + properties.setProperty("acks", "all"); + properties.setProperty("retries", "3"); + properties.setProperty("key.serializer", ByteArraySerializer.class.getName()); + properties.setProperty("value.serializer", ByteArraySerializer.class.getName()); + this.txnEnabled = txnEnabled; + if (txnEnabled) { + properties.setProperty("enable.idempotence", "true"); + properties.setProperty("transactional.id", TaskIdUtils.getRandomId()); + } + this.producer = new KafkaProducer<>( + properties, + new StringSerializer(), + new StringSerializer() + ); + } + + @Override + public void write(String topic, List events) throws Exception + { + if (txnEnabled) { + producer.initTransactions(); + producer.beginTransaction(); + } + + for (String event : events) + { + write(topic, event); + } + + if (txnEnabled) { + producer.commitTransaction(); + } + } + + /** + * This method does not handle transaction. For transaction functionality use + * {@link #write(String, List future = producer.send(new ProducerRecord<>(topic, event)); + pendingWriteRecords.add(future); + } + + @Override + public void shutdown() + { + producer.close(); + } + + @Override + public void flush() throws Exception + { + for (Future future : pendingWriteRecords) { + future.get(); + } + pendingWriteRecords.clear(); + } + + private void addFilteredProperties(IntegrationTestingConfig config, Properties properties) + { + for (Map.Entry entry : config.getProperties().entrySet()) { + if (entry.getKey().startsWith(TEST_PROPERTY_PREFIX)) { + properties.setProperty(entry.getKey().substring(TEST_PROPERTY_PREFIX.length()), entry.getValue()); + } + } + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java index bc5ace2d1b48..6833947b39ac 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.Properties; -public class KinesisAdminClient +public class KinesisAdminClient implements StreamAdminClient { private AmazonKinesis amazonKinesis; @@ -70,6 +70,7 @@ public KinesisAdminClient(String endpoint) throws Exception )).build(); } + @Override public void createStream(String streamName, int shardCount, Map tags) { CreateStreamResult createStreamResult = amazonKinesis.createStream(streamName, shardCount); @@ -88,6 +89,7 @@ public void createStream(String streamName, int shardCount, Map } + @Override public void deleteStream(String streamName) { DeleteStreamResult deleteStreamResult = amazonKinesis.deleteStream(streamName); @@ -101,6 +103,7 @@ public void deleteStream(String streamName) * If {@param blocksUntilStarted} is set to true, then this method will blocks until the resharding * started (but not nessesary finished), otherwise, the method will returns right after issue the reshard command */ + @Override public void updateShardCount(String streamName, int newShardCount, boolean blocksUntilStarted) { int originalShardCount = getStreamShardCount(streamName); @@ -129,12 +132,14 @@ public void updateShardCount(String streamName, int newShardCount, boolean block } } + @Override public boolean isStreamActive(String streamName) { StreamDescription streamDescription = getStreamDescription(streamName); return verifyStreamStatus(streamDescription, StreamStatus.ACTIVE); } + @Override public int getStreamShardCount(String streamName) { StreamDescription streamDescription = getStreamDescription(streamName); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java index 09950f132047..735c13f5fa4d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java @@ -30,6 +30,7 @@ import java.io.FileInputStream; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Properties; public class KinesisEventWriter implements StreamEventWriter @@ -63,6 +64,14 @@ public KinesisEventWriter(String endpoint, boolean aggregate) throws Exception this.kinesisProducer = new KinesisProducer(kinesisProducerConfiguration); } + @Override + public void write(String streamName, List events) + { + for (String event : events) { + write(streamName, event); + } + } + @Override public void write(String streamName, String event) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java new file mode 100644 index 000000000000..fc0ba05739aa --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java @@ -0,0 +1,35 @@ +/* + * 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 tags) throws Exception; + + void deleteStream(String streamName) throws Exception; + + void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception; + + boolean isStreamActive(String streamName) throws Exception; + + int getStreamShardCount(String streamName) throws Exception; +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java index 1bfd6b675919..df9593d2a381 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java @@ -19,11 +19,15 @@ package org.apache.druid.testing.utils; +import java.util.List; + public interface StreamEventWriter { - void write(String topic, String event); + void write(String topic, List events) throws Exception; + + void write(String topic, String event) throws Exception; void shutdown(); - void flush(); + void flush() throws Exception; } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java index a20254d2cf07..6e2a7e88191f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java @@ -54,264 +54,5 @@ abstract class AbstractKafkaIndexerTest extends AbstractIndexerTest { - private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); - protected static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/stream_supervisor_spec_legacy_parser.json"; - protected 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 String TOPIC_NAME = "kafka_indexing_service_topic"; - private static final int NUM_EVENTS_TO_SEND = 60; - private static final long WAIT_TIME_MILLIS = 2 * 60 * 1000L; - private static final String TEST_PROPERTY_PREFIX = "kafka.test.property."; - - // We'll fill in the current time and numbers for added, deleted and changed - // before sending the event. - private static final String EVENT_TEMPLATE = - "{\"timestamp\": \"%s\"," + - "\"page\": \"Gypsy Danger\"," + - "\"language\" : \"en\"," + - "\"user\" : \"nuclear\"," + - "\"unpatrolled\" : \"true\"," + - "\"newPage\" : \"true\"," + - "\"robot\": \"false\"," + - "\"anonymous\": \"false\"," + - "\"namespace\":\"article\"," + - "\"continent\":\"North America\"," + - "\"country\":\"United States\"," + - "\"region\":\"Bay Area\"," + - "\"city\":\"San Francisco\"," + - "\"added\":%d," + - "\"deleted\":%d," + - "\"delta\":%d}"; - - private ZkUtils zkUtils; - private boolean segmentsExist; // to tell if we should remove segments during teardown - - // format for the querying interval - private static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); - // format for the expected timestamp in a query response - private static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'"); - - @Inject - private TestQueryHelper queryHelper; - @Inject - private IntegrationTestingConfig config; - - private String fullDatasourceName; - - void doKafkaIndexTest(String dataSourceName, String supervisorSpecPath, boolean txnEnabled) - { - fullDatasourceName = dataSourceName + config.getExtraDatasourceNameSuffix(); - // create topic - try { - int sessionTimeoutMs = 10000; - int connectionTimeoutMs = 10000; - String zkHosts = config.getZookeeperHosts(); - ZkClient zkClient = new ZkClient(zkHosts, sessionTimeoutMs, connectionTimeoutMs, ZKStringSerializer$.MODULE$); - zkUtils = new ZkUtils(zkClient, new ZkConnection(zkHosts, sessionTimeoutMs), false); - if (config.manageKafkaTopic()) { - int numPartitions = 4; - int replicationFactor = 1; - Properties topicConfig = new Properties(); - AdminUtils.createTopic( - zkUtils, - TOPIC_NAME, - numPartitions, - replicationFactor, - topicConfig, - RackAwareMode.Disabled$.MODULE$ - ); - } - } - catch (Exception e) { - throw new ISE(e, "could not create kafka topic"); - } - - String spec; - try { - LOG.info("supervisorSpec name: [%s]", supervisorSpecPath); - final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); - final Properties consumerProperties = new Properties(); - consumerProperties.putAll(consumerConfigs); - consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); - - spec = getResourceAsString(supervisorSpecPath); - spec = StringUtils.replace(spec, "%%DATASOURCE%%", fullDatasourceName); - spec = StringUtils.replace(spec, "%%STREAM_TYPE%%", "kafka"); - spec = StringUtils.replace(spec, "%%TOPIC_KEY%%", "topic"); - spec = StringUtils.replace(spec, "%%TOPIC_VALUE%%", TOPIC_NAME); - spec = StringUtils.replace(spec, "%%USE_EARLIEST_KEY%%", "useEarliestOffset"); - spec = StringUtils.replace(spec, "%%STREAM_PROPERTIES_KEY%%", "consumerProperties"); - spec = StringUtils.replace(spec, "%%STREAM_PROPERTIES_VALUE%%", jsonMapper.writeValueAsString(consumerProperties)); - LOG.info("supervisorSpec: [%s]\n", spec); - } - catch (Exception e) { - LOG.error("could not read file [%s]", supervisorSpecPath); - throw new ISE(e, "could not read file [%s]", supervisorSpecPath); - } - - // start supervisor - String supervisorId = indexer.submitSupervisor(spec); - LOG.info("Submitted supervisor"); - - // set up kafka producer - Properties properties = new Properties(); - addFilteredProperties(config, properties); - properties.setProperty("bootstrap.servers", config.getKafkaHost()); - LOG.info("Kafka bootstrap.servers: [%s]", config.getKafkaHost()); - properties.setProperty("acks", "all"); - properties.setProperty("retries", "3"); - properties.setProperty("key.serializer", ByteArraySerializer.class.getName()); - properties.setProperty("value.serializer", ByteArraySerializer.class.getName()); - if (txnEnabled) { - properties.setProperty("enable.idempotence", "true"); - properties.setProperty("transactional.id", TaskIdUtils.getRandomId()); - } - - KafkaProducer producer = new KafkaProducer<>( - properties, - new StringSerializer(), - new StringSerializer() - ); - - DateTimeZone zone = DateTimes.inferTzFromString("UTC"); - // format for putting into events - DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"); - - DateTime dt = new DateTime(zone); // timestamp to put on events - DateTime dtFirst = dt; // timestamp of 1st event - DateTime dtLast = dt; // timestamp of last event - - // these are used to compute the expected aggregations - int added = 0; - int num_events = 0; - - // send data to kafka - if (txnEnabled) { - producer.initTransactions(); - producer.beginTransaction(); - } - while (num_events < NUM_EVENTS_TO_SEND) { - num_events++; - added += num_events; - // construct the event to send - String event = StringUtils.format(EVENT_TEMPLATE, event_fmt.print(dt), num_events, 0, num_events); - LOG.info("sending event: [%s]", event); - try { - - producer.send(new ProducerRecord<>(TOPIC_NAME, event)).get(); - - } - catch (Exception ioe) { - throw Throwables.propagate(ioe); - } - - dtLast = dt; - dt = new DateTime(zone); - } - if (txnEnabled) { - producer.commitTransaction(); - } - producer.close(); - - LOG.info("Waiting for [%s] millis for Kafka indexing tasks to consume events", WAIT_TIME_MILLIS); - try { - Thread.sleep(WAIT_TIME_MILLIS); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - - InputStream is = AbstractKafkaIndexerTest.class.getResourceAsStream(QUERIES_FILE); - if (null == is) { - throw new ISE("could not open query file: %s", QUERIES_FILE); - } - - // put the timestamps into the query structure - String query_response_template; - try { - query_response_template = IOUtils.toString(is, StandardCharsets.UTF_8); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", QUERIES_FILE); - } - - String queryStr = query_response_template; - queryStr = StringUtils.replace(queryStr, "%%DATASOURCE%%", fullDatasourceName); - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)); - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2))); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_ADDED%%", Integer.toString(added)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events)); - - // this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing - try { - this.queryHelper.testQueriesFromString(queryStr, 2); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - - LOG.info("Shutting down Kafka Supervisor"); - indexer.shutdownSupervisor(supervisorId); - - // wait for all kafka indexing tasks to finish - LOG.info("Waiting for all kafka indexing tasks to finish"); - ITRetryUtil.retryUntilTrue( - () -> (indexer.getPendingTasks().size() - + indexer.getRunningTasks().size() - + indexer.getWaitingTasks().size()) == 0, - "Waiting for Tasks Completion" - ); - - // wait for segments to be handed off - try { - ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - true, - 10000, - 30, - "Real-time generated segments loaded" - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - LOG.info("segments are present"); - segmentsExist = true; - - // this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4 - try { - this.queryHelper.testQueriesFromString(queryStr, 2); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - private void addFilteredProperties(IntegrationTestingConfig config, Properties properties) - { - for (Map.Entry entry : config.getProperties().entrySet()) { - if (entry.getKey().startsWith(TEST_PROPERTY_PREFIX)) { - properties.setProperty(entry.getKey().substring(TEST_PROPERTY_PREFIX.length()), entry.getValue()); - } - } - } - - void doTearDown() - { - if (config.manageKafkaTopic()) { - // delete kafka topic - AdminUtils.deleteTopic(zkUtils, TOPIC_NAME); - } - - // remove segments - if (segmentsExist && fullDatasourceName != null) { - unloadAndKillData(fullDatasourceName); - } - } } diff --git a/integration-tests/stop_cluster.sh b/integration-tests/stop_cluster.sh index 4ce4268806de..2828a0ff8a96 100755 --- a/integration-tests/stop_cluster.sh +++ b/integration-tests/stop_cluster.sh @@ -14,6 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. +# Skip stopping docker if flag set (For use during development) +if [ -n "$DRUID_INTEGRATION_TEST_SKIP_STOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_SKIP_STOP_DOCKER" == true ] + then + exit 0 + fi + for node in druid-historical druid-coordinator druid-overlord druid-router druid-router-permissive-tls druid-router-no-client-auth-tls druid-router-custom-check-tls druid-broker druid-middlemanager druid-zookeeper-kafka druid-metadata-storage druid-it-hadoop; do From eee34658803c721228147af4138c068c5a4f139e Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 16 Apr 2020 22:52:44 -1000 Subject: [PATCH 02/17] refactor kinesis IT --- .../indexer/AbstractKafkaIndexerTest.java | 58 --- .../indexer/AbstractStreamIndexingTest.java | 387 ++++++++++++++++++ .../indexer/ITKafkaIndexingServiceTest.java | 40 +- ...KafkaIndexingServiceTransactionalTest.java | 66 --- .../indexer/ITKinesisIndexingServiceTest.java | 330 ++------------- 5 files changed, 414 insertions(+), 467 deletions(-) delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java deleted file mode 100644 index 6e2a7e88191f..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.base.Throwables; -import com.google.inject.Inject; -import kafka.admin.AdminUtils; -import kafka.admin.RackAwareMode; -import kafka.utils.ZKStringSerializer$; -import kafka.utils.ZkUtils; -import org.I0Itec.zkclient.ZkClient; -import org.I0Itec.zkclient.ZkConnection; -import org.apache.commons.io.IOUtils; -import org.apache.druid.indexer.TaskIdUtils; -import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.testing.utils.TestQueryHelper; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.Map; -import java.util.Properties; - -abstract class AbstractKafkaIndexerTest extends AbstractIndexerTest -{ - -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java new file mode 100644 index 000000000000..f4025ddb763f --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -0,0 +1,387 @@ +/* + * 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 org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; + +import java.io.Closeable; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTest +{ + 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; + + private 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; + + String streamName; + String fullDatasourceName; + + private StreamAdminClient streamAdminClient; + private StreamEventWriter streamEventWriter; + private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; + private Function streamIngestionPropsTransform; + private Function streamQueryPropsTransform; + private String supervisorId; + private int secondsToGenerateRemaining; + + abstract StreamAdminClient getStreamAdminClient() throws Exception; + abstract StreamEventWriter getStreamEventWriter() throws Exception; + abstract Function getStreamIngestionPropsTransform(); + abstract Function getStreamQueryPropsTransform(); + + @BeforeClass + public void beforeClass() throws Exception + { + streamAdminClient = getStreamAdminClient(); + streamEventWriter = getStreamEventWriter(); + wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); + } + + @AfterClass + public void tearDown() + { + wikipediaStreamEventGenerator.shutdown(); + streamEventWriter.shutdown(); + } + + @BeforeMethod + public void before() throws Exception + { + streamName = "kinesis_index_test_" + UUID.randomUUID(); + String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); + Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTimes.nowUtc().plusMinutes(30).getMillis())); + streamAdminClient.createStream(streamName, STREAM_SHARD_COUNT, tags); + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(streamName), + true, + 10000, + 30, + "Wait for stream active" + ); + secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND; + fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); + streamIngestionPropsTransform = getStreamIngestionPropsTransform(); + streamQueryPropsTransform = getStreamQueryPropsTransform(); + } + + @AfterMethod + public void teardown() + { + try { + streamEventWriter.flush(); + indexer.shutdownSupervisor(supervisorId); + } + catch (Exception e) { + // Best effort cleanup as the supervisor may have already went Bye-Bye + } + try { + unloader(fullDatasourceName); + } + catch (Exception e) { + // Best effort cleanup as the datasource may have already went Bye-Bye + } + try { + streamAdminClient.deleteStream(streamName); + } + catch (Exception e) { + // Best effort cleanup as the stream may have already went Bye-Bye + } + } + + void doTestIndexDataWithLegacyParserStableState() throws Exception + { + try ( + final Closeable ignored1 = unloader(fullDatasourceName) + ) { + final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start data generator + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } + + void doTestIndexDataWithInputFormatStableState() throws Exception + { + try ( + final Closeable ignored1 = unloader(fullDatasourceName) + ) { + final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start data generator + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } + + 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()); + } + + void doTestIndexDataWithStartStopSupervisor() throws Exception + { + try ( + final Closeable ignored1 = unloader(fullDatasourceName) + ) { + final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start generating half of the data + int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + // Verify supervisor is healthy before suspension + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + true, + 10000, + 30, + "Waiting for supervisor to be healthy" + ); + // Suspend the supervisor + indexer.suspendSupervisor(supervisorId); + // Start generating remainning half of the data + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + // Resume the supervisor + indexer.resumeSupervisor(supervisorId); + // Verify supervisor is healthy after suspension + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + true, + 10000, + 30, + "Waiting for supervisor to be healthy" + ); + // Verify that supervisor can catch up with the stream + verifyIngestedData(supervisorId); + } + } + + void doTestIndexDataWithKinesisReshardSplit() throws Exception + { + // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2 + testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2); + } + + void doTestIndexDataWithKinesisReshardMerge() 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 + { + try ( + final Closeable ignored1 = unloader(fullDatasourceName) + ) { + final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start generating one third of the data (before restarting) + int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + // Verify supervisor is healthy before restart + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + 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(streamName, 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(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + // Verify supervisor is healthy + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + true, + 10000, + 30, + "Waiting for supervisor to be healthy" + ); + // Verify that supervisor ingested all data + verifyIngestedData(supervisorId); + } + } + + private void testIndexWithStreamReshardHelper(int newShardCount) throws Exception + { + try ( + final Closeable ignored1 = unloader(fullDatasourceName) + ) { + final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start generating one third of the data (before resharding) + int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + // Verify supervisor is healthy before resahrding + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + 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(streamName, newShardCount, true); + // Start generating one third of the data (while resharding) + int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + // Wait for kinesis stream to finish resharding + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(streamName), + true, + 10000, + 30, + "Waiting for Kinesis stream to finish resharding" + ); + ITRetryUtil.retryUntil( + () -> streamAdminClient.getStreamShardCount(streamName) == newShardCount, + true, + 10000, + 30, + "Waiting for Kinesis stream to finish resharding" + ); + // Start generating remainding data (after resharding) + wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + // Verify supervisor is healthy after resahrding + ITRetryUtil.retryUntil( + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + true, + 10000, + 30, + "Waiting for supervisor to be healthy" + ); + // Verify that supervisor can catch up with the stream + verifyIngestedData(supervisorId); + } + } + + private void verifyIngestedData(String supervisorId) throws Exception + { + // Wait for supervisor to consume events + LOG.info("Waiting for [%s] millis for Kinesis indexing tasks to consume events", WAIT_TIME_MILLIS); + Thread.sleep(WAIT_TIME_MILLIS); + // Query data + final String querySpec = streamQueryPropsTransform.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(supervisorId); + // wait for all indexing tasks to finish + LOG.info("Waiting for all indexing tasks to finish"); + ITRetryUtil.retryUntilTrue( + () -> (indexer.getPendingTasks().size() + + indexer.getRunningTasks().size() + + indexer.getWaitingTasks().size()) == 0, + "Waiting for Tasks Completion" + ); + // wait for segments to be handed off + ITRetryUtil.retryUntil( + () -> coordinator.areSegmentsLoaded(fullDatasourceName), + 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; + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java index 30e4bab23e1c..d3b6ca8a060d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java @@ -19,45 +19,7 @@ package org.apache.druid.tests.indexer; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -@Test(groups = TestNGGroup.KAFKA_INDEX) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITKafkaIndexingServiceTest extends AbstractKafkaIndexerTest +abstract class ITKafkaIndexingServiceTest extends AbstractIndexerTest { - private static final Logger LOG = new Logger(ITKafkaIndexingServiceTest.class); - private static final String DATASOURCE = "kafka_indexing_service_test"; - - @DataProvider - public static Object[][] testParams() - { - return new Object[][]{ - {"legacy_parser"}, - {"input_format"} - }; - } - - @Test(dataProvider = "testParams") - public void testKafka(String param) - { - final String supervisorSpecPath = "legacy_parser".equals(param) - ? INDEXER_FILE_LEGACY_PARSER - : INDEXER_FILE_INPUT_FORMAT; - LOG.info("Starting test: ITKafkaIndexingServiceTest"); - doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, false); - } - @AfterMethod - public void afterClass() - { - LOG.info("teardown"); - doTearDown(); - } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java deleted file mode 100644 index f32b82433f1a..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -/** - * This is a test for the Kafka indexing service with transactional topics - */ -@Test(groups = TestNGGroup.KAFKA_INDEX) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITKafkaIndexingServiceTransactionalTest extends AbstractKafkaIndexerTest -{ - private static final Logger LOG = new Logger(ITKafkaIndexingServiceTransactionalTest.class); - private static final String DATASOURCE = "kafka_indexing_service_txn_test"; - - @DataProvider - public static Object[][] testParams() - { - return new Object[][]{ - {"legacy_parser"}, - {"input_format"} - }; - } - - @Test(dataProvider = "testParams") - public void testKafka(String param) - { - final String supervisorSpecPath = "legacy_parser".equals(param) - ? INDEXER_FILE_LEGACY_PARSER - : INDEXER_FILE_INPUT_FORMAT; - LOG.info("Starting test: ITKafkaIndexingServiceTransactionalTest"); - doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, true); - } - - @AfterMethod - public void afterClass() - { - LOG.info("teardown"); - doTearDown(); - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java index b539b5d547c2..54231654ec2c 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java @@ -19,102 +19,39 @@ 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.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.DruidClusterAdminClient; -import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.KinesisAdminClient; import org.apache.druid.testing.utils.KinesisEventWriter; -import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator; +import org.apache.druid.testing.utils.StreamAdminClient; +import org.apache.druid.testing.utils.StreamEventWriter; import org.apache.druid.tests.TestNGGroup; -import org.joda.time.DateTime; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Guice; import org.testng.annotations.Test; -import java.io.Closeable; -import java.util.Map; -import java.util.UUID; import java.util.function.Function; @Test(groups = TestNGGroup.KINESIS_INDEX) @Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest +public class ITKinesisIndexingServiceTest extends AbstractStreamIndexingTest { - private static final Logger LOG = new Logger(ITKinesisIndexingServiceTest.class); - private static final int KINESIS_SHARD_COUNT = 2; - // 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 long WAIT_TIME_MILLIS = 3 * 60 * 1000L; - private static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0); - 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"; - // format for the querying interval - private static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); - // format for the expected timestamp in a query response - private static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'"); - private static final int EVENTS_PER_SECOND = 6; - private static final long CYCLE_PADDING_MS = 100; - private static final int TOTAL_NUMBER_OF_SECOND = 10; - @Inject - private DruidClusterAdminClient druidClusterAdminClient; - - private String streamName; - private String fullDatasourceName; - private KinesisAdminClient kinesisAdminClient; - private KinesisEventWriter kinesisEventWriter; - private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; - private Function kinesisIngestionPropsTransform; - private Function kinesisQueryPropsTransform; - private String supervisorId; - private int secondsToGenerateRemaining; - - @BeforeClass - public void beforeClass() throws Exception + @Override + StreamAdminClient getStreamAdminClient() throws Exception { - kinesisAdminClient = new KinesisAdminClient(config.getStreamEndpoint()); - kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), false); - wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); + return new KinesisAdminClient(config.getStreamEndpoint()); } - @AfterClass - public void tearDown() + @Override + StreamEventWriter getStreamEventWriter() throws Exception { - wikipediaStreamEventGenerator.shutdown(); - kinesisEventWriter.shutdown(); + return new KinesisEventWriter(config.getStreamEndpoint(), false); } - @BeforeMethod - public void before() + @Override + Function getStreamIngestionPropsTransform() { - streamName = "kinesis_index_test_" + UUID.randomUUID(); - String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); - Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTimes.nowUtc().plusMinutes(30).getMillis())); - kinesisAdminClient.createStream(streamName, KINESIS_SHARD_COUNT, tags); - ITRetryUtil.retryUntil( - () -> kinesisAdminClient.isStreamActive(streamName), - true, - 10000, - 30, - "Wait for stream active" - ); - secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND; - fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); - kinesisIngestionPropsTransform = spec -> { + return spec -> { try { spec = StringUtils.replace( spec, @@ -156,7 +93,12 @@ public void before() throw new RuntimeException(e); } }; - kinesisQueryPropsTransform = spec -> { + } + + @Override + Function getStreamQueryPropsTransform() + { + return spec -> { try { spec = StringUtils.replace( spec, @@ -210,271 +152,51 @@ public void before() }; } - @AfterMethod - public void teardown() - { - try { - kinesisEventWriter.flush(); - indexer.shutdownSupervisor(supervisorId); - } - catch (Exception e) { - // Best effort cleanup as the supervisor may have already went Bye-Bye - } - try { - unloader(fullDatasourceName); - } - catch (Exception e) { - // Best effort cleanup as the datasource may have already went Bye-Bye - } - try { - kinesisAdminClient.deleteStream(streamName); - } - catch (Exception e) { - // Best effort cleanup as the stream may have already went Bye-Bye - } - } - @Test public void testKinesisIndexDataWithLegacyParserStableState() throws Exception { - try ( - final Closeable ignored1 = unloader(fullDatasourceName) - ) { - final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER)); - LOG.info("supervisorSpec: [%s]\n", taskSpec); - // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); - } + doTestIndexDataWithLegacyParserStableState(); } @Test public void testKinesisIndexDataWithInputFormatStableState() throws Exception { - try ( - final Closeable ignored1 = unloader(fullDatasourceName) - ) { - final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); - LOG.info("supervisorSpec: [%s]\n", taskSpec); - // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); - } + doTestIndexDataWithInputFormatStableState(); } @Test public void testKinesisIndexDataWithLosingCoordinator() throws Exception { - testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady()); + doTestIndexDataWithLosingCoordinator(); } @Test public void testKinesisIndexDataWithLosingOverlord() throws Exception { - testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady()); + doTestIndexDataWithLosingOverlord(); } @Test public void testKinesisIndexDataWithLosingHistorical() throws Exception { - testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady()); + doTestIndexDataWithLosingHistorical(); } @Test public void testKinesisIndexDataWithStartStopSupervisor() throws Exception { - try ( - final Closeable ignored1 = unloader(fullDatasourceName) - ) { - final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); - LOG.info("supervisorSpec: [%s]\n", taskSpec); - // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start generating half of the data - int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2; - secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); - // Verify supervisor is healthy before suspension - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - true, - 10000, - 30, - "Waiting for supervisor to be healthy" - ); - // Suspend the supervisor - indexer.suspendSupervisor(supervisorId); - // Start generating remainning half of the data - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); - // Resume the supervisor - indexer.resumeSupervisor(supervisorId); - // Verify supervisor is healthy after suspension - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - true, - 10000, - 30, - "Waiting for supervisor to be healthy" - ); - // Verify that supervisor can catch up with the stream - verifyIngestedData(supervisorId); - } + doTestIndexDataWithStartStopSupervisor(); } @Test public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception { - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 - testIndexWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); + doTestIndexDataWithKinesisReshardSplit(); } @Test public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception { - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 - testIndexWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); - } - - private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception - { - try ( - final Closeable ignored1 = unloader(fullDatasourceName) - ) { - final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); - LOG.info("supervisorSpec: [%s]\n", taskSpec); - // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start generating one third of the data (before restarting) - int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; - secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); - // Verify supervisor is healthy before restart - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - 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(streamName, kinesisEventWriter, 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(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); - // Verify supervisor is healthy - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - true, - 10000, - 30, - "Waiting for supervisor to be healthy" - ); - // Verify that supervisor ingested all data - verifyIngestedData(supervisorId); - } - } - - private void testIndexWithKinesisReshardHelper(int newShardCount) throws Exception - { - try ( - final Closeable ignored1 = unloader(fullDatasourceName) - ) { - final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); - LOG.info("supervisorSpec: [%s]\n", taskSpec); - // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start generating one third of the data (before resharding) - int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; - secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); - // Verify supervisor is healthy before resahrding - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - true, - 10000, - 30, - "Waiting for supervisor to be healthy" - ); - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to newShardCount and waits until the resharding starts - kinesisAdminClient.updateShardCount(streamName, newShardCount, true); - // Start generating one third of the data (while resharding) - int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; - secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); - // Wait for kinesis stream to finish resharding - ITRetryUtil.retryUntil( - () -> kinesisAdminClient.isStreamActive(streamName), - true, - 10000, - 30, - "Waiting for Kinesis stream to finish resharding" - ); - // Start generating remainding data (after resharding) - wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); - // Verify supervisor is healthy after resahrding - ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), - true, - 10000, - 30, - "Waiting for supervisor to be healthy" - ); - // Verify that supervisor can catch up with the stream - verifyIngestedData(supervisorId); - } - } - - private void verifyIngestedData(String supervisorId) throws Exception - { - // Wait for supervisor to consume events - LOG.info("Waiting for [%s] millis for Kinesis indexing tasks to consume events", WAIT_TIME_MILLIS); - Thread.sleep(WAIT_TIME_MILLIS); - // Query data - final String querySpec = kinesisQueryPropsTransform.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(supervisorId); - // wait for all Kinesis indexing tasks to finish - LOG.info("Waiting for all indexing tasks to finish"); - ITRetryUtil.retryUntilTrue( - () -> (indexer.getPendingTasks().size() - + indexer.getRunningTasks().size() - + indexer.getWaitingTasks().size()) == 0, - "Waiting for Tasks Completion" - ); - // wait for segments to be handed off - ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - 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); - } - private long getSumOfEventSequence(int numEvents) - { - return (numEvents * (1 + numEvents)) / 2; + doTestIndexDataWithKinesisReshardMerge(); } } From 3cd2135246cc3946d3f363d4e34efe355b7a1711 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 16 Apr 2020 23:20:16 -1000 Subject: [PATCH 03/17] fix typo refactor --- .../tests/indexer/AbstractStreamIndexingTest.java | 15 +++++++-------- .../indexer/ITKinesisIndexingServiceTest.java | 4 ++-- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index f4025ddb763f..7f00f9290a49 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -51,8 +51,7 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes 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; - - private static final Logger LOG = new Logger(AbstractStreamIndexingTest.class); + 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. @@ -230,13 +229,13 @@ void doTestIndexDataWithStartStopSupervisor() throws Exception } } - void doTestIndexDataWithKinesisReshardSplit() throws Exception + void doTestIndexDataWithStreamReshardSplit() throws Exception { // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2 testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2); } - void doTestIndexDataWithKinesisReshardMerge() throws Exception + void doTestIndexDataWithStreamReshardMerge() throws Exception { // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT / 2 testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT / 2); @@ -319,20 +318,20 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); - // Wait for kinesis stream to finish resharding + // Wait for stream to finish resharding ITRetryUtil.retryUntil( () -> streamAdminClient.isStreamActive(streamName), true, 10000, 30, - "Waiting for Kinesis stream to finish resharding" + "Waiting for stream to finish resharding" ); ITRetryUtil.retryUntil( () -> streamAdminClient.getStreamShardCount(streamName) == newShardCount, true, 10000, 30, - "Waiting for Kinesis stream to finish resharding" + "Waiting for stream to finish resharding" ); // Start generating remainding data (after resharding) wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); @@ -352,7 +351,7 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio private void verifyIngestedData(String supervisorId) throws Exception { // Wait for supervisor to consume events - LOG.info("Waiting for [%s] millis for Kinesis indexing tasks to consume events", WAIT_TIME_MILLIS); + 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 = streamQueryPropsTransform.apply(getResourceAsString(QUERIES_FILE)); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java index 54231654ec2c..a55d83816f0a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java @@ -191,12 +191,12 @@ public void testKinesisIndexDataWithStartStopSupervisor() throws Exception @Test public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception { - doTestIndexDataWithKinesisReshardSplit(); + doTestIndexDataWithStreamReshardSplit(); } @Test public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception { - doTestIndexDataWithKinesisReshardMerge(); + doTestIndexDataWithStreamReshardMerge(); } } From 8714cff257fb57f976bca760a5c1eea18ab459a5 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 17 Apr 2020 21:56:20 -1000 Subject: [PATCH 04/17] parallel --- integration-tests/pom.xml | 6 +- .../druid/testing/utils/KafkaAdminClient.java | 5 ++ .../testing/utils/KinesisAdminClient.java | 8 ++ .../testing/utils/StreamAdminClient.java | 2 + ...> AbstractKinesisIndexingServiceTest.java} | 61 ++----------- .../indexer/AbstractStreamIndexingTest.java | 33 +++---- ...TKinesisIndexingServiceSerializedTest.java | 76 ++++++++++++++++ ...inesisIndexingServiceParallelizedTest.java | 89 +++++++++++++++++++ .../{testng.xml => testng-parallelized.xml} | 20 ++--- .../src/test/resources/testng-serialized.xml | 35 ++++++++ 10 files changed, 247 insertions(+), 88 deletions(-) rename integration-tests/src/test/java/org/apache/druid/tests/indexer/{ITKinesisIndexingServiceTest.java => AbstractKinesisIndexingServiceTest.java} (75%) create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java rename integration-tests/src/test/resources/{testng.xml => testng-parallelized.xml} (71%) create mode 100644 integration-tests/src/test/resources/testng-serialized.xml diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index ede1de0abfaa..5f7f1f68ed38 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -432,7 +432,8 @@ plain - src/test/resources/testng.xml + src/test/resources/testng-serialized.xml + src/test/resources/testng-parallelized.xml @@ -487,7 +488,8 @@ -Ddruid.client.https.keyStorePassword=druid123 - src/test/resources/testng.xml + src/test/resources/testng-serialized.xml + src/test/resources/testng-parallelized.xml diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java index dc0cf5213dae..a20c2b95a142 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java @@ -95,4 +95,9 @@ public int getStreamShardCount(String streamName) throws Exception TopicDescription topicDescription = result.values().get(streamName).get(); return topicDescription.partitions().size(); } + + @Override + public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception { + return getStreamShardCount(streamName) == newShardCount; + } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java index 6833947b39ac..b4938e90811c 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java @@ -146,6 +146,14 @@ public int getStreamShardCount(String streamName) return getStreamShardCount(streamDescription); } + @Override + public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) + { + int actualShardCount = getStreamShardCount(streamName); + return actualShardCount == oldShardCount + newShardCount; + } + + private boolean verifyStreamStatus(StreamDescription streamDescription, StreamStatus streamStatusToCheck) { return streamStatusToCheck.toString().equals(streamDescription.getStreamStatus()); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java index fc0ba05739aa..aa409bdf4a27 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java @@ -32,4 +32,6 @@ public interface StreamAdminClient boolean isStreamActive(String streamName) throws Exception; int getStreamShardCount(String streamName) throws Exception; + + boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception; } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java similarity index 75% rename from integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java rename to integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java index a55d83816f0a..70d5d0f34987 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java @@ -20,21 +20,20 @@ package org.apache.druid.tests.indexer; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.testing.utils.KinesisAdminClient; import org.apache.druid.testing.utils.KinesisEventWriter; import org.apache.druid.testing.utils.StreamAdminClient; import org.apache.druid.testing.utils.StreamEventWriter; -import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; import java.util.function.Function; -@Test(groups = TestNGGroup.KINESIS_INDEX) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITKinesisIndexingServiceTest extends AbstractStreamIndexingTest +public abstract class AbstractKinesisIndexingServiceTest extends AbstractStreamIndexingTest { + @Override + String getTestNamePrefix() + { + return "kinesis"; + } @Override StreamAdminClient getStreamAdminClient() throws Exception @@ -151,52 +150,4 @@ Function getStreamQueryPropsTransform() } }; } - - @Test - public void testKinesisIndexDataWithLegacyParserStableState() throws Exception - { - doTestIndexDataWithLegacyParserStableState(); - } - - @Test - public void testKinesisIndexDataWithInputFormatStableState() throws Exception - { - doTestIndexDataWithInputFormatStableState(); - } - - @Test - public void testKinesisIndexDataWithLosingCoordinator() throws Exception - { - doTestIndexDataWithLosingCoordinator(); - } - - @Test - public void testKinesisIndexDataWithLosingOverlord() throws Exception - { - doTestIndexDataWithLosingOverlord(); - } - - @Test - public void testKinesisIndexDataWithLosingHistorical() throws Exception - { - doTestIndexDataWithLosingHistorical(); - } - - @Test - public void testKinesisIndexDataWithStartStopSupervisor() throws Exception - { - doTestIndexDataWithStartStopSupervisor(); - } - - @Test - public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception - { - doTestIndexDataWithStreamReshardSplit(); - } - - @Test - public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception - { - doTestIndexDataWithStreamReshardMerge(); - } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 7f00f9290a49..9078afea56b4 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -32,10 +32,6 @@ import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import java.io.Closeable; import java.util.Map; @@ -81,27 +77,25 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes abstract StreamEventWriter getStreamEventWriter() throws Exception; abstract Function getStreamIngestionPropsTransform(); abstract Function getStreamQueryPropsTransform(); + abstract String getTestNamePrefix(); - @BeforeClass - public void beforeClass() throws Exception + protected void doBeforeClass() throws Exception { streamAdminClient = getStreamAdminClient(); streamEventWriter = getStreamEventWriter(); wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); } - @AfterClass - public void tearDown() + protected void doClassTeardown() { wikipediaStreamEventGenerator.shutdown(); streamEventWriter.shutdown(); } - @BeforeMethod - public void before() throws Exception + protected void doBefore() throws Exception { - streamName = "kinesis_index_test_" + UUID.randomUUID(); - String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); + streamName = getTestNamePrefix() + "_index_test_" + UUID.randomUUID(); + String datasource = getTestNamePrefix() + "_indexing_service_test_" + UUID.randomUUID(); Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTimes.nowUtc().plusMinutes(30).getMillis())); streamAdminClient.createStream(streamName, STREAM_SHARD_COUNT, tags); ITRetryUtil.retryUntil( @@ -117,8 +111,7 @@ public void before() throws Exception streamQueryPropsTransform = getStreamQueryPropsTransform(); } - @AfterMethod - public void teardown() + protected void doMethodTeardown() { try { streamEventWriter.flush(); @@ -141,7 +134,7 @@ public void teardown() } } - void doTestIndexDataWithLegacyParserStableState() throws Exception + protected void doTestIndexDataWithLegacyParserStableState() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -157,7 +150,7 @@ void doTestIndexDataWithLegacyParserStableState() throws Exception } } - void doTestIndexDataWithInputFormatStableState() throws Exception + protected void doTestIndexDataWithInputFormatStableState() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -188,7 +181,7 @@ void doTestIndexDataWithLosingHistorical() throws Exception testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady()); } - void doTestIndexDataWithStartStopSupervisor() throws Exception + protected void doTestIndexDataWithStartStopSupervisor() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -229,13 +222,13 @@ void doTestIndexDataWithStartStopSupervisor() throws Exception } } - void doTestIndexDataWithStreamReshardSplit() throws Exception + protected void doTestIndexDataWithStreamReshardSplit() throws Exception { // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT * 2 testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT * 2); } - void doTestIndexDataWithStreamReshardMerge() throws Exception + protected void doTestIndexDataWithStreamReshardMerge() throws Exception { // Reshard the stream from STREAM_SHARD_COUNT to STREAM_SHARD_COUNT / 2 testIndexWithStreamReshardHelper(STREAM_SHARD_COUNT / 2); @@ -327,7 +320,7 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio "Waiting for stream to finish resharding" ); ITRetryUtil.retryUntil( - () -> streamAdminClient.getStreamShardCount(streamName) == newShardCount, + () -> streamAdminClient.verfiyShardCountUpdated(streamName, STREAM_SHARD_COUNT, newShardCount), true, 10000, 30, diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java new file mode 100644 index 000000000000..2a40627639e1 --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java @@ -0,0 +1,76 @@ +/* + * 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.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +@Test(groups = TestNGGroup.KINESIS_INDEX) +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKinesisIndexingServiceSerializedTest extends AbstractKinesisIndexingServiceTest +{ + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @BeforeMethod + public void before() throws Exception + { + doBefore(); + } + + @AfterMethod + public void teardown() + { + doMethodTeardown(); + } + + @Test + public void testKinesisIndexDataWithLosingCoordinator() throws Exception + { + doTestIndexDataWithLosingCoordinator(); + } + + @Test + public void testKinesisIndexDataWithLosingOverlord() throws Exception + { + doTestIndexDataWithLosingOverlord(); + } + + @Test + public void testKinesisIndexDataWithLosingHistorical() throws Exception + { + doTestIndexDataWithLosingHistorical(); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java new file mode 100644 index 000000000000..8dbc29cb8e09 --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java @@ -0,0 +1,89 @@ +/* + * 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.AbstractKinesisIndexingServiceTest; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +@Test(groups = TestNGGroup.KINESIS_INDEX) +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKinesisIndexingServiceParallelizedTest extends AbstractKinesisIndexingServiceTest +{ + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @BeforeMethod + public void before() throws Exception + { + doBefore(); + } + + @AfterMethod + public void teardown() + { + doMethodTeardown(); + } + + @Test + public void testKinesisIndexDataWithLegacyParserStableState() throws Exception + { + doTestIndexDataWithLegacyParserStableState(); + } + + @Test + public void testKinesisIndexDataWithInputFormatStableState() throws Exception + { + doTestIndexDataWithInputFormatStableState(); + } + + @Test + public void testKinesisIndexDataWithStartStopSupervisor() throws Exception + { + doTestIndexDataWithStartStopSupervisor(); + } + + @Test + public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception + { + doTestIndexDataWithStreamReshardSplit(); + } + + @Test + public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception + { + doTestIndexDataWithStreamReshardMerge(); + } +} diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng-parallelized.xml similarity index 71% rename from integration-tests/src/test/resources/testng.xml rename to integration-tests/src/test/resources/testng-parallelized.xml index 333029c363c3..ed767a136fd8 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng-parallelized.xml @@ -20,15 +20,13 @@ - - - - - - - - - - - + + + + + + + + + diff --git a/integration-tests/src/test/resources/testng-serialized.xml b/integration-tests/src/test/resources/testng-serialized.xml new file mode 100644 index 000000000000..e3226382e23e --- /dev/null +++ b/integration-tests/src/test/resources/testng-serialized.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + + + From 2c97efd81d55b181e69d64f7868fcea5bb046b5e Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 18 Apr 2020 01:21:28 -1000 Subject: [PATCH 05/17] parallel --- integration-tests/pom.xml | 6 +- .../druid/testing/utils/SuiteListener.java | 65 +++++++++++++++++++ .../org/testng/DruidTestRunnerFactory.java | 42 ------------ .../test/resources/testng-parallelized.xml | 32 --------- .../{testng-serialized.xml => testng.xml} | 8 ++- 5 files changed, 74 insertions(+), 79 deletions(-) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/SuiteListener.java delete mode 100644 integration-tests/src/test/resources/testng-parallelized.xml rename integration-tests/src/test/resources/{testng-serialized.xml => testng.xml} (83%) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 5f7f1f68ed38..ede1de0abfaa 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -432,8 +432,7 @@ plain - src/test/resources/testng-serialized.xml - src/test/resources/testng-parallelized.xml + src/test/resources/testng.xml @@ -488,8 +487,7 @@ -Ddruid.client.https.keyStorePassword=druid123 - src/test/resources/testng-serialized.xml - src/test/resources/testng-parallelized.xml + src/test/resources/testng.xml diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SuiteListener.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SuiteListener.java new file mode 100644 index 000000000000..6259156b03b2 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SuiteListener.java @@ -0,0 +1,65 @@ +/* + * 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.inject.Injector; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.DruidTestModuleFactory; +import org.testng.ISuite; +import org.testng.ISuiteListener; + +public class SuiteListener implements ISuiteListener +{ + private static final Logger LOG = new Logger(SuiteListener.class); + + @Override + public void onStart(ISuite suite) + { + Injector injector = DruidTestModuleFactory.getInjector(); + IntegrationTestingConfig config = injector.getInstance(IntegrationTestingConfig.class); + DruidClusterAdminClient druidClusterAdminClient = injector.getInstance(DruidClusterAdminClient.class); + + druidClusterAdminClient.waitUntilCoordinatorReady(); + druidClusterAdminClient.waitUntilIndexerReady(); + druidClusterAdminClient.waitUntilBrokerReady(); + String routerHost = config.getRouterUrl(); + if (null != routerHost) { + druidClusterAdminClient.waitUntilRouterReady(); + } + Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + try { + lifecycle.start(); + } + catch (Exception e) { + LOG.error(e, ""); + throw new RuntimeException(e); + } + } + + @Override + public void onFinish(ISuite suite) + { + Injector injector = DruidTestModuleFactory.getInjector(); + Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + lifecycle.stop(); + } +} diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java index 53ef663b5c33..b71dfc16e017 100644 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -19,12 +19,6 @@ package /*CHECKSTYLE.OFF: PackageName*/org.testng/*CHECKSTYLE.ON: PackageName*/; -import com.google.inject.Injector; -import org.apache.druid.java.util.common.lifecycle.Lifecycle; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.DruidClusterAdminClient; import org.testng.internal.IConfiguration; import org.testng.internal.annotations.IAnnotationFinder; import org.testng.xml.XmlTest; @@ -36,8 +30,6 @@ */ public class DruidTestRunnerFactory implements ITestRunnerFactory { - private static final Logger LOG = new Logger(DruidTestRunnerFactory.class); - @Override public TestRunner newTestRunner(ISuite suite, XmlTest test, List listeners) { @@ -71,39 +63,5 @@ protected DruidTestRunner( { super(configuration, suite, test, outputDirectory, finder, skipFailedInvocationCounts, invokedMethodListeners); } - - @Override - public void run() - { - Injector injector = DruidTestModuleFactory.getInjector(); - IntegrationTestingConfig config = injector.getInstance(IntegrationTestingConfig.class); - DruidClusterAdminClient druidClusterAdminClient = injector.getInstance(DruidClusterAdminClient.class); - - druidClusterAdminClient.waitUntilCoordinatorReady(); - druidClusterAdminClient.waitUntilIndexerReady(); - druidClusterAdminClient.waitUntilBrokerReady(); - String routerHost = config.getRouterUrl(); - if (null != routerHost) { - druidClusterAdminClient.waitUntilRouterReady(); - } - Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - try { - lifecycle.start(); - runTests(); - } - catch (Exception e) { - LOG.error(e, ""); - throw new RuntimeException(e); - } - finally { - lifecycle.stop(); - } - - } - - private void runTests() - { - super.run(); - } } } diff --git a/integration-tests/src/test/resources/testng-parallelized.xml b/integration-tests/src/test/resources/testng-parallelized.xml deleted file mode 100644 index ed767a136fd8..000000000000 --- a/integration-tests/src/test/resources/testng-parallelized.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - - - - - - - - - diff --git a/integration-tests/src/test/resources/testng-serialized.xml b/integration-tests/src/test/resources/testng.xml similarity index 83% rename from integration-tests/src/test/resources/testng-serialized.xml rename to integration-tests/src/test/resources/testng.xml index e3226382e23e..6a468fdd70c7 100644 --- a/integration-tests/src/test/resources/testng-serialized.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -20,9 +20,10 @@ - + + @@ -32,4 +33,9 @@ + + + + + From 8a737c519f7763bb054ab8d3b9b4d6396c2e9eb4 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 18 Apr 2020 02:18:28 -1000 Subject: [PATCH 06/17] parallel --- integration-tests/README.md | 2 +- .../tests/indexer/AbstractKinesisIndexingServiceTest.java | 6 ------ .../druid/tests/indexer/AbstractStreamIndexingTest.java | 2 +- .../indexer/ITKinesisIndexingServiceSerializedTest.java | 6 ++++++ .../ITKinesisIndexingServiceParallelizedTest.java | 6 ++++++ integration-tests/src/test/resources/testng.xml | 2 +- 6 files changed, 15 insertions(+), 9 deletions(-) diff --git a/integration-tests/README.md b/integration-tests/README.md index f13ac9fd3015..226bdd1eab80 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -72,7 +72,7 @@ The file must contain one property per line, the key must start with `druid_` an ### Useful mvn command flags -- -Dskip.start.docker=true to skip starting docker containers. This can save ~6 minutes by skipping building and bringing +- -Dskip.start.docker=true to skip starting docker containers. This can save ~3 minutes by skipping building and bringing up the docker containers (Druid, Kafka, Hadoop, MYSQL, zookeeper, etc). Please make sure that you actually do have these containers already running if using this flag. Additionally, please make sure that the running containers are in the same state that the setup script (run_cluster.sh) would have brought it up in. diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java index 70d5d0f34987..dfb107973b2b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java @@ -29,12 +29,6 @@ public abstract class AbstractKinesisIndexingServiceTest extends AbstractStreamIndexingTest { - @Override - String getTestNamePrefix() - { - return "kinesis"; - } - @Override StreamAdminClient getStreamAdminClient() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 9078afea56b4..79b1ac44f136 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -77,7 +77,7 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes abstract StreamEventWriter getStreamEventWriter() throws Exception; abstract Function getStreamIngestionPropsTransform(); abstract Function getStreamQueryPropsTransform(); - abstract String getTestNamePrefix(); + public abstract String getTestNamePrefix(); protected void doBeforeClass() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java index 2a40627639e1..51c3511d50e6 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java @@ -32,6 +32,12 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKinesisIndexingServiceSerializedTest extends AbstractKinesisIndexingServiceTest { + @Override + public String getTestNamePrefix() + { + return "kinesis_serialized"; + } + @BeforeClass public void beforeClass() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java index 8dbc29cb8e09..8951756b3d2f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java @@ -33,6 +33,12 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKinesisIndexingServiceParallelizedTest extends AbstractKinesisIndexingServiceTest { + @Override + public String getTestNamePrefix() + { + return "kinesis_parallelized"; + } + @BeforeClass public void beforeClass() throws Exception { diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml index 6a468fdd70c7..b5c255da9fcd 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -33,7 +33,7 @@ - + From e12449d9ca951db4b51b0082bd6032abe394ac3f Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 18 Apr 2020 12:36:07 -1000 Subject: [PATCH 07/17] parallel works now --- integration-tests/pom.xml | 12 - .../clients/OverlordResourceTestClient.java | 10 + .../testing/guice/DruidTestModuleFactory.java | 1 - .../org/testng/DruidTestRunnerFactory.java | 67 ----- .../AbstractKinesisIndexingServiceTest.java | 4 +- .../indexer/AbstractStreamIndexingTest.java | 257 +++++++++++------- ...TKinesisIndexingServiceSerializedTest.java | 12 - ...inesisIndexingServiceParallelizedTest.java | 12 - pom.xml | 3 +- 9 files changed, 170 insertions(+), 208 deletions(-) delete mode 100644 integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index ede1de0abfaa..60532b7b2d6f 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -410,12 +410,6 @@ - - - testrunfactory - org.testng.DruidTestRunnerFactory - - -Duser.timezone=UTC -Dfile.encoding=UTF-8 @@ -468,12 +462,6 @@ - - - testrunfactory - org.testng.DruidTestRunnerFactory - - -Duser.timezone=UTC -Dfile.encoding=UTF-8 diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java index 20f6a76540d5..7995595bb2be 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java @@ -43,6 +43,7 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus; import java.net.URL; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; @@ -156,6 +157,15 @@ public List getCompleteTasksForDataSource(final String dataS return getTasks(StringUtils.format("tasks?state=complete&datasource=%s", StringUtils.urlEncode(dataSource))); } + public List getUncompletedTasksForDataSource(final String dataSource) + { + List uncompletedTasks = new ArrayList<>(); + uncompletedTasks.addAll(getTasks(StringUtils.format("tasks?state=pending&datasource=%s", StringUtils.urlEncode(dataSource)))); + uncompletedTasks.addAll(getTasks(StringUtils.format("tasks?state=running&datasource=%s", StringUtils.urlEncode(dataSource)))); + uncompletedTasks.addAll(getTasks(StringUtils.format("tasks?state=waiting&datasource=%s", StringUtils.urlEncode(dataSource)))); + return uncompletedTasks; + } + private List getTasks(String identifier) { try { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java index e082ffc9f66f..2c3b59d91522 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java @@ -57,7 +57,6 @@ private static List getModules() @Override public Module createModule(ITestContext context, Class testClass) { - context.addGuiceModule(DruidTestModule.class, MODULE); context.addInjector(Collections.singletonList(MODULE), INJECTOR); return MODULE; } diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java deleted file mode 100644 index b71dfc16e017..000000000000 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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 /*CHECKSTYLE.OFF: PackageName*/org.testng/*CHECKSTYLE.ON: PackageName*/; - -import org.testng.internal.IConfiguration; -import org.testng.internal.annotations.IAnnotationFinder; -import org.testng.xml.XmlTest; - -import java.util.List; - -/** - * This class must be in package org.testng to access protected methods like TestNG.getDefault().getConfiguration() - */ -public class DruidTestRunnerFactory implements ITestRunnerFactory -{ - @Override - public TestRunner newTestRunner(ISuite suite, XmlTest test, List listeners) - { - IConfiguration configuration = TestNG.getDefault().getConfiguration(); - String outputDirectory = suite.getOutputDirectory(); - IAnnotationFinder annotationFinder = configuration.getAnnotationFinder(); - Boolean skipFailedInvocationCounts = suite.getXmlSuite().skipFailedInvocationCounts(); - return new DruidTestRunner( - configuration, - suite, - test, - outputDirectory, - annotationFinder, - skipFailedInvocationCounts, - listeners - ); - } - - private static class DruidTestRunner extends TestRunner - { - - protected DruidTestRunner( - IConfiguration configuration, - ISuite suite, - XmlTest test, - String outputDirectory, - IAnnotationFinder finder, - boolean skipFailedInvocationCounts, - List invokedMethodListeners - ) - { - super(configuration, suite, test, outputDirectory, finder, skipFailedInvocationCounts, invokedMethodListeners); - } - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java index dfb107973b2b..e2b524121825 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java @@ -42,7 +42,7 @@ StreamEventWriter getStreamEventWriter() throws Exception } @Override - Function getStreamIngestionPropsTransform() + Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName) { return spec -> { try { @@ -89,7 +89,7 @@ Function getStreamIngestionPropsTransform() } @Override - Function getStreamQueryPropsTransform() + Function generateStreamQueryPropsTransform(String streamName, String fullDatasourceName) { return spec -> { try { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 79b1ac44f136..efc3617b7dda 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -24,6 +24,7 @@ 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.clients.TaskResponseObject; import org.apache.druid.testing.utils.DruidClusterAdminClient; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.StreamAdminClient; @@ -34,6 +35,8 @@ import org.joda.time.format.DateTimeFormatter; import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.function.Function; @@ -62,21 +65,14 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes @Inject private DruidClusterAdminClient druidClusterAdminClient; - String streamName; - String fullDatasourceName; - private StreamAdminClient streamAdminClient; private StreamEventWriter streamEventWriter; private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; - private Function streamIngestionPropsTransform; - private Function streamQueryPropsTransform; - private String supervisorId; - private int secondsToGenerateRemaining; abstract StreamAdminClient getStreamAdminClient() throws Exception; abstract StreamEventWriter getStreamEventWriter() throws Exception; - abstract Function getStreamIngestionPropsTransform(); - abstract Function getStreamQueryPropsTransform(); + abstract Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName); + abstract Function generateStreamQueryPropsTransform(String streamName, String fullDatasourceName); public abstract String getTestNamePrefix(); protected void doBeforeClass() throws Exception @@ -92,77 +88,43 @@ protected void doClassTeardown() streamEventWriter.shutdown(); } - protected void doBefore() throws Exception - { - streamName = getTestNamePrefix() + "_index_test_" + UUID.randomUUID(); - String datasource = getTestNamePrefix() + "_indexing_service_test_" + UUID.randomUUID(); - Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTimes.nowUtc().plusMinutes(30).getMillis())); - streamAdminClient.createStream(streamName, STREAM_SHARD_COUNT, tags); - ITRetryUtil.retryUntil( - () -> streamAdminClient.isStreamActive(streamName), - true, - 10000, - 30, - "Wait for stream active" - ); - secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND; - fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); - streamIngestionPropsTransform = getStreamIngestionPropsTransform(); - streamQueryPropsTransform = getStreamQueryPropsTransform(); - } - - protected void doMethodTeardown() - { - try { - streamEventWriter.flush(); - indexer.shutdownSupervisor(supervisorId); - } - catch (Exception e) { - // Best effort cleanup as the supervisor may have already went Bye-Bye - } - try { - unloader(fullDatasourceName); - } - catch (Exception e) { - // Best effort cleanup as the datasource may have already went Bye-Bye - } - try { - streamAdminClient.deleteStream(streamName); - } - catch (Exception e) { - // Best effort cleanup as the stream may have already went Bye-Bye - } - } - protected void doTestIndexDataWithLegacyParserStableState() throws Exception { + final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( - final Closeable ignored1 = unloader(fullDatasourceName) + final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) ) { - final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER)); + final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); + generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec)); LOG.info("Submitted supervisor"); // Start data generator - wikipediaStreamEventGenerator.start(streamName, streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + verifyIngestedData(generatedTestConfig); + } + finally { + doMethodTeardown(generatedTestConfig); } } protected void doTestIndexDataWithInputFormatStableState() throws Exception { + final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( - final Closeable ignored1 = unloader(fullDatasourceName) + final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) ) { - final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); + generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec)); LOG.info("Submitted supervisor"); // Start data generator - wikipediaStreamEventGenerator.start(streamName, streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + verifyIngestedData(generatedTestConfig); + } + finally { + doMethodTeardown(generatedTestConfig); } } @@ -183,42 +145,47 @@ void doTestIndexDataWithLosingHistorical() throws Exception protected void doTestIndexDataWithStartStopSupervisor() throws Exception { + final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( - final Closeable ignored1 = unloader(fullDatasourceName) + final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) ) { - final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); + 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(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before suspension ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), true, 10000, 30, "Waiting for supervisor to be healthy" ); // Suspend the supervisor - indexer.suspendSupervisor(supervisorId); + indexer.suspendSupervisor(generatedTestConfig.getSupervisorId()); // Start generating remainning half of the data - wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Resume the supervisor - indexer.resumeSupervisor(supervisorId); + indexer.resumeSupervisor(generatedTestConfig.getSupervisorId()); // Verify supervisor is healthy after suspension ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + () -> 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(supervisorId); + verifyIngestedData(generatedTestConfig); + } + finally { + doMethodTeardown(generatedTestConfig); } } @@ -236,21 +203,23 @@ protected void doTestIndexDataWithStreamReshardMerge() throws Exception private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception { + final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( - final Closeable ignored1 = unloader(fullDatasourceName) + final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) ) { - final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); + 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(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before restart ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), true, 10000, 30, @@ -263,106 +232,112 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa // Start generating one third of the data (while restarting) int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; - wikipediaStreamEventGenerator.start(streamName, streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + 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(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); // Verify supervisor is healthy ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), true, 10000, 30, "Waiting for supervisor to be healthy" ); // Verify that supervisor ingested all data - verifyIngestedData(supervisorId); + verifyIngestedData(generatedTestConfig); + } + finally { + doMethodTeardown(generatedTestConfig); } } private void testIndexWithStreamReshardHelper(int newShardCount) throws Exception { + final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( - final Closeable ignored1 = unloader(fullDatasourceName) + final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) ) { - final String taskSpec = streamIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + final String taskSpec = generatedTestConfig.getStreamIngestionPropsTransform().apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - supervisorId = indexer.submitSupervisor(taskSpec); + 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(streamName, streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before resahrding ITRetryUtil.retryUntil( - () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), + () -> 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(streamName, newShardCount, true); + 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(streamName, streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Wait for stream to finish resharding ITRetryUtil.retryUntil( - () -> streamAdminClient.isStreamActive(streamName), + () -> streamAdminClient.isStreamActive(generatedTestConfig.getStreamName()), true, 10000, 30, "Waiting for stream to finish resharding" ); ITRetryUtil.retryUntil( - () -> streamAdminClient.verfiyShardCountUpdated(streamName, STREAM_SHARD_COUNT, newShardCount), + () -> 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(streamName, streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + 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(supervisorId)), + () -> 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(supervisorId); + verifyIngestedData(generatedTestConfig); + } + finally { + doMethodTeardown(generatedTestConfig); } } - private void verifyIngestedData(String supervisorId) throws Exception + 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 = streamQueryPropsTransform.apply(getResourceAsString(QUERIES_FILE)); + 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(supervisorId); + indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId()); // wait for all indexing tasks to finish LOG.info("Waiting for all indexing tasks to finish"); ITRetryUtil.retryUntilTrue( - () -> (indexer.getPendingTasks().size() - + indexer.getRunningTasks().size() - + indexer.getWaitingTasks().size()) == 0, + () -> (indexer.getUncompletedTasksForDataSource(generatedTestConfig.getFullDatasourceName()).size() == 0), "Waiting for Tasks Completion" ); // wait for segments to be handed off ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), + () -> coordinator.areSegmentsLoaded(generatedTestConfig.getFullDatasourceName()), true, 10000, 30, @@ -372,8 +347,90 @@ private void verifyIngestedData(String supervisorId) throws Exception // 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) + { + try { + streamEventWriter.flush(); + indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId()); + } + catch (Exception e) { + // Best effort cleanup as the supervisor may have already went Bye-Bye + } + try { + unloader(generatedTestConfig.getFullDatasourceName()); + } + catch (Exception e) { + // Best effort cleanup as the datasource may have already went Bye-Bye + } + try { + streamAdminClient.deleteStream(generatedTestConfig.getStreamName()); + } + catch (Exception e) { + // Best effort cleanup as the stream may have already went Bye-Bye + } + } + + private class GeneratedTestConfig + { + private String streamName; + private String fullDatasourceName; + private String supervisorId; + private Function streamIngestionPropsTransform; + private Function streamQueryPropsTransform; + + GeneratedTestConfig() throws Exception + { + streamName = getTestNamePrefix() + "_index_test_" + UUID.randomUUID(); + String datasource = getTestNamePrefix() + "_indexing_service_test_" + UUID.randomUUID(); + Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTimes.nowUtc().plusMinutes(30).getMillis())); + streamAdminClient.createStream(streamName, STREAM_SHARD_COUNT, tags); + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(streamName), + true, + 10000, + 30, + "Wait for stream active" + ); + fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); + streamIngestionPropsTransform = generateStreamIngestionPropsTransform(streamName, fullDatasourceName); + streamQueryPropsTransform = generateStreamQueryPropsTransform(streamName, fullDatasourceName); + } + + public String getSupervisorId() + { + return supervisorId; + } + + public void setSupervisorId(String supervisorId) + { + this.supervisorId = supervisorId; + } + + public String getStreamName() + { + return streamName; + } + + public String getFullDatasourceName() + { + return fullDatasourceName; + } + + public Function getStreamIngestionPropsTransform() + { + return streamIngestionPropsTransform; + } + + public Function getStreamQueryPropsTransform() + { + return streamQueryPropsTransform; + } + + } +} \ No newline at end of file diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java index 51c3511d50e6..a05238acb476 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java @@ -50,18 +50,6 @@ public void tearDown() doClassTeardown(); } - @BeforeMethod - public void before() throws Exception - { - doBefore(); - } - - @AfterMethod - public void teardown() - { - doMethodTeardown(); - } - @Test public void testKinesisIndexDataWithLosingCoordinator() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java index 8951756b3d2f..fdebb801fc6f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java @@ -51,18 +51,6 @@ public void tearDown() doClassTeardown(); } - @BeforeMethod - public void before() throws Exception - { - doBefore(); - } - - @AfterMethod - public void teardown() - { - doMethodTeardown(); - } - @Test public void testKinesisIndexDataWithLegacyParserStableState() throws Exception { diff --git a/pom.xml b/pom.xml index 4ab46a8884fe..6c98e55ff6e6 100644 --- a/pom.xml +++ b/pom.xml @@ -1196,7 +1196,7 @@ org.testng testng - 6.8.7 + 6.14.3 org.hamcrest @@ -1253,7 +1253,6 @@ org/apache/druid/benchmark/**/* org/apache/druid/**/*Benchmark* - org/testng/DruidTestRunnerFactory* org/apache/druid/testing/**/* From fae658f2b1eddfd92e3d802118942bcfafae653a Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 18 Apr 2020 17:47:40 -1000 Subject: [PATCH 08/17] add kafka it --- .../druid/testing/utils/KafkaEventWriter.java | 31 ++-- .../testing/utils/KinesisEventWriter.java | 18 +- .../testing/utils/StreamEventWriter.java | 8 +- .../utils/SyntheticStreamGenerator.java | 8 + .../indexer/AbstractITBatchIndexTest.java | 2 +- .../AbstractKafkaIndexingServiceTest.java | 156 ++++++++++++++++++ .../indexer/AbstractStreamIndexingTest.java | 25 ++- ...ServiceNonTransactionalSerializedTest.java | 74 +++++++++ .../indexer/ITKafkaIndexingServiceTest.java | 25 --- ...ingServiceTransactionalSerializedTest.java | 74 +++++++++ ...rviceNonTransactionalParallelizedTest.java | 81 +++++++++ ...gServiceTransactionalParallelizedTest.java | 81 +++++++++ .../src/test/resources/testng.xml | 2 +- 13 files changed, 531 insertions(+), 54 deletions(-) create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java index eb2409bcb9d2..7ec88f23ea3e 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java @@ -59,32 +59,39 @@ public KafkaEventWriter(IntegrationTestingConfig config, boolean txnEnabled) new StringSerializer(), new StringSerializer() ); + if (txnEnabled) { + producer.initTransactions(); + } + } + + @Override + public boolean isTransactionEnabled() + { + return txnEnabled; } @Override - public void write(String topic, List events) throws Exception + public void initTransaction() { if (txnEnabled) { - producer.initTransactions(); producer.beginTransaction(); + } else { + throw new IllegalStateException("Kafka writer was initialized with transaction disabled"); } + } - for (String event : events) - { - write(topic, event); - } - + @Override + public void commitTransaction() + { if (txnEnabled) { producer.commitTransaction(); + } else { + throw new IllegalStateException("Kafka writer was initialized with transaction disabled"); } } - /** - * This method does not handle transaction. For transaction functionality use - * {@link #write(String, List future = producer.send(new ProducerRecord<>(topic, event)); pendingWriteRecords.add(future); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java index 735c13f5fa4d..7867cf43a08e 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java @@ -65,11 +65,21 @@ public KinesisEventWriter(String endpoint, boolean aggregate) throws Exception } @Override - public void write(String streamName, List events) + public boolean isTransactionEnabled() { - for (String event : events) { - write(streamName, event); - } + return false; + } + + @Override + public void initTransaction() + { + // No-Op as Kinesis does not support transaction + } + + @Override + public void commitTransaction() + { + // No-Op as Kinesis does not support transaction } @Override diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java index df9593d2a381..7b0f5194c81c 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java @@ -23,11 +23,15 @@ public interface StreamEventWriter { - void write(String topic, List events) throws Exception; - void write(String topic, String event) throws Exception; void shutdown(); void flush() throws Exception; + + boolean isTransactionEnabled(); + + void initTransaction(); + + void commitTransaction(); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java index 748a6ed2d3bc..460fe46a3fda 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java @@ -94,6 +94,10 @@ public void start(String streamTopic, StreamEventWriter streamEventWriter, int t nowCeilingToSecond.plusSeconds(1).minus(cyclePaddingMs) ); + if (streamEventWriter.isTransactionEnabled()) { + streamEventWriter.initTransaction(); + } + for (int i = 1; i <= eventsPerSecond; i++) { streamEventWriter.write(streamTopic, MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); @@ -107,6 +111,10 @@ public void start(String streamTopic, StreamEventWriter streamEventWriter, int t } } + if (streamEventWriter.isTransactionEnabled()) { + streamEventWriter.commitTransaction(); + } + nowCeilingToSecond = nowCeilingToSecond.plusSeconds(1); eventTimestamp = eventTimestamp.plusSeconds(1); seconds++; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index beaeac5330f2..cb6192a3a427 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -85,7 +85,7 @@ public String getFolderSuffix() private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class); @Inject - IntegrationTestingConfig config; + public IntegrationTestingConfig config; @Inject protected SqlTestQueryHelper sqlQueryHelper; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java new file mode 100644 index 000000000000..e14aa5222539 --- /dev/null +++ b/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() throws Exception; + + @Override + StreamAdminClient getStreamAdminClient() throws Exception + { + return new KafkaAdminClient(config.getKafkaHost()); + } + + @Override + public StreamEventWriter getStreamEventWriter() throws Exception + { + return new KafkaEventWriter(config, isKafkaWriterTransactionalEnabled()); + } + + @Override + Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName) + { + final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); + final Properties consumerProperties = new Properties(); + consumerProperties.putAll(consumerConfigs); + consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); + return spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%STREAM_TYPE%%", + "kafka" + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_KEY%%", + "topic" + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_VALUE%%", + streamName + ); + spec = StringUtils.replace( + spec, + "%%USE_EARLIEST_KEY%%", + "useEarliestOffset" + ); + spec = StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_KEY%%", + "consumerProperties" + ); + return StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_VALUE%%", + jsonMapper.writeValueAsString(consumerProperties) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + } + + @Override + Function generateStreamQueryPropsTransform(String streamName, String fullDatasourceName) + { + return spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", + TIMESTAMP_FMT.print(FIRST_EVENT_TIME) + ); + spec = StringUtils.replace( + spec, + "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", + TIMESTAMP_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND - 1)) + ); + spec = StringUtils.replace( + spec, + "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", + TIMESTAMP_FMT.print(FIRST_EVENT_TIME) + ); + spec = StringUtils.replace( + spec, + "%%TIMESERIES_QUERY_START%%", + INTERVAL_FMT.print(FIRST_EVENT_TIME) + ); + spec = StringUtils.replace( + spec, + "%%TIMESERIES_QUERY_END%%", + INTERVAL_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND - 1).plusMinutes(2)) + ); + spec = StringUtils.replace( + spec, + "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + TIMESTAMP_FMT.print(FIRST_EVENT_TIME) + ); + spec = StringUtils.replace( + spec, + "%%TIMESERIES_ADDED%%", + Long.toString(getSumOfEventSequence(EVENTS_PER_SECOND) * TOTAL_NUMBER_OF_SECOND) + ); + return StringUtils.replace( + spec, + "%%TIMESERIES_NUMEVENTS%%", + Integer.toString(EVENTS_PER_SECOND * TOTAL_NUMBER_OF_SECOND) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index efc3617b7dda..7573cf28e847 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -66,7 +66,6 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes private DruidClusterAdminClient druidClusterAdminClient; private StreamAdminClient streamAdminClient; - private StreamEventWriter streamEventWriter; private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; abstract StreamAdminClient getStreamAdminClient() throws Exception; @@ -78,18 +77,17 @@ public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTes protected void doBeforeClass() throws Exception { streamAdminClient = getStreamAdminClient(); - streamEventWriter = getStreamEventWriter(); wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); } protected void doClassTeardown() { wikipediaStreamEventGenerator.shutdown(); - streamEventWriter.shutdown(); } protected void doTestIndexDataWithLegacyParserStableState() throws Exception { + StreamEventWriter streamEventWriter = getStreamEventWriter(); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -104,12 +102,13 @@ protected void doTestIndexDataWithLegacyParserStableState() throws Exception verifyIngestedData(generatedTestConfig); } finally { - doMethodTeardown(generatedTestConfig); + doMethodTeardown(generatedTestConfig, streamEventWriter); } } protected void doTestIndexDataWithInputFormatStableState() throws Exception { + StreamEventWriter streamEventWriter = getStreamEventWriter(); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -124,7 +123,7 @@ protected void doTestIndexDataWithInputFormatStableState() throws Exception verifyIngestedData(generatedTestConfig); } finally { - doMethodTeardown(generatedTestConfig); + doMethodTeardown(generatedTestConfig, streamEventWriter); } } @@ -145,6 +144,7 @@ void doTestIndexDataWithLosingHistorical() throws Exception protected void doTestIndexDataWithStartStopSupervisor() throws Exception { + StreamEventWriter streamEventWriter = getStreamEventWriter(); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -185,7 +185,7 @@ protected void doTestIndexDataWithStartStopSupervisor() throws Exception verifyIngestedData(generatedTestConfig); } finally { - doMethodTeardown(generatedTestConfig); + doMethodTeardown(generatedTestConfig, streamEventWriter); } } @@ -203,6 +203,7 @@ protected void doTestIndexDataWithStreamReshardMerge() throws Exception private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception { + StreamEventWriter streamEventWriter = getStreamEventWriter(); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -251,12 +252,13 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa verifyIngestedData(generatedTestConfig); } finally { - doMethodTeardown(generatedTestConfig); + 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()) @@ -314,7 +316,7 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio verifyIngestedData(generatedTestConfig); } finally { - doMethodTeardown(generatedTestConfig); + doMethodTeardown(generatedTestConfig, streamEventWriter); } } @@ -353,10 +355,15 @@ long getSumOfEventSequence(int numEvents) return (numEvents * (1 + numEvents)) / 2; } - private void doMethodTeardown(GeneratedTestConfig generatedTestConfig) + 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 + } + try { indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId()); } catch (Exception e) { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java new file mode 100644 index 000000000000..b894f7f222f4 --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java @@ -0,0 +1,74 @@ +/* + * 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.KAFKA_INDEX) +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKafkaIndexingServiceNonTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest +{ + @Override + public boolean isKafkaWriterTransactionalEnabled() + { + return false; + } + + @Override + public String getTestNamePrefix() + { + return "kafka_nontransactional_serialized"; + } + + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @Test + public void testKafkaIndexDataWithLosingCoordinator() throws Exception + { + doTestIndexDataWithLosingCoordinator(); + } + + @Test + public void testKafkaIndexDataWithLosingOverlord() throws Exception + { + doTestIndexDataWithLosingOverlord(); + } + + @Test + public void testKafkaIndexDataWithLosingHistorical() throws Exception + { + doTestIndexDataWithLosingHistorical(); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java deleted file mode 100644 index d3b6ca8a060d..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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; - -abstract class ITKafkaIndexingServiceTest extends AbstractIndexerTest -{ - -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java new file mode 100644 index 000000000000..daa5562c1ff8 --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -0,0 +1,74 @@ +/* + * 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.KAFKA_INDEX) +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest +{ + @Override + public boolean isKafkaWriterTransactionalEnabled() + { + return true; + } + + @Override + public String getTestNamePrefix() + { + return "kafka_transactional_serialized"; + } + + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @Test + public void testKafkaIndexDataWithLosingCoordinator() throws Exception + { + doTestIndexDataWithLosingCoordinator(); + } + + @Test + public void testKafkaIndexDataWithLosingOverlord() throws Exception + { + doTestIndexDataWithLosingOverlord(); + } + + @Test + public void testKafkaIndexDataWithLosingHistorical() throws Exception + { + doTestIndexDataWithLosingHistorical(); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java new file mode 100644 index 000000000000..d367a608c452 --- /dev/null +++ b/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 +{ + @Override + public boolean isKafkaWriterTransactionalEnabled() + { + return false; + } + + @Override + public String getTestNamePrefix() + { + return "kafka_non_transactional_parallelized"; + } + + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @Test + public void testKafkaIndexDataWithLegacyParserStableState() throws Exception + { + doTestIndexDataWithLegacyParserStableState(); + } + + @Test + public void testKafkaIndexDataWithInputFormatStableState() throws Exception + { + doTestIndexDataWithInputFormatStableState(); + } + + @Test + public void testKafkaIndexDataWithStartStopSupervisor() throws Exception + { + doTestIndexDataWithStartStopSupervisor(); + } + + @Test + public void testKafkaIndexDataWithKafkaReshardSplit() throws Exception + { + doTestIndexDataWithStreamReshardSplit(); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java new file mode 100644 index 000000000000..efbe9c19df6b --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.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 ITKafkaIndexingServiceTransactionalParallelizedTest extends AbstractKafkaIndexingServiceTest +{ + @Override + public boolean isKafkaWriterTransactionalEnabled() + { + return true; + } + + @Override + public String getTestNamePrefix() + { + return "kafka_transactional_parallelized"; + } + + @BeforeClass + public void beforeClass() throws Exception + { + doBeforeClass(); + } + + @AfterClass + public void tearDown() + { + doClassTeardown(); + } + + @Test + public void testKafkaIndexDataWithLegacyParserStableState() throws Exception + { + doTestIndexDataWithLegacyParserStableState(); + } + + @Test + public void testKafkaIndexDataWithInputFormatStableState() throws Exception + { + doTestIndexDataWithInputFormatStableState(); + } + + @Test + public void testKafkaIndexDataWithStartStopSupervisor() throws Exception + { + doTestIndexDataWithStartStopSupervisor(); + } + + @Test + public void testKafkaIndexDataWithKafkaReshardSplit() throws Exception + { + doTestIndexDataWithStreamReshardSplit(); + } +} diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml index b5c255da9fcd..f83ae72d5465 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -33,7 +33,7 @@ - + From 33837300dee1eb0fb23c70f03303434bdf927765 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 18 Apr 2020 18:37:04 -1000 Subject: [PATCH 09/17] add doc to readme --- integration-tests/README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/integration-tests/README.md b/integration-tests/README.md index 226bdd1eab80..f2b0a13a6dd8 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -314,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" +test tag section in integration-tests/src/test/resources/testng.xml +Please be mindful when adding tests to the "AllParallelizedTests" test tag that the tests can run in parallel with +other tests at the same time. i.e. test does not modify/restart/stop the druid cluster or other dependecy containers, +test does not use excessive memory straving other concurent task, test does not modify and/or use other task, +supervisor, datasource it did not create. From 29c39cf9f8d97fedd6280a4b3a3f6839aedf5439 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 19 Apr 2020 14:25:35 -1000 Subject: [PATCH 10/17] fix tests --- .travis.yml | 13 +++++++++++++ integration-tests/pom.xml | 5 ----- .../druid/testing/utils/KafkaAdminClient.java | 8 ++++---- .../druid/testing/utils/KinesisEventWriter.java | 1 - .../druid/testing/utils/StreamAdminClient.java | 2 +- .../druid/testing/utils/StreamEventWriter.java | 4 +--- .../java/org/apache/druid/tests/TestNGGroup.java | 2 ++ .../indexer/AbstractKafkaIndexingServiceTest.java | 2 +- .../tests/indexer/AbstractStreamIndexingTest.java | 6 +----- ...aIndexingServiceTransactionalSerializedTest.java | 2 +- .../ITKinesisIndexingServiceSerializedTest.java | 2 -- ...ndexingServiceTransactionalParallelizedTest.java | 2 +- .../ITKinesisIndexingServiceParallelizedTest.java | 2 -- 13 files changed, 25 insertions(+), 26 deletions(-) diff --git a/.travis.yml b/.travis.yml index 45caa749a7a5..ac5f7f4b8ac7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -313,6 +313,14 @@ jobs: script: *run_integration_test after_failure: *integration_test_diags + - &integration_kafka_transactional_index + name: "(Compile=openjdk8, Run=openjdk8) transactional kafka index integration test" + jdk: openjdk8 + services: *integration_test_services + env: TESTNG_GROUPS='-Dgroups=kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=8' + script: *run_integration_test + after_failure: *integration_test_diags + - &integration_query name: "(Compile=openjdk8, Run=openjdk8) query integration test" jdk: openjdk8 @@ -362,6 +370,11 @@ jobs: jdk: openjdk8 env: TESTNG_GROUPS='-Dgroups=kafka-index' JVM_RUNTIME='-Djvm.runtime=11' + - <<: *integration_kafka_transactional_index + name: "(Compile=openjdk8, Run=openjdk11) transactional kafka index integration test" + jdk: openjdk8 + env: TESTNG_GROUPS='-Dgroups=kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=11' + - <<: *integration_query name: "(Compile=openjdk8, Run=openjdk11) query integration test" jdk: openjdk8 diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 60532b7b2d6f..9c4c429f7384 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -233,11 +233,6 @@ com.google.guava guava - - com.101tec - zkclient - 0.10 - javax.validation validation-api diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java index a20c2b95a142..411e5d65a901 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java @@ -39,7 +39,7 @@ public class KafkaAdminClient implements StreamAdminClient { AdminClient adminClient; - public KafkaAdminClient(String kafkaInternalHost) throws Exception + public KafkaAdminClient(String kafkaInternalHost) { Properties config = new Properties(); config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost); @@ -50,7 +50,6 @@ public KafkaAdminClient(String kafkaInternalHost) throws Exception public void createStream(String streamName, int partitionCount, Map tags) throws Exception { final short replicationFactor = 1; - Map configs = new HashMap<>(); final NewTopic newTopic = new NewTopic(streamName, partitionCount, replicationFactor); final CreateTopicsResult createTopicsResult = adminClient.createTopics(Collections.singleton(newTopic)); // Wait for create topic to compelte @@ -83,7 +82,7 @@ public void updateShardCount(String streamName, int newPartitionCount, boolean b } @Override - public boolean isStreamActive(String streamName) throws Exception + public boolean isStreamActive(String streamName) { return true; } @@ -97,7 +96,8 @@ public int getStreamShardCount(String streamName) throws Exception } @Override - public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception { + public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception + { return getStreamShardCount(streamName) == newShardCount; } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java index 7867cf43a08e..0377e9e42e0f 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java @@ -30,7 +30,6 @@ import java.io.FileInputStream; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.List; import java.util.Properties; public class KinesisEventWriter implements StreamEventWriter diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java index aa409bdf4a27..23e0e422b3d8 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java @@ -29,7 +29,7 @@ public interface StreamAdminClient void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception; - boolean isStreamActive(String streamName) throws Exception; + boolean isStreamActive(String streamName); int getStreamShardCount(String streamName) throws Exception; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java index 7b0f5194c81c..0baab934cbca 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java @@ -19,11 +19,9 @@ package org.apache.druid.testing.utils; -import java.util.List; - public interface StreamEventWriter { - void write(String topic, String event) throws Exception; + void write(String topic, String event); void shutdown(); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index c0116f9149e4..360787e12f4a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -31,6 +31,8 @@ public class TestNGGroup public static final String KAFKA_INDEX = "kafka-index"; + public static final String TRANSACTIONAL_KAFKA_INDEX = "kafka-transactional_index"; + public static final String OTHER_INDEX = "other-index"; public static final String PERFECT_ROLLUP_PARALLEL_BATCH_INDEX = "perfect-rollup-parallel-batch-index"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java index e14aa5222539..a1e7c47e51d1 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java @@ -32,7 +32,7 @@ public abstract class AbstractKafkaIndexingServiceTest extends AbstractStreamIndexingTest { - public abstract boolean isKafkaWriterTransactionalEnabled() throws Exception; + public abstract boolean isKafkaWriterTransactionalEnabled(); @Override StreamAdminClient getStreamAdminClient() throws Exception diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 7573cf28e847..228f7970d2a5 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -24,7 +24,6 @@ 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.clients.TaskResponseObject; import org.apache.druid.testing.utils.DruidClusterAdminClient; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.StreamAdminClient; @@ -35,8 +34,6 @@ import org.joda.time.format.DateTimeFormatter; import java.io.Closeable; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import java.util.UUID; import java.util.function.Function; @@ -438,6 +435,5 @@ public Function getStreamQueryPropsTransform() { return streamQueryPropsTransform; } - } -} \ No newline at end of file +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java index daa5562c1ff8..3f1df22e17bf 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -26,7 +26,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.KAFKA_INDEX) +@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java index a05238acb476..fc31e2fa99c4 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java @@ -22,9 +22,7 @@ import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Guice; import org.testng.annotations.Test; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java index efbe9c19df6b..362e16807636 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java @@ -27,7 +27,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.KAFKA_INDEX) +@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceTransactionalParallelizedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java index fdebb801fc6f..6b138bec1556 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java @@ -23,9 +23,7 @@ import org.apache.druid.tests.TestNGGroup; import org.apache.druid.tests.indexer.AbstractKinesisIndexingServiceTest; import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Guice; import org.testng.annotations.Test; From 3f7e05e4ee781a32848df2e568d03c96047926ac Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 19 Apr 2020 18:10:22 -1000 Subject: [PATCH 11/17] fix failing test --- .travis.yml | 4 ++-- .../druid/tests/indexer/AbstractKafkaIndexingServiceTest.java | 4 ++-- integration-tests/src/test/resources/testng.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index ac5f7f4b8ac7..83e6070b8f69 100644 --- a/.travis.yml +++ b/.travis.yml @@ -349,7 +349,7 @@ jobs: name: "(Compile=openjdk8, Run=openjdk8) other integration test" jdk: openjdk8 services: *integration_test_services - env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index' JVM_RUNTIME='-Djvm.runtime=8' + env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=8' script: *run_integration_test after_failure: *integration_test_diags # END - Integration tests for Compile with Java 8 and Run with Java 8 @@ -393,7 +393,7 @@ jobs: - <<: *integration_tests name: "(Compile=openjdk8, Run=openjdk11) other integration test" jdk: openjdk8 - env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index' JVM_RUNTIME='-Djvm.runtime=11' + env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=11' # END - Integration tests for Compile with Java 8 and Run with Java 11 - name: "security vulnerabilities" diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java index a1e7c47e51d1..1c5ea8bb3c9f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java @@ -35,13 +35,13 @@ public abstract class AbstractKafkaIndexingServiceTest extends AbstractStreamInd public abstract boolean isKafkaWriterTransactionalEnabled(); @Override - StreamAdminClient getStreamAdminClient() throws Exception + StreamAdminClient getStreamAdminClient() { return new KafkaAdminClient(config.getKafkaHost()); } @Override - public StreamEventWriter getStreamEventWriter() throws Exception + public StreamEventWriter getStreamEventWriter() { return new KafkaEventWriter(config, isKafkaWriterTransactionalEnabled()); } diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml index f83ae72d5465..5a0735a05782 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -33,7 +33,7 @@ - + From d0a8c1a30b8d88eb0176d00f84534a741018969f Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 19 Apr 2020 19:40:28 -1000 Subject: [PATCH 12/17] test --- .travis.yml | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/.travis.yml b/.travis.yml index 83e6070b8f69..2d8e46157786 100644 --- a/.travis.yml +++ b/.travis.yml @@ -287,7 +287,7 @@ jobs: - docker env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' script: &run_integration_test - - ${MVN} verify -pl integration-tests -P integration-tests ${TESTNG_GROUPS} ${JVM_RUNTIME} ${MAVEN_SKIP} + - travis_wait ${MVN} verify -pl integration-tests -P integration-tests ${TESTNG_GROUPS} ${JVM_RUNTIME} ${MAVEN_SKIP} after_failure: &integration_test_diags - for v in ~/shared/logs/*.log ; do echo $v logtail ======================== ; tail -100 $v ; @@ -365,16 +365,6 @@ jobs: jdk: openjdk8 env: TESTNG_GROUPS='-Dgroups=perfect-rollup-parallel-batch-index' JVM_RUNTIME='-Djvm.runtime=11' - - <<: *integration_kafka_index - name: "(Compile=openjdk8, Run=openjdk11) kafka index integration test" - jdk: openjdk8 - env: TESTNG_GROUPS='-Dgroups=kafka-index' JVM_RUNTIME='-Djvm.runtime=11' - - - <<: *integration_kafka_transactional_index - name: "(Compile=openjdk8, Run=openjdk11) transactional kafka index integration test" - jdk: openjdk8 - env: TESTNG_GROUPS='-Dgroups=kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=11' - - <<: *integration_query name: "(Compile=openjdk8, Run=openjdk11) query integration test" jdk: openjdk8 From 9febc083410b4967dc1e0b919a6690846281ed81 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 19 Apr 2020 23:27:53 -1000 Subject: [PATCH 13/17] test --- integration-tests/src/test/resources/testng.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml index 5a0735a05782..b5c255da9fcd 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -33,7 +33,7 @@ - + From 6db038ec35c92af13fe5ebb947091973bb50eb47 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 20 Apr 2020 00:18:43 -1000 Subject: [PATCH 14/17] test --- .travis.yml | 22 ++++++++++++++++--- .../org/apache/druid/tests/TestNGGroup.java | 6 ++++- ...ServiceNonTransactionalSerializedTest.java | 2 +- ...ingServiceTransactionalSerializedTest.java | 2 +- .../src/test/resources/testng.xml | 2 +- 5 files changed, 27 insertions(+), 7 deletions(-) diff --git a/.travis.yml b/.travis.yml index 2d8e46157786..c231f7b20a6a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -313,11 +313,27 @@ jobs: script: *run_integration_test after_failure: *integration_test_diags + - &integration_kafka_index_slow + name: "(Compile=openjdk8, Run=openjdk8) kafka index integration test slow" + jdk: openjdk8 + services: *integration_test_services + env: TESTNG_GROUPS='-Dgroups=kafka-index-slow' JVM_RUNTIME='-Djvm.runtime=8' + script: *run_integration_test + after_failure: *integration_test_diags + - &integration_kafka_transactional_index name: "(Compile=openjdk8, Run=openjdk8) transactional kafka index integration test" jdk: openjdk8 services: *integration_test_services - env: TESTNG_GROUPS='-Dgroups=kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=8' + env: TESTNG_GROUPS='-Dgroups=kafka-transactional-index' JVM_RUNTIME='-Djvm.runtime=8' + script: *run_integration_test + after_failure: *integration_test_diags + + - &integration_kafka_transactional_index_slow + name: "(Compile=openjdk8, Run=openjdk8) transactional kafka index integration test slow" + jdk: openjdk8 + services: *integration_test_services + env: TESTNG_GROUPS='-Dgroups=kafka-transactional-index-slow' JVM_RUNTIME='-Djvm.runtime=8' script: *run_integration_test after_failure: *integration_test_diags @@ -349,7 +365,7 @@ jobs: name: "(Compile=openjdk8, Run=openjdk8) other integration test" jdk: openjdk8 services: *integration_test_services - env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=8' + env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow' JVM_RUNTIME='-Djvm.runtime=8' script: *run_integration_test after_failure: *integration_test_diags # END - Integration tests for Compile with Java 8 and Run with Java 8 @@ -383,7 +399,7 @@ jobs: - <<: *integration_tests name: "(Compile=openjdk8, Run=openjdk11) other integration test" jdk: openjdk8 - env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional_index' JVM_RUNTIME='-Djvm.runtime=11' + env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow' JVM_RUNTIME='-Djvm.runtime=11' # END - Integration tests for Compile with Java 8 and Run with Java 11 - name: "security vulnerabilities" diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index 360787e12f4a..9d8b7fe0b4ea 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -31,7 +31,11 @@ public class TestNGGroup public static final String KAFKA_INDEX = "kafka-index"; - public static final String TRANSACTIONAL_KAFKA_INDEX = "kafka-transactional_index"; + public static final String KAFKA_INDEX_SLOW = "kafka-index-slow"; + + public static final String TRANSACTIONAL_KAFKA_INDEX = "kafka-transactional-index"; + + public static final String TRANSACTIONAL_KAFKA_INDEX_SLOW = "kafka-transactional-index-slow"; public static final String OTHER_INDEX = "other-index"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java index b894f7f222f4..dc4a968eae16 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java @@ -26,7 +26,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.KAFKA_INDEX) +@Test(groups = TestNGGroup.KAFKA_INDEX_SLOW) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceNonTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java index 3f1df22e17bf..5aa81e15b274 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -26,7 +26,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX) +@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX_SLOW) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml index b5c255da9fcd..5a0735a05782 100644 --- a/integration-tests/src/test/resources/testng.xml +++ b/integration-tests/src/test/resources/testng.xml @@ -33,7 +33,7 @@ - + From fcbfcc2403ab2c8bd90fbfef9a42bc5f4812e473 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 20 Apr 2020 07:39:16 -1000 Subject: [PATCH 15/17] test --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index c231f7b20a6a..93ddf47e5be0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -287,7 +287,7 @@ jobs: - docker env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' script: &run_integration_test - - travis_wait ${MVN} verify -pl integration-tests -P integration-tests ${TESTNG_GROUPS} ${JVM_RUNTIME} ${MAVEN_SKIP} + - ${MVN} verify -pl integration-tests -P integration-tests ${TESTNG_GROUPS} ${JVM_RUNTIME} ${MAVEN_SKIP} after_failure: &integration_test_diags - for v in ~/shared/logs/*.log ; do echo $v logtail ======================== ; tail -100 $v ; From f31dc7ba95a1fc8a5d7df041c9a37368a4fb9b88 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 20 Apr 2020 20:44:56 -1000 Subject: [PATCH 16/17] address comments --- integration-tests/README.md | 4 ++-- .../org/apache/druid/testing/utils/KinesisAdminClient.java | 3 +++ .../druid/tests/coordinator/duty/ITAutoCompactionTest.java | 3 +++ .../apache/druid/tests/indexer/AbstractITBatchIndexTest.java | 2 +- .../org/apache/druid/tests/indexer/AbstractIndexerTest.java | 2 +- .../druid/tests/indexer/AbstractStreamIndexingTest.java | 2 +- 6 files changed, 11 insertions(+), 5 deletions(-) diff --git a/integration-tests/README.md b/integration-tests/README.md index f2b0a13a6dd8..ab9f9c412d0d 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -321,6 +321,6 @@ class can be set to run in parallel (multiple test methods of the given class ru the given class/package from the "AllSerializedTests" test tag section and including it in the "AllParallelizedTests" test tag section in integration-tests/src/test/resources/testng.xml Please be mindful when adding tests to the "AllParallelizedTests" test tag that the tests can run in parallel with -other tests at the same time. i.e. test does not modify/restart/stop the druid cluster or other dependecy containers, -test does not use excessive memory straving other concurent task, test does not modify and/or use other task, +other tests at the same time. i.e. test does not modify/restart/stop the druid cluster or other dependency containers, +test does not use excessive memory starving other concurent task, test does not modify and/or use other task, supervisor, datasource it did not create. diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java index b4938e90811c..a3f7dddd005b 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java @@ -150,6 +150,9 @@ public int getStreamShardCount(String streamName) public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) { int actualShardCount = getStreamShardCount(streamName); + // Kinesis does not immediately drop the old shards after the resharding and hence, + // would still returns both open shards and closed shards from the API call. + // To verify, we sum the old count (closed shareds) and the expected new count (open shards) return actualShardCount == oldShardCount + newShardCount; } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 592c91034daf..7b560e1107c1 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/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); + // Verify that the compaction config is updated correctly. CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); DataSourceCompactionConfig foundDataSourceCompactionConfig = null; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index cb6192a3a427..beaeac5330f2 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -85,7 +85,7 @@ public String getFolderSuffix() private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class); @Inject - public IntegrationTestingConfig config; + IntegrationTestingConfig config; @Inject protected SqlTestQueryHelper sqlQueryHelper; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index 03df90eb99d6..56774eff9073 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -57,7 +57,7 @@ public abstract class AbstractIndexerTest protected TestQueryHelper queryHelper; @Inject - private IntegrationTestingConfig config; + public IntegrationTestingConfig config; protected Closeable unloader(final String dataSource) { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 228f7970d2a5..1e32e4ba63d9 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -38,7 +38,7 @@ import java.util.UUID; import java.util.function.Function; -public abstract class AbstractStreamIndexingTest extends AbstractITBatchIndexTest +public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest { static final DateTime FIRST_EVENT_TIME = DateTimes.of(1994, 4, 29, 1, 0); // format for the querying interval From 47d3f624cd0b171d343eb081df41db51a141250a Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 21 Apr 2020 21:26:49 -1000 Subject: [PATCH 17/17] addressed comments --- integration-tests/README.md | 4 +- .../druid/testing/utils/KafkaAdminClient.java | 19 +++-- .../druid/testing/utils/KafkaEventWriter.java | 6 +- .../testing/utils/KinesisAdminClient.java | 10 +-- .../testing/utils/StreamAdminClient.java | 13 +++- .../testing/utils/StreamEventWriter.java | 5 ++ .../druid/testing/utils/StreamGenerator.java | 4 +- .../utils/SyntheticStreamGenerator.java | 6 +- .../tests/indexer/AbstractIndexerTest.java | 2 +- .../AbstractKafkaIndexingServiceTest.java | 11 +-- .../AbstractKinesisIndexingServiceTest.java | 9 ++- .../indexer/AbstractStreamIndexingTest.java | 69 +++++++++++-------- ...ServiceNonTransactionalSerializedTest.java | 11 ++- ...ingServiceTransactionalSerializedTest.java | 11 ++- ...TKinesisIndexingServiceSerializedTest.java | 9 +++ ...rviceNonTransactionalParallelizedTest.java | 18 ++++- ...gServiceTransactionalParallelizedTest.java | 18 ++++- ...inesisIndexingServiceParallelizedTest.java | 20 ++++++ 18 files changed, 179 insertions(+), 66 deletions(-) diff --git a/integration-tests/README.md b/integration-tests/README.md index ab9f9c412d0d..f390e8586806 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -317,10 +317,10 @@ 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 +class can be set to run in parallel (multiple test methods of each class running at the same time) by excluding the given class/package from the "AllSerializedTests" test tag section and including it in the "AllParallelizedTests" test tag section in integration-tests/src/test/resources/testng.xml Please be mindful when adding tests to the "AllParallelizedTests" test tag that the tests can run in parallel with -other tests at the same time. i.e. test does not modify/restart/stop the druid cluster or other dependency containers, +other tests from the same class at the same time. i.e. test does not modify/restart/stop the druid cluster or other dependency containers, test does not use excessive memory starving other concurent task, test does not modify and/or use other task, supervisor, datasource it did not create. diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java index 411e5d65a901..d63d08833cd7 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaAdminClient.java @@ -37,12 +37,12 @@ public class KafkaAdminClient implements StreamAdminClient { - AdminClient adminClient; + private AdminClient adminClient; public KafkaAdminClient(String kafkaInternalHost) { Properties config = new Properties(); - config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost); + config.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaInternalHost); adminClient = AdminClient.create(config); } @@ -67,10 +67,11 @@ public void deleteStream(String streamName) throws Exception * 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 + * started (but not nessesary finished), otherwise, the method will returns right after issue the + * repartitioning command */ @Override - public void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception + public void updatePartitionCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception { Map counts = new HashMap<>(); counts.put(streamName, NewPartitions.increaseTo(newPartitionCount)); @@ -81,6 +82,10 @@ public void updateShardCount(String streamName, int newPartitionCount, boolean b } } + /** + * Stream state such as active/non-active does not applies to Kafka. + * Returning true since Kafka stream is always active and can always be writen and read to. + */ @Override public boolean isStreamActive(String streamName) { @@ -88,7 +93,7 @@ public boolean isStreamActive(String streamName) } @Override - public int getStreamShardCount(String streamName) throws Exception + public int getStreamPartitionCount(String streamName) throws Exception { DescribeTopicsResult result = adminClient.describeTopics(ImmutableList.of(streamName)); TopicDescription topicDescription = result.values().get(streamName).get(); @@ -96,8 +101,8 @@ public int getStreamShardCount(String streamName) throws Exception } @Override - public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception + public boolean verfiyPartitionCountUpdated(String streamName, int oldPartitionCount, int newPartitionCount) throws Exception { - return getStreamShardCount(streamName) == newShardCount; + return getStreamPartitionCount(streamName) == newPartitionCount; } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java index 7ec88f23ea3e..f7ec75507b9a 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KafkaEventWriter.java @@ -36,9 +36,9 @@ public class KafkaEventWriter implements StreamEventWriter { private static final String TEST_PROPERTY_PREFIX = "kafka.test.property."; - private KafkaProducer producer; - private boolean txnEnabled; - private List> pendingWriteRecords = new ArrayList<>(); + private final KafkaProducer producer; + private final boolean txnEnabled; + private final List> pendingWriteRecords = new ArrayList<>(); public KafkaEventWriter(IntegrationTestingConfig config, boolean txnEnabled) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java index a3f7dddd005b..7c8759ae0e36 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java @@ -104,9 +104,9 @@ public void deleteStream(String streamName) * started (but not nessesary finished), otherwise, the method will returns right after issue the reshard command */ @Override - public void updateShardCount(String streamName, int newShardCount, boolean blocksUntilStarted) + public void updatePartitionCount(String streamName, int newShardCount, boolean blocksUntilStarted) { - int originalShardCount = getStreamShardCount(streamName); + int originalShardCount = getStreamPartitionCount(streamName); UpdateShardCountRequest updateShardCountRequest = new UpdateShardCountRequest(); updateShardCountRequest.setStreamName(streamName); updateShardCountRequest.setTargetShardCount(newShardCount); @@ -140,16 +140,16 @@ public boolean isStreamActive(String streamName) } @Override - public int getStreamShardCount(String streamName) + public int getStreamPartitionCount(String streamName) { StreamDescription streamDescription = getStreamDescription(streamName); return getStreamShardCount(streamDescription); } @Override - public boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) + public boolean verfiyPartitionCountUpdated(String streamName, int oldShardCount, int newShardCount) { - int actualShardCount = getStreamShardCount(streamName); + int actualShardCount = getStreamPartitionCount(streamName); // Kinesis does not immediately drop the old shards after the resharding and hence, // would still returns both open shards and closed shards from the API call. // To verify, we sum the old count (closed shareds) and the expected new count (open shards) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java index 23e0e422b3d8..ea36d1c969a6 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamAdminClient.java @@ -21,17 +21,24 @@ import java.util.Map; +/** + * This interface provides the administrative client contract for any stream storage (such as Kafka and Kinesis) + * which supports managing and inspecting streams (aka topics) and stream's partitions (aka shards). + * This is used for setting up, tearing down and any other administrative changes required in integration tests. + * Each method resulting in a change of state for the stream is intended to be synchronous to help + * make integration tests deterministic and easy to write. + */ public interface StreamAdminClient { void createStream(String streamName, int partitionCount, Map tags) throws Exception; void deleteStream(String streamName) throws Exception; - void updateShardCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception; + void updatePartitionCount(String streamName, int newPartitionCount, boolean blocksUntilStarted) throws Exception; boolean isStreamActive(String streamName); - int getStreamShardCount(String streamName) throws Exception; + int getStreamPartitionCount(String streamName) throws Exception; - boolean verfiyShardCountUpdated(String streamName, int oldShardCount, int newShardCount) throws Exception; + boolean verfiyPartitionCountUpdated(String streamName, int oldPartitionCount, int newPartitionCount) throws Exception; } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java index 0baab934cbca..5d25916b6f62 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java @@ -19,6 +19,11 @@ package org.apache.druid.testing.utils; + +/** + * This interface is use to write test event data to the underlying stream (such as Kafka, Kinesis) + * This can also be use with {@link StreamGenerator} to write particular set of test data + */ public interface StreamEventWriter { void write(String topic, String event); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java index a232c59a8d65..f2d1f489d886 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java @@ -23,9 +23,9 @@ public interface StreamGenerator { - void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds); + void run(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds); - void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime); + void run(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime); void shutdown(); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java index 460fe46a3fda..f2bfde857c61 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java @@ -61,13 +61,13 @@ public SyntheticStreamGenerator(int eventsPerSecond, long cyclePaddingMs) abstract Object getEvent(int row, DateTime timestamp); @Override - public void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds) + public void run(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds) { - start(streamTopic, streamEventWriter, totalNumberOfSeconds, null); + run(streamTopic, streamEventWriter, totalNumberOfSeconds, null); } @Override - public void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime) + public void run(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime) { // The idea here is that we will send [eventsPerSecond] events that will either use [nowFlooredToSecond] // or the [overrrideFirstEventTime] as the primary timestamp. diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index 56774eff9073..03df90eb99d6 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -57,7 +57,7 @@ public abstract class AbstractIndexerTest protected TestQueryHelper queryHelper; @Inject - public IntegrationTestingConfig config; + private IntegrationTestingConfig config; protected Closeable unloader(final String dataSource) { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java index 1c5ea8bb3c9f..ce769bfb59ba 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexingServiceTest.java @@ -21,6 +21,7 @@ import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.utils.KafkaAdminClient; import org.apache.druid.testing.utils.KafkaEventWriter; import org.apache.druid.testing.utils.StreamAdminClient; @@ -32,22 +33,24 @@ public abstract class AbstractKafkaIndexingServiceTest extends AbstractStreamIndexingTest { - public abstract boolean isKafkaWriterTransactionalEnabled(); + protected abstract boolean isKafkaWriterTransactionalEnabled(); @Override - StreamAdminClient getStreamAdminClient() + StreamAdminClient createStreamAdminClient(IntegrationTestingConfig config) { return new KafkaAdminClient(config.getKafkaHost()); } @Override - public StreamEventWriter getStreamEventWriter() + public StreamEventWriter createStreamEventWriter(IntegrationTestingConfig config) { return new KafkaEventWriter(config, isKafkaWriterTransactionalEnabled()); } @Override - Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName) + Function generateStreamIngestionPropsTransform(String streamName, + String fullDatasourceName, + IntegrationTestingConfig config) { final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); final Properties consumerProperties = new Properties(); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java index e2b524121825..14c9cac8ac2a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKinesisIndexingServiceTest.java @@ -20,6 +20,7 @@ package org.apache.druid.tests.indexer; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.utils.KinesisAdminClient; import org.apache.druid.testing.utils.KinesisEventWriter; import org.apache.druid.testing.utils.StreamAdminClient; @@ -30,19 +31,21 @@ public abstract class AbstractKinesisIndexingServiceTest extends AbstractStreamIndexingTest { @Override - StreamAdminClient getStreamAdminClient() throws Exception + StreamAdminClient createStreamAdminClient(IntegrationTestingConfig config) throws Exception { return new KinesisAdminClient(config.getStreamEndpoint()); } @Override - StreamEventWriter getStreamEventWriter() throws Exception + StreamEventWriter createStreamEventWriter(IntegrationTestingConfig config) throws Exception { return new KinesisEventWriter(config.getStreamEndpoint(), false); } @Override - Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName) + Function generateStreamIngestionPropsTransform(String streamName, + String fullDatasourceName, + IntegrationTestingConfig config) { return spec -> { try { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java index 1e32e4ba63d9..2f0c65afaebc 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractStreamIndexingTest.java @@ -24,6 +24,7 @@ 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.IntegrationTestingConfig; import org.apache.druid.testing.utils.DruidClusterAdminClient; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.StreamAdminClient; @@ -47,7 +48,6 @@ public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest 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. @@ -58,22 +58,28 @@ public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest 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; + private static final Logger LOG = new Logger(AbstractStreamIndexingTest.class); @Inject private DruidClusterAdminClient druidClusterAdminClient; + @Inject + private IntegrationTestingConfig config; + private StreamAdminClient streamAdminClient; private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; - abstract StreamAdminClient getStreamAdminClient() throws Exception; - abstract StreamEventWriter getStreamEventWriter() throws Exception; - abstract Function generateStreamIngestionPropsTransform(String streamName, String fullDatasourceName); + abstract StreamAdminClient createStreamAdminClient(IntegrationTestingConfig config) throws Exception; + abstract StreamEventWriter createStreamEventWriter(IntegrationTestingConfig config) throws Exception; + abstract Function generateStreamIngestionPropsTransform(String streamName, + String fullDatasourceName, + IntegrationTestingConfig config); abstract Function generateStreamQueryPropsTransform(String streamName, String fullDatasourceName); public abstract String getTestNamePrefix(); protected void doBeforeClass() throws Exception { - streamAdminClient = getStreamAdminClient(); + streamAdminClient = createStreamAdminClient(config); wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); } @@ -84,7 +90,7 @@ protected void doClassTeardown() protected void doTestIndexDataWithLegacyParserStableState() throws Exception { - StreamEventWriter streamEventWriter = getStreamEventWriter(); + StreamEventWriter streamEventWriter = createStreamEventWriter(config); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -95,7 +101,7 @@ protected void doTestIndexDataWithLegacyParserStableState() throws Exception generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec)); LOG.info("Submitted supervisor"); // Start data generator - wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); verifyIngestedData(generatedTestConfig); } finally { @@ -105,7 +111,7 @@ protected void doTestIndexDataWithLegacyParserStableState() throws Exception protected void doTestIndexDataWithInputFormatStableState() throws Exception { - StreamEventWriter streamEventWriter = getStreamEventWriter(); + StreamEventWriter streamEventWriter = createStreamEventWriter(config); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -116,7 +122,7 @@ protected void doTestIndexDataWithInputFormatStableState() throws Exception generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec)); LOG.info("Submitted supervisor"); // Start data generator - wikipediaStreamEventGenerator.start(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); verifyIngestedData(generatedTestConfig); } finally { @@ -141,7 +147,7 @@ void doTestIndexDataWithLosingHistorical() throws Exception protected void doTestIndexDataWithStartStopSupervisor() throws Exception { - StreamEventWriter streamEventWriter = getStreamEventWriter(); + StreamEventWriter streamEventWriter = createStreamEventWriter(config); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -155,7 +161,7 @@ protected void doTestIndexDataWithStartStopSupervisor() throws Exception 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); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before suspension ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), @@ -167,7 +173,7 @@ protected void doTestIndexDataWithStartStopSupervisor() throws Exception // 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)); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Resume the supervisor indexer.resumeSupervisor(generatedTestConfig.getSupervisorId()); // Verify supervisor is healthy after suspension @@ -200,7 +206,7 @@ protected void doTestIndexDataWithStreamReshardMerge() throws Exception private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable waitForReadyRunnable) throws Exception { - StreamEventWriter streamEventWriter = getStreamEventWriter(); + StreamEventWriter streamEventWriter = createStreamEventWriter(config); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -214,7 +220,7 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa 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); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before restart ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), @@ -230,13 +236,13 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa // 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)); + wikipediaStreamEventGenerator.run(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)); + // Start generating remaining data (after restarting) + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); // Verify supervisor is healthy ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), @@ -255,7 +261,7 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa private void testIndexWithStreamReshardHelper(int newShardCount) throws Exception { - StreamEventWriter streamEventWriter = getStreamEventWriter(); + StreamEventWriter streamEventWriter = createStreamEventWriter(config); final GeneratedTestConfig generatedTestConfig = new GeneratedTestConfig(); try ( final Closeable ignored1 = unloader(generatedTestConfig.getFullDatasourceName()) @@ -269,7 +275,7 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio 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); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Verify supervisor is healthy before resahrding ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(generatedTestConfig.getSupervisorId())), @@ -279,11 +285,11 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio "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); + streamAdminClient.updatePartitionCount(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)); + wikipediaStreamEventGenerator.run(generatedTestConfig.getStreamName(), streamEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Wait for stream to finish resharding ITRetryUtil.retryUntil( () -> streamAdminClient.isStreamActive(generatedTestConfig.getStreamName()), @@ -293,14 +299,14 @@ private void testIndexWithStreamReshardHelper(int newShardCount) throws Exceptio "Waiting for stream to finish resharding" ); ITRetryUtil.retryUntil( - () -> streamAdminClient.verfiyShardCountUpdated(generatedTestConfig.getStreamName(), STREAM_SHARD_COUNT, newShardCount), + () -> streamAdminClient.verfiyPartitionCountUpdated(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)); + // Start generating remaining data (after resharding) + wikipediaStreamEventGenerator.run(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())), @@ -356,27 +362,32 @@ private void doMethodTeardown(GeneratedTestConfig generatedTestConfig, StreamEve { try { streamEventWriter.flush(); + streamEventWriter.shutdown(); } catch (Exception e) { - // Best effort cleanup as the writer may have already went Bye-Bye + // Best effort cleanup as the writer may have already been cleanup + LOG.warn(e, "Failed to cleanup writer. This might be expected depending on the test method"); } try { indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId()); } catch (Exception e) { - // Best effort cleanup as the supervisor may have already went Bye-Bye + // Best effort cleanup as the supervisor may have already been cleanup + LOG.warn(e, "Failed to cleanup supervisor. This might be expected depending on the test method"); } try { unloader(generatedTestConfig.getFullDatasourceName()); } catch (Exception e) { - // Best effort cleanup as the datasource may have already went Bye-Bye + // Best effort cleanup as the datasource may have already been cleanup + LOG.warn(e, "Failed to cleanup datasource. This might be expected depending on the test method"); } try { streamAdminClient.deleteStream(generatedTestConfig.getStreamName()); } catch (Exception e) { - // Best effort cleanup as the stream may have already went Bye-Bye + // Best effort cleanup as the stream may have already been cleanup + LOG.warn(e, "Failed to cleanup stream. This might be expected depending on the test method"); } } @@ -402,7 +413,7 @@ private class GeneratedTestConfig "Wait for stream active" ); fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); - streamIngestionPropsTransform = generateStreamIngestionPropsTransform(streamName, fullDatasourceName); + streamIngestionPropsTransform = generateStreamIngestionPropsTransform(streamName, fullDatasourceName, config); streamQueryPropsTransform = generateStreamQueryPropsTransform(streamName, fullDatasourceName); } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java index dc4a968eae16..99713a729847 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java @@ -31,7 +31,7 @@ public class ITKafkaIndexingServiceNonTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { @Override - public boolean isKafkaWriterTransactionalEnabled() + protected boolean isKafkaWriterTransactionalEnabled() { return false; } @@ -54,18 +54,27 @@ public void tearDown() doClassTeardown(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingCoordinator() throws Exception { doTestIndexDataWithLosingCoordinator(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingOverlord() throws Exception { doTestIndexDataWithLosingOverlord(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingHistorical() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java index 5aa81e15b274..06bcf050d678 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -31,7 +31,7 @@ public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { @Override - public boolean isKafkaWriterTransactionalEnabled() + protected boolean isKafkaWriterTransactionalEnabled() { return true; } @@ -54,18 +54,27 @@ public void tearDown() doClassTeardown(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingCoordinator() throws Exception { doTestIndexDataWithLosingCoordinator(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingOverlord() throws Exception { doTestIndexDataWithLosingOverlord(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKafkaIndexDataWithLosingHistorical() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java index fc31e2fa99c4..8e64abb65560 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceSerializedTest.java @@ -48,18 +48,27 @@ public void tearDown() doClassTeardown(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKinesisIndexDataWithLosingCoordinator() throws Exception { doTestIndexDataWithLosingCoordinator(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKinesisIndexDataWithLosingOverlord() throws Exception { doTestIndexDataWithLosingOverlord(); } + /** + * This test must be run individually since the test affect and modify the state of the Druid cluster + */ @Test public void testKinesisIndexDataWithLosingHistorical() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java index d367a608c452..199530e0a32a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceNonTransactionalParallelizedTest.java @@ -32,7 +32,7 @@ public class ITKafkaIndexingServiceNonTransactionalParallelizedTest extends AbstractKafkaIndexingServiceTest { @Override - public boolean isKafkaWriterTransactionalEnabled() + protected boolean isKafkaWriterTransactionalEnabled() { return false; } @@ -55,24 +55,40 @@ public void tearDown() doClassTeardown(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithLegacyParserStableState() throws Exception { doTestIndexDataWithLegacyParserStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithInputFormatStableState() throws Exception { doTestIndexDataWithInputFormatStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithStartStopSupervisor() throws Exception { doTestIndexDataWithStartStopSupervisor(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithKafkaReshardSplit() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java index 362e16807636..7db3a7fd832c 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceTransactionalParallelizedTest.java @@ -32,7 +32,7 @@ public class ITKafkaIndexingServiceTransactionalParallelizedTest extends AbstractKafkaIndexingServiceTest { @Override - public boolean isKafkaWriterTransactionalEnabled() + protected boolean isKafkaWriterTransactionalEnabled() { return true; } @@ -55,24 +55,40 @@ public void tearDown() doClassTeardown(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithLegacyParserStableState() throws Exception { doTestIndexDataWithLegacyParserStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithInputFormatStableState() throws Exception { doTestIndexDataWithInputFormatStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithStartStopSupervisor() throws Exception { doTestIndexDataWithStartStopSupervisor(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKafkaIndexDataWithKafkaReshardSplit() throws Exception { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java index 6b138bec1556..38816dc1328f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKinesisIndexingServiceParallelizedTest.java @@ -49,30 +49,50 @@ public void tearDown() doClassTeardown(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKinesisIndexDataWithLegacyParserStableState() throws Exception { doTestIndexDataWithLegacyParserStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKinesisIndexDataWithInputFormatStableState() throws Exception { doTestIndexDataWithInputFormatStableState(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKinesisIndexDataWithStartStopSupervisor() throws Exception { doTestIndexDataWithStartStopSupervisor(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception { doTestIndexDataWithStreamReshardSplit(); } + /** + * This test can be run concurrently with other tests as it creates/modifies/teardowns a unique datasource + * and supervisor maintained and scoped within this test only + */ @Test public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception {