From 5f0f36fbb37614d13bc63a0dbb5b19a240b603f4 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 24 Mar 2020 12:18:53 -1000 Subject: [PATCH 01/22] kinesis IT --- integration-tests/pom.xml | 6 ++ .../testing/utils/KinesisAdminClient.java | 60 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 7aa13e00d006..2fcbf3bd69ea 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -237,6 +237,12 @@ + + com.amazonaws + aws-java-sdk-kinesis + 1.11.400 + compile + 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 new file mode 100644 index 000000000000..5052cbf675b9 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisAdminClient.java @@ -0,0 +1,60 @@ +/* + * 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.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.util.AwsHostNameUtils; + +import java.io.FileInputStream; +import java.util.Properties; + +public class KinesisAdminClient +{ + private AmazonKinesisClient amazonKinesisClient; + + KinesisAdminClient() throws Exception + { + String pathToConfigFile = System.getProperty("override.config.path"); + Properties prop = new Properties(); + prop.load(new FileInputStream(pathToConfigFile)); + + AWSStaticCredentialsProvider credentials = new AWSStaticCredentialsProvider( + new BasicAWSCredentials( + prop.getProperty("druid_kinesis_accessKey"), + prop.getProperty("druid_kinesis_secretKey") + ) + ); + amazonKinesisClient = AmazonKinesisClientBuilder.standard() + .withCredentials(credentials) + .withClientConfiguration(new ClientConfiguration()) + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( + endpoint, + AwsHostNameUtils.parseRegion( + endpoint, + null + ) + )).build(); + } +} From 8d6a35390d4de09199f59b508ad1bc3a11fa543f Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 26 Mar 2020 09:06:08 -1000 Subject: [PATCH 02/22] Kinesis IT --- integration-tests/pom.xml | 27 +++- .../druid/testing/utils/EventWriter.java | 27 ++++ .../apache/druid/testing/utils/Generator.java | 27 ++++ .../testing/utils/KinesisAdminClient.java | 62 ++++++- .../testing/utils/KinesisEventWriter.java | 84 ++++++++++ .../utils/StreamVerifierEventGenerator.java | 54 +++++++ .../utils/StreamVerifierSyntheticEvent.java | 104 ++++++++++++ .../testing/utils/SyntheticGenerator.java | 152 ++++++++++++++++++ .../indexer/ITKinesisIndexingServiceTest.java | 24 +++ 9 files changed, 551 insertions(+), 10 deletions(-) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 18046558c85c..25397dbe71c7 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -31,7 +31,28 @@ 0.18.0-SNAPSHOT + + 0.13.1 + + + + com.amazonaws + amazon-kinesis-producer + ${kinesis.producer.version} + + + com.amazonaws + aws-java-sdk-kinesis + ${aws.sdk.version} + + + + com.amazonaws + aws-java-sdk-core + + + org.apache.druid druid-core @@ -243,12 +264,6 @@ - - com.amazonaws - aws-java-sdk-kinesis - 1.11.400 - compile - diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java new file mode 100644 index 000000000000..be66bbe8fa0c --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java @@ -0,0 +1,27 @@ +/* + * 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; + +public interface EventWriter +{ + void write(String event); + + void shutdown(); +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java new file mode 100644 index 000000000000..0da51d18b8f3 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java @@ -0,0 +1,27 @@ +/* + * 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; + +public interface Generator +{ + void start(EventWriter eventWriter); + + void shutdown(); +} 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 5052cbf675b9..6dda2872dc38 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 @@ -23,18 +23,26 @@ import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.model.CreateStreamResult; +import com.amazonaws.services.kinesis.model.DeleteStreamResult; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.ScalingType; +import com.amazonaws.services.kinesis.model.StreamStatus; +import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; +import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; +import org.apache.druid.java.util.common.ISE; import java.io.FileInputStream; import java.util.Properties; public class KinesisAdminClient { - private AmazonKinesisClient amazonKinesisClient; + private AmazonKinesis amazonKinesis; - KinesisAdminClient() throws Exception + KinesisAdminClient(String endpoint) throws Exception { String pathToConfigFile = System.getProperty("override.config.path"); Properties prop = new Properties(); @@ -46,7 +54,7 @@ public class KinesisAdminClient prop.getProperty("druid_kinesis_secretKey") ) ); - amazonKinesisClient = AmazonKinesisClientBuilder.standard() + amazonKinesis = AmazonKinesisClientBuilder.standard() .withCredentials(credentials) .withClientConfiguration(new ClientConfiguration()) .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( @@ -57,4 +65,50 @@ public class KinesisAdminClient ) )).build(); } + + void createStream(String streamName, int shardCount) + { + CreateStreamResult createStreamResult = amazonKinesis.createStream(streamName, shardCount); + if (createStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot create stream for integration test"); + } + } + + void deleteStream(String streamName) + { + DeleteStreamResult deleteStreamResult = amazonKinesis.deleteStream(streamName); + if (deleteStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot delete stream for integration test"); + } + } + + void updateShardCount(String streamName, int newShardCount) + { + UpdateShardCountRequest updateShardCountRequest = new UpdateShardCountRequest(); + updateShardCountRequest.setStreamName(streamName); + updateShardCountRequest.setTargetShardCount(newShardCount); + updateShardCountRequest.setScalingType(ScalingType.UNIFORM_SCALING); + UpdateShardCountResult updateShardCountResult = amazonKinesis.updateShardCount(updateShardCountRequest); + if (updateShardCountResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot update stream's shard count for integration test"); + } + } + + boolean isStreamActive(String streamName) + { + DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); + if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot get stream status for integration test"); + } + return StreamStatus.ACTIVE.toString().equals(describeStreamResult.getStreamDescription().getStreamStatus()); + } + + int getStreamShardCount(String streamName) + { + DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); + if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot get stream status for integration test"); + } + return describeStreamResult.getStreamDescription().getShards().size(); + } } 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 new file mode 100644 index 000000000000..61790ed00870 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/KinesisEventWriter.java @@ -0,0 +1,84 @@ +/* + * 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.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.services.kinesis.producer.KinesisProducer; +import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; +import com.amazonaws.util.AwsHostNameUtils; +import org.apache.commons.codec.digest.DigestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileInputStream; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Properties; + +public class KinesisEventWriter implements EventWriter +{ + private static final Logger LOG = LoggerFactory.getLogger(KinesisEventWriter.class); + + private final String streamName; + private final KinesisProducer kinesisProducer; + + public KinesisEventWriter(String endpoint, String streamName, boolean aggregate) throws Exception + { + String pathToConfigFile = System.getProperty("override.config.path"); + Properties prop = new Properties(); + prop.load(new FileInputStream(pathToConfigFile)); + + AWSStaticCredentialsProvider credentials = new AWSStaticCredentialsProvider( + new BasicAWSCredentials( + prop.getProperty("druid_kinesis_accessKey"), + prop.getProperty("druid_kinesis_secretKey") + ) + ); + + KinesisProducerConfiguration kinesisProducerConfiguration = new KinesisProducerConfiguration() + .setCredentialsProvider(credentials) + .setRegion(AwsHostNameUtils.parseRegion(endpoint, null)) + .setRequestTimeout(20000L) + .setRecordTtl(9223372036854775807L) + .setMetricsLevel("none") + .setAggregationEnabled(aggregate); + + this.kinesisProducer = new KinesisProducer(kinesisProducerConfiguration); + this.streamName = streamName; + } + + @Override + public void write(String event) + { + kinesisProducer.addUserRecord( + streamName, + DigestUtils.sha1Hex(event), + ByteBuffer.wrap(event.getBytes(StandardCharsets.UTF_8)) + ); + } + + @Override + public void shutdown() + { + LOG.info("Shutting down Kinesis client"); + kinesisProducer.flushSync(); + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java new file mode 100644 index 000000000000..1bc348723876 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java @@ -0,0 +1,54 @@ +/* + * 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.joda.time.DateTime; + +import java.util.UUID; + +public class StreamVerifierEventGenerator extends SyntheticGenerator +{ + public StreamVerifierEventGenerator(int eventsPerHour, long cyclePaddingMs, int totalNumberOfHours) + { + super(eventsPerHour, cyclePaddingMs, totalNumberOfHours); + } + + @Override + Object getEvent(int i, DateTime timestamp) + { + return StreamVerifierSyntheticEvent.of( + UUID.randomUUID().toString(), + timestamp.getMillis(), + DateTime.now().getMillis(), + i, + i == getEventsPerHour() ? getSumOfEventSequence(getEventsPerHour()) : null, + i == 1 + ); + } + + + /** + * Assumes the first number in the sequence is 1, incrementing by 1, until numEvents. + */ + private long getSumOfEventSequence(int numEvents) + { + return (numEvents * (1 + numEvents)) / 2; + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java new file mode 100644 index 000000000000..78abdc96d247 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java @@ -0,0 +1,104 @@ +/* + * 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.fasterxml.jackson.annotation.JsonProperty; + +public class StreamVerifierSyntheticEvent +{ + private String id; + private long groupingTimestamp; + private long insertionTimestamp; + private long sequenceNumber; + private Long expectedSequenceNumberSum; + private boolean firstEvent; + + private StreamVerifierSyntheticEvent( + String id, + long groupingTimestamp, + long insertionTimestamp, + long sequenceNumber, + Long expectedSequenceNumberSum, + boolean firstEvent + ) + { + this.id = id; + this.groupingTimestamp = groupingTimestamp; + this.insertionTimestamp = insertionTimestamp; + this.sequenceNumber = sequenceNumber; + this.expectedSequenceNumberSum = expectedSequenceNumberSum; + this.firstEvent = firstEvent; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public long getGroupingTimestamp() + { + return groupingTimestamp; + } + + @JsonProperty + public long getInsertionTimestamp() + { + return insertionTimestamp; + } + + @JsonProperty + public long getSequenceNumber() + { + return sequenceNumber; + } + + @JsonProperty + public Long getExpectedSequenceNumberSum() + { + return expectedSequenceNumberSum; + } + + @JsonProperty + public Integer getFirstEventFlag() + { + return firstEvent ? 1 : null; + } + + public static StreamVerifierSyntheticEvent of( + String id, + long groupingTimestamp, + long insertionTimestamp, + long sequenceNumber, + Long expectedSequenceNumberSum, + boolean firstEvent + ) + { + return new StreamVerifierSyntheticEvent( + id, + groupingTimestamp, + insertionTimestamp, + sequenceNumber, + expectedSequenceNumberSum, + firstEvent + ); + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java new file mode 100644 index 000000000000..cad509521a8e --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -0,0 +1,152 @@ +/* + * 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.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class SyntheticGenerator implements Generator +{ + private static final Logger log = LoggerFactory.getLogger(SyntheticGenerator.class); + static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + static { + MAPPER.setInjectableValues( + new InjectableValues.Std() + .addValue(ObjectMapper.class.getName(), MAPPER) + ); + MAPPER.setSerializationInclusion(JsonInclude.Include.NON_NULL); + } + + public int getEventsPerSecond() + { + return eventsPerSecond; + } + + private final int eventsPerSecond; + + // When calculating rates, leave this buffer to minimize overruns where we're still writing messages from the previous + // second. If the generator finishes sending [eventsPerSecond] events and the second is not up, it will wait for the next + // second to begin. + private final long cyclePaddingMs; + + private final int totalNumberOfSecond; + + public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs, int totalNumberOfSecond) + { + this.eventsPerSecond = eventsPerSecond; + this.cyclePaddingMs = cyclePaddingMs; + this.totalNumberOfSecond = totalNumberOfSecond; + } + + abstract Object getEvent(int row, DateTime timestamp); + + @Override + public void start(EventWriter eventWriter) + { + // The idea here is that we will send [eventsPerSecond] events that will use this value as the primary timestamp. + // Having a fixed number of events that use the same timestamp will help in allowing us to determine if any events + // were dropped or duplicated. We will try to space the event generation over the remainder of the second so that it + // roughly completes at the top of the second, but if it doesn't complete, it will still send the remainder of the + // events with the original timestamp, even after wall time has moved onto the next second. + DateTime nowFlooredToSecond = DateTime.now().secondOfDay().roundFloorCopy(); + int seconds = 0; + + while (true) { + try { + long sleepMillis = nowFlooredToSecond.getMillis() - DateTime.now().getMillis(); + if (sleepMillis > 0) { + log.info("Waiting {} ms for next run cycle (at {})", sleepMillis, nowFlooredToSecond); + Thread.sleep(sleepMillis); + continue; + } + + log.info( + "Beginning run cycle with {} events, target completion time: {}", + eventsPerSecond, + nowFlooredToSecond.plusSeconds(1).minus(cyclePaddingMs) + ); + + for (int i = 1; i <= eventsPerSecond; i++) { + eventWriter.write(MAPPER.writeValueAsString(getEvent(i, nowFlooredToSecond))); + + long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowFlooredToSecond); + if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { + log.info("Event: {}/{}, sleep time: {} ms", i, eventsPerSecond, sleepTime); + } + + if (sleepTime > 0) { + Thread.sleep(sleepTime); + } + } + + nowFlooredToSecond = nowFlooredToSecond.plusSeconds(1); + seconds++; + + log.info( + "Finished writing {} events, current time: {} - updating next timestamp to: {}", + eventsPerSecond, + DateTime.now(), + nowFlooredToSecond + ); + + if (seconds > totalNumberOfSecond) { + log.info( + "Finished writing {} seconds", + seconds + ); + break; + } + } + catch (Exception e) { + throw new RuntimeException("Exception in event generation loop", e); + } + } + } + + @Override + public void shutdown() + { + } + + /** + * Dynamically adjust delay between messages to spread them out over the remaining time left in the second. + */ + private long calculateSleepTimeMs(long eventsRemaining, DateTime secondBeingProcessed) + { + if (eventsRemaining == 0) { + return 0; + } + + DateTime now = DateTime.now(); + DateTime nextSecondToProcessMinusBuffer = secondBeingProcessed.plusSeconds(1).minus(cyclePaddingMs); + + if (nextSecondToProcessMinusBuffer.isBefore(now)) { + return 0; // We're late!! Write messages as fast as you can + } + + return (nextSecondToProcessMinusBuffer.getMillis() - now.getMillis()) / eventsRemaining; + } +} 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 new file mode 100644 index 000000000000..5319c76be70d --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKinesisIndexingServiceTest.java @@ -0,0 +1,24 @@ +/* + * 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; + +public class ITKinesisIndexingServiceTest +{ +} From 747dcf7aa5469ac2be0b6c83ca859eac4a3b0d4d Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 26 Mar 2020 21:11:42 -1000 Subject: [PATCH 03/22] Kinesis IT --- .../override-examples/kinesis | 22 ++ integration-tests/pom.xml | 6 + integration-tests/run_cluster.sh | 3 + .../testing/ConfigFileConfigProvider.java | 8 + .../druid/testing/DockerConfigProvider.java | 9 + .../testing/IntegrationTestingConfig.java | 2 + .../apache/druid/testing/utils/Generator.java | 4 + .../testing/utils/KinesisAdminClient.java | 12 +- .../utils/StreamVerifierEventGenerator.java | 6 +- .../utils/StreamVerifierSyntheticEvent.java | 2 +- .../testing/utils/SyntheticGenerator.java | 15 +- .../utils/WikipediaStreamEventGenerator.java | 63 +++++ .../org/apache/druid/tests/TestNGGroup.java | 1 + .../indexer/AbstractKafkaIndexerTest.java | 6 +- .../indexer/ITKinesisIndexingServiceTest.java | 216 +++++++++++++++++- ...queries.json => stream_index_queries.json} | 0 ... stream_supervisor_spec_input_format.json} | 10 +- ...stream_supervisor_spec_legacy_parser.json} | 10 +- 18 files changed, 368 insertions(+), 27 deletions(-) create mode 100644 integration-tests/docker/environment-configs/override-examples/kinesis create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java rename integration-tests/src/test/resources/indexer/{kafka_index_queries.json => stream_index_queries.json} (100%) rename integration-tests/src/test/resources/indexer/{kafka_supervisor_spec_input_format.json => stream_supervisor_spec_input_format.json} (86%) rename integration-tests/src/test/resources/indexer/{kafka_supervisor_spec_legacy_parser.json => stream_supervisor_spec_legacy_parser.json} (87%) diff --git a/integration-tests/docker/environment-configs/override-examples/kinesis b/integration-tests/docker/environment-configs/override-examples/kinesis new file mode 100644 index 000000000000..33d4bac48ebe --- /dev/null +++ b/integration-tests/docker/environment-configs/override-examples/kinesis @@ -0,0 +1,22 @@ +# +# 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. +# +druid_kinesis_accessKey= +druid_kinesis_secretKey= +AWS_REGION= +druid_extensions_loadList=["druid-kinesis-indexing-service"] \ No newline at end of file diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 25397dbe71c7..aa847da09c7e 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -70,6 +70,12 @@ ${project.parent.version} runtime + + org.apache.druid.extensions + druid-kinesis-indexing-service + ${project.parent.version} + runtime + org.apache.druid.extensions druid-azure-extensions diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh index 87fd8540a482..81f0e991da8f 100755 --- a/integration-tests/run_cluster.sh +++ b/integration-tests/run_cluster.sh @@ -71,6 +71,9 @@ $ For druid-hdfs-storage mkdir -p $SHARED_DIR/docker/extensions/druid-hdfs-storage mv $SHARED_DIR/docker/lib/druid-hdfs-storage-* $SHARED_DIR/docker/extensions/druid-hdfs-storage + # For druid-kinesis-indexing-service + mkdir -p $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service + mv $SHARED_DIR/docker/lib/druid-kinesis-indexing-service-* $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service # Pull Hadoop dependency if needed if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] diff --git a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java index 769b71216114..1fec42c0e445 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java @@ -57,6 +57,7 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide private String password; private String cloudBucket; private String cloudPath; + private String streamEndpoint; @JsonCreator ConfigFileConfigProvider(@JsonProperty("configFile") String configFile) @@ -192,6 +193,7 @@ private void loadProperties(String configFile) cloudBucket = props.get("cloud_bucket"); cloudPath = props.get("cloud_path"); + streamEndpoint = props.get("stream_endpoint"); LOG.info("router: [%s], [%s]", routerUrl, routerTLSUrl); LOG.info("broker: [%s], [%s]", brokerUrl, brokerTLSUrl); @@ -354,6 +356,12 @@ public String getCloudPath() return cloudPath; } + @Override + public String getStreamEndpoint() + { + return streamEndpoint; + } + @Override public Map getProperties() { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/DockerConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/DockerConfigProvider.java index 83d80e7870a8..e33e12188d5d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/DockerConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/DockerConfigProvider.java @@ -46,6 +46,9 @@ public class DockerConfigProvider implements IntegrationTestingConfigProvider @JsonProperty private String cloudBucket; + @JsonProperty + private String streamEndpoint; + @Override public IntegrationTestingConfig get() { @@ -229,6 +232,12 @@ public String getCloudPath() { return cloudPath; } + + @Override + public String getStreamEndpoint() + { + return streamEndpoint; + } }; } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfig.java b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfig.java index d178f90dd1d4..17f2aab844a7 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfig.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfig.java @@ -88,4 +88,6 @@ default String getKafkaInternalHost() String getCloudBucket(); String getCloudPath(); + + String getStreamEndpoint(); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java index 0da51d18b8f3..185a4d60fa44 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java @@ -19,9 +19,13 @@ package org.apache.druid.testing.utils; +import org.joda.time.DateTime; + public interface Generator { void start(EventWriter eventWriter); + void start(EventWriter eventWriter, DateTime overrrideFirstEventTime); + void shutdown(); } 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 6dda2872dc38..0629c0569ac0 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 @@ public class KinesisAdminClient { private AmazonKinesis amazonKinesis; - KinesisAdminClient(String endpoint) throws Exception + public KinesisAdminClient(String endpoint) throws Exception { String pathToConfigFile = System.getProperty("override.config.path"); Properties prop = new Properties(); @@ -66,7 +66,7 @@ public class KinesisAdminClient )).build(); } - void createStream(String streamName, int shardCount) + public void createStream(String streamName, int shardCount) { CreateStreamResult createStreamResult = amazonKinesis.createStream(streamName, shardCount); if (createStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { @@ -74,7 +74,7 @@ void createStream(String streamName, int shardCount) } } - void deleteStream(String streamName) + public void deleteStream(String streamName) { DeleteStreamResult deleteStreamResult = amazonKinesis.deleteStream(streamName); if (deleteStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { @@ -82,7 +82,7 @@ void deleteStream(String streamName) } } - void updateShardCount(String streamName, int newShardCount) + public void updateShardCount(String streamName, int newShardCount) { UpdateShardCountRequest updateShardCountRequest = new UpdateShardCountRequest(); updateShardCountRequest.setStreamName(streamName); @@ -94,7 +94,7 @@ void updateShardCount(String streamName, int newShardCount) } } - boolean isStreamActive(String streamName) + public boolean isStreamActive(String streamName) { DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { @@ -103,7 +103,7 @@ boolean isStreamActive(String streamName) return StreamStatus.ACTIVE.toString().equals(describeStreamResult.getStreamDescription().getStreamStatus()); } - int getStreamShardCount(String streamName) + public int getStreamShardCount(String streamName) { DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java index 1bc348723876..8d4ccddafd1c 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java @@ -25,9 +25,9 @@ public class StreamVerifierEventGenerator extends SyntheticGenerator { - public StreamVerifierEventGenerator(int eventsPerHour, long cyclePaddingMs, int totalNumberOfHours) + public StreamVerifierEventGenerator(int eventsPerSeconds, long cyclePaddingMs, int totalNumberOfSeconds) { - super(eventsPerHour, cyclePaddingMs, totalNumberOfHours); + super(eventsPerSeconds, cyclePaddingMs, totalNumberOfSeconds); } @Override @@ -38,7 +38,7 @@ Object getEvent(int i, DateTime timestamp) timestamp.getMillis(), DateTime.now().getMillis(), i, - i == getEventsPerHour() ? getSumOfEventSequence(getEventsPerHour()) : null, + i == getEventsPerSecond() ? getSumOfEventSequence(getEventsPerSecond()) : null, i == 1 ); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java index 78abdc96d247..e8c314a6b4bd 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierSyntheticEvent.java @@ -30,7 +30,7 @@ public class StreamVerifierSyntheticEvent private Long expectedSequenceNumberSum; private boolean firstEvent; - private StreamVerifierSyntheticEvent( + public StreamVerifierSyntheticEvent( String id, long groupingTimestamp, long insertionTimestamp, diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index cad509521a8e..03cd4978718c 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -66,12 +66,20 @@ public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs, int totalNum @Override public void start(EventWriter eventWriter) { - // The idea here is that we will send [eventsPerSecond] events that will use this value as the primary timestamp. + start(eventWriter, null); + } + + @Override + public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) + { + // The idea here is that we will send [eventsPerSecond] events that will either use [nowFlooredToSecond] + // or the [overrrideFirstEventTime] as the primary timestamp. // Having a fixed number of events that use the same timestamp will help in allowing us to determine if any events // were dropped or duplicated. We will try to space the event generation over the remainder of the second so that it // roughly completes at the top of the second, but if it doesn't complete, it will still send the remainder of the // events with the original timestamp, even after wall time has moved onto the next second. DateTime nowFlooredToSecond = DateTime.now().secondOfDay().roundFloorCopy(); + DateTime eventTimestamp = overrrideFirstEventTime == null ? nowFlooredToSecond : overrrideFirstEventTime; int seconds = 0; while (true) { @@ -90,7 +98,7 @@ public void start(EventWriter eventWriter) ); for (int i = 1; i <= eventsPerSecond; i++) { - eventWriter.write(MAPPER.writeValueAsString(getEvent(i, nowFlooredToSecond))); + eventWriter.write(MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowFlooredToSecond); if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { @@ -103,6 +111,7 @@ public void start(EventWriter eventWriter) } nowFlooredToSecond = nowFlooredToSecond.plusSeconds(1); + eventTimestamp = eventTimestamp.plusSeconds(1); seconds++; log.info( @@ -112,7 +121,7 @@ public void start(EventWriter eventWriter) nowFlooredToSecond ); - if (seconds > totalNumberOfSecond) { + if (seconds >= totalNumberOfSecond) { log.info( "Finished writing {} seconds", seconds diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java new file mode 100644 index 000000000000..4464430ef080 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java @@ -0,0 +1,63 @@ +/* + * 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.calcite.plan.Strong; +import org.apache.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +public class WikipediaStreamEventGenerator extends SyntheticGenerator +{ + private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"); + + public WikipediaStreamEventGenerator(int eventsPerSeconds, long cyclePaddingMs, int totalNumberOfSeconds) + { + super(eventsPerSeconds, cyclePaddingMs, totalNumberOfSeconds); + } + + @Override + Object getEvent(int i, DateTime timestamp) + { + Map event = new HashMap<>(); + event.put("page", "Gypsy Danger"); + event.put("language", "en"); + event.put("user", "nuclear"); + event.put("unpatrolled", "true"); + event.put("newPage", "true"); + event.put("robot", "false"); + event.put("anonymous", "false"); + event.put("namespace", "article"); + event.put("continent", "North Americ"); + event.put("country", "United States"); + event.put("region", "Bay Area"); + event.put("city", "San Francisco"); + event.put("timestamp", DATE_TIME_FORMATTER.print(timestamp)); + event.put("added", i); + event.put("deleted", 0); + event.put("delta", i); + return event; + } +} 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 79be37bd1ee2..e444bd41c256 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 @@ -25,6 +25,7 @@ */ public class TestNGGroup { + public static final String KINESIS_INDEX = "kinesis-index"; public static final String BATCH_INDEX = "batch-index"; public static final String HADOOP_INDEX = "hadoop-index"; public static final String KAFKA_INDEX = "kafka-index"; 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 87a2cceece20..f56ccbddf3a4 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 @@ -55,9 +55,9 @@ abstract class AbstractKafkaIndexerTest extends AbstractIndexerTest { private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); - protected static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/kafka_supervisor_spec_legacy_parser.json"; - protected static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/kafka_supervisor_spec_input_format.json"; - private static final String QUERIES_FILE = "/indexer/kafka_index_queries.json"; + 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; 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 5319c76be70d..1c2b98a7359b 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,6 +19,220 @@ package org.apache.druid.tests.indexer; -public class ITKinesisIndexingServiceTest +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.ITRetryUtil; +import org.apache.druid.testing.utils.KinesisAdminClient; +import org.apache.druid.testing.utils.KinesisEventWriter; +import org.apache.druid.testing.utils.WikipediaStreamEventGenerator; +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.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.UUID; +import java.util.function.Function; + +@Test(groups = TestNGGroup.KINESIS_INDEX) +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest { + private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); + private static final long WAIT_TIME_MILLIS = 2 * 60 * 1000L; + private static final DateTime FIRST_EVENT_TIME = DateTime.parse("1994-04-29T00:00:00.000Z"); + 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; + + private String streamName; + private String fullDatasourceName; + private KinesisAdminClient kinesisAdminClient; + private KinesisEventWriter kinesisEventWriter; + private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; + private Function kinesisIngestionPropsTransform; + private Function kinesisQueryPropsTransform; + + @BeforeClass + public void beforeClass() throws Exception + { + kinesisAdminClient = new KinesisAdminClient(config.getStreamEndpoint()); + } + + @BeforeMethod + public void before() throws Exception + { + streamName = "kinesis_index_test_" + UUID.randomUUID(); + String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); + kinesisAdminClient.createStream(streamName, 2); + ITRetryUtil.retryUntil( + () -> kinesisAdminClient.isStreamActive(streamName), + true, + 10000, + 30, + "Wait for stream active" + ); + kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), streamName, false); + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, TOTAL_NUMBER_OF_SECOND); + fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); + kinesisIngestionPropsTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%STREAM_TYPE%%", + "kinesis" + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_KEY%%", + "stream" + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_VALUE%%", + streamName + ); + spec = StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_KEY%%", + "endpoint" + ); + return StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_VALUE%%", + jsonMapper.writeValueAsString(config.getStreamEndpoint()) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + kinesisQueryPropsTransform = 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)) + ); + 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).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); + } + }; + } + + @AfterMethod + public void teardown() + { +// kinesisAdminClient.deleteStream(streamName); +// wikipediaStreamEventGenerator.shutdown(); +// kinesisEventWriter.shutdown(); + } + + @Test + public void test_x() 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 + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + // Wait for supervisor to consume events + LOG.info("Waiting for [%s] millis for Kafka 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 kafka 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; + } } diff --git a/integration-tests/src/test/resources/indexer/kafka_index_queries.json b/integration-tests/src/test/resources/indexer/stream_index_queries.json similarity index 100% rename from integration-tests/src/test/resources/indexer/kafka_index_queries.json rename to integration-tests/src/test/resources/indexer/stream_index_queries.json diff --git a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json similarity index 86% rename from integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json rename to integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json index 4ba59afdcac7..a2850bb561b0 100644 --- a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json +++ b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json @@ -1,5 +1,5 @@ { - "type": "kafka", + "type": "%%STREAM_TYPE%%", "dataSchema": { "dataSource": "%%DATASOURCE%%", "timestampSpec": { @@ -39,15 +39,15 @@ } }, "tuningConfig": { - "type": "kafka", + "type": "%%STREAM_TYPE%%", "intermediatePersistPeriod": "PT30S", "maxRowsPerSegment": 5000000, "maxRowsInMemory": 500000 }, "ioConfig": { - "topic": "%%TOPIC%%", - "consumerProperties": %%CONSUMER_PROPERTIES%%, - "taskCount": 2, + "%%TOPIC_KEY%%": "%%TOPIC_VALUE%%", + "%%STREAM_PROPERTIES_KEY%%": %%STREAM_PROPERTIES_VALUE%%, + "taskCount": 1, "replicas": 1, "taskDuration": "PT2M", "useEarliestOffset": true, diff --git a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_legacy_parser.json b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json similarity index 87% rename from integration-tests/src/test/resources/indexer/kafka_supervisor_spec_legacy_parser.json rename to integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json index 511b65dcffc8..09979d622685 100644 --- a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_legacy_parser.json +++ b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json @@ -1,5 +1,5 @@ { - "type": "kafka", + "type": "%%STREAM_TYPE%%", "dataSchema": { "dataSource": "%%DATASOURCE%%", "parser": { @@ -45,15 +45,15 @@ } }, "tuningConfig": { - "type": "kafka", + "type": "%%STREAM_TYPE%%", "intermediatePersistPeriod": "PT30S", "maxRowsPerSegment": 5000000, "maxRowsInMemory": 500000 }, "ioConfig": { - "topic": "%%TOPIC%%", - "consumerProperties": %%CONSUMER_PROPERTIES%%, - "taskCount": 2, + "%%TOPIC_KEY%%": "%%TOPIC_VALUE%%", + "%%STREAM_PROPERTIES_KEY%%": %%STREAM_PROPERTIES_VALUE%%, + "taskCount": 1, "replicas": 1, "taskDuration": "PT2M", "useEarliestOffset": true From e0b0e30a49fbdeff3a354027b32e7e30a664f2d1 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 27 Mar 2020 17:15:22 -1000 Subject: [PATCH 04/22] Kinesis IT --- .../extensions-core/kinesis-ingestion.md | 2 +- integration-tests/README.md | 17 +++++++- .../docker/environment-configs/broker | 2 +- .../docker/environment-configs/coordinator | 2 +- .../docker/environment-configs/historical | 2 +- .../docker/environment-configs/middlemanager | 2 +- .../docker/environment-configs/overlord | 2 +- .../docker/environment-configs/router | 2 +- .../router-custom-check-tls | 2 +- .../router-no-client-auth-tls | 2 +- .../environment-configs/router-permissive-tls | 2 +- integration-tests/run_cluster.sh | 18 ++++---- .../testing/utils/KinesisAdminClient.java | 16 +++++++- .../testing/utils/SyntheticGenerator.java | 16 ++++---- .../utils/WikipediaStreamEventGenerator.java | 3 -- .../org/apache/druid/tests/TestNGGroup.java | 8 +++- .../indexer/AbstractKafkaIndexerTest.java | 8 +++- ...KafkaIndexingServiceTransactionalTest.java | 2 +- .../indexer/ITKinesisIndexingServiceTest.java | 41 +++++++++++-------- .../stream_supervisor_spec_input_format.json | 6 +-- .../stream_supervisor_spec_legacy_parser.json | 6 +-- 21 files changed, 103 insertions(+), 58 deletions(-) diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index bb54e4be9cdd..e0fcc2d00730 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -137,7 +137,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon | `indexSpecForIntermediatePersists` | | Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. This can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values. | no (default = same as indexSpec) | | `reportParseExceptions` | Boolean | If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped. | no (default == false) | | `handoffConditionTimeout` | Long | Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. | no (default == 0) | -| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kinesis messages that are no longer available.

If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.html#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.

If true, Druid will automatically reset to the earlier or latest sequence number available in Kinesis, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Please note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. | no (default == false) | +| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kinesis messages that are no longer available.

If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.html#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.

If true, Druid will automatically reset to the earlier or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Please note that this can lead to data being _DROPPED_ (if `useEarliestSequenceNumber` is false) or _DUPLICATED_ (if `useEarliestSequenceNumber` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. | no (default == false) | | `skipSequenceNumberAvailabilityCheck` | Boolean | Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`. | no (default == false) | | `workerThreads` | Integer | The number of threads that will be used by the supervisor for asynchronous operations. | no (default == min(10, taskCount)) | | `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. | no (default == min(10, taskCount * replicas)) | diff --git a/integration-tests/README.md b/integration-tests/README.md index 0cc5b01c61b3..d1a94a10bf2b 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -66,7 +66,22 @@ Integration tests can also be run with either Java 8 or Java 11 by adding -Djvm. 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. +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 + +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): + +- Overlord process at port 5009 +- Middlemanager process at port 5008 +- Historical process at port 5007 +- Coordinator process at port 5006 +- Broker process at port 5005 +- Router process at port 5004 +- Router with custom check tls process at port 5003 +- Router with no client auth tls process at port 5002 +- Router with permissive tls process at port 5001 Running Tests Using A Configuration File for Any Cluster ------------------- diff --git a/integration-tests/docker/environment-configs/broker b/integration-tests/docker/environment-configs/broker index b8794d4c0bd7..ae2d5611f241 100644 --- a/integration-tests/docker/environment-configs/broker +++ b/integration-tests/docker/environment-configs/broker @@ -21,7 +21,7 @@ DRUID_SERVICE=broker DRUID_LOG_PATH=/shared/logs/broker.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx512m -Xms512m -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx512m -Xms512m -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005 # Druid configs druid_processing_buffer_sizeBytes=25000000 diff --git a/integration-tests/docker/environment-configs/coordinator b/integration-tests/docker/environment-configs/coordinator index de779f62db7d..6bd0260b8131 100644 --- a/integration-tests/docker/environment-configs/coordinator +++ b/integration-tests/docker/environment-configs/coordinator @@ -21,7 +21,7 @@ DRUID_SERVICE=coordinator DRUID_LOG_PATH=/shared/logs/coordinator.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -Xms128m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -Xms128m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5006 # Druid configs druid_metadata_storage_type=mysql diff --git a/integration-tests/docker/environment-configs/historical b/integration-tests/docker/environment-configs/historical index 1f74b0ce2678..a2fcf33a6665 100644 --- a/integration-tests/docker/environment-configs/historical +++ b/integration-tests/docker/environment-configs/historical @@ -21,7 +21,7 @@ DRUID_SERVICE=historical DRUID_LOG_PATH=/shared/logs/historical.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx512m -Xms512m -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx512m -Xms512m -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5007 # Druid configs druid_processing_buffer_sizeBytes=25000000 diff --git a/integration-tests/docker/environment-configs/middlemanager b/integration-tests/docker/environment-configs/middlemanager index c37c3fee8859..9cbe41bce8f8 100644 --- a/integration-tests/docker/environment-configs/middlemanager +++ b/integration-tests/docker/environment-configs/middlemanager @@ -21,7 +21,7 @@ DRUID_SERVICE=middleManager DRUID_LOG_PATH=/shared/logs/middlemanager.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx64m -Xms64m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx64m -Xms64m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5008 # Druid configs druid_server_http_numThreads=100 diff --git a/integration-tests/docker/environment-configs/overlord b/integration-tests/docker/environment-configs/overlord index d86eb196f566..ebb3d5bf18e4 100644 --- a/integration-tests/docker/environment-configs/overlord +++ b/integration-tests/docker/environment-configs/overlord @@ -21,7 +21,7 @@ DRUID_SERVICE=overlord DRUID_LOG_PATH=/shared/logs/overlord.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -Xms128m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -Xms128m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5009 # Druid configs druid_metadata_storage_type=mysql diff --git a/integration-tests/docker/environment-configs/router b/integration-tests/docker/environment-configs/router index b3636b72ceb8..f25b23ee8cfd 100644 --- a/integration-tests/docker/environment-configs/router +++ b/integration-tests/docker/environment-configs/router @@ -21,7 +21,7 @@ DRUID_SERVICE=router DRUID_LOG_PATH=/shared/logs/router.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseG1GC +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5004 # Druid configs druid_auth_basic_common_cacheDirectory=/tmp/authCache/router diff --git a/integration-tests/docker/environment-configs/router-custom-check-tls b/integration-tests/docker/environment-configs/router-custom-check-tls index 07b072495a1e..ece8531d677c 100644 --- a/integration-tests/docker/environment-configs/router-custom-check-tls +++ b/integration-tests/docker/environment-configs/router-custom-check-tls @@ -21,7 +21,7 @@ DRUID_SERVICE=router DRUID_LOG_PATH=/shared/logs/router-custom-check-tls.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5003 # Druid configs druid_plaintextPort=8891 diff --git a/integration-tests/docker/environment-configs/router-no-client-auth-tls b/integration-tests/docker/environment-configs/router-no-client-auth-tls index bc6959cf6134..4b703bac5ee7 100644 --- a/integration-tests/docker/environment-configs/router-no-client-auth-tls +++ b/integration-tests/docker/environment-configs/router-no-client-auth-tls @@ -21,7 +21,7 @@ DRUID_SERVICE=router DRUID_LOG_PATH=/shared/logs/router-no-client-auth-tls.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5002 # Druid configs druid_plaintextPort=8890 diff --git a/integration-tests/docker/environment-configs/router-permissive-tls b/integration-tests/docker/environment-configs/router-permissive-tls index b4beb9fc0808..41346cb15610 100644 --- a/integration-tests/docker/environment-configs/router-permissive-tls +++ b/integration-tests/docker/environment-configs/router-permissive-tls @@ -21,7 +21,7 @@ DRUID_SERVICE=router DRUID_LOG_PATH=/shared/logs/router-permissive-tls.log # JAVA OPTS -SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails +SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5001 # Druid configs druid_plaintextPort=8889 diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh index 81f0e991da8f..f6b5ba206ccc 100755 --- a/integration-tests/run_cluster.sh +++ b/integration-tests/run_cluster.sh @@ -206,29 +206,29 @@ fi docker run -d --privileged --net druid-it-net --ip 172.172.172.3 ${COMMON_ENV} --name druid-metadata-storage -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster # Start Overlord - docker run -d --privileged --net druid-it-net --ip 172.172.172.4 ${COMMON_ENV} ${OVERLORD_ENV} ${OVERRIDE_ENV} --name druid-overlord -p 8090:8090 -p 8290:8290 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.4 ${COMMON_ENV} ${OVERLORD_ENV} ${OVERRIDE_ENV} --name druid-overlord -p 5009:5009 -p 8090:8090 -p 8290:8290 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster # Start Coordinator - docker run -d --privileged --net druid-it-net --ip 172.172.172.5 ${COMMON_ENV} ${COORDINATOR_ENV} ${OVERRIDE_ENV} --name druid-coordinator -p 8081:8081 -p 8281:8281 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-overlord:druid-overlord --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.5 ${COMMON_ENV} ${COORDINATOR_ENV} ${OVERRIDE_ENV} --name druid-coordinator -p 5006:5006 -p 8081:8081 -p 8281:8281 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-overlord:druid-overlord --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster # Start Historical - docker run -d --privileged --net druid-it-net --ip 172.172.172.6 ${COMMON_ENV} ${HISTORICAL_ENV} ${OVERRIDE_ENV} --name druid-historical -p 8083:8083 -p 8283:8283 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.6 ${COMMON_ENV} ${HISTORICAL_ENV} ${OVERRIDE_ENV} --name druid-historical -p 5007:5007 -p 8083:8083 -p 8283:8283 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster # Start Middlemanger - docker run -d --privileged --net druid-it-net --ip 172.172.172.7 ${COMMON_ENV} ${MIDDLEMANAGER_ENV} ${OVERRIDE_ENV} --name druid-middlemanager -p 8091:8091 -p 8291:8291 -p 8100:8100 -p 8101:8101 -p 8102:8102 -p 8103:8103 -p 8104:8104 -p 8105:8105 -p 8300:8300 -p 8301:8301 -p 8302:8302 -p 8303:8303 -p 8304:8304 -p 8305:8305 -v $RESOURCEDIR:/resources -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-overlord:druid-overlord druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.7 ${COMMON_ENV} ${MIDDLEMANAGER_ENV} ${OVERRIDE_ENV} --name druid-middlemanager -p 5008:5008 -p 8091:8091 -p 8291:8291 -p 8100:8100 -p 8101:8101 -p 8102:8102 -p 8103:8103 -p 8104:8104 -p 8105:8105 -p 8300:8300 -p 8301:8301 -p 8302:8302 -p 8303:8303 -p 8304:8304 -p 8305:8305 -v $RESOURCEDIR:/resources -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-overlord:druid-overlord druid/cluster # Start Broker - docker run -d --privileged --net druid-it-net --ip 172.172.172.8 ${COMMON_ENV} ${BROKER_ENV} ${OVERRIDE_ENV} --name druid-broker -p 8082:8082 -p 8282:8282 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-middlemanager:druid-middlemanager --link druid-historical:druid-historical druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.8 ${COMMON_ENV} ${BROKER_ENV} ${OVERRIDE_ENV} --name druid-broker -p 5005:5005 -p 8082:8082 -p 8282:8282 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-middlemanager:druid-middlemanager --link druid-historical:druid-historical druid/cluster # Start Router - docker run -d --privileged --net druid-it-net --ip 172.172.172.9 ${COMMON_ENV} ${ROUTER_ENV} ${OVERRIDE_ENV} --name druid-router -p 8888:8888 -p 9088:9088 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.9 ${COMMON_ENV} ${ROUTER_ENV} ${OVERRIDE_ENV} --name druid-router -p 8888:8888 -p 5004:5004 -p 9088:9088 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster # Start Router with permissive TLS settings (client auth enabled, no hostname verification, no revocation check) - docker run -d --privileged --net druid-it-net --ip 172.172.172.10 ${COMMON_ENV} ${ROUTER_PERMISSIVE_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-permissive-tls -p 8889:8889 -p 9089:9089 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.10 ${COMMON_ENV} ${ROUTER_PERMISSIVE_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-permissive-tls -p 5001:5001 -p 8889:8889 -p 9089:9089 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster # Start Router with TLS but no client auth - docker run -d --privileged --net druid-it-net --ip 172.172.172.11 ${COMMON_ENV} ${ROUTER_NO_CLIENT_AUTH_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-no-client-auth-tls -p 8890:8890 -p 9090:9090 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.11 ${COMMON_ENV} ${ROUTER_NO_CLIENT_AUTH_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-no-client-auth-tls -p 5002:5002 -p 8890:8890 -p 9090:9090 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster # Start Router with custom TLS cert checkers - docker run -d --privileged --net druid-it-net --ip 172.172.172.12 ${COMMON_ENV} ${ROUTER_CUSTOM_CHECK_TLS_ENV} ${OVERRIDE_ENV} --hostname druid-router-custom-check-tls --name druid-router-custom-check-tls -p 8891:8891 -p 9091:9091 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.12 ${COMMON_ENV} ${ROUTER_CUSTOM_CHECK_TLS_ENV} ${OVERRIDE_ENV} --hostname druid-router-custom-check-tls --name druid-router-custom-check-tls -p 5003:5003 -p 8891:8891 -p 9091:9091 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster } \ No newline at end of file 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 0629c0569ac0..a3bfd794b067 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 @@ -25,6 +25,8 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.model.AddTagsToStreamRequest; +import com.amazonaws.services.kinesis.model.AddTagsToStreamResult; import com.amazonaws.services.kinesis.model.CreateStreamResult; import com.amazonaws.services.kinesis.model.DeleteStreamResult; import com.amazonaws.services.kinesis.model.DescribeStreamResult; @@ -33,9 +35,11 @@ import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; +import org.apache.commons.lang.StringUtils; import org.apache.druid.java.util.common.ISE; import java.io.FileInputStream; +import java.util.Map; import java.util.Properties; public class KinesisAdminClient @@ -66,12 +70,22 @@ public KinesisAdminClient(String endpoint) throws Exception )).build(); } - public void createStream(String streamName, int shardCount) + public void createStream(String streamName, int shardCount, Map tags) { CreateStreamResult createStreamResult = amazonKinesis.createStream(streamName, shardCount); if (createStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { throw new ISE("Cannot create stream for integration test"); } + if (tags != null && !tags.isEmpty()) { + AddTagsToStreamRequest addTagsToStreamRequest = new AddTagsToStreamRequest(); + addTagsToStreamRequest.setStreamName(streamName); + addTagsToStreamRequest.setTags(tags); + AddTagsToStreamResult addTagsToStreamResult = amazonKinesis.addTagsToStream(addTagsToStreamRequest); + if (addTagsToStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { + throw new ISE("Cannot tag stream for integration test"); + } + } + } public void deleteStream(String streamName) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index 03cd4978718c..232f07681cfd 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -78,15 +78,15 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) // were dropped or duplicated. We will try to space the event generation over the remainder of the second so that it // roughly completes at the top of the second, but if it doesn't complete, it will still send the remainder of the // events with the original timestamp, even after wall time has moved onto the next second. - DateTime nowFlooredToSecond = DateTime.now().secondOfDay().roundFloorCopy(); - DateTime eventTimestamp = overrrideFirstEventTime == null ? nowFlooredToSecond : overrrideFirstEventTime; + DateTime nowCeilingToSecond = DateTime.now().secondOfDay().roundCeilingCopy(); + DateTime eventTimestamp = overrrideFirstEventTime == null ? nowCeilingToSecond : overrrideFirstEventTime; int seconds = 0; while (true) { try { - long sleepMillis = nowFlooredToSecond.getMillis() - DateTime.now().getMillis(); + long sleepMillis = nowCeilingToSecond.getMillis() - DateTime.now().getMillis(); if (sleepMillis > 0) { - log.info("Waiting {} ms for next run cycle (at {})", sleepMillis, nowFlooredToSecond); + log.info("Waiting {} ms for next run cycle (at {})", sleepMillis, nowCeilingToSecond); Thread.sleep(sleepMillis); continue; } @@ -94,13 +94,13 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) log.info( "Beginning run cycle with {} events, target completion time: {}", eventsPerSecond, - nowFlooredToSecond.plusSeconds(1).minus(cyclePaddingMs) + nowCeilingToSecond.plusSeconds(1).minus(cyclePaddingMs) ); for (int i = 1; i <= eventsPerSecond; i++) { eventWriter.write(MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); - long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowFlooredToSecond); + long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowCeilingToSecond); if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { log.info("Event: {}/{}, sleep time: {} ms", i, eventsPerSecond, sleepTime); } @@ -110,7 +110,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) } } - nowFlooredToSecond = nowFlooredToSecond.plusSeconds(1); + nowCeilingToSecond = nowCeilingToSecond.plusSeconds(1); eventTimestamp = eventTimestamp.plusSeconds(1); seconds++; @@ -118,7 +118,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) "Finished writing {} events, current time: {} - updating next timestamp to: {}", eventsPerSecond, DateTime.now(), - nowFlooredToSecond + nowCeilingToSecond ); if (seconds >= totalNumberOfSecond) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java index 4464430ef080..0a77c1876710 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java @@ -19,15 +19,12 @@ package org.apache.druid.testing.utils; -import org.apache.calcite.plan.Strong; -import org.apache.druid.java.util.common.StringUtils; import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; import java.util.HashMap; import java.util.Map; -import java.util.UUID; public class WikipediaStreamEventGenerator extends SyntheticGenerator { 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 e444bd41c256..ec7f605beb8b 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 @@ -25,12 +25,18 @@ */ public class TestNGGroup { - public static final String KINESIS_INDEX = "kinesis-index"; public static final String BATCH_INDEX = "batch-index"; public static final String HADOOP_INDEX = "hadoop-index"; public static final String KAFKA_INDEX = "kafka-index"; public static final String OTHER_INDEX = "other-index"; public static final String PERFECT_ROLLUP_PARALLEL_BATCH_INDEX = "perfect-rollup-parallel-batch-index"; + /** + * This group is not part of CI. To run this group, kinesis configs/credentials for your kinesis must be provided in a file. + * The path of the file must then be pass to mvn with -Doverride.config.path= + * See integration-tests/docker/environment-configs/override-examples/s3 for env vars to provide. + * + */ + public static final String KINESIS_INDEX = "kinesis-index"; // This group can only be run individually using -Dgroups=query since it requires specific test data setup. public static final String QUERY = "query"; public static final String REALTIME_INDEX = "realtime-index"; 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 f56ccbddf3a4..eea0f7fb2ae5 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 @@ -137,8 +137,12 @@ void doKafkaIndexTest(String dataSourceName, String supervisorSpecPath, boolean spec = getResourceAsString(supervisorSpecPath); spec = StringUtils.replace(spec, "%%DATASOURCE%%", fullDatasourceName); - spec = StringUtils.replace(spec, "%%TOPIC%%", TOPIC_NAME); - spec = StringUtils.replace(spec, "%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties)); + 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) { 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 index 04c52b2f97d7..f32b82433f1a 100644 --- 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 @@ -54,7 +54,7 @@ public void testKafka(String param) ? INDEXER_FILE_LEGACY_PARSER : INDEXER_FILE_INPUT_FORMAT; LOG.info("Starting test: ITKafkaIndexingServiceTransactionalTest"); - doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, false); + doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, true); } @AfterMethod 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 1c2b98a7359b..b24e8ba9964b 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,6 +19,7 @@ package org.apache.druid.tests.indexer; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; @@ -37,6 +38,7 @@ import org.testng.annotations.Test; import java.io.Closeable; +import java.util.Map; import java.util.UUID; import java.util.function.Function; @@ -45,7 +47,8 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest { private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); - private static final long WAIT_TIME_MILLIS = 2 * 60 * 1000L; + private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after"; + private static final long WAIT_TIME_MILLIS = 60 * 1000L; private static final DateTime FIRST_EVENT_TIME = DateTime.parse("1994-04-29T00:00:00.000Z"); 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"; @@ -77,7 +80,8 @@ public void before() throws Exception { streamName = "kinesis_index_test_" + UUID.randomUUID(); String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); - kinesisAdminClient.createStream(streamName, 2); + Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTime.now().plusMinutes(30).getMillis())); + kinesisAdminClient.createStream(streamName, 2, tags); ITRetryUtil.retryUntil( () -> kinesisAdminClient.isStreamActive(streamName), true, @@ -110,6 +114,11 @@ public void before() throws Exception "%%TOPIC_VALUE%%", streamName ); + spec = StringUtils.replace( + spec, + "%%USE_EARLIEST_KEY%%", + "useEarliestSequenceNumber" + ); spec = StringUtils.replace( spec, "%%STREAM_PROPERTIES_KEY%%", @@ -140,7 +149,7 @@ public void before() throws Exception spec = StringUtils.replace( spec, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", - TIMESTAMP_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND)) + TIMESTAMP_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND-1)) ); spec = StringUtils.replace( spec, @@ -155,7 +164,7 @@ public void before() throws Exception spec = StringUtils.replace( spec, "%%TIMESERIES_QUERY_END%%", - INTERVAL_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND).plusMinutes(2)) + INTERVAL_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND-1).plusMinutes(2)) ); spec = StringUtils.replace( spec, @@ -165,7 +174,7 @@ public void before() throws Exception spec = StringUtils.replace( spec, "%%TIMESERIES_ADDED%%", - Long.toString(getSumOfEventSequence(EVENTS_PER_SECOND*TOTAL_NUMBER_OF_SECOND)) + Long.toString(getSumOfEventSequence(EVENTS_PER_SECOND) * TOTAL_NUMBER_OF_SECOND) ); return StringUtils.replace( spec, @@ -182,9 +191,9 @@ public void before() throws Exception @AfterMethod public void teardown() { -// kinesisAdminClient.deleteStream(streamName); -// wikipediaStreamEventGenerator.shutdown(); -// kinesisEventWriter.shutdown(); + kinesisAdminClient.deleteStream(streamName); + wikipediaStreamEventGenerator.shutdown(); + kinesisEventWriter.shutdown(); } @Test @@ -209,14 +218,14 @@ public void test_x() throws Exception this.queryHelper.testQueriesFromString(querySpec, 2); LOG.info("Shutting down supervisor"); indexer.shutdownSupervisor(supervisorId); - // wait for all kafka 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 all kafka 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), diff --git a/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json index a2850bb561b0..ce9bedc84431 100644 --- a/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json +++ b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_input_format.json @@ -47,10 +47,10 @@ "ioConfig": { "%%TOPIC_KEY%%": "%%TOPIC_VALUE%%", "%%STREAM_PROPERTIES_KEY%%": %%STREAM_PROPERTIES_VALUE%%, - "taskCount": 1, + "taskCount": 2, "replicas": 1, - "taskDuration": "PT2M", - "useEarliestOffset": true, + "taskDuration": "PT5M", + "%%USE_EARLIEST_KEY%%": true, "inputFormat" : { "type" : "json" } diff --git a/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json index 09979d622685..623aadf6583b 100644 --- a/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json +++ b/integration-tests/src/test/resources/indexer/stream_supervisor_spec_legacy_parser.json @@ -53,9 +53,9 @@ "ioConfig": { "%%TOPIC_KEY%%": "%%TOPIC_VALUE%%", "%%STREAM_PROPERTIES_KEY%%": %%STREAM_PROPERTIES_VALUE%%, - "taskCount": 1, + "taskCount": 2, "replicas": 1, - "taskDuration": "PT2M", - "useEarliestOffset": true + "taskDuration": "PT5M", + "%%USE_EARLIEST_KEY%%": true } } From 77599e6a247af7d56aefaf56ed6dfd6a7477676c Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 27 Mar 2020 17:39:35 -1000 Subject: [PATCH 05/22] Kinesis IT --- .travis.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index e7582d5ae716..d91d92497c13 100644 --- a/.travis.yml +++ b/.travis.yml @@ -344,7 +344,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' 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' 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 +383,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' 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' JVM_RUNTIME='-Djvm.runtime=11' # END - Integration tests for Compile with Java 8 and Run with Java 11 - name: "security vulnerabilities" From d76fcc44dd3cfd3168af07df405d73732df62b50 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 28 Mar 2020 22:46:38 -1000 Subject: [PATCH 06/22] Kinesis IT --- integration-tests/pom.xml | 2 +- .../clients/OverlordResourceTestClient.java | 99 ++++++++ .../indexer/ITKinesisIndexingServiceTest.java | 215 ++++++++++++++++-- 3 files changed, 290 insertions(+), 26 deletions(-) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index aa847da09c7e..baf6b7696555 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -74,7 +74,7 @@ org.apache.druid.extensions druid-kinesis-indexing-service ${project.parent.version} - runtime + compile
org.apache.druid.extensions 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 6ad64116351f..4a3d314ec057 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 @@ -24,8 +24,13 @@ import com.google.common.base.Predicates; import com.google.inject.Inject; import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorReportPayload; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; @@ -287,6 +292,100 @@ public void shutdownSupervisor(String id) } } + public SupervisorStateManager.BasicState getSupervisorStatus(String id) + { + try { + StatusResponseHolder response = httpClient.go( + new Request( + HttpMethod.GET, + new URL(StringUtils.format( + "%ssupervisor/%s/status", + getIndexerURL(), + StringUtils.urlEncode(id) + )) + ), + StatusResponseHandler.getInstance() + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while getting supervisor status, response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + Map responseData = jsonMapper.readValue( + response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + Map payload = jsonMapper.convertValue( + responseData.get("payload"), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + String state = (String) payload.get("state"); + LOG.info("Supervisor id[%s] has state [%s]", id, state); + return SupervisorStateManager.BasicState.valueOf(state); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void suspendSupervisor(String id) + { + try { + StatusResponseHolder response = httpClient.go( + new Request( + HttpMethod.POST, + new URL(StringUtils.format( + "%ssupervisor/%s/suspend", + getIndexerURL(), + StringUtils.urlEncode(id) + )) + ), + StatusResponseHandler.getInstance() + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while suspending supervisor, response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + LOG.info("Suspended supervisor with id[%s]", id); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void resumeSupervisor(String id) + { + try { + StatusResponseHolder response = httpClient.go( + new Request( + HttpMethod.POST, + new URL(StringUtils.format( + "%ssupervisor/%s/resume", + getIndexerURL(), + StringUtils.urlEncode(id) + )) + ), + StatusResponseHandler.getInstance() + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while resuming supervisor, response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + LOG.info("Resumed supervisor with id[%s]", id); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + private StatusResponseHolder makeRequest(HttpMethod method, String url) { try { 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 b24e8ba9964b..3c9f2d00a07a 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 @@ -20,6 +20,7 @@ package org.apache.druid.tests.indexer; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; @@ -47,6 +48,7 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest { private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); + private static final int KINESIS_SHARD_COUNT = 2; private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after"; private static final long WAIT_TIME_MILLIS = 60 * 1000L; private static final DateTime FIRST_EVENT_TIME = DateTime.parse("1994-04-29T00:00:00.000Z"); @@ -68,6 +70,8 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; private Function kinesisIngestionPropsTransform; private Function kinesisQueryPropsTransform; + private int secondsToGenerateRemaining; + @BeforeClass public void beforeClass() throws Exception @@ -81,7 +85,7 @@ 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(DateTime.now().plusMinutes(30).getMillis())); - kinesisAdminClient.createStream(streamName, 2, tags); + kinesisAdminClient.createStream(streamName, KINESIS_SHARD_COUNT, tags); ITRetryUtil.retryUntil( () -> kinesisAdminClient.isStreamActive(streamName), true, @@ -90,6 +94,7 @@ public void before() throws Exception "Wait for stream active" ); kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), streamName, false); + secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND; wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, TOTAL_NUMBER_OF_SECOND); fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); kinesisIngestionPropsTransform = spec -> { @@ -196,8 +201,7 @@ public void teardown() kinesisEventWriter.shutdown(); } - @Test - public void test_x() throws Exception + public void testKineseIndexDataWithLegacyParserStableState() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -209,37 +213,198 @@ public void test_x() throws Exception LOG.info("Submitted supervisor"); // Start Kinesis data generator wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - // Wait for supervisor to consume events - LOG.info("Waiting for [%s] millis for Kafka 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 kafka 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 + verifyIngestedData(supervisorId); + } + } + + public void testKineseIndexDataWithInputFormatStableState() 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 + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } + +// public void testKineseIndexDataWithLosingCoordinator() 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 +//// String supervisorId = indexer.submitSupervisor(taskSpec); +//// LOG.info("Submitted supervisor"); +//// // Start Kinesis data generator +//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +//// verifyIngestedData(supervisorId); +// } +// } +// +// public void testKineseIndexDataWithLosingOverlord() 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 +//// String supervisorId = indexer.submitSupervisor(taskSpec); +//// LOG.info("Submitted supervisor"); +//// // Start Kinesis data generator +//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +//// verifyIngestedData(supervisorId); +// } +// } +// +// public void testKineseIndexDataWithLosingHistorical() 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 +//// String supervisorId = indexer.submitSupervisor(taskSpec); +//// LOG.info("Submitted supervisor"); +//// // Start Kinesis data generator +//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +//// verifyIngestedData(supervisorId); +// } +// } +// + public void testKineseIndexDataWithStartStopSupervisor() 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 + String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + // Suspend the supervisor + indexer.suspendSupervisor(supervisorId); + // Start generating remainning half of the data + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + // Resume the supervisor + indexer.resumeSupervisor(supervisorId); + // Verify supervisor is healthy after suspension ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), + () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), true, 10000, 30, - "Real-time generated segments loaded" + "Waiting for supervisor to be healthy" ); + // Verify that supervisor can catch up with the stream + verifyIngestedData(supervisorId); + } + } + + public void testKineseIndexDataWithKinesisReshardSplit() throws Exception + { + // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 + testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); + } - // 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); + public void testKineseIndexDataWithKinesisReshardMerge() throws Exception + { + // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 + testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); + } + + private void testKineseIndexDataWithKinesisReshardHelper(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 + String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + // Reshard the supervisor by split from KINESIS_SHARD_COUNT to newShardCount + kinesisAdminClient.updateShardCount(streamName, newShardCount); + // Start generating one third of the data (while resharding) + int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateSecondRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + // 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); } } + private void verifyIngestedData(String supervisorId) throws Exception + { + // Wait for supervisor to consume events + LOG.info("Waiting for [%s] millis for Kafka 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 kafka 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; From fd928f2ca33274e26b1a351840556927163c4c17 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 29 Mar 2020 17:27:41 -1000 Subject: [PATCH 07/22] Kinesis IT --- integration-tests/pom.xml | 28 +++ .../testing/utils/DruidDockerAdminClient.java | 38 +++ .../druid/tests/indexer/ITIndexerTest.java | 2 + .../indexer/ITKinesisIndexingServiceTest.java | 222 +++++++++--------- 4 files changed, 182 insertions(+), 108 deletions(-) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index baf6b7696555..119039a8a793 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -234,6 +234,27 @@ com.google.code.findbugs jsr305 + + com.github.docker-java + docker-java + 3.2.0 + + + com.github.docker-java + docker-java-transport-jersey + + + io.netty + netty-transport-native-kqueue + + + + + io.netty + netty-transport-native-kqueue + ${netty4.version} + ${os.detected.name}-${os.detected.arch} + @@ -273,6 +294,13 @@ + + + kr.motd.maven + os-maven-plugin + 1.6.1 + + org.apache.maven.plugins diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java new file mode 100644 index 000000000000..664352e176f7 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java @@ -0,0 +1,38 @@ +/* + * 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.github.dockerjava.api.DockerClient; +import com.github.dockerjava.core.DockerClientBuilder; +import com.github.dockerjava.netty.NettyDockerCmdExecFactory; + +public class DruidDockerAdminClient +{ + private static final DockerClient docker = DockerClientBuilder.getInstance() + .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()) + .withConnectTimeout(10 * 1000)) + .build(); + + public static void test() + { + System.out.println("**********"); + System.out.println(docker.listImagesCmd().withShowAll(true).exec()); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index 95b16dee3d7f..7922231c95ba 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -20,6 +20,7 @@ package org.apache.druid.tests.indexer; import org.apache.druid.testing.guice.DruidTestModuleFactory; +import org.apache.druid.testing.utils.DruidDockerAdminClient; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; import org.testng.annotations.Test; @@ -48,6 +49,7 @@ public class ITIndexerTest extends AbstractITBatchIndexTest @Test public void testIndexData() throws Exception { + DruidDockerAdminClient.test(); final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testIndexData-druidInputSource"; try ( 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 3c9f2d00a07a..e65f2c6b3d49 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 @@ -201,132 +201,138 @@ public void teardown() kinesisEventWriter.shutdown(); } - public void testKineseIndexDataWithLegacyParserStableState() throws Exception +// @Test +// public void testKineseIndexDataWithLegacyParserStableState() 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 +// String supervisorId = indexer.submitSupervisor(taskSpec); +// LOG.info("Submitted supervisor"); +// // Start Kinesis data generator +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// verifyIngestedData(supervisorId); +// } +// } +// +// @Test +// public void testKineseIndexDataWithInputFormatStableState() 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 +// String supervisorId = indexer.submitSupervisor(taskSpec); +// LOG.info("Submitted supervisor"); +// // Start Kinesis data generator +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// verifyIngestedData(supervisorId); +// } +// } + + public void testKineseIndexDataWithLosingCoordinator() 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 - String supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); +// final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); +// LOG.info("supervisorSpec: [%s]\n", taskSpec); +// // Start supervisor +// String supervisorId = indexer.submitSupervisor(taskSpec); +// LOG.info("Submitted supervisor"); +// // Start Kinesis data generator +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// verifyIngestedData(supervisorId); } } - public void testKineseIndexDataWithInputFormatStableState() throws Exception + public void testKineseIndexDataWithLosingOverlord() 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 - String supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); +// final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); +// LOG.info("supervisorSpec: [%s]\n", taskSpec); +// // Start supervisor +// String supervisorId = indexer.submitSupervisor(taskSpec); +// LOG.info("Submitted supervisor"); +// // Start Kinesis data generator +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// verifyIngestedData(supervisorId); } } -// public void testKineseIndexDataWithLosingCoordinator() throws Exception + public void testKineseIndexDataWithLosingHistorical() 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 +// String supervisorId = indexer.submitSupervisor(taskSpec); +// LOG.info("Submitted supervisor"); +// // Start Kinesis data generator +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// verifyIngestedData(supervisorId); + } + } + +// +// @Test +// public void testKineseIndexDataWithStartStopSupervisor() 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 -//// String supervisorId = indexer.submitSupervisor(taskSpec); -//// LOG.info("Submitted supervisor"); -//// // Start Kinesis data generator -//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -//// verifyIngestedData(supervisorId); +// final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); +// LOG.info("supervisorSpec: [%s]\n", taskSpec); +// // Start supervisor +// String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); +// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); +// // Suspend the supervisor +// indexer.suspendSupervisor(supervisorId); +// // Start generating remainning half of the data +// wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); +// wikipediaStreamEventGenerator.start(kinesisEventWriter, 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); // } // } // -// public void testKineseIndexDataWithLosingOverlord() throws Exception +// @Test +// public void testKineseIndexDataWithKinesisReshardSplit() 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 -//// String supervisorId = indexer.submitSupervisor(taskSpec); -//// LOG.info("Submitted supervisor"); -//// // Start Kinesis data generator -//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -//// verifyIngestedData(supervisorId); -// } +// // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 +// testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); // } // -// public void testKineseIndexDataWithLosingHistorical() throws Exception +// @Test +// public void testKineseIndexDataWithKinesisReshardMerge() 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 -//// String supervisorId = indexer.submitSupervisor(taskSpec); -//// LOG.info("Submitted supervisor"); -//// // Start Kinesis data generator -//// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -//// verifyIngestedData(supervisorId); -// } +// // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 +// testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); // } -// - public void testKineseIndexDataWithStartStopSupervisor() 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 - String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - // Suspend the supervisor - indexer.suspendSupervisor(supervisorId); - // Start generating remainning half of the data - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); - wikipediaStreamEventGenerator.start(kinesisEventWriter, 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); - } - } - - public void testKineseIndexDataWithKinesisReshardSplit() throws Exception - { - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 - testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); - } - - public void testKineseIndexDataWithKinesisReshardMerge() throws Exception - { - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 - testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); - } private void testKineseIndexDataWithKinesisReshardHelper(int newShardCount) throws Exception { @@ -385,14 +391,14 @@ private void verifyIngestedData(String supervisorId) throws Exception this.queryHelper.testQueriesFromString(querySpec, 2); LOG.info("Shutting down supervisor"); indexer.shutdownSupervisor(supervisorId); -// // wait for all kafka 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 all kafka 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), From 08edce8a4a796ae50c205fab182de4709e677ae5 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 29 Mar 2020 18:07:35 -1000 Subject: [PATCH 08/22] Kinesis IT --- integration-tests/pom.xml | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 119039a8a793..bd8eef411218 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -31,15 +31,11 @@ 0.18.0-SNAPSHOT - - 0.13.1 - - com.amazonaws amazon-kinesis-producer - ${kinesis.producer.version} + 0.13.1 com.amazonaws @@ -253,7 +249,7 @@ io.netty netty-transport-native-kqueue ${netty4.version} - ${os.detected.name}-${os.detected.arch} + osx-x86_64 @@ -294,13 +290,6 @@ - - - kr.motd.maven - os-maven-plugin - 1.6.1 - - org.apache.maven.plugins From b70712a292c7c2e8f4ff67e550a3fb83fb88fac3 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 29 Mar 2020 20:53:03 -1000 Subject: [PATCH 09/22] Kinesis IT --- integration-tests/pom.xml | 22 +- .../clients/OverlordResourceTestClient.java | 4 - .../testing/utils/DruidDockerAdminClient.java | 11 +- .../testing/utils/KinesisAdminClient.java | 3 +- .../testing/utils/KinesisEventWriter.java | 5 +- .../utils/StreamVerifierEventGenerator.java | 5 +- .../testing/utils/SyntheticGenerator.java | 14 +- .../druid/tests/indexer/ITIndexerTest.java | 2 - .../indexer/ITKinesisIndexingServiceTest.java | 225 +++++++++--------- 9 files changed, 153 insertions(+), 138 deletions(-) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index bd8eef411218..b86a1d41f9cb 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -49,6 +49,15 @@ + + com.amazonaws + aws-java-sdk-core + ${aws.sdk.version} + + + commons-codec + commons-codec + org.apache.druid druid-core @@ -70,7 +79,7 @@ org.apache.druid.extensions druid-kinesis-indexing-service ${project.parent.version} - compile + runtime org.apache.druid.extensions @@ -245,11 +254,22 @@ + + com.github.docker-java + docker-java-transport-netty + 3.2.0 + + + com.github.docker-java + docker-java-api + 3.2.0 + io.netty netty-transport-native-kqueue ${netty4.version} osx-x86_64 + runtime 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 4a3d314ec057..20f6a76540d5 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 @@ -24,13 +24,9 @@ import com.google.common.base.Predicates; import com.google.inject.Inject; import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorReportPayload; -import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java index 664352e176f7..219b68a3e2e4 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java @@ -25,14 +25,13 @@ public class DruidDockerAdminClient { - private static final DockerClient docker = DockerClientBuilder.getInstance() - .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()) - .withConnectTimeout(10 * 1000)) - .build(); - + private static final DockerClient DOCKER_CLIENT = DockerClientBuilder.getInstance() + .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()) + .withConnectTimeout(10 * 1000)) + .build(); public static void test() { System.out.println("**********"); - System.out.println(docker.listImagesCmd().withShowAll(true).exec()); + System.out.println(DOCKER_CLIENT.listImagesCmd().withShowAll(true).exec()); } } 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 a3bfd794b067..49099e3838be 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 @@ -35,7 +35,6 @@ import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; -import org.apache.commons.lang.StringUtils; import org.apache.druid.java.util.common.ISE; import java.io.FileInputStream; @@ -70,7 +69,7 @@ public KinesisAdminClient(String endpoint) throws Exception )).build(); } - public void createStream(String streamName, int shardCount, Map tags) + public void createStream(String streamName, int shardCount, Map tags) { CreateStreamResult createStreamResult = amazonKinesis.createStream(streamName, shardCount); if (createStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { 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 61790ed00870..e015b8ff2f45 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 @@ -25,8 +25,7 @@ import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; import com.amazonaws.util.AwsHostNameUtils; import org.apache.commons.codec.digest.DigestUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.druid.java.util.common.logger.Logger; import java.io.FileInputStream; import java.nio.ByteBuffer; @@ -35,7 +34,7 @@ public class KinesisEventWriter implements EventWriter { - private static final Logger LOG = LoggerFactory.getLogger(KinesisEventWriter.class); + private static final Logger LOG = new Logger(KinesisEventWriter.class); private final String streamName; private final KinesisProducer kinesisProducer; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java index 8d4ccddafd1c..35659256eecb 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java @@ -19,6 +19,7 @@ package org.apache.druid.testing.utils; +import org.apache.druid.java.util.common.DateTimes; import org.joda.time.DateTime; import java.util.UUID; @@ -33,10 +34,10 @@ public StreamVerifierEventGenerator(int eventsPerSeconds, long cyclePaddingMs, i @Override Object getEvent(int i, DateTime timestamp) { - return StreamVerifierSyntheticEvent.of( + return StreamVerifierSyntheticEvent.of( UUID.randomUUID().toString(), timestamp.getMillis(), - DateTime.now().getMillis(), + DateTimes.nowUtc().getMillis(), i, i == getEventsPerSecond() ? getSumOfEventSequence(getEventsPerSecond()) : null, i == 1 diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index 232f07681cfd..65db43bc5178 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -23,13 +23,13 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.logger.Logger; import org.joda.time.DateTime; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public abstract class SyntheticGenerator implements Generator { - private static final Logger log = LoggerFactory.getLogger(SyntheticGenerator.class); + private static final Logger log = new Logger(SyntheticGenerator.class); static final ObjectMapper MAPPER = new DefaultObjectMapper(); static { @@ -78,13 +78,13 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) // were dropped or duplicated. We will try to space the event generation over the remainder of the second so that it // roughly completes at the top of the second, but if it doesn't complete, it will still send the remainder of the // events with the original timestamp, even after wall time has moved onto the next second. - DateTime nowCeilingToSecond = DateTime.now().secondOfDay().roundCeilingCopy(); + DateTime nowCeilingToSecond = DateTimes.nowUtc().secondOfDay().roundCeilingCopy(); DateTime eventTimestamp = overrrideFirstEventTime == null ? nowCeilingToSecond : overrrideFirstEventTime; int seconds = 0; while (true) { try { - long sleepMillis = nowCeilingToSecond.getMillis() - DateTime.now().getMillis(); + long sleepMillis = nowCeilingToSecond.getMillis() - DateTimes.nowUtc().getMillis(); if (sleepMillis > 0) { log.info("Waiting {} ms for next run cycle (at {})", sleepMillis, nowCeilingToSecond); Thread.sleep(sleepMillis); @@ -117,7 +117,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) log.info( "Finished writing {} events, current time: {} - updating next timestamp to: {}", eventsPerSecond, - DateTime.now(), + DateTimes.nowUtc(), nowCeilingToSecond ); @@ -149,7 +149,7 @@ private long calculateSleepTimeMs(long eventsRemaining, DateTime secondBeingProc return 0; } - DateTime now = DateTime.now(); + DateTime now = DateTimes.nowUtc(); DateTime nextSecondToProcessMinusBuffer = secondBeingProcessed.plusSeconds(1).minus(cyclePaddingMs); if (nextSecondToProcessMinusBuffer.isBefore(now)) { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index 7922231c95ba..95b16dee3d7f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -20,7 +20,6 @@ package org.apache.druid.tests.indexer; import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.DruidDockerAdminClient; import org.apache.druid.tests.TestNGGroup; import org.testng.annotations.Guice; import org.testng.annotations.Test; @@ -49,7 +48,6 @@ public class ITIndexerTest extends AbstractITBatchIndexTest @Test public void testIndexData() throws Exception { - DruidDockerAdminClient.test(); final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testIndexData-druidInputSource"; try ( 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 e65f2c6b3d49..18ca091bdb11 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 @@ -21,9 +21,11 @@ import com.google.common.collect.ImmutableMap; 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.DruidDockerAdminClient; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.KinesisAdminClient; import org.apache.druid.testing.utils.KinesisEventWriter; @@ -82,9 +84,10 @@ public void beforeClass() throws Exception @BeforeMethod public void before() throws Exception { + DruidDockerAdminClient.test(); streamName = "kinesis_index_test_" + UUID.randomUUID(); String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); - Map tags = ImmutableMap.of(STREAM_EXPIRE_TAG, Long.toString(DateTime.now().plusMinutes(30).getMillis())); + 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), @@ -154,7 +157,7 @@ public void before() throws Exception spec = StringUtils.replace( spec, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", - TIMESTAMP_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND-1)) + TIMESTAMP_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND - 1)) ); spec = StringUtils.replace( spec, @@ -169,7 +172,7 @@ public void before() throws Exception spec = StringUtils.replace( spec, "%%TIMESERIES_QUERY_END%%", - INTERVAL_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND-1).plusMinutes(2)) + INTERVAL_FMT.print(FIRST_EVENT_TIME.plusSeconds(TOTAL_NUMBER_OF_SECOND - 1).plusMinutes(2)) ); spec = StringUtils.replace( spec, @@ -184,7 +187,7 @@ public void before() throws Exception return StringUtils.replace( spec, "%%TIMESERIES_NUMEVENTS%%", - Integer.toString(EVENTS_PER_SECOND*TOTAL_NUMBER_OF_SECOND) + Integer.toString(EVENTS_PER_SECOND * TOTAL_NUMBER_OF_SECOND) ); } catch (Exception e) { @@ -201,53 +204,53 @@ public void teardown() kinesisEventWriter.shutdown(); } -// @Test -// public void testKineseIndexDataWithLegacyParserStableState() 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 -// String supervisorId = indexer.submitSupervisor(taskSpec); -// LOG.info("Submitted supervisor"); -// // Start Kinesis data generator -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// verifyIngestedData(supervisorId); -// } -// } -// -// @Test -// public void testKineseIndexDataWithInputFormatStableState() 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 -// String supervisorId = indexer.submitSupervisor(taskSpec); -// LOG.info("Submitted supervisor"); -// // Start Kinesis data generator -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// verifyIngestedData(supervisorId); -// } -// } + @Test + public void testKineseIndexDataWithLegacyParserStableState() 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 + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } + + @Test + public void testKineseIndexDataWithInputFormatStableState() 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 + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } public void testKineseIndexDataWithLosingCoordinator() 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 -// String supervisorId = indexer.submitSupervisor(taskSpec); -// LOG.info("Submitted supervisor"); -// // Start Kinesis data generator -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// verifyIngestedData(supervisorId); + final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); } } @@ -256,14 +259,14 @@ public void testKineseIndexDataWithLosingOverlord() 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 -// String supervisorId = indexer.submitSupervisor(taskSpec); -// LOG.info("Submitted supervisor"); -// // Start Kinesis data generator -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// verifyIngestedData(supervisorId); + final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); } } @@ -272,67 +275,67 @@ public void testKineseIndexDataWithLosingHistorical() 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 -// String supervisorId = indexer.submitSupervisor(taskSpec); -// LOG.info("Submitted supervisor"); -// // Start Kinesis data generator -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// verifyIngestedData(supervisorId); + final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); + LOG.info("supervisorSpec: [%s]\n", taskSpec); + // Start supervisor + String supervisorId = indexer.submitSupervisor(taskSpec); + LOG.info("Submitted supervisor"); + // Start Kinesis data generator + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + verifyIngestedData(supervisorId); + } + } + + + @Test + public void testKineseIndexDataWithStartStopSupervisor() 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 + String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + // Suspend the supervisor + indexer.suspendSupervisor(supervisorId); + // Start generating remainning half of the data + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); + wikipediaStreamEventGenerator.start(kinesisEventWriter, 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); } } -// -// @Test -// public void testKineseIndexDataWithStartStopSupervisor() 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 -// String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); -// wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); -// // Suspend the supervisor -// indexer.suspendSupervisor(supervisorId); -// // Start generating remainning half of the data -// wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); -// wikipediaStreamEventGenerator.start(kinesisEventWriter, 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); -// } -// } -// -// @Test -// public void testKineseIndexDataWithKinesisReshardSplit() throws Exception -// { -// // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 -// testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); -// } -// -// @Test -// public void testKineseIndexDataWithKinesisReshardMerge() throws Exception -// { -// // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 -// testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); -// } + @Test + public void testKineseIndexDataWithKinesisReshardSplit() throws Exception + { + // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 + testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); + } + + @Test + public void testKineseIndexDataWithKinesisReshardMerge() throws Exception + { + // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 + testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); + } private void testKineseIndexDataWithKinesisReshardHelper(int newShardCount) throws Exception { From 7a26fa690b8ff851be47a9d83771c852f1938672 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sun, 29 Mar 2020 22:19:18 -1000 Subject: [PATCH 10/22] Kinesis IT --- .../utils/DruidClusterAdminClient.java | 161 ++++++++++++++++++ .../testing/utils/DruidDockerAdminClient.java | 37 ---- .../testing/utils/KinesisAdminClient.java | 6 +- .../org/testng/DruidTestRunnerFactory.java | 33 +--- .../indexer/ITKinesisIndexingServiceTest.java | 104 +++++------ 5 files changed, 225 insertions(+), 116 deletions(-) create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java delete mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java new file mode 100644 index 000000000000..48519e103594 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java @@ -0,0 +1,161 @@ +/* + * 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.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.model.Container; +import com.github.dockerjava.core.DockerClientBuilder; +import com.github.dockerjava.netty.NettyDockerCmdExecFactory; +import com.google.inject.Inject; +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.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.TestClient; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.net.URL; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +public class DruidClusterAdminClient +{ + private static final Logger LOG = new Logger(DruidClusterAdminClient.class); + private static final String COORDINATOR_DOCKER_CONTAINER_NAME = "druid-coordinator"; + private static final String HISTORICAL_DOCKER_CONTAINER_NAME = "druid-historical"; + private static final String INDEXER_DOCKER_CONTAINER_NAME = "druid-overlord"; + private static final String BROKERR_DOCKER_CONTAINER_NAME = "druid-broker"; + private static final String ROUTER_DOCKER_CONTAINER_NAME = "druid-router"; + private static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "druid-middlemanager"; + + private final HttpClient httpClient; + private IntegrationTestingConfig config; + + @Inject + DruidClusterAdminClient( + @TestClient HttpClient httpClient, + IntegrationTestingConfig config + ) + { + this.httpClient = httpClient; + this.config = config; + } + + public void restartCoordinatorContainer() + { + restartDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME); + } + + public void restartHistoricalContainer() + { + restartDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME); + } + + public void restartIndexerContainer() + { + restartDockerContainer(INDEXER_DOCKER_CONTAINER_NAME); + } + + public void restartBrokerContainer() + { + restartDockerContainer(BROKERR_DOCKER_CONTAINER_NAME); + } + + public void restartRouterContainer() + { + restartDockerContainer(ROUTER_DOCKER_CONTAINER_NAME); + } + + public void restartMiddleManagerContainer() + { + restartDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME); + } + + public void waitUntilCoordinatorReady() + { + waitUntilInstanceReady(config.getCoordinatorUrl()); + } + + public void waitUntilHistoricalReady() + { + waitUntilInstanceReady(config.getHistoricalUrl()); + } + + public void waitUntilIndexerReady() + { + waitUntilInstanceReady(config.getIndexerUrl()); + } + + public void waitUntilBrokerReady() + { + waitUntilInstanceReady(config.getBrokerUrl()); + } + + public void waitUntilRouterReady() + { + waitUntilInstanceReady(config.getRouterUrl()); + } + + private void restartDockerContainer(String serviceName) + { + DockerClient dockerClient = DockerClientBuilder.getInstance() + .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()) + .withConnectTimeout(10 * 1000)) + .build(); + List containers = dockerClient.listContainersCmd().exec(); + Optional containerName = containers.stream() + .filter(container -> Arrays.asList(container.getNames()).contains(serviceName)) + .findFirst() + .map(container -> container.getId()); + + if (!containerName.isPresent()) { + throw new ISE("Cannot find docker container for " + serviceName); + } + dockerClient.restartContainerCmd(containerName.get()); + } + + private void waitUntilInstanceReady(final String host) + { + ITRetryUtil.retryUntilTrue( + () -> { + try { + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status/health", host))), + StatusResponseHandler.getInstance() + ).get(); + + LOG.info("%s %s", response.getStatus(), response.getContent()); + return response.getStatus().equals(HttpResponseStatus.OK); + } + catch (Throwable e) { + LOG.error(e, ""); + return false; + } + }, + "Waiting for instance to be ready: [" + host + "]" + ); + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java deleted file mode 100644 index 219b68a3e2e4..000000000000 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidDockerAdminClient.java +++ /dev/null @@ -1,37 +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.testing.utils; - -import com.github.dockerjava.api.DockerClient; -import com.github.dockerjava.core.DockerClientBuilder; -import com.github.dockerjava.netty.NettyDockerCmdExecFactory; - -public class DruidDockerAdminClient -{ - private static final DockerClient DOCKER_CLIENT = DockerClientBuilder.getInstance() - .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()) - .withConnectTimeout(10 * 1000)) - .build(); - public static void test() - { - System.out.println("**********"); - System.out.println(DOCKER_CLIENT.listImagesCmd().withShowAll(true).exec()); - } -} 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 49099e3838be..319c1d054384 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 @@ -35,7 +35,9 @@ import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; +import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testing.IntegrationTestingConfig; import java.io.FileInputStream; import java.util.Map; @@ -45,8 +47,10 @@ public class KinesisAdminClient { private AmazonKinesis amazonKinesis; - public KinesisAdminClient(String endpoint) throws Exception + @Inject + public KinesisAdminClient(IntegrationTestingConfig config) throws Exception { + String endpoint = config.getStreamEndpoint(); String pathToConfigFile = System.getProperty("override.config.path"); Properties prop = new Properties(); prop.load(new FileInputStream(pathToConfigFile)); diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java index d6b244813bd0..cd243efc96f9 100644 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -31,6 +31,7 @@ import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.utils.DruidClusterAdminClient; import org.apache.druid.testing.utils.ITRetryUtil; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -87,14 +88,14 @@ public void run() { Injector injector = DruidTestModuleFactory.getInjector(); IntegrationTestingConfig config = injector.getInstance(IntegrationTestingConfig.class); - HttpClient client = injector.getInstance(Key.get(HttpClient.class, TestClient.class)); + DruidClusterAdminClient druidClusterAdminClient = injector.getInstance(DruidClusterAdminClient.class); - waitUntilInstanceReady(client, config.getCoordinatorUrl()); - waitUntilInstanceReady(client, config.getIndexerUrl()); - waitUntilInstanceReady(client, config.getBrokerUrl()); + druidClusterAdminClient.waitUntilCoordinatorReady(); + druidClusterAdminClient.waitUntilIndexerReady(); + druidClusterAdminClient.waitUntilBrokerReady(); String routerHost = config.getRouterUrl(); if (null != routerHost) { - waitUntilInstanceReady(client, config.getRouterUrl()); + druidClusterAdminClient.waitUntilRouterReady(); } Lifecycle lifecycle = injector.getInstance(Lifecycle.class); try { @@ -115,27 +116,5 @@ private void runTests() { super.run(); } - - public void waitUntilInstanceReady(final HttpClient client, final String host) - { - ITRetryUtil.retryUntilTrue( - () -> { - try { - StatusResponseHolder response = client.go( - new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status/health", host))), - StatusResponseHandler.getInstance() - ).get(); - - LOG.info("%s %s", response.getStatus(), response.getContent()); - return response.getStatus().equals(HttpResponseStatus.OK); - } - catch (Throwable e) { - LOG.error(e, ""); - return false; - } - }, - "Waiting for instance to be ready: [" + host + "]" - ); - } } } 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 18ca091bdb11..e242ce113ec6 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 @@ -20,12 +20,13 @@ 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.DruidDockerAdminClient; +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; @@ -35,7 +36,6 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; 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; @@ -65,26 +65,22 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private static final long CYCLE_PADDING_MS = 100; private static final int TOTAL_NUMBER_OF_SECOND = 10; + @Inject + private DruidClusterAdminClient druidClusterAdminClient; + @Inject + private KinesisAdminClient kinesisAdminClient; + private String streamName; private String fullDatasourceName; - private KinesisAdminClient kinesisAdminClient; private KinesisEventWriter kinesisEventWriter; private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; private Function kinesisIngestionPropsTransform; private Function kinesisQueryPropsTransform; private int secondsToGenerateRemaining; - - @BeforeClass - public void beforeClass() throws Exception - { - kinesisAdminClient = new KinesisAdminClient(config.getStreamEndpoint()); - } - @BeforeMethod public void before() throws Exception { - DruidDockerAdminClient.test(); 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())); @@ -240,53 +236,19 @@ public void testKineseIndexDataWithInputFormatStableState() throws Exception public void testKineseIndexDataWithLosingCoordinator() 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 - String supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); - } + testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady()); } public void testKineseIndexDataWithLosingOverlord() 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 - String supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); - } + testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady()); } public void testKineseIndexDataWithLosingHistorical() 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 - String supervisorId = indexer.submitSupervisor(taskSpec); - LOG.info("Submitted supervisor"); - // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); - verifyIngestedData(supervisorId); - } + testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady()); } - @Test public void testKineseIndexDataWithStartStopSupervisor() throws Exception { @@ -327,17 +289,57 @@ public void testKineseIndexDataWithStartStopSupervisor() throws Exception public void testKineseIndexDataWithKinesisReshardSplit() throws Exception { // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 - testKineseIndexDataWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); + testIndexWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); } @Test public void testKineseIndexDataWithKinesisReshardMerge() throws Exception { // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 - testKineseIndexDataWithKinesisReshardHelper(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 + String 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + // Restart Druid process + restartRunnable.run(); + // Start generating one third of the data (while restarting) + int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; + secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateSecondRound); + wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + // Wait for Druid process to be available + waitForReadyRunnable.run(); + // Start generating remainding data (after restarting) + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); + wikipediaStreamEventGenerator.start(kinesisEventWriter, 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 testKineseIndexDataWithKinesisReshardHelper(int newShardCount) throws Exception + private void testIndexWithKinesisReshardHelper(int newShardCount) throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) From 65e0ca4101bee4539fef7730de10860c5c324b93 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 00:47:03 -1000 Subject: [PATCH 11/22] Kinesis IT --- .../tls/generate-server-certs-and-keystores.sh | 6 ++++++ .../testing/utils/DruidClusterAdminClient.java | 15 ++++++++------- .../indexer/ITKinesisIndexingServiceTest.java | 7 +++++++ 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/integration-tests/docker/tls/generate-server-certs-and-keystores.sh b/integration-tests/docker/tls/generate-server-certs-and-keystores.sh index 8f38be303a8d..2719b1f22f12 100755 --- a/integration-tests/docker/tls/generate-server-certs-and-keystores.sh +++ b/integration-tests/docker/tls/generate-server-certs-and-keystores.sh @@ -17,6 +17,12 @@ cd /tls +FILE_CHECK_IF_RAN=/tls/server.key +if [ -f "$FILE_CHECK_IF_RAN" ]; then + echo "Script was ran already. Skip running again." + exit +fi + rm -f cert_db.txt touch cert_db.txt diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java index 48519e103594..4c6518d535bc 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java @@ -44,12 +44,12 @@ public class DruidClusterAdminClient { private static final Logger LOG = new Logger(DruidClusterAdminClient.class); - private static final String COORDINATOR_DOCKER_CONTAINER_NAME = "druid-coordinator"; - private static final String HISTORICAL_DOCKER_CONTAINER_NAME = "druid-historical"; - private static final String INDEXER_DOCKER_CONTAINER_NAME = "druid-overlord"; - private static final String BROKERR_DOCKER_CONTAINER_NAME = "druid-broker"; - private static final String ROUTER_DOCKER_CONTAINER_NAME = "druid-router"; - private static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "druid-middlemanager"; + private static final String COORDINATOR_DOCKER_CONTAINER_NAME = "/druid-coordinator"; + private static final String HISTORICAL_DOCKER_CONTAINER_NAME = "/druid-historical"; + private static final String INDEXER_DOCKER_CONTAINER_NAME = "/druid-overlord"; + private static final String BROKERR_DOCKER_CONTAINER_NAME = "/druid-broker"; + private static final String ROUTER_DOCKER_CONTAINER_NAME = "/druid-router"; + private static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "/druid-middlemanager"; private final HttpClient httpClient; private IntegrationTestingConfig config; @@ -132,9 +132,10 @@ private void restartDockerContainer(String serviceName) .map(container -> container.getId()); if (!containerName.isPresent()) { + LOG.error("Cannot find docker container for " + serviceName); throw new ISE("Cannot find docker container for " + serviceName); } - dockerClient.restartContainerCmd(containerName.get()); + dockerClient.restartContainerCmd(containerName.get()).exec(); } private void waitUntilInstanceReady(final String host) 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 e242ce113ec6..acc79d4928e7 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 @@ -234,16 +234,19 @@ public void testKineseIndexDataWithInputFormatStableState() throws Exception } } + @Test public void testKineseIndexDataWithLosingCoordinator() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady()); } + @Test public void testKineseIndexDataWithLosingOverlord() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady()); } + @Test public void testKineseIndexDataWithLosingHistorical() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady()); @@ -315,14 +318,18 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); // 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateSecondRound); wikipediaStreamEventGenerator.start(kinesisEventWriter, 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); From 2ddfe1abc87b80028174289aeb3b30710156ea62 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 00:54:20 -1000 Subject: [PATCH 12/22] Kinesis IT --- integration-tests/README.md | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/integration-tests/README.md b/integration-tests/README.md index 8b32ea451852..9300bb476ddb 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -168,20 +168,26 @@ The integration test that indexes from Cloud or uses Cloud as deep storage is no of the integration test run discussed above. Running these tests requires the user to provide their own Cloud. -Currently, the integration test supports Google Cloud Storage, Amazon S3, and Microsoft Azure. -These can be run by providing "gcs-deep-storage", "s3-deep-storage", or "azure-deep-storage" -to -Dgroups for Google Cloud Storage, Amazon S3, and Microsoft Azure respectively. Note that only +Currently, the integration test supports Amazon Kinesis, Google Cloud Storage, Amazon S3, and Microsoft Azure. +These can be run by providing "kinesis-index", "gcs-deep-storage", "s3-deep-storage", or "azure-deep-storage" +to -Dgroups for Amazon Kinesis, Google Cloud Storage, Amazon S3, and Microsoft Azure respectively. Note that only one group should be run per mvn command. -In addition to specifying the -Dgroups to mvn command, the following will need to be provided: +For all of the Cloud Integration tests, the following will also need to be provided: +1) Provide -Doverride.config.path= with your Cloud credentials/configs set. See +integration-tests/docker/environment-configs/override-examples/ directory for env vars to provide for each Cloud. + +For Amazon Kinesis, the following will also need to be provided: +1) Provide -Ddruid.test.config.streamEndpoint= with the endpoint of your stream set. +For example, kinesis.us-east-1.amazonaws.com + +For Google Cloud Storage, Amazon S3, and Microsoft Azure, the following will also need to be provided: 1) Set the bucket and path for your test data. This can be done by setting -Ddruid.test.config.cloudBucket and -Ddruid.test.config.cloudPath in the mvn command or setting "cloud_bucket" and "cloud_path" in the config file. 2) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json located in integration-tests/src/test/resources/data/batch_index to your Cloud storage at the location set in step 1. -3) Provide -Doverride.config.path= with your Cloud credentials/configs set. See -integration-tests/docker/environment-configs/override-examples/ directory for env vars to provide for each Cloud storage. -For running Google Cloud Storage, in addition to the above, you will also have to: +For Google Cloud Storage, in addition to the above, you will also have to: 1) Provide -Dresource.file.dir.path= with folder that contains GOOGLE_APPLICATION_CREDENTIALS file For example, to run integration test for Google Cloud Storage: From 8505e79a9f2301fc49bdfbea99896c5b2a0fd501 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 10:44:19 -1000 Subject: [PATCH 13/22] Kinesis IT --- .../druid/testing/utils/KinesisAdminClient.java | 2 -- .../druid/testing/utils/SyntheticGenerator.java | 10 +++++----- .../main/java/org/testng/DruidTestRunnerFactory.java | 11 ----------- .../tests/indexer/ITKinesisIndexingServiceTest.java | 12 +++++++++--- 4 files changed, 14 insertions(+), 21 deletions(-) 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 319c1d054384..2c5ff8a2b230 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 @@ -35,7 +35,6 @@ import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; -import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.testing.IntegrationTestingConfig; @@ -47,7 +46,6 @@ public class KinesisAdminClient { private AmazonKinesis amazonKinesis; - @Inject public KinesisAdminClient(IntegrationTestingConfig config) throws Exception { String endpoint = config.getStreamEndpoint(); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index 65db43bc5178..aeb5261cc1dd 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -86,13 +86,13 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) try { long sleepMillis = nowCeilingToSecond.getMillis() - DateTimes.nowUtc().getMillis(); if (sleepMillis > 0) { - log.info("Waiting {} ms for next run cycle (at {})", sleepMillis, nowCeilingToSecond); + log.info("Waiting %s ms for next run cycle (at %s)", sleepMillis, nowCeilingToSecond); Thread.sleep(sleepMillis); continue; } log.info( - "Beginning run cycle with {} events, target completion time: {}", + "Beginning run cycle with %s events, target completion time: %s", eventsPerSecond, nowCeilingToSecond.plusSeconds(1).minus(cyclePaddingMs) ); @@ -102,7 +102,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowCeilingToSecond); if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { - log.info("Event: {}/{}, sleep time: {} ms", i, eventsPerSecond, sleepTime); + log.info("Event: %s/%s, sleep time: %s ms", i, eventsPerSecond, sleepTime); } if (sleepTime > 0) { @@ -115,7 +115,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) seconds++; log.info( - "Finished writing {} events, current time: {} - updating next timestamp to: {}", + "Finished writing %s events, current time: %s - updating next timestamp to: %s", eventsPerSecond, DateTimes.nowUtc(), nowCeilingToSecond @@ -123,7 +123,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) if (seconds >= totalNumberOfSecond) { log.info( - "Finished writing {} seconds", + "Finished writing %s seconds", seconds ); break; diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java index cd243efc96f9..53ef663b5c33 100644 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -20,26 +20,15 @@ package /*CHECKSTYLE.OFF: PackageName*/org.testng/*CHECKSTYLE.ON: PackageName*/; import com.google.inject.Injector; -import com.google.inject.Key; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StatusResponseHandler; -import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.guice.TestClient; import org.apache.druid.testing.utils.DruidClusterAdminClient; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.testng.internal.IConfiguration; import org.testng.internal.annotations.IAnnotationFinder; import org.testng.xml.XmlTest; -import java.net.URL; import java.util.List; /** 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 acc79d4928e7..071760696fe1 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 @@ -35,6 +35,7 @@ import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; +import org.junit.BeforeClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Guice; @@ -53,7 +54,7 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private static final int KINESIS_SHARD_COUNT = 2; private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after"; private static final long WAIT_TIME_MILLIS = 60 * 1000L; - private static final DateTime FIRST_EVENT_TIME = DateTime.parse("1994-04-29T00:00:00.000Z"); + 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"; @@ -67,17 +68,22 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest @Inject private DruidClusterAdminClient druidClusterAdminClient; - @Inject - private KinesisAdminClient kinesisAdminClient; private String streamName; private String fullDatasourceName; + private KinesisAdminClient kinesisAdminClient; private KinesisEventWriter kinesisEventWriter; private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; private Function kinesisIngestionPropsTransform; private Function kinesisQueryPropsTransform; private int secondsToGenerateRemaining; + @BeforeClass + public void beforeClass() throws Exception + { + kinesisAdminClient = new KinesisAdminClient(config); + } + @BeforeMethod public void before() throws Exception { From 7c89471549778d0e9d415a74d66e5fca5af04c33 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 10:55:03 -1000 Subject: [PATCH 14/22] Kinesis IT --- .../druid/tests/indexer/ITKinesisIndexingServiceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 071760696fe1..3f10b20f95fc 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 @@ -35,8 +35,8 @@ import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import org.junit.BeforeClass; 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 1d7a20ea5ba966a7ab50658410eb7b8f41df3340 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 20:54:56 -1000 Subject: [PATCH 15/22] Kinesis IT --- integration-tests/run_cluster.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh index f6b5ba206ccc..04d166df7dde 100755 --- a/integration-tests/run_cluster.sh +++ b/integration-tests/run_cluster.sh @@ -203,7 +203,7 @@ fi docker run -d --privileged --net druid-it-net --ip 172.172.172.2 ${COMMON_ENV} --name druid-zookeeper-kafka -p 2181:2181 -p 9092:9092 -p 9093:9093 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/zookeeper.conf:$SUPERVISORDIR/zookeeper.conf -v $SERVICE_SUPERVISORDS_DIR/kafka.conf:$SUPERVISORDIR/kafka.conf druid/cluster # Start MYSQL - docker run -d --privileged --net druid-it-net --ip 172.172.172.3 ${COMMON_ENV} --name druid-metadata-storage -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster + docker run -d --privileged --net druid-it-net --ip 172.172.172.3 ${COMMON_ENV} --name druid-metadata-storage -p 3306:3306 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster # Start Overlord docker run -d --privileged --net druid-it-net --ip 172.172.172.4 ${COMMON_ENV} ${OVERLORD_ENV} ${OVERRIDE_ENV} --name druid-overlord -p 5009:5009 -p 8090:8090 -p 8290:8290 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster From 29a43884199ecbe3424868d0f0358143c904a4fc Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 22:43:29 -1000 Subject: [PATCH 16/22] fix kinesis timeout --- .../apache/druid/testing/utils/EventWriter.java | 2 ++ .../druid/testing/utils/KinesisEventWriter.java | 17 ++++++++++++++++- .../druid/testing/utils/SyntheticGenerator.java | 1 + 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java index be66bbe8fa0c..7fa3c5b2d536 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java @@ -24,4 +24,6 @@ public interface EventWriter void write(String event); void shutdown(); + + void flush(); } 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 e015b8ff2f45..d76d1503aecb 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 @@ -55,7 +55,9 @@ public KinesisEventWriter(String endpoint, String streamName, boolean aggregate) KinesisProducerConfiguration kinesisProducerConfiguration = new KinesisProducerConfiguration() .setCredentialsProvider(credentials) .setRegion(AwsHostNameUtils.parseRegion(endpoint, null)) - .setRequestTimeout(20000L) + .setRequestTimeout(600000L) + .setConnectTimeout(300000L) + .setRecordMaxBufferedTime(15000) .setRecordTtl(9223372036854775807L) .setMetricsLevel("none") .setAggregationEnabled(aggregate); @@ -80,4 +82,17 @@ public void shutdown() LOG.info("Shutting down Kinesis client"); kinesisProducer.flushSync(); } + + @Override + public void flush() + { + kinesisProducer.flushSync(); + ITRetryUtil.retryUntil( + () -> kinesisProducer.getOutstandingRecordsCount() == 0, + true, + 10000, + 30, + "Waiting for all Kinesis tasks to be flushed" + ); + } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index aeb5261cc1dd..97e1b78781ab 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -128,6 +128,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) ); break; } + eventWriter.flush(); } catch (Exception e) { throw new RuntimeException("Exception in event generation loop", e); From 2f3ae22f87ed3f6304c5e5b5d6cf567f2b030e10 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 30 Mar 2020 23:42:52 -1000 Subject: [PATCH 17/22] Kinesis IT --- .../druid/testing/utils/EventWriter.java | 2 +- .../apache/druid/testing/utils/Generator.java | 4 +- .../testing/utils/KinesisAdminClient.java | 4 +- .../testing/utils/KinesisEventWriter.java | 6 +-- .../utils/StreamVerifierEventGenerator.java | 4 +- .../testing/utils/SyntheticGenerator.java | 17 +++---- .../utils/WikipediaStreamEventGenerator.java | 4 +- .../indexer/ITKinesisIndexingServiceTest.java | 48 +++++++++---------- 8 files changed, 40 insertions(+), 49 deletions(-) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java index 7fa3c5b2d536..c572e56a7b2e 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java @@ -21,7 +21,7 @@ public interface EventWriter { - void write(String event); + void write(String topic, String event); void shutdown(); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java index 185a4d60fa44..0e2a1bc90c32 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java @@ -23,9 +23,9 @@ public interface Generator { - void start(EventWriter eventWriter); + void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds); - void start(EventWriter eventWriter, DateTime overrrideFirstEventTime); + void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime); void shutdown(); } 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 2c5ff8a2b230..49099e3838be 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 @@ -36,7 +36,6 @@ import com.amazonaws.services.kinesis.model.UpdateShardCountResult; import com.amazonaws.util.AwsHostNameUtils; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.testing.IntegrationTestingConfig; import java.io.FileInputStream; import java.util.Map; @@ -46,9 +45,8 @@ public class KinesisAdminClient { private AmazonKinesis amazonKinesis; - public KinesisAdminClient(IntegrationTestingConfig config) throws Exception + public KinesisAdminClient(String endpoint) throws Exception { - String endpoint = config.getStreamEndpoint(); String pathToConfigFile = System.getProperty("override.config.path"); Properties prop = new Properties(); prop.load(new FileInputStream(pathToConfigFile)); 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 d76d1503aecb..cec0971c781c 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 @@ -36,10 +36,9 @@ public class KinesisEventWriter implements EventWriter { private static final Logger LOG = new Logger(KinesisEventWriter.class); - private final String streamName; private final KinesisProducer kinesisProducer; - public KinesisEventWriter(String endpoint, String streamName, boolean aggregate) throws Exception + public KinesisEventWriter(String endpoint, boolean aggregate) throws Exception { String pathToConfigFile = System.getProperty("override.config.path"); Properties prop = new Properties(); @@ -63,11 +62,10 @@ public KinesisEventWriter(String endpoint, String streamName, boolean aggregate) .setAggregationEnabled(aggregate); this.kinesisProducer = new KinesisProducer(kinesisProducerConfiguration); - this.streamName = streamName; } @Override - public void write(String event) + public void write(String streamName, String event) { kinesisProducer.addUserRecord( streamName, diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java index 35659256eecb..d387716f53b6 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java @@ -26,9 +26,9 @@ public class StreamVerifierEventGenerator extends SyntheticGenerator { - public StreamVerifierEventGenerator(int eventsPerSeconds, long cyclePaddingMs, int totalNumberOfSeconds) + public StreamVerifierEventGenerator(int eventsPerSeconds, long cyclePaddingMs) { - super(eventsPerSeconds, cyclePaddingMs, totalNumberOfSeconds); + super(eventsPerSeconds, cyclePaddingMs); } @Override diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java index 97e1b78781ab..baf0af3ad649 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java @@ -52,25 +52,22 @@ public int getEventsPerSecond() // second to begin. private final long cyclePaddingMs; - private final int totalNumberOfSecond; - - public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs, int totalNumberOfSecond) + public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs) { this.eventsPerSecond = eventsPerSecond; this.cyclePaddingMs = cyclePaddingMs; - this.totalNumberOfSecond = totalNumberOfSecond; } abstract Object getEvent(int row, DateTime timestamp); @Override - public void start(EventWriter eventWriter) + public void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds) { - start(eventWriter, null); + start(streamTopic, eventWriter, totalNumberOfSeconds, null); } @Override - public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) + public void start(String streamTopic, EventWriter eventWriter, 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. @@ -98,7 +95,7 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) ); for (int i = 1; i <= eventsPerSecond; i++) { - eventWriter.write(MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); + eventWriter.write(streamTopic, MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowCeilingToSecond); if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { @@ -121,14 +118,14 @@ public void start(EventWriter eventWriter, DateTime overrrideFirstEventTime) nowCeilingToSecond ); - if (seconds >= totalNumberOfSecond) { + if (seconds >= totalNumberOfSeconds) { + eventWriter.flush(); log.info( "Finished writing %s seconds", seconds ); break; } - eventWriter.flush(); } catch (Exception e) { throw new RuntimeException("Exception in event generation loop", e); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java index 0a77c1876710..7685819d6066 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java @@ -30,9 +30,9 @@ public class WikipediaStreamEventGenerator extends SyntheticGenerator { private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"); - public WikipediaStreamEventGenerator(int eventsPerSeconds, long cyclePaddingMs, int totalNumberOfSeconds) + public WikipediaStreamEventGenerator(int eventsPerSeconds, long cyclePaddingMs) { - super(eventsPerSeconds, cyclePaddingMs, totalNumberOfSeconds); + super(eventsPerSeconds, cyclePaddingMs); } @Override 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 3f10b20f95fc..8fdb002f383e 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 @@ -35,6 +35,7 @@ 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; @@ -81,11 +82,20 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest @BeforeClass public void beforeClass() throws Exception { - kinesisAdminClient = new KinesisAdminClient(config); + kinesisAdminClient = new KinesisAdminClient(config.getStreamEndpoint()); + kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), false); + wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); + } + + @AfterClass + public void tearDown() + { + wikipediaStreamEventGenerator.shutdown(); + kinesisEventWriter.shutdown(); } @BeforeMethod - public void before() throws Exception + public void before() { streamName = "kinesis_index_test_" + UUID.randomUUID(); String datasource = "kinesis_indexing_service_test_" + UUID.randomUUID(); @@ -98,9 +108,7 @@ public void before() throws Exception 30, "Wait for stream active" ); - kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), streamName, false); secondsToGenerateRemaining = TOTAL_NUMBER_OF_SECOND; - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, TOTAL_NUMBER_OF_SECOND); fullDatasourceName = datasource + config.getExtraDatasourceNameSuffix(); kinesisIngestionPropsTransform = spec -> { try { @@ -201,9 +209,7 @@ public void before() throws Exception @AfterMethod public void teardown() { - kinesisAdminClient.deleteStream(streamName); - wikipediaStreamEventGenerator.shutdown(); - kinesisEventWriter.shutdown(); +// kinesisAdminClient.deleteStream(streamName); } @Test @@ -218,7 +224,7 @@ public void testKineseIndexDataWithLegacyParserStableState() throws Exception String supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); verifyIngestedData(supervisorId); } } @@ -235,7 +241,7 @@ public void testKineseIndexDataWithInputFormatStableState() throws Exception String supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start Kinesis data generator - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); verifyIngestedData(supervisorId); } } @@ -272,13 +278,11 @@ public void testKineseIndexDataWithStartStopSupervisor() throws Exception // Start generating half of the data int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Suspend the supervisor indexer.suspendSupervisor(supervisorId); // Start generating remainning half of the data - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Resume the supervisor indexer.resumeSupervisor(supervisorId); // Verify supervisor is healthy after suspension @@ -321,8 +325,7 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa // Start generating one third of the data (before restarting) int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Restart Druid process LOG.info("Restarting Druid process"); restartRunnable.run(); @@ -330,15 +333,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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateSecondRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + 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 = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); // Verify supervisor is healthy ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), @@ -365,15 +366,13 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti // Start generating one third of the data (before resharding) int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateFirstRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); // Reshard the supervisor by split from KINESIS_SHARD_COUNT to newShardCount kinesisAdminClient.updateShardCount(streamName, newShardCount); // Start generating one third of the data (while resharding) int secondsToGenerateSecondRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateSecondRound; - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateSecondRound); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateSecondRound, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound)); // Wait for kinesis stream to finish resharding ITRetryUtil.retryUntil( () -> kinesisAdminClient.isStreamActive(streamName), @@ -383,8 +382,7 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti "Waiting for Kinesis stream to finish resharding" ); // Start generating remainding data (after resharding) - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS, secondsToGenerateRemaining); - wikipediaStreamEventGenerator.start(kinesisEventWriter, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); + wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); // Verify supervisor is healthy after suspension ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), From c5f4b141f79b8ea2fe11713015f787021d4c89fd Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 31 Mar 2020 00:47:30 -1000 Subject: [PATCH 18/22] Kinesis IT --- .../indexer/ITKinesisIndexingServiceTest.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) 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 8fdb002f383e..0e4782e76bc8 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 @@ -54,7 +54,7 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); private static final int KINESIS_SHARD_COUNT = 2; private static final String STREAM_EXPIRE_TAG = "druid-ci-expire-after"; - private static final long WAIT_TIME_MILLIS = 60 * 1000L; + 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"; @@ -77,6 +77,7 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; private Function kinesisIngestionPropsTransform; private Function kinesisQueryPropsTransform; + private String supervisorId; private int secondsToGenerateRemaining; @BeforeClass @@ -209,7 +210,14 @@ public void before() @AfterMethod public void teardown() { -// kinesisAdminClient.deleteStream(streamName); + try { + kinesisEventWriter.flush(); + indexer.shutdownSupervisor(supervisorId); + unloader(fullDatasourceName); + kinesisAdminClient.deleteStream(streamName); + } catch (Exception e) { + // Best effort cleanup + } } @Test @@ -221,7 +229,7 @@ public void testKineseIndexDataWithLegacyParserStableState() throws Exception final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_LEGACY_PARSER)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - String supervisorId = indexer.submitSupervisor(taskSpec); + supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start Kinesis data generator wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); @@ -238,7 +246,7 @@ public void testKineseIndexDataWithInputFormatStableState() throws Exception final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - String supervisorId = indexer.submitSupervisor(taskSpec); + supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start Kinesis data generator wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, TOTAL_NUMBER_OF_SECOND, FIRST_EVENT_TIME); @@ -273,7 +281,7 @@ public void testKineseIndexDataWithStartStopSupervisor() throws Exception final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - String supervisorId = indexer.submitSupervisor(taskSpec); + supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start generating half of the data int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 2; @@ -320,7 +328,7 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - String supervisorId = indexer.submitSupervisor(taskSpec); + supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start generating one third of the data (before restarting) int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; @@ -361,7 +369,7 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti final String taskSpec = kinesisIngestionPropsTransform.apply(getResourceAsString(INDEXER_FILE_INPUT_FORMAT)); LOG.info("supervisorSpec: [%s]\n", taskSpec); // Start supervisor - String supervisorId = indexer.submitSupervisor(taskSpec); + supervisorId = indexer.submitSupervisor(taskSpec); LOG.info("Submitted supervisor"); // Start generating one third of the data (before resharding) int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; From bb24f1cf3c8d15071bdfe27628ad027b70c02fe5 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 31 Mar 2020 02:15:17 -1000 Subject: [PATCH 19/22] fix checkstyle --- .../druid/testing/utils/KinesisEventWriter.java | 4 ++-- .../{EventWriter.java => StreamEventWriter.java} | 2 +- .../{Generator.java => StreamGenerator.java} | 6 +++--- .../utils/StreamVerifierEventGenerator.java | 2 +- ...erator.java => SyntheticStreamGenerator.java} | 16 ++++++++-------- ... => WikipediaStreamEventStreamGenerator.java} | 4 ++-- .../indexer/ITKinesisIndexingServiceTest.java | 9 +++++---- 7 files changed, 22 insertions(+), 21 deletions(-) rename integration-tests/src/main/java/org/apache/druid/testing/utils/{EventWriter.java => StreamEventWriter.java} (96%) rename integration-tests/src/main/java/org/apache/druid/testing/utils/{Generator.java => StreamGenerator.java} (77%) rename integration-tests/src/main/java/org/apache/druid/testing/utils/{SyntheticGenerator.java => SyntheticStreamGenerator.java} (88%) rename integration-tests/src/main/java/org/apache/druid/testing/utils/{WikipediaStreamEventGenerator.java => WikipediaStreamEventStreamGenerator.java} (91%) 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 cec0971c781c..f7c0045af2ea 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 @@ -32,7 +32,7 @@ import java.nio.charset.StandardCharsets; import java.util.Properties; -public class KinesisEventWriter implements EventWriter +public class KinesisEventWriter implements StreamEventWriter { private static final Logger LOG = new Logger(KinesisEventWriter.class); @@ -90,7 +90,7 @@ public void flush() true, 10000, 30, - "Waiting for all Kinesis tasks to be flushed" + "Waiting for all Kinesis writes to be flushed" ); } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java similarity index 96% rename from integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java rename to integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java index c572e56a7b2e..1bfd6b675919 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/EventWriter.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamEventWriter.java @@ -19,7 +19,7 @@ package org.apache.druid.testing.utils; -public interface EventWriter +public interface StreamEventWriter { void write(String topic, String event); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java similarity index 77% rename from integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java rename to integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java index 0e2a1bc90c32..a232c59a8d65 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/Generator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamGenerator.java @@ -21,11 +21,11 @@ import org.joda.time.DateTime; -public interface Generator +public interface StreamGenerator { - void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds); + void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds); - void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime); + void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime); void shutdown(); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java index d387716f53b6..bb56c794a152 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/StreamVerifierEventGenerator.java @@ -24,7 +24,7 @@ import java.util.UUID; -public class StreamVerifierEventGenerator extends SyntheticGenerator +public class StreamVerifierEventGenerator extends SyntheticStreamGenerator { public StreamVerifierEventGenerator(int eventsPerSeconds, long cyclePaddingMs) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java similarity index 88% rename from integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java rename to integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java index baf0af3ad649..748a6ed2d3bc 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/SyntheticStreamGenerator.java @@ -27,9 +27,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.joda.time.DateTime; -public abstract class SyntheticGenerator implements Generator +public abstract class SyntheticStreamGenerator implements StreamGenerator { - private static final Logger log = new Logger(SyntheticGenerator.class); + private static final Logger log = new Logger(SyntheticStreamGenerator.class); static final ObjectMapper MAPPER = new DefaultObjectMapper(); static { @@ -52,7 +52,7 @@ public int getEventsPerSecond() // second to begin. private final long cyclePaddingMs; - public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs) + public SyntheticStreamGenerator(int eventsPerSecond, long cyclePaddingMs) { this.eventsPerSecond = eventsPerSecond; this.cyclePaddingMs = cyclePaddingMs; @@ -61,13 +61,13 @@ public SyntheticGenerator(int eventsPerSecond, long cyclePaddingMs) abstract Object getEvent(int row, DateTime timestamp); @Override - public void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds) + public void start(String streamTopic, StreamEventWriter streamEventWriter, int totalNumberOfSeconds) { - start(streamTopic, eventWriter, totalNumberOfSeconds, null); + start(streamTopic, streamEventWriter, totalNumberOfSeconds, null); } @Override - public void start(String streamTopic, EventWriter eventWriter, int totalNumberOfSeconds, DateTime overrrideFirstEventTime) + public void start(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. @@ -95,7 +95,7 @@ public void start(String streamTopic, EventWriter eventWriter, int totalNumberOf ); for (int i = 1; i <= eventsPerSecond; i++) { - eventWriter.write(streamTopic, MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); + streamEventWriter.write(streamTopic, MAPPER.writeValueAsString(getEvent(i, eventTimestamp))); long sleepTime = calculateSleepTimeMs(eventsPerSecond - i, nowCeilingToSecond); if ((i <= 100 && i % 10 == 0) || i % 100 == 0) { @@ -119,7 +119,7 @@ public void start(String streamTopic, EventWriter eventWriter, int totalNumberOf ); if (seconds >= totalNumberOfSeconds) { - eventWriter.flush(); + streamEventWriter.flush(); log.info( "Finished writing %s seconds", seconds diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java similarity index 91% rename from integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java rename to integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java index 7685819d6066..77dc7c870c10 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java @@ -26,11 +26,11 @@ import java.util.HashMap; import java.util.Map; -public class WikipediaStreamEventGenerator extends SyntheticGenerator +public class WikipediaStreamEventStreamGenerator extends SyntheticStreamGenerator { private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"); - public WikipediaStreamEventGenerator(int eventsPerSeconds, long cyclePaddingMs) + public WikipediaStreamEventStreamGenerator(int eventsPerSeconds, long cyclePaddingMs) { super(eventsPerSeconds, cyclePaddingMs); } 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 0e4782e76bc8..a8a09d84fb21 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 @@ -30,7 +30,7 @@ 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.WikipediaStreamEventGenerator; +import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator; import org.apache.druid.tests.TestNGGroup; import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; @@ -74,7 +74,7 @@ public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest private String fullDatasourceName; private KinesisAdminClient kinesisAdminClient; private KinesisEventWriter kinesisEventWriter; - private WikipediaStreamEventGenerator wikipediaStreamEventGenerator; + private WikipediaStreamEventStreamGenerator wikipediaStreamEventGenerator; private Function kinesisIngestionPropsTransform; private Function kinesisQueryPropsTransform; private String supervisorId; @@ -85,7 +85,7 @@ public void beforeClass() throws Exception { kinesisAdminClient = new KinesisAdminClient(config.getStreamEndpoint()); kinesisEventWriter = new KinesisEventWriter(config.getStreamEndpoint(), false); - wikipediaStreamEventGenerator = new WikipediaStreamEventGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); + wikipediaStreamEventGenerator = new WikipediaStreamEventStreamGenerator(EVENTS_PER_SECOND, CYCLE_PADDING_MS); } @AfterClass @@ -215,7 +215,8 @@ public void teardown() indexer.shutdownSupervisor(supervisorId); unloader(fullDatasourceName); kinesisAdminClient.deleteStream(streamName); - } catch (Exception e) { + } + catch (Exception e) { // Best effort cleanup } } From d7637d53ff7fdee5ff28b15c49178b176bbf3a39 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 31 Mar 2020 02:20:50 -1000 Subject: [PATCH 20/22] Kinesis IT --- .../java/org/apache/druid/testing/utils/KinesisEventWriter.java | 1 - 1 file changed, 1 deletion(-) 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 f7c0045af2ea..09950f132047 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 @@ -56,7 +56,6 @@ public KinesisEventWriter(String endpoint, boolean aggregate) throws Exception .setRegion(AwsHostNameUtils.parseRegion(endpoint, null)) .setRequestTimeout(600000L) .setConnectTimeout(300000L) - .setRecordMaxBufferedTime(15000) .setRecordTtl(9223372036854775807L) .setMetricsLevel("none") .setAggregationEnabled(aggregate); From 0aa938c2a859abcdb45d6406938b309ceeba5d60 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Wed, 1 Apr 2020 14:52:27 -1000 Subject: [PATCH 21/22] address comments --- integration-tests/README.md | 1 - .../generate-server-certs-and-keystores.sh | 2 +- .../testing/utils/KinesisAdminClient.java | 49 ++++++++++--- .../WikipediaStreamEventStreamGenerator.java | 2 +- .../indexer/ITKinesisIndexingServiceTest.java | 68 +++++++++++++++---- 5 files changed, 96 insertions(+), 26 deletions(-) diff --git a/integration-tests/README.md b/integration-tests/README.md index 9300bb476ddb..b8a03c7b1259 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -122,7 +122,6 @@ Then run the tests using a command similar to: # Run all integration tests that have been verified to work against a quickstart cluster. mvn verify -P int-tests-config-file -Dgroups=quickstart-compatible ``` ->>>>>>> upstream/master Running Tests Using A Configuration File for Any Cluster ------------------- diff --git a/integration-tests/docker/tls/generate-server-certs-and-keystores.sh b/integration-tests/docker/tls/generate-server-certs-and-keystores.sh index 2719b1f22f12..6c40a0704a38 100755 --- a/integration-tests/docker/tls/generate-server-certs-and-keystores.sh +++ b/integration-tests/docker/tls/generate-server-certs-and-keystores.sh @@ -19,7 +19,7 @@ cd /tls FILE_CHECK_IF_RAN=/tls/server.key if [ -f "$FILE_CHECK_IF_RAN" ]; then - echo "Script was ran already. Skip running again." + echo "Using existing certs/keys since /tls/server.key exists. Skipping generation (most likely this script was ran previously). To generate new certs, delete /tls/server.key" exit fi 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 49099e3838be..17ebc938c08f 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 @@ -31,6 +31,7 @@ import com.amazonaws.services.kinesis.model.DeleteStreamResult; import com.amazonaws.services.kinesis.model.DescribeStreamResult; import com.amazonaws.services.kinesis.model.ScalingType; +import com.amazonaws.services.kinesis.model.StreamDescription; import com.amazonaws.services.kinesis.model.StreamStatus; import com.amazonaws.services.kinesis.model.UpdateShardCountRequest; import com.amazonaws.services.kinesis.model.UpdateShardCountResult; @@ -95,8 +96,14 @@ public void deleteStream(String streamName) } } - public void updateShardCount(String streamName, int newShardCount) + /** + * This method updates the shard count of {@param streamName} to have a final shard count of {@param newShardCount} + * 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 + */ + public void updateShardCount(String streamName, int newShardCount, boolean blocksUntilStarted) { + int originalShardCount = getStreamShardCount(streamName); UpdateShardCountRequest updateShardCountRequest = new UpdateShardCountRequest(); updateShardCountRequest.setStreamName(streamName); updateShardCountRequest.setTargetShardCount(newShardCount); @@ -105,23 +112,49 @@ public void updateShardCount(String streamName, int newShardCount) if (updateShardCountResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { throw new ISE("Cannot update stream's shard count for integration test"); } + if (blocksUntilStarted) { + // Wait until the resharding started (or finished) + ITRetryUtil.retryUntil( + () -> { + StreamDescription streamDescription = getStreamDescription(streamName); + int updatedShardCount = getStreamShardCount(streamDescription); + return verifyStreamStatus(streamDescription, StreamStatus.UPDATING) || + (verifyStreamStatus(streamDescription, StreamStatus.ACTIVE) && updatedShardCount > originalShardCount); + }, + true, + 30, + 30, + "Kinesis stream resharding to start (or finished)" + ); + } } public boolean isStreamActive(String streamName) { - DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); - if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { - throw new ISE("Cannot get stream status for integration test"); - } - return StreamStatus.ACTIVE.toString().equals(describeStreamResult.getStreamDescription().getStreamStatus()); + StreamDescription streamDescription = getStreamDescription(streamName); + return verifyStreamStatus(streamDescription, StreamStatus.ACTIVE); } public int getStreamShardCount(String streamName) { + StreamDescription streamDescription = getStreamDescription(streamName); + return getStreamShardCount(streamDescription); + } + + private boolean verifyStreamStatus(StreamDescription streamDescription, StreamStatus streamStatusToCheck) { + return streamStatusToCheck.toString().equals(streamDescription.getStreamStatus()); + } + + private int getStreamShardCount(StreamDescription streamDescription) + { + return streamDescription.getShards().size(); + } + + private StreamDescription getStreamDescription(String streamName) { DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { - throw new ISE("Cannot get stream status for integration test"); + throw new ISE("Cannot get stream description for integration test"); } - return describeStreamResult.getStreamDescription().getShards().size(); + return describeStreamResult.getStreamDescription(); } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java index 77dc7c870c10..4fea67d7be57 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/WikipediaStreamEventStreamGenerator.java @@ -47,7 +47,7 @@ Object getEvent(int i, DateTime timestamp) event.put("robot", "false"); event.put("anonymous", "false"); event.put("namespace", "article"); - event.put("continent", "North Americ"); + event.put("continent", "North America"); event.put("country", "United States"); event.put("region", "Bay Area"); event.put("city", "San Francisco"); 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 a8a09d84fb21..e1965ca15d25 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 @@ -20,6 +20,7 @@ package org.apache.druid.tests.indexer; import com.google.common.collect.ImmutableMap; +import com.google.gson.annotations.Since; import com.google.inject.Inject; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.DateTimes; @@ -51,8 +52,11 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKinesisIndexingServiceTest extends AbstractITBatchIndexTest { - private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); + 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); @@ -213,16 +217,26 @@ 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 + // Best effort cleanup as the stream may have already went Bye-Bye } } @Test - public void testKineseIndexDataWithLegacyParserStableState() throws Exception + public void testKinesisIndexDataWithLegacyParserStableState() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -239,7 +253,7 @@ public void testKineseIndexDataWithLegacyParserStableState() throws Exception } @Test - public void testKineseIndexDataWithInputFormatStableState() throws Exception + public void testKinesisIndexDataWithInputFormatStableState() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -256,25 +270,25 @@ public void testKineseIndexDataWithInputFormatStableState() throws Exception } @Test - public void testKineseIndexDataWithLosingCoordinator() throws Exception + public void testKinesisIndexDataWithLosingCoordinator() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartCoordinatorContainer(), () -> druidClusterAdminClient.waitUntilCoordinatorReady()); } @Test - public void testKineseIndexDataWithLosingOverlord() throws Exception + public void testKinesisIndexDataWithLosingOverlord() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartIndexerContainer(), () -> druidClusterAdminClient.waitUntilIndexerReady()); } @Test - public void testKineseIndexDataWithLosingHistorical() throws Exception + public void testKinesisIndexDataWithLosingHistorical() throws Exception { testIndexWithLosingNodeHelper(() -> druidClusterAdminClient.restartHistoricalContainer(), () -> druidClusterAdminClient.waitUntilHistoricalReady()); } @Test - public void testKineseIndexDataWithStartStopSupervisor() throws Exception + public void testKinesisIndexDataWithStartStopSupervisor() throws Exception { try ( final Closeable ignored1 = unloader(fullDatasourceName) @@ -288,6 +302,14 @@ public void testKineseIndexDataWithStartStopSupervisor() throws Exception 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 @@ -308,14 +330,14 @@ public void testKineseIndexDataWithStartStopSupervisor() throws Exception } @Test - public void testKineseIndexDataWithKinesisReshardSplit() throws Exception + public void testKinesisIndexDataWithKinesisReshardSplit() throws Exception { // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT * 2 testIndexWithKinesisReshardHelper(KINESIS_SHARD_COUNT * 2); } @Test - public void testKineseIndexDataWithKinesisReshardMerge() throws Exception + public void testKinesisIndexDataWithKinesisReshardMerge() throws Exception { // Reshard the supervisor by split from KINESIS_SHARD_COUNT to KINESIS_SHARD_COUNT / 2 testIndexWithKinesisReshardHelper(KINESIS_SHARD_COUNT / 2); @@ -335,6 +357,14 @@ private void testIndexWithLosingNodeHelper(Runnable restartRunnable, Runnable wa 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(); @@ -376,8 +406,16 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti int secondsToGenerateFirstRound = TOTAL_NUMBER_OF_SECOND / 3; secondsToGenerateRemaining = secondsToGenerateRemaining - secondsToGenerateFirstRound; wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateFirstRound, FIRST_EVENT_TIME); - // Reshard the supervisor by split from KINESIS_SHARD_COUNT to newShardCount - kinesisAdminClient.updateShardCount(streamName, newShardCount); + // 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; @@ -392,7 +430,7 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti ); // Start generating remainding data (after resharding) wikipediaStreamEventGenerator.start(streamName, kinesisEventWriter, secondsToGenerateRemaining, FIRST_EVENT_TIME.plusSeconds(secondsToGenerateFirstRound + secondsToGenerateSecondRound)); - // Verify supervisor is healthy after suspension + // Verify supervisor is healthy after resahrding ITRetryUtil.retryUntil( () -> SupervisorStateManager.BasicState.RUNNING.equals(indexer.getSupervisorStatus(supervisorId)), true, @@ -408,7 +446,7 @@ private void testIndexWithKinesisReshardHelper(int newShardCount) throws Excepti private void verifyIngestedData(String supervisorId) throws Exception { // Wait for supervisor to consume events - LOG.info("Waiting for [%s] millis for Kafka indexing tasks to consume events", WAIT_TIME_MILLIS); + 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)); @@ -416,7 +454,7 @@ private void verifyIngestedData(String supervisorId) throws Exception this.queryHelper.testQueriesFromString(querySpec, 2); LOG.info("Shutting down supervisor"); indexer.shutdownSupervisor(supervisorId); - // wait for all kafka indexing tasks to finish + // wait for all Kinesis indexing tasks to finish LOG.info("Waiting for all indexing tasks to finish"); ITRetryUtil.retryUntilTrue( () -> (indexer.getPendingTasks().size() From c094119f17e6533ba703aae78151e537837e0f21 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Wed, 1 Apr 2020 15:47:00 -1000 Subject: [PATCH 22/22] fix checkstyle --- .../org/apache/druid/testing/utils/KinesisAdminClient.java | 6 ++++-- .../druid/tests/indexer/ITKinesisIndexingServiceTest.java | 1 - 2 files changed, 4 insertions(+), 3 deletions(-) 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 17ebc938c08f..bc5ace2d1b48 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 @@ -141,7 +141,8 @@ public int getStreamShardCount(String streamName) return getStreamShardCount(streamDescription); } - private boolean verifyStreamStatus(StreamDescription streamDescription, StreamStatus streamStatusToCheck) { + private boolean verifyStreamStatus(StreamDescription streamDescription, StreamStatus streamStatusToCheck) + { return streamStatusToCheck.toString().equals(streamDescription.getStreamStatus()); } @@ -150,7 +151,8 @@ private int getStreamShardCount(StreamDescription streamDescription) return streamDescription.getShards().size(); } - private StreamDescription getStreamDescription(String streamName) { + private StreamDescription getStreamDescription(String streamName) + { DescribeStreamResult describeStreamResult = amazonKinesis.describeStream(streamName); if (describeStreamResult.getSdkHttpMetadata().getHttpStatusCode() != 200) { throw new ISE("Cannot get stream description for integration test"); 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 e1965ca15d25..b539b5d547c2 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 @@ -20,7 +20,6 @@ package org.apache.druid.tests.indexer; import com.google.common.collect.ImmutableMap; -import com.google.gson.annotations.Since; import com.google.inject.Inject; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.DateTimes;