diff --git a/.github/scripts/run_docker-tests b/.github/scripts/run_docker-tests new file mode 100755 index 000000000000..564ed61482dc --- /dev/null +++ b/.github/scripts/run_docker-tests @@ -0,0 +1,37 @@ +#!/bin/bash + +# 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. + +# Runs Docker tests +# Requires environment variable DRUID_DIST_IMAGE_NAME + +set -e + +DRUID_IMAGE_NAME_ENV="DRUID_DIST_IMAGE_NAME" +DRUID_IMAGE_NAME=$DRUID_DIST_IMAGE_NAME + +DRUID_IMAGE_SYS_PROPERTY="druid.testing.docker.image" + +if [ -z "${DRUID_IMAGE_NAME}" ]; then + echo "ERROR!! Environment variable [$DRUID_IMAGE_NAME_ENV] not set!" + echo "Run 'export $DRUID_IMAGE_NAME_ENV=' to specify the image to use in the Docker tests." + exit 1 +else + echo "Running Docker tests with image[$DRUID_IMAGE_NAME]" +fi + +OPTS+="-pl embedded-tests" +mvn -B $OPTS verify -Pdocker-tests -D$DRUID_IMAGE_SYS_PROPERTY=$DRUID_IMAGE_NAME "-DjfrProfilerArgLine=$JFR_PROFILER_ARG_LINE" "$@" diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index b3a40f1decf9..0e5717ea0df3 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -88,7 +88,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [ query, query-retry, query-error, security, high-availability ] + testing_group: [ query, query-retry, query-error, security ] uses: ./.github/workflows/reusable-standard-its.yml needs: build with: @@ -109,7 +109,7 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,high-availability,custom-coordinator-duties + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,custom-coordinator-duties use_indexer: ${{ matrix.indexer }} group: other diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml new file mode 100644 index 000000000000..516288810164 --- /dev/null +++ b/.github/workflows/docker-tests.yml @@ -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. + +name: "Docker Tests using Distribution Image" +on: + workflow_call: + +jobs: + run-docker-tests: + name: Run Docker tests + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v4 + - name: Set Docker image env var + run: echo "DRUID_DIST_IMAGE_NAME=apache/druid:docker-tests" >> $GITHUB_ENV + - name: Build the Docker image + run: DOCKER_BUILDKIT=1 docker build -t $DRUID_DIST_IMAGE_NAME -f distribution/docker/Dockerfile . + - name: Save Docker image to archive + run: | + echo "Saving image $DRUID_DIST_IMAGE_NAME in archive druid-dist-container.tar.gz" + docker save "$DRUID_DIST_IMAGE_NAME" | gzip > druid-dist-container.tar.gz + - name: Stop and remove Druid Docker containers + run: | + echo "Force stopping all Druid containers and pruning" + docker ps -aq --filter "ancestor=apache/druid" | xargs -r docker rm -f + docker system prune -af --volumes + - name: Load Docker image + run: | + docker load --input druid-dist-container.tar.gz + docker images + - name: Setup Java + uses: actions/setup-java@v4 + with: + distribution: 'zulu' + java-version: 17 + cache: 'maven' + - name: Run Docker tests + id: run-it + run: .github/scripts/run_docker-tests + timeout-minutes: 60 + + - name: Collect docker logs on failure + if: ${{ failure() && steps.run-it.conclusion == 'failure' }} + run: | + mkdir docker-logs + for c in $(docker ps -a --format="{{.Names}}") + do + docker logs $c > ./docker-logs/$c.log + done + + - name: Tar docker logs + if: ${{ failure() && steps.run-it.conclusion == 'failure' }} + run: tar cvzf ./docker-logs.tgz ./docker-logs + + - name: Upload docker logs to GitHub + if: ${{ failure() && steps.run-it.conclusion == 'failure' }} + uses: actions/upload-artifact@v4 + with: + name: failure-docker-logs + path: docker-logs.tgz + + - name: Collect surefire reports on failure + if: ${{ failure() && steps.run-it.conclusion == 'failure' }} + run: | + tar cvzf ./surefire-logs.tgz ./embedded-tests/target/surefire-reports + + - name: Upload surefire reports to GitHub + if: ${{ failure() && steps.run-it.conclusion == 'failure' }} + uses: actions/upload-artifact@v4 + with: + name: failure-surefire-logs + path: surefire-logs.tgz diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 575f28c25acf..8bfa51f142f3 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -67,7 +67,7 @@ jobs: fail-fast: false matrix: jdk: [17] - it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Query, DruidExactCountBitmap] + it: [MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Query, DruidExactCountBitmap] indexer: [middleManager] uses: ./.github/workflows/reusable-revised-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index 66ffdd1a32a0..5da028c5dd30 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -78,7 +78,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [query, query-retry, query-error, security, high-availability, centralized-datasource-schema] + testing_group: [query, query-retry, query-error, security, centralized-datasource-schema] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -181,6 +181,6 @@ jobs: with: build_jdk: 17 runtime_jdk: 17 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,high-availability,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled use_indexer: ${{ matrix.indexer }} group: other diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 21c6f3c3f39d..95648f0087fa 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -174,8 +174,12 @@ jobs: DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} + docker-tests: + needs: [build, unit-tests] + uses: ./.github/workflows/docker-tests.yml + actions-timeline: - needs: [build, unit-tests, revised-its, standard-its] + needs: [build, unit-tests, revised-its, standard-its, docker-tests] runs-on: ubuntu-latest if: ${{ !cancelled() }} steps: diff --git a/distribution/docker/druid.sh b/distribution/docker/druid.sh index bb5c1362ac17..e43ad0d9e41d 100755 --- a/distribution/docker/druid.sh +++ b/distribution/docker/druid.sh @@ -129,6 +129,13 @@ SCONFIG=$(eval echo \$$(echo $SCONFIG)) if [ -n "${SCONFIG}" ] then + # Create service conf directory as it may not exist for custom node roles + if [ ! -d "$SERVICE_CONF_DIR" ] + then + echo "Creating conf directory '$SERVICE_CONF_DIR'" + mkdir -p $SERVICE_CONF_DIR + fi + cp -f "${SCONFIG}" $SERVICE_CONF_DIR/runtime.properties fi diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index bf8840ecd47b..6d4ad968fd8c 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -80,12 +80,28 @@ ${project.parent.version} test + + org.apache.druid.extensions + postgresql-metadata-storage + ${project.parent.version} + test + org.apache.druid.extensions druid-s3-extensions ${project.parent.version} test + + org.apache.druid.extensions + druid-testcontainers + ${project.parent.version} + + + org.apache.druid.extensions + druid-testing-tools + ${project.parent.version} + org.apache.druid.extensions druid-basic-security @@ -100,6 +116,14 @@ com.google.guava guava + + com.google.inject + guice + + + org.apache.httpcomponents + httpclient + io.netty netty @@ -215,6 +239,11 @@ junit junit + + org.junit.platform + junit-platform-launcher + test + org.hamcrest hamcrest-all @@ -268,6 +297,12 @@ ${testcontainers.version} test + + org.testcontainers + postgresql + ${testcontainers.version} + test + org.testcontainers kafka @@ -313,6 +348,51 @@ true + + + org.apache.maven.plugins + maven-surefire-plugin + + docker-test + + + + + + docker-tests + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + org.apache.maven.plugins + maven-failsafe-plugin + + docker-test + + + *DockerTest* + + true + + + + + integration-test + verify + + + + + + + + diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 29991844f03a..bd89df768a2c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -112,7 +112,7 @@ public class AutoCompactionTest extends CompactionTestBase { private static final Logger LOG = new Logger(AutoCompactionTest.class); - private static final Supplier INDEX_TASK = MoreResources.Task.BASIC_INDEX; + private static final Supplier INDEX_TASK = MoreResources.Task.INDEX_TASK_WITH_AGGREGATORS; private static final Supplier INDEX_TASK_WITH_GRANULARITY_SPEC = () -> INDEX_TASK.get().dimensions("language").dynamicPartitionWithMaxRows(10); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java index 99eb428cf9c7..b14a696f87c9 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionTaskTest.java @@ -61,7 +61,7 @@ public class CompactionTaskTest extends CompactionTestBase { - private static final Supplier INDEX_TASK = MoreResources.Task.BASIC_INDEX; + private static final Supplier INDEX_TASK = MoreResources.Task.INDEX_TASK_WITH_AGGREGATORS; private static final List> INDEX_QUERIES_RESOURCE = List.of( Pair.of(Resources.Query.SELECT_MIN_MAX_TIME, "2013-08-31T01:02:33.000Z,2013-09-01T12:41:27.000Z"), @@ -102,7 +102,7 @@ public class CompactionTaskTest extends CompactionTestBase .ioConfig(new CompactionIntervalSpec(Intervals.of("2013-08-31/2013-09-02"), null), true); private static final Supplier INDEX_TASK_WITH_TIMESTAMP = - () -> MoreResources.Task.BASIC_INDEX.get().dimensions( + () -> MoreResources.Task.INDEX_TASK_WITH_AGGREGATORS.get().dimensions( "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", "namespace", "continent", "country", "region", "city", "timestamp" diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/CustomNodeRoleDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/CustomNodeRoleDockerTest.java new file mode 100644 index 000000000000..ca5f932c622a --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/CustomNodeRoleDockerTest.java @@ -0,0 +1,93 @@ +/* + * 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.embedded.docker; + +import org.apache.druid.testing.DruidCommand; +import org.apache.druid.testing.cli.CliEventCollector; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.server.HighAvailabilityTest; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +/** + * Docker test to verify running a Druid service with a custom node role provided + * by an extension. This test uses Docker containers so that we can verify + * starting the custom node from the command line. + *

+ * See {@code HttpEmitterEventCollector} for running the custom node as an + * embedded server. + */ +public class CustomNodeRoleDockerTest extends EmbeddedClusterTestBase implements LatestImageDockerTest +{ + // Server used only for its bindings + private final EmbeddedRouter router = new EmbeddedRouter(); + + @Override + protected EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster + .withZookeeper() + .useContainerFriendlyHostname() + .addCommonProperty("druid.extensions.loadList", "[\"druid-testing-tools\"]") + .addResource(new DruidContainerResource(new EventCollectorCommand()).usingTestImage()) + .addServer(router); + } + + @Test + public void test_customNode_isDiscoveredByOtherServices() + { + HighAvailabilityTest.verifyNodeRoleHasServerCount( + CliEventCollector.NODE_ROLE, + 1, + router.bindings().nodeDiscovery(), + router.bindings().escalatedHttpClient() + ); + } + + private static class EventCollectorCommand implements DruidCommand + { + @Override + public String getName() + { + return "eventCollector"; + } + + @Override + public String getJavaOpts() + { + return "-Xms128m -Xmx128m"; + } + + @Override + public Integer[] getExposedPorts() + { + return new Integer[]{9301}; + } + + @Override + public Map getDefaultProperties() + { + return Map.of(); + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/DruidContainerResource.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/DruidContainerResource.java new file mode 100644 index 000000000000..fc7a18d41fdf --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/DruidContainerResource.java @@ -0,0 +1,233 @@ +/* + * 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.embedded.docker; + +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.DruidCommand; +import org.apache.druid.testing.DruidContainer; +import org.apache.druid.testing.MountedDir; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHostname; +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.utility.DockerImageName; + +import java.io.File; +import java.nio.file.Files; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * {@link TestcontainerResource} to run Druid services. + * Currently, only core extensions can be used out-of-the-box with these containers + * such as {@code druid-s3-extensions} or {@code postgresql-metadata-storage}, + * simply by adding them to {@code druid.extensions.loadList}. + * + * @see LatestImageDockerTest + */ +public class DruidContainerResource extends TestcontainerResource +{ + /** + * Java system property to specify the name of the Docker test image. + */ + public static final String PROPERTY_TEST_IMAGE = "druid.testing.docker.image"; + + private static final Logger log = new Logger(DruidContainerResource.class); + + /** + * Forbidden server properties that may be used by EmbeddedDruidServers but + * interfere with the functioning of DruidContainer-based services. + */ + private static final Set FORBIDDEN_PROPERTIES = Set.of( + "druid.extensions.modulesForEmbeddedTests" + ); + + /** + * A static incremental ID is used instead of a random number to ensure that + * tests are more deterministic and easier to debug. + */ + private static final AtomicInteger SERVER_ID = new AtomicInteger(0); + + private final String name; + private final DruidCommand command; + private final Map properties = new HashMap<>(); + + private DockerImageName imageName; + private EmbeddedDruidCluster cluster; + + private File containerDirectory; + + private MountedDir indexerLogsDeepStorageDirectory; + private MountedDir serviceLogsDirectory; + private MountedDir segmentDeepStorageDirectory; + + public DruidContainerResource(DruidCommand command) + { + this.name = StringUtils.format( + "container_%s_%d", + command.getName(), + SERVER_ID.incrementAndGet() + ); + this.command = command; + addProperty("druid.host", EmbeddedHostname.containerFriendly().toString()); + } + + public DruidContainerResource usingImage(DockerImageName imageName) + { + this.imageName = imageName; + return this; + } + + /** + * Uses the Docker test image specified by the system property + * {@link #PROPERTY_TEST_IMAGE} for this container. + */ + public DruidContainerResource usingTestImage() + { + final String imageName = System.getProperty(PROPERTY_TEST_IMAGE); + InvalidInput.conditionalException( + imageName != null, + StringUtils.format( + "System property[%s] must be set while running Docker tests locally" + + " to specify which Druid image to use. Update your run configuration" + + " to include '-D%s='.", + PROPERTY_TEST_IMAGE, PROPERTY_TEST_IMAGE + ) + ); + return usingImage(DockerImageName.parse(imageName)); + } + + public DruidContainerResource addProperty(String key, String value) + { + properties.put(key, value); + return this; + } + + @Override + public void beforeStart(EmbeddedDruidCluster cluster) + { + this.cluster = cluster; + + // Mount directories used by the entire cluster (including embedded servers) + this.segmentDeepStorageDirectory = new MountedDir( + new File("/druid/deep-store"), + cluster.getTestFolder().getOrCreateFolder("deep-store") + ); + this.indexerLogsDeepStorageDirectory = new MountedDir( + new File("/druid/indexer-logs"), + cluster.getTestFolder().getOrCreateFolder("indexer-logs") + ); + + // Mount directories used by this container for easier debugging with service logs + this.containerDirectory = cluster.getTestFolder().getOrCreateFolder(name); + + final File logDirectory = new File(containerDirectory, "log"); + this.serviceLogsDirectory = new MountedDir(new File("/opt/druid/log"), logDirectory); + + // Create the log directory upfront to avoid permission issues + createLogDirectory(logDirectory); + } + + @Override + protected DruidContainer createContainer() + { + final DruidContainer container = new DruidContainer(command, imageName) + .withFileSystemBind(segmentDeepStorageDirectory) + .withFileSystemBind(indexerLogsDeepStorageDirectory) + .withFileSystemBind(serviceLogsDirectory) + .withEnv( + Map.of( + "DRUID_SET_HOST_IP", "0", + "DRUID_SET_HOST", "0" + ) + ); + + log.info( + "Starting Druid container[%s] with image[%s], exposed ports[%s] and mounted directory[%s].", + name, imageName, Arrays.toString(command.getExposedPorts()), containerDirectory + ); + + setCommonProperties(container); + setServerProperties(container); + + return container; + } + + private void setCommonProperties(DruidContainer container) + { + final Properties commonProperties = new Properties(); + commonProperties.putAll(cluster.getCommonProperties()); + FORBIDDEN_PROPERTIES.forEach(commonProperties::remove); + + commonProperties.setProperty( + "druid.storage.storageDirectory", + segmentDeepStorageDirectory.containerFile().getAbsolutePath() + ); + commonProperties.setProperty( + "druid.indexer.logs.directory", + indexerLogsDeepStorageDirectory.containerFile().getAbsolutePath() + ); + + log.info( + "Writing common properties for Druid container[%s]: [%s]", + name, commonProperties + ); + + for (String key : commonProperties.stringPropertyNames()) { + container.withCommonProperty(key, commonProperties.getProperty(key)); + } + } + + private void setServerProperties(DruidContainer container) + { + FORBIDDEN_PROPERTIES.forEach(properties::remove); + log.info( + "Writing runtime properties for Druid container[%s]: [%s]", + name, properties + ); + + properties.forEach(container::withServiceProperty); + } + + private static void createLogDirectory(File dir) + { + try { + FileUtils.mkdirp(dir); + Files.setPosixFilePermissions(dir.toPath(), PosixFilePermissions.fromString("rwxrwxrwx")); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public String toString() + { + return name; + } + +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionBackwardCompatibilityDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionBackwardCompatibilityDockerTest.java new file mode 100644 index 000000000000..5c814c2cb903 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionBackwardCompatibilityDockerTest.java @@ -0,0 +1,112 @@ +/* + * 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.embedded.docker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.indexing.SegmentUpdateResponse; +import org.apache.druid.testing.DruidCommand; +import org.apache.druid.testing.DruidContainer; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.IngestionSmokeTest; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; + +/** + * Runs some basic ingestion tests using Coordinator and Overlord at version + * {@link DruidContainer.Image#APACHE_31} and other services at current version + * to test backward compatibility. + *

+ * This test does not implement {@link LatestImageDockerTest} and runs in the + * {@code mvn test} phase. + */ +public class IngestionBackwardCompatibilityDockerTest extends IngestionSmokeTest +{ + @Override + protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) + { + DruidContainerResource containerOverlord = new DruidContainerResource(DruidCommand.Server.OVERLORD) + .usingImage(DruidContainer.Image.APACHE_31); + DruidContainerResource containerCoordinator = new DruidContainerResource(DruidCommand.Server.COORDINATOR) + .usingImage(DruidContainer.Image.APACHE_31); + + // Add an EmbeddedOverlord to the cluster to use its client and mapper bindings. + // but ensure that it is not the leader. + overlord.addProperty("druid.plaintextPort", "7090"); + + return cluster + .useContainerFriendlyHostname() + .addResource(containerOverlord) + .addResource(containerCoordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()) + .addServer(eventCollector) + .addCommonProperty( + "druid.extensions.loadList", + "[\"druid-s3-extensions\", \"druid-kafka-indexing-service\"," + + "\"druid-multi-stage-query\", \"postgresql-metadata-storage\"]" + ); + } + + @BeforeEach + public void verifyOverlordLeader() + { + // Verify that the EmbeddedOverlord is not leader i.e. the container Overlord is leader + Assertions.assertFalse( + overlord.bindings().overlordLeaderSelector().isLeader() + ); + } + + @Override + protected int markSegmentsAsUnused(String dataSource) + { + // For old Druid versions, use Coordinator to mark segments as unused + final ServiceClient coordinatorClient = + overlord.bindings().getInstance(ServiceClient.class, Coordinator.class); + + try { + RequestBuilder req = new RequestBuilder( + HttpMethod.DELETE, + StringUtils.format("/druid/coordinator/v1/datasources/%s", dataSource) + ); + BytesFullResponseHolder responseHolder = coordinatorClient.request( + req, + new BytesFullResponseHandler() + ); + + final ObjectMapper mapper = overlord.bindings().jsonMapper(); + return mapper.readValue(responseHolder.getContent(), SegmentUpdateResponse.class).getNumChangedSegments(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionDockerTest.java new file mode 100644 index 000000000000..25371fae269c --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/IngestionDockerTest.java @@ -0,0 +1,79 @@ +/* + * 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.embedded.docker; + +import org.apache.druid.testing.DruidCommand; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.indexing.IngestionSmokeTest; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; + +/** + * Runs some basic ingestion tests using {@code DruidContainers} to verify the + * functionality of latest Druid Docker images. The underlying cluster also uses + * embedded servers either to provide visibility into the cluster state. + */ +public class IngestionDockerTest extends IngestionSmokeTest implements LatestImageDockerTest +{ + @Override + protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) + { + DruidContainerResource containerOverlord = new DruidContainerResource(DruidCommand.Server.OVERLORD) + .usingTestImage(); + DruidContainerResource containerCoordinator = new DruidContainerResource(DruidCommand.Server.COORDINATOR) + .usingTestImage(); + DruidContainerResource historical = new DruidContainerResource(DruidCommand.Server.HISTORICAL) + .usingTestImage(); + DruidContainerResource router = new DruidContainerResource(DruidCommand.Server.ROUTER) + .usingTestImage(); + DruidContainerResource middleManager = new DruidContainerResource(DruidCommand.Server.MIDDLE_MANAGER) + .usingTestImage() + .addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); + + // Add an EmbeddedOverlord to the cluster to use its client and mapper bindings. + // but ensure that it is not the leader. + overlord.addProperty("druid.plaintextPort", "7090"); + + return cluster + .useContainerFriendlyHostname() + .addResource(containerOverlord) + .addResource(containerCoordinator) + .addResource(middleManager) + .addResource(historical) + .addResource(router) + .addServer(overlord) + .addServer(broker) + .addServer(eventCollector) + .addCommonProperty( + "druid.extensions.loadList", + "[\"druid-s3-extensions\", \"druid-kafka-indexing-service\"," + + "\"druid-multi-stage-query\", \"postgresql-metadata-storage\"]" + ); + } + + @BeforeEach + public void verifyOverlordLeader() + { + // Verify that the EmbeddedOverlord is not leader i.e. the container Overlord is leader + Assertions.assertFalse( + overlord.bindings().overlordLeaderSelector().isLeader() + ); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/LatestImageDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/LatestImageDockerTest.java new file mode 100644 index 000000000000..058e5616b1fe --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/LatestImageDockerTest.java @@ -0,0 +1,41 @@ +/* + * 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.embedded.docker; + +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +/** + * Interface for embedded tests which use {@link DruidContainerResource} running + * the latest Druid image. All test classes implementing this interface should be + * named {@code *DockerTest*} to allow maven to recognize them as integration tests, + * and should use {@code EmbeddedHostname.containerFriendly()} to allow Druid + * containers to be reachable by embedded servers and vice-versa. + *

+ * DO NOT write a new Docker test unless absolutely necessary. + * For all testing needs, use regular EmbeddedDruidServer-based tests only. + * as they are much faster, easy to debug and do not require image downloads. + */ +@Tag("docker-test") +@EnabledIfSystemProperty(named = DruidContainerResource.PROPERTY_TEST_IMAGE, matches = ".+") +public interface LatestImageDockerTest +{ + +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java index a93ef2ddb79d..6b8fa1bc6d3f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IndexTaskTest.java @@ -20,7 +20,6 @@ package org.apache.druid.testing.embedded.indexing; import org.apache.druid.indexing.common.task.IndexTask; -import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; @@ -143,14 +142,11 @@ public void test_run100Tasks_concurrently() private IndexTask createIndexTaskForInlineData(String taskId, String inlineDataCsv) { - return TaskBuilder.ofTypeIndex() - .dataSource(dataSource) - .isoTimestampColumn("time") - .csvInputFormatWithColumns("time", "item", "value") - .inlineInputSourceWithData(inlineDataCsv) - .segmentGranularity("DAY") - .dimensions() - .withId(taskId); + return MoreResources.Task.BASIC_INDEX + .get() + .inlineInputSourceWithData(inlineDataCsv) + .dataSource(dataSource) + .withId(taskId); } /** diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java new file mode 100644 index 000000000000..4e70a09909b6 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java @@ -0,0 +1,409 @@ +/* + * 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.embedded.indexing; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule; +import org.apache.druid.msq.guice.IndexerMemoryManagementModule; +import org.apache.druid.msq.guice.MSQDurableStorageModule; +import org.apache.druid.msq.guice.MSQExternalDataSourceModule; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.guice.MSQSqlModule; +import org.apache.druid.msq.guice.SqlTaskModule; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.emitter.LatchableEmitterModule; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.embedded.minio.MinIOStorageResource; +import org.apache.druid.testing.embedded.msq.EmbeddedMSQApis; +import org.apache.druid.testing.embedded.psql.PostgreSQLMetadataResource; +import org.apache.druid.testing.embedded.server.EmbeddedEventCollector; +import org.apache.druid.timeline.DataSegment; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +/** + * Contains a variety of basic ingestion tests. + */ +public class IngestionSmokeTest extends EmbeddedClusterTestBase +{ + protected final EmbeddedOverlord overlord = new EmbeddedOverlord(); + + /** + * Indexer with 2 slots and 200MB each as the minimum required memory for the + * MSQ tasks in {@link #test_ingestWikipedia1DayWithMSQ_andQueryData()} is 133 MB. + */ + protected EmbeddedIndexer indexer = new EmbeddedIndexer() + .setServerMemory(300_000_000) + .addProperty("druid.worker.capacity", "2") + .addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); + + /** + * Broker with a short metadata refresh period. + */ + protected EmbeddedBroker broker = new EmbeddedBroker() + .addProperty("druid.sql.planner.metadataRefreshPeriod", "PT1s"); + + /** + * Event collector used to wait for metric events to occur. + */ + protected final EmbeddedEventCollector eventCollector = new EmbeddedEventCollector() + .addProperty("druid.emitter", "latching"); + + protected final KafkaResource kafkaServer = new KafkaResource(); + + @Override + public EmbeddedDruidCluster createCluster() + { + return addServers( + EmbeddedDruidCluster + .withZookeeper() + .addExtensions( + KafkaIndexTaskModule.class, + LatchableEmitterModule.class, + PostgreSQLMetadataStorageModule.class, + MSQSqlModule.class, + SqlTaskModule.class, + MSQIndexingModule.class, + MSQDurableStorageModule.class, + MSQExternalDataSourceModule.class, + IndexerMemoryManagementModule.class + ) + .addResource(new PostgreSQLMetadataResource()) + .addResource(new MinIOStorageResource()) + .addResource(kafkaServer) + .addCommonProperty("druid.emitter", "http") + .addCommonProperty("druid.emitter.http.recipientBaseUrl", eventCollector.getMetricsUrl()) + .addCommonProperty("druid.emitter.http.flushMillis", "500") + ); + } + + /** + * Adds servers to the given cluster. + * + * @return The updated cluster. + */ + protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) + { + return cluster + .addServer(new EmbeddedCoordinator()) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(eventCollector) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedRouter()); + } + + @AfterEach + public void cleanUp() + { + markSegmentsAsUnused(dataSource); + } + + protected int markSegmentsAsUnused(String dataSource) + { + return cluster.callApi() + .onLeaderOverlord(o -> o.markSegmentsAsUnused(dataSource)) + .getNumChangedSegments(); + } + + @Test + public void test_runIndexTask_andKillData() + { + final int numSegments = 10; + + // Run an 'index' task and verify the ingested data + final String taskId = IdUtils.getRandomId(); + final IndexTask task = MoreResources.Task.BASIC_INDEX.get().dataSource(dataSource).withId(taskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); + cluster.callApi().waitForTaskToSucceed(taskId, eventCollector.latchableEmitter()); + + verifyUsedSegmentCount(numSegments); + waitForSegmentsToBeQueryable(numSegments); + + cluster.callApi().verifySqlQuery("SELECT COUNT(*) FROM sys.segments WHERE datasource='%s'", dataSource, "10"); + cluster.callApi().verifySqlQuery("SELECT * FROM %s", dataSource, Resources.InlineData.CSV_10_DAYS); + + // Mark all segments as unused and verify state + Assertions.assertEquals( + numSegments, + markSegmentsAsUnused(dataSource) + ); + verifyUsedSegmentCount(0); + eventCollector.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("segment/loadQueue/success") + .hasDimension(DruidMetrics.DESCRIPTION, "DROP") + .hasService("druid/coordinator"), + agg -> agg.hasSumAtLeast(numSegments) + ); + cluster.callApi().verifySqlQuery("SELECT * FROM sys.segments WHERE datasource='%s'", dataSource, ""); + + // Kill all unused segments + final String killTaskId = cluster.callApi().onLeaderOverlord( + o -> o.runKillTask(IdUtils.getRandomId(), dataSource, Intervals.ETERNITY, null, null, null) + ); + cluster.callApi().waitForTaskToSucceed(killTaskId, eventCollector.latchableEmitter()); + + eventCollector.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("segment/nuked/bytes") + .hasService("druid/overlord"), + agg -> agg.hasCountAtLeast(numSegments) + ); + } + + @Test + public void test_runIndexParallelTask_andCompactData() + { + final int numInitialSegments = 1; + + // Run an 'index_parallel' task and verify the ingested data + final String taskId = IdUtils.getRandomId(); + final ParallelIndexSupervisorTask task = TaskBuilder + .ofTypeIndexParallel() + .timestampColumn("timestamp") + .jsonInputFormat() + .inputSource(Resources.HttpData.wikipedia1Day()) + .dimensions() + .tuningConfig(t -> t.withMaxNumConcurrentSubTasks(1)) + .dataSource(dataSource) + .withId(taskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); + cluster.callApi().waitForTaskToSucceed(taskId, eventCollector.latchableEmitter()); + + waitForSegmentsToBeQueryable(numInitialSegments); + cluster.callApi().verifySqlQuery("SELECT COUNT(*) FROM %s", dataSource, "24433"); + cluster.callApi().verifySqlQuery("SELECT COUNT(*) FROM sys.segments WHERE datasource='%s'", dataSource, "1"); + + final String[] segmentIntervalParts = cluster.runSql( + "SELECT \"start\", \"end\" FROM sys.segments WHERE datasource='%s'", + dataSource + ).split(","); + final Interval segmentInterval = Intervals.of("%s/%s", segmentIntervalParts[0], segmentIntervalParts[1]); + + // Run compaction for this interval + final String compactTaskId = IdUtils.getRandomId(); + final CompactionTask compactionTask = TaskBuilder + .ofTypeCompact() + .dataSource(dataSource) + .interval(segmentInterval) + .dynamicPartitionWithMaxRows(5000) + .withId(compactTaskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(compactTaskId, compactionTask)); + cluster.callApi().waitForTaskToSucceed(taskId, eventCollector.latchableEmitter()); + + // Verify the compacted data + final int numCompactedSegments = 5; + waitForSegmentsToBeQueryable(numInitialSegments + numCompactedSegments); + cluster.callApi().verifySqlQuery("SELECT COUNT(*) FROM %s", dataSource, "24433"); + cluster.callApi().verifySqlQuery( + "SELECT COUNT(*) FROM sys.segments WHERE datasource='%s' AND is_overshadowed=0", + dataSource, + "5" + ); + } + + @Test + public void test_ingestWikipedia1DayWithMSQ_andQueryData() + { + final String sql = + "INSERT INTO %s" + + " SELECT " + + " TIME_PARSE(\"timestamp\") AS __time, *" + + " FROM TABLE(" + + " EXTERN(" + + " '{\"type\":\"http\",\"uris\":[\"https://druid.apache.org/data/wikipedia.json.gz\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"name\":\"isRobot\",\"type\":\"string\"},{\"name\":\"channel\",\"type\":\"string\"},{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"flags\",\"type\":\"string\"},{\"name\":\"isUnpatrolled\",\"type\":\"string\"},{\"name\":\"page\",\"type\":\"string\"},{\"name\":\"diffUrl\",\"type\":\"string\"},{\"name\":\"added\",\"type\":\"long\"},{\"name\":\"comment\",\"type\":\"string\"},{\"name\":\"commentLength\",\"type\":\"long\"},{\"name\":\"isNew\",\"type\":\"string\"},{\"name\":\"isMinor\",\"type\":\"string\"},{\"name\":\"delta\",\"type\":\"long\"},{\"name\":\"isAnonymous\",\"type\":\"string\"},{\"name\":\"user\",\"type\":\"string\"},{\"name\":\"deltaBucket\",\"type\":\"long\"},{\"name\":\"deleted\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"},{\"name\":\"cityName\",\"type\":\"string\"},{\"name\":\"countryName\",\"type\":\"string\"},{\"name\":\"regionIsoCode\",\"type\":\"string\"},{\"name\":\"metroCode\",\"type\":\"long\"},{\"name\":\"countryIsoCode\",\"type\":\"string\"},{\"name\":\"regionName\",\"type\":\"string\"}]'" + + " )" + + ")" + + " PARTITIONED BY DAY" + + " CLUSTERED BY countryName"; + + final EmbeddedMSQApis msqApis = new EmbeddedMSQApis(cluster, overlord); + final SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql, dataSource); + cluster.callApi().waitForTaskToSucceed(taskStatus.getTaskId(), eventCollector.latchableEmitter()); + + waitForSegmentsToBeQueryable(1); + cluster.callApi().verifySqlQuery("SELECT COUNT(*) FROM %s", dataSource, "24433"); + } + + @Test + public void test_runKafkaSupervisor() + { + final String topic = dataSource; + kafkaServer.createTopicWithPartitions(topic, 2); + + kafkaServer.produceRecordsToTopic( + generateRecordsForTopic(topic, 10, DateTimes.of("2025-06-01")) + ); + + // Submit and start a supervisor + final String supervisorId = dataSource; + final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor(topic); + + final Map startSupervisorResult = cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(kafkaSupervisorSpec) + ); + Assertions.assertEquals(Map.of("id", supervisorId), startSupervisorResult); + + waitForSegmentsToBeQueryable(1); + + SupervisorStatus supervisorStatus = cluster.callApi().getSupervisorStatus(supervisorId); + Assertions.assertFalse(supervisorStatus.isSuspended()); + Assertions.assertTrue(supervisorStatus.isHealthy()); + Assertions.assertEquals("RUNNING", supervisorStatus.getState()); + Assertions.assertEquals(topic, supervisorStatus.getSource()); + + // Get the task statuses + List taskStatuses = ImmutableList.copyOf( + (CloseableIterator) + cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(null, dataSource, 1)) + ); + Assertions.assertEquals(1, taskStatuses.size()); + Assertions.assertEquals(TaskState.RUNNING, taskStatuses.get(0).getStatusCode()); + + // Suspend the supervisor and verify the state + cluster.callApi().onLeaderOverlord( + o -> o.postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()) + ); + supervisorStatus = cluster.callApi().getSupervisorStatus(supervisorId); + Assertions.assertTrue(supervisorStatus.isSuspended()); + } + + private KafkaSupervisorSpec createKafkaSupervisor(String topic) + { + return new KafkaSupervisorSpec( + null, + null, + DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(), + createTuningConfig(), + new KafkaSupervisorIOConfig( + topic, + null, + new CsvInputFormat(List.of("timestamp", "item"), null, null, false, 0, false), + null, null, + null, + kafkaServer.consumerProperties(), + null, null, null, null, null, + true, + null, null, null, null, null, null, null, null + ), + null, null, null, null, null, null, null, null, null, null, null + ); + } + + private KafkaSupervisorTuningConfig createTuningConfig() + { + return new KafkaSupervisorTuningConfig( + null, + null, null, null, + 1, + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null + ); + } + + private List> generateRecordsForTopic( + String topic, + int numRecords, + DateTime startTime + ) + { + final List> records = new ArrayList<>(); + for (int i = 0; i < numRecords; ++i) { + String valueCsv = StringUtils.format( + "%s,%s,%d", + startTime.plusDays(i), + IdUtils.getRandomId(), + ThreadLocalRandom.current().nextInt(1000) + ); + records.add( + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(valueCsv)) + ); + } + return records; + } + + private void waitForSegmentsToBeQueryable(int numSegments) + { + eventCollector.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("segment/schemaCache/refresh/count") + .hasService("druid/broker") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(numSegments) + ); + } + + /** + * Verifies the total number of used segments in {@link #dataSource}. + */ + private void verifyUsedSegmentCount(int expectedCount) + { + final Set allUsedSegments = overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.INCLUDING_OVERSHADOWED); + Assertions.assertEquals(expectedCount, allUsedSegments.size()); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java index 57a18694e10d..5fd0785dc1b6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/MoreResources.java @@ -39,7 +39,20 @@ public class MoreResources */ public static class Task { + /** + * A minimal 'index' task that ingests inline data {@link Resources.InlineData#CSV_10_DAYS} + * using "DAY" granularity. + */ public static final Supplier BASIC_INDEX = + () -> TaskBuilder + .ofTypeIndex() + .isoTimestampColumn("time") + .csvInputFormatWithColumns("time", "item", "value") + .inlineInputSourceWithData(Resources.InlineData.CSV_10_DAYS) + .segmentGranularity("DAY") + .dimensions(); + + public static final Supplier INDEX_TASK_WITH_AGGREGATORS = () -> TaskBuilder .ofTypeIndex() .jsonInputFormat() diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResource.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResource.java index 57c91bfffb70..d16a61319d84 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResource.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResource.java @@ -85,7 +85,7 @@ public void onStarted(EmbeddedDruidCluster cluster) cluster.addCommonProperty("druid.indexer.logs.s3Prefix", "druid/indexing-logs"); // Configure S3 connection properties - cluster.addCommonProperty("druid.s3.endpoint.url", getEndpointUrl()); + cluster.addCommonProperty("druid.s3.endpoint.url", cluster.getEmbeddedHostname().useInUri(getEndpointUrl())); cluster.addCommonProperty("druid.s3.accessKey", getAccessKey()); cluster.addCommonProperty("druid.s3.secretKey", getSecretKey()); cluster.addCommonProperty("druid.s3.enablePathStyleAccess", "true"); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResourceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResourceTest.java index a7569a963956..0cdcb4b2b9a7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResourceTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/minio/MinIOStorageResourceTest.java @@ -20,6 +20,7 @@ package org.apache.druid.testing.embedded.minio; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHostname; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentMatchers; @@ -38,6 +39,8 @@ public void testMinIOContainerLifecycle() { final MinIOStorageResource resource = new MinIOStorageResource("test-bucket", "test/base"); final EmbeddedDruidCluster cluster = Mockito.mock(EmbeddedDruidCluster.class); + Mockito.when(cluster.getEmbeddedHostname()).thenReturn(EmbeddedHostname.localhost()); + resource.beforeStart(cluster); resource.start(); assertTrue(resource.isRunning()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java index bce1362254f1..0d82c6f7850e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java @@ -40,7 +40,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.testing.embedded.EmbeddedClusterApis; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -import org.apache.druid.testing.embedded.EmbeddedOverlord; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.apache.kafka.clients.producer.ProducerRecord; import org.joda.time.Period; @@ -85,7 +84,7 @@ void setupCreateKafkaTopic() /** * Submits a supervisor spec to the Overlord. */ - protected void submitSupervisor(EmbeddedOverlord overlord) + protected void submitSupervisor() { // Submit a supervisor. final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeQueryTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeQueryTest.java index a7071714f3e3..165e6b06cc12 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeQueryTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeQueryTest.java @@ -167,7 +167,7 @@ protected void setupLookups() throws Exception void setUpEach() { msqApis = new EmbeddedMSQApis(cluster, overlord); - submitSupervisor(overlord); + submitSupervisor(); publishToKafka(TestIndex.getMMappedWikipediaIndex()); // Wait for it to be loaded. @@ -291,7 +291,7 @@ public void test_selectBroadcastJoin_task_withRealtime() SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql); String taskId = taskStatus.getTaskId(); - cluster.callApi().waitForTaskToFinish(taskId, overlord); + cluster.callApi().waitForTaskToFinish(taskId, overlord.latchableEmitter()); final TaskStatusResponse currentStatus = cluster.callApi().onLeaderOverlord( o -> o.taskStatus(taskId) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeUnnestQueryTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeUnnestQueryTest.java index 2c9e7c6258cf..cb38ce54acc6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeUnnestQueryTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeUnnestQueryTest.java @@ -113,7 +113,7 @@ void setUpEach() QueryableIndex index = TestIndex.getMMappedTestIndex(); - submitSupervisor(overlord); + submitSupervisor(); publishToKafka(index); final int totalRows = index.getNumRows(); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResource.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResource.java new file mode 100644 index 000000000000..c9a60fd443f3 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResource.java @@ -0,0 +1,90 @@ +/* + * 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.embedded.psql; + +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.PostgreSQLContainer; + +/** + * Resource that creates a PostgreSQL metadata store. + */ +public class PostgreSQLMetadataResource extends TestcontainerResource> +{ + private static final String DATABASE_NAME = "druid_test"; + private static final String USERNAME = "sally"; + private static final String PASSWORD = "diurd"; + + private String connectURI; + + @Override + protected PostgreSQLContainer createContainer() + { + return new PostgreSQLContainer<>("postgres:16-alpine") + .withDatabaseName(DATABASE_NAME) + .withUsername(USERNAME) + .withPassword(PASSWORD); + } + + @Override + public void onStarted(EmbeddedDruidCluster cluster) + { + connectURI = createConnectURI(cluster); + cluster.addExtension(PostgreSQLMetadataStorageModule.class); + cluster.addCommonProperty("druid.metadata.storage.type", PostgreSQLMetadataStorageModule.TYPE); + cluster.addCommonProperty("druid.metadata.storage.connector.connectURI", createConnectURI(cluster)); + cluster.addCommonProperty("druid.metadata.storage.connector.user", getUsername()); + cluster.addCommonProperty("druid.metadata.storage.connector.password", getPassword()); + } + + public String getDatabaseName() + { + return DATABASE_NAME; + } + + public String getUsername() + { + return USERNAME; + } + + public String getPassword() + { + return PASSWORD; + } + + public String getConnectURI() + { + ensureRunning(); + return connectURI; + } + + private String createConnectURI(EmbeddedDruidCluster cluster) + { + ensureRunning(); + return StringUtils.format( + "jdbc:postgresql://%s:%d/%s", + cluster.getEmbeddedHostname(), + getContainer().getMappedPort(5432), + DATABASE_NAME + ); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResourceTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResourceTest.java new file mode 100644 index 000000000000..54d62c50e1e3 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataResourceTest.java @@ -0,0 +1,62 @@ +/* + * 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.embedded.psql; + +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.Statement; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class PostgreSQLMetadataResourceTest +{ + @Test + @Timeout(60) + public void testPostgreSQLMetdataStore() throws Exception + { + final PostgreSQLMetadataResource resource = new PostgreSQLMetadataResource(); + + resource.start(); + assertTrue(resource.isRunning()); + + // Test database connection + resource.onStarted(EmbeddedDruidCluster.empty()); + final String uri = resource.getConnectURI(); + final String userName = resource.getUsername(); + final String password = resource.getPassword(); + + try (Connection connection = DriverManager.getConnection(uri, userName, password); + Statement statement = connection.createStatement()) { + final ResultSet resultSet = statement.executeQuery("SELECT CURRENT_DATABASE()"); + assertTrue(resultSet.next()); + assertEquals(resource.getDatabaseName(), resultSet.getString(1)); + } + + resource.stop(); + assertFalse(resource.isRunning()); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataStoreTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataStoreTest.java new file mode 100644 index 000000000000..a3741eb4d82c --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/psql/PostgreSQLMetadataStoreTest.java @@ -0,0 +1,44 @@ +/* + * 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.embedded.psql; + +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.indexing.IndexTaskTest; + +/** + * Same as {@link IndexTaskTest}, but using a PostgreSQL metadata store instead of Derby. + */ +public class PostgreSQLMetadataStoreTest extends IndexTaskTest +{ + @Override + public EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withZookeeper() + .useLatchableEmitter() + .addResource(new PostgreSQLMetadataResource()) + .addServer(overlord) + .addServer(coordinator) + .addServer(indexer) + .addServer(historical) + .addServer(broker) + .addServer(new EmbeddedRouter()); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedEventCollector.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedEventCollector.java new file mode 100644 index 000000000000..90fab4ac1cf5 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/EmbeddedEventCollector.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.embedded.server; + +import com.google.inject.Injector; +import com.google.inject.Module; +import org.apache.druid.cli.ServerRunnable; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.testing.cli.CliEventCollector; +import org.apache.druid.testing.embedded.EmbeddedDruidServer; +import org.apache.druid.testing.embedded.EmbeddedHostname; +import org.apache.http.client.utils.URIBuilder; + +import java.util.ArrayList; +import java.util.List; + +/** + * Implementation of {@link EmbeddedDruidServer} provided by the extension {@code testing-tools}. + * This server collects events emitted using the {@code HttpPostEmitter}. + */ +public class EmbeddedEventCollector extends EmbeddedDruidServer +{ + @Override + protected ServerRunnable createRunnable(LifecycleInitHandler handler) + { + return new EventCollector(handler); + } + + public String getMetricsUrl() + { + return new URIBuilder() + .setScheme("http") + .setHost(EmbeddedHostname.containerFriendly().toString()) + .setPort(CliEventCollector.PORT) + .setPath("/druid-ext/testing-tools/events") + .toString(); + } + + private class EventCollector extends CliEventCollector + { + private final LifecycleInitHandler handler; + + private EventCollector(LifecycleInitHandler handler) + { + this.handler = handler; + } + + @Override + public Lifecycle initLifecycle(Injector injector) + { + final Lifecycle lifecycle = super.initLifecycle(injector); + handler.onLifecycleInit(lifecycle); + return lifecycle; + } + + @Override + protected List getModules() + { + final List modules = new ArrayList<>(super.getModules()); + modules.add(EmbeddedEventCollector.this::bindReferenceHolder); + return modules; + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java index e8d7382179d1..303ad16afadf 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java @@ -128,7 +128,7 @@ public void test_switchLeader_andVerifyUsingSysTables() coordinator1.latchableEmitter().waitForEvent( event -> event.hasMetricName("segment/metadataCache/used/count") .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValue(10) + .hasValueAtLeast(10) ); // Run sys queries, switch leaders, repeat @@ -153,7 +153,7 @@ public void test_switchLeader_andVerifyUsingSysTables() } } - private void verifyNodeRoleHasServerCount( + public static void verifyNodeRoleHasServerCount( NodeRole role, int expectedCount, DruidNodeDiscoveryProvider discovery, @@ -174,7 +174,7 @@ private void verifyNodeRoleHasServerCount( } } - private void verifySelfDiscoveredStatusReturnsOk( + private static void verifySelfDiscoveredStatusReturnsOk( DiscoveryDruidNode node, HttpClient httpClient ) throws Exception diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HttpEmitterEventCollectorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HttpEmitterEventCollectorTest.java new file mode 100644 index 000000000000..59246d05d3b2 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HttpEmitterEventCollectorTest.java @@ -0,0 +1,94 @@ +/* + * 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.embedded.server; + +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.emitter.LatchableEmitterModule; +import org.apache.druid.testing.embedded.indexing.MoreResources; +import org.apache.druid.testing.embedded.indexing.Resources; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.junit.jupiter.api.Test; + +/** + * Embedded test to verify basic functionality of {@code HttpPostEmitter} and + * {@link EmbeddedEventCollector}. + */ +public class HttpEmitterEventCollectorTest extends EmbeddedClusterTestBase +{ + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedEventCollector eventCollector = new EmbeddedEventCollector() + .addProperty("druid.emitter", "latching"); + + @Override + protected EmbeddedDruidCluster createCluster() + { + // Event collector node uses a latchable emitter + // All other servers use http emitter + return EmbeddedDruidCluster + .withEmbeddedDerbyAndZookeeper() + .addExtension(LatchableEmitterModule.class) + .addCommonProperty("druid.emitter", "http") + .addCommonProperty("druid.emitter.http.recipientBaseUrl", eventCollector.getMetricsUrl()) + .addCommonProperty("druid.emitter.http.flushMillis", "500") + .addServer(overlord) + .addServer(eventCollector) + .addServer(coordinator) + .addServer(new EmbeddedHistorical()) + .addServer(new EmbeddedBroker()) + .addServer(new EmbeddedIndexer()); + } + + @Test + public void test_runIndexTask_andQueryData() + { + // Run a task + final String taskId = IdUtils.getRandomId(); + final IndexTask task = MoreResources.Task.BASIC_INDEX.get().dataSource(dataSource).withId(taskId); + cluster.callApi().onLeaderOverlord(o -> o.runTask(taskId, task)); + + // Wait for the task finish metric to be emitted by the Overlord + eventCollector.latchableEmitter().waitForEvent( + event -> event.hasMetricName("task/run/time") + .hasService("druid/overlord") + .hasHost(overlord.bindings().selfNode().getHostAndPort()) + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasDimension(DruidMetrics.TASK_ID, taskId) + ); + + // Wait for the Broker to refresh schema + eventCollector.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("segment/schemaCache/refresh/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasService("druid/broker"), + agg -> agg.hasSumAtLeast(10) + ); + + cluster.callApi().verifySqlQuery("SELECT * FROM %s", dataSource, Resources.InlineData.CSV_10_DAYS); + } +} diff --git a/extensions-core/druid-testcontainers/README.md b/extensions-core/druid-testcontainers/README.md new file mode 100644 index 000000000000..fb490253ce0a --- /dev/null +++ b/extensions-core/druid-testcontainers/README.md @@ -0,0 +1,42 @@ + + +# Testcontainers for Apache Druid + +This module contains Testcontainers implementation for running Apache Druid services. +The dependencies in the pom must be minimal so that this module can be used independently by libraries and codebases outside Druid to run Testcontainers. + +## Usage + +``` +final DruidContainer container = new DruidContainer("overlord", "apache/druid:33.0.0"); +container.start(); +``` + +## Standard Druid Images + +1. `apache/druid:33.0.0` +2. `apache/druid:32.0.1` +3. `apache/druid:31.0.2` + +## Next steps + +- Add Druid command for running a starter cluster in a single container +- Publish this extension as a community module for Testcontainers +- Contribute the code to the main Testcontainers repository diff --git a/extensions-core/druid-testcontainers/pom.xml b/extensions-core/druid-testcontainers/pom.xml new file mode 100644 index 000000000000..1a577ecb1cb9 --- /dev/null +++ b/extensions-core/druid-testcontainers/pom.xml @@ -0,0 +1,65 @@ + + + + + 4.0.0 + + org.apache.druid.extensions + druid-testcontainers + druid-testcontainers + Testcontainers for running Apache Druid services in tests + + + org.apache.druid + druid + 35.0.0-SNAPSHOT + ../../pom.xml + + + + + + org.testcontainers + testcontainers + ${testcontainers.version} + compile + + + junit + junit + provided + + + + + + + + org.jacoco + jacoco-maven-plugin + + true + + + + + + diff --git a/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidCommand.java b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidCommand.java new file mode 100644 index 000000000000..f9d3ef89a1d7 --- /dev/null +++ b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidCommand.java @@ -0,0 +1,199 @@ +/* + * 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; + +import java.util.Map; + +/** + * Druid command to launch a {@link DruidContainer}. + * + * @see Server + */ +public interface DruidCommand +{ + String getName(); + + String getJavaOpts(); + + Integer[] getExposedPorts(); + + Map getDefaultProperties(); + + /** + * Standard Druid commands to launch a server. + * + * @see #OVERLORD + * @see #COORDINATOR + */ + enum Server implements DruidCommand + { + /** + * Command to run a Druid Coordinator which coordinates segment assignments, + * balancing and centralized schema. + */ + COORDINATOR( + "coordinator", + "-Xms128m -Xmx128m", + Map.of( + "druid.coordinator.startDelay", "PT0.1S", + "druid.coordinator.period", "PT0.5S", + "druid.manager.segments.pollDuration", "PT0.1S" + ), + 8081 + ), + + /** + * Command to run a Druid Overlord which manages ingestion tasks and publishes + * segment metadata to Metadata Store. + */ + OVERLORD( + "overlord", + "-Xms256m -Xmx256m", + Map.of( + "druid.indexer.storage.type", "metadata", + "druid.indexer.queue.startDelay", "PT0S", + "druid.indexer.queue.restartDelay", "PT0S", + // Keep a small sync timeout so that Peons and Indexers are not stuck + // handling a change request when Overlord has already shutdown + "druid.indexer.runner.syncRequestTimeout", "PT1S" + ), + 8090 + ), + + /** + * Command to run a Druid Indexer which is a lightweight ingestion worker + * that launches ingestion tasks as separate threads in the same JVM. + *

+ * By default, this Indexer has a task capacity of 2. + */ + INDEXER( + "indexer", + "-Xms128m -Xmx128m", + Map.of( + "druid.lookup.enableLookupSyncOnStartup", "false", + "druid.worker.capacity", "2", + "druid.processing.buffer.sizeBytes", "50MiB", + "druid.processing.numMergeBuffers", "2", + "druid.processing.numThreads", "5" + ), + 8091 + ), + + /** + * Command to run a Druid MiddleManager which is an ingestion worker + * that launches ingestion tasks as child processes. + *

+ * By default, this MiddleManager has a task capacity of 2 running at exposed + * ports 8100 and 8101. + */ + MIDDLE_MANAGER( + "middleManager", + "-Xms128m -Xmx128m", + Map.of( + "druid.lookup.enableLookupSyncOnStartup", "false", + "druid.worker.capacity", "2", + "druid.processing.buffer.sizeBytes", "50MiB", + "druid.processing.numMergeBuffers", "2", + "druid.processing.numThreads", "5" + ), + 8091, 8100, 8101 + ), + + /** + * Command to run Druid Historical service which hosts segment data and can + * be queried by a Broker. + */ + HISTORICAL( + "historical", + "-Xms128m -Xmx128m", + Map.of( + "druid.segmentCache.locations", "[{\"path\":\"/opt/druid/var/segment-cache\",\"maxSize\":\"50M\"}]", + "druid.processing.buffer.sizeBytes", "10MiB", + "druid.processing.numMergeBuffers", "2", + "druid.processing.numThreads", "5" + ), + 8083 + ), + + /** + * Command to run a Druid Broker which can handle all SQL and JSON queries + * over HTTP and JDBC. + */ + BROKER( + "broker", + "-Xms128m -Xmx128m", + Map.of( + "druid.lookup.enableLookupSyncOnStartup", "false", + "druid.processing.buffer.sizeBytes", "50MiB", + "druid.processing.numMergeBuffers", "2", + "druid.processing.numThreads", "5" + ), + 8082 + ), + + /** + * Command to run a Druid Router which routes queries to different Brokers + * and serves the Druid Web-Console UI. + */ + ROUTER( + "router", + "-Xms128m -Xmx128m", + Map.of("druid.router.managementProxy.enabled", "true"), + 8888 + ); + + private final String name; + private final String javaOpts; + private final Integer[] exposedPorts; + private final Map defaultProperties; + + Server(String name, String javaOpts, Map defaultProperties, Integer... exposedPorts) + { + this.name = name; + this.javaOpts = javaOpts; + this.defaultProperties = defaultProperties; + this.exposedPorts = exposedPorts; + } + + @Override + public String getName() + { + return name; + } + + @Override + public String getJavaOpts() + { + return javaOpts; + } + + @Override + public Integer[] getExposedPorts() + { + return exposedPorts; + } + + @Override + public Map getDefaultProperties() + { + return defaultProperties; + } + } +} diff --git a/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidContainer.java b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidContainer.java new file mode 100644 index 000000000000..81c38ad6dbad --- /dev/null +++ b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/DruidContainer.java @@ -0,0 +1,164 @@ +/* + * 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; + +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.images.builder.Transferable; +import org.testcontainers.utility.DockerImageName; + +import java.io.StringWriter; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Testcontainer for running Apache Druid services. + *

+ * Usage: + *

+ *
+ * 
+ * Dependencies of Druid services: + *
    + *
  • All Druid services need to be able to connect to a Zookeeper cluster for + * service discovery. Specified by property {@code druid.service.zk.host}
  • + *
  • Coordinator and Overlord services need to be able to connect to a SQL + * metadata store, specified by property {@code druid.metadata.storage.connector.connectURI}.
  • + *
+ */ +public class DruidContainer extends GenericContainer +{ + /** + * Standard images for {@link DruidContainer}. + */ + public static class Image + { + public static final DockerImageName APACHE_31 = DockerImageName.parse("apache/druid:31.0.2"); + public static final DockerImageName APACHE_32 = DockerImageName.parse("apache/druid:32.0.1"); + public static final DockerImageName APACHE_33 = DockerImageName.parse("apache/druid:33.0.0"); + } + + private static final String COMMON_PROPERTIES_PATH = "/tmp/druid_conf_common.runtime.properties"; + private static final String SERVICE_PROPERTIES_PATH = "/tmp/druid_conf_runtime.properties"; + + private final Properties commonProperties = new Properties(); + private final Properties serviceProperties = new Properties(); + + /** + * Creates a new {@link DruidContainer} which uses the given image name. + * + * @param command Druid command to run. e.g. {@code DruidCommand.Server.COORDINATOR}. + * @param imageName Name of the Druid image to use + * @see DruidCommand for standard Druid commands. + * @see Image for standard Druid images + */ + public DruidContainer(DruidCommand command, String imageName) + { + this(command, DockerImageName.parse(imageName)); + } + + /** + * Creates a new {@link DruidContainer} which uses the given image name. + * + * @param command Druid command to run. e.g. {@code DruidCommand.Server.OVERLORD}. + * @param imageName Name of the Druid image to use + * @see DruidCommand for standard Druid commands. + * @see Image for standard Druid images + */ + public DruidContainer(DruidCommand command, DockerImageName imageName) + { + super(imageName); + + setCommand(command.getName()); + withEnv("DRUID_CONFIG_COMMON", COMMON_PROPERTIES_PATH); + withEnv("DRUID_CONFIG_" + command.getName(), SERVICE_PROPERTIES_PATH); + withEnv("JAVA_OPTS", command.getJavaOpts()); + + final Integer[] exposedPorts = command.getExposedPorts(); + withExposedPorts(exposedPorts); + + serviceProperties.putAll(command.getDefaultProperties()); + + final int servicePort = exposedPorts[0]; + serviceProperties.setProperty("druid.plaintextPort", String.valueOf(servicePort)); + waitingFor(Wait.forHttp("/status/health").forPort(servicePort)); + + // Bind the ports statically (rather than using a mapped port) so that this + // Druid service is discoverable with the Druid service discovery + List portBindings = Stream.of(exposedPorts).map( + port -> port + ":" + port + ).collect(Collectors.toList()); + setPortBindings(portBindings); + } + + /** + * Binds the host path in the given {@link MountedDir} to the corresponding + * container path with read-write permissions. + */ + public DruidContainer withFileSystemBind(MountedDir mountedDir) + { + return withFileSystemBind( + mountedDir.hostFile().getAbsolutePath(), + mountedDir.containerFile().getAbsolutePath(), + BindMode.READ_WRITE + ); + } + + /** + * Sets a common property to be used for the service running on this container. + * The properties are written out to the file {@link #COMMON_PROPERTIES_PATH}. + */ + public DruidContainer withCommonProperty(String key, String value) + { + commonProperties.setProperty(key, value); + return this; + } + + /** + * Sets a runtime property to be used for the service running on this container. + * The properties are written out to the file {@link #SERVICE_PROPERTIES_PATH}. + */ + public DruidContainer withServiceProperty(String key, String value) + { + serviceProperties.setProperty(key, value); + return this; + } + + @Override + protected void containerIsCreated(String containerId) + { + copyFileToContainer(Transferable.of(toString(commonProperties), 0777), COMMON_PROPERTIES_PATH); + copyFileToContainer(Transferable.of(toString(serviceProperties), 0777), SERVICE_PROPERTIES_PATH); + } + + private static String toString(Properties properties) + { + try (StringWriter writer = new StringWriter()) { + properties.store(writer, "Druid Runtime Properties"); + return writer.toString(); + } + catch (Exception e) { + throw new RuntimeException("Could not serialize Druid runtime properties", e); + } + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/MountedDir.java similarity index 63% rename from integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java rename to extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/MountedDir.java index de8970b20ee5..844a7bd6f3fb 100644 --- a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java +++ b/extensions-core/druid-testcontainers/src/main/java/org/apache/druid/testing/MountedDir.java @@ -17,16 +17,31 @@ * under the License. */ -package org.apache.druid.cli; +package org.apache.druid.testing; -import com.github.rvesse.airline.builder.CliBuilder; +import java.io.File; -public class CustomNodeRoleCommandCreator implements CliCommandCreator +/** + * Represents a directory mounted on a {@link DruidContainer}. + */ +public class MountedDir { - @SuppressWarnings({ "rawtypes", "unchecked" }) - @Override - public void addCommands(CliBuilder builder) + private final File hostFile; + private final File containerFile; + + public MountedDir(File containerFile, File hostFile) + { + this.hostFile = hostFile; + this.containerFile = containerFile; + } + + public File containerFile() + { + return containerFile; + } + + public File hostFile() { - builder.withGroup("server").withCommands(CliCustomNodeRole.class); + return hostFile; } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java index c109516e4875..c0c373ba2e74 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kafka.simulate; import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.TestcontainerResource; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.NewTopic; @@ -39,17 +40,29 @@ */ public class KafkaResource extends TestcontainerResource { - public KafkaResource() + private static final String KAFKA_IMAGE = "apache/kafka:4.0.0"; + + private EmbeddedDruidCluster cluster; + + @Override + public void beforeStart(EmbeddedDruidCluster cluster) { - super(); + this.cluster = cluster; } - private static final String KAFKA_IMAGE = "apache/kafka:4.0.0"; - @Override protected KafkaContainer createContainer() { - return new KafkaContainer(KAFKA_IMAGE); + // The result of getBootstrapServers() is the first entry in KafkaContainer.advertisedListeners. + // Override getBootstrapServers() to ensure that both DruidContainers and + // EmbeddedDruidServers can connect to the Kafka brokers. + return new KafkaContainer(KAFKA_IMAGE) { + @Override + public String getBootstrapServers() + { + return cluster.getEmbeddedHostname().useInHostAndPort(super.getBootstrapServers()); + } + }; } public String getBootstrapServerUrl() diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResourceTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResourceTest.java index 437c89183b4f..9b8237417727 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResourceTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kafka.simulate; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -37,6 +38,7 @@ public void testKafka() { final KafkaResource resource = new KafkaResource(); + resource.beforeStart(EmbeddedDruidCluster.empty()); resource.start(); assertTrue(resource.isRunning()); diff --git a/extensions-core/testing-tools/pom.xml b/extensions-core/testing-tools/pom.xml index 1d84d97b4428..ed41e242bb82 100644 --- a/extensions-core/testing-tools/pom.xml +++ b/extensions-core/testing-tools/pom.xml @@ -131,6 +131,32 @@ provided
+ + org.eclipse.jetty + jetty-servlet + provided + + + com.github.rvesse + airline + provided + + + org.eclipse.jetty + jetty-server + provided + + + com.sun.jersey + jersey-server + provided + + + javax.ws.rs + jsr311-api + provided + + commons-io diff --git a/integration-tests/src/main/java/org/apache/druid/cli/CliCustomNodeRole.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CliEventCollector.java similarity index 81% rename from integration-tests/src/main/java/org/apache/druid/cli/CliCustomNodeRole.java rename to extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CliEventCollector.java index e0e1605b3026..60b85845b5e6 100644 --- a/integration-tests/src/main/java/org/apache/druid/cli/CliCustomNodeRole.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CliEventCollector.java @@ -17,18 +17,16 @@ * under the License. */ -package org.apache.druid.cli; +package org.apache.druid.testing.cli; import com.fasterxml.jackson.databind.ObjectMapper; import com.github.rvesse.airline.annotations.Command; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.name.Names; -import com.google.inject.servlet.GuiceFilter; +import org.apache.druid.cli.ServerRunnable; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; import org.apache.druid.guice.LazySingleton; @@ -42,11 +40,13 @@ import org.apache.druid.server.security.AuthenticationUtils; import org.apache.druid.server.security.Authenticator; import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.testing.metrics.EventCollectorResource; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.server.handler.StatisticsHandler; import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -55,19 +55,19 @@ import java.util.Set; @Command( - name = CliCustomNodeRole.SERVICE_NAME, - description = "Some custom druid node role defined in an extension" + name = CliEventCollector.SERVICE_NAME, + description = "Custom Druid node role used to collect metrics over HTTP" ) -public class CliCustomNodeRole extends ServerRunnable +public class CliEventCollector extends ServerRunnable { - private static final Logger LOG = new Logger(CliCustomNodeRole.class); + private static final Logger LOG = new Logger(CliEventCollector.class); - public static final String SERVICE_NAME = "custom-node-role"; + public static final String SERVICE_NAME = "eventCollector"; public static final int PORT = 9301; public static final int TLS_PORT = 9501; - public static final NodeRole NODE_ROLE = new NodeRole(CliCustomNodeRole.SERVICE_NAME); + public static final NodeRole NODE_ROLE = new NodeRole(CliEventCollector.SERVICE_NAME); - public CliCustomNodeRole() + public CliEventCollector() { super(LOG); } @@ -75,21 +75,25 @@ public CliCustomNodeRole() @Override protected Set getNodeRoles(Properties properties) { - return ImmutableSet.of(NODE_ROLE); + return Set.of(NODE_ROLE); } @Override protected List getModules() { - return ImmutableList.of( + return List.of( binder -> { - LOG.info("starting up"); - binder.bindConstant().annotatedWith(Names.named("serviceName")).to(CliCustomNodeRole.SERVICE_NAME); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(CliCustomNodeRole.PORT); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(CliCustomNodeRole.TLS_PORT); + LOG.info( + "Starting Test Service [eventCollector]. This is a test only" + + " Druid service which must never be used in production." + ); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to(CliEventCollector.SERVICE_NAME); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(CliEventCollector.PORT); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(CliEventCollector.TLS_PORT); binder.bind(JettyServerInitializer.class).to(CustomJettyServiceInitializer.class).in(LazySingleton.class); LifecycleModule.register(binder, Server.class); + Jerseys.addResource(binder, EventCollectorResource.class); bindAnnouncer( binder, @@ -105,9 +109,7 @@ protected List getModules() // ugly mimic of other jetty initializers private static class CustomJettyServiceInitializer implements JettyServerInitializer { - private static List UNSECURED_PATHS = ImmutableList.of( - "/status/health" - ); + private static final List UNSECURED_PATHS = List.of("/status/health"); private final ServerConfig serverConfig; @@ -145,7 +147,8 @@ public void initialize(Server server, Injector injector) jsonMapper ); - root.addFilter(GuiceFilter.class, "/*", null); + final FilterHolder guiceFilterHolder = JettyServerInitUtils.getGuiceFilterHolder(injector); + root.addFilter(guiceFilterHolder, "/*", null); final HandlerList handlerList = new HandlerList(); // Do not change the order of the handlers that have already been added diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CustomNodeRoleCommandCreator.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CustomNodeRoleCommandCreator.java similarity index 91% rename from integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CustomNodeRoleCommandCreator.java rename to extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CustomNodeRoleCommandCreator.java index 028f574f22c0..e77c9156b934 100644 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CustomNodeRoleCommandCreator.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cli/CustomNodeRoleCommandCreator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.testing.tools; +package org.apache.druid.testing.cli; import com.github.rvesse.airline.builder.CliBuilder; import org.apache.druid.cli.CliCommandCreator; @@ -28,6 +28,6 @@ public class CustomNodeRoleCommandCreator implements CliCommandCreator @Override public void addCommands(CliBuilder builder) { - builder.withGroup("server").withCommands(CliCustomNodeRole.class); + builder.withGroup("server").withCommands(CliEventCollector.class); } } diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventCollectorResource.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventCollectorResource.java new file mode 100644 index 000000000000..95dda687d44f --- /dev/null +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventCollectorResource.java @@ -0,0 +1,100 @@ +/* + * 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.metrics; + +import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.server.http.security.StateResourceFilter; +import org.apache.druid.testing.cli.CliEventCollector; + +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Used by {@link CliEventCollector}. + *

+ * Collects metrics emitted by other Druid services using {@code HttpPostEmitter}, + * and forwards them to {@link ServiceEmitter}. This can be used in conjunction + * with {@code LatchableEmitter} to watch for certain events to be emitted by a + * specific Druid service. + */ +@Path("/druid-ext/testing-tools/events") +public class EventCollectorResource +{ + private static final Logger log = new Logger(EventCollectorResource.class); + + private final ServiceEmitter emitter; + + @Inject + public EventCollectorResource( + ServiceEmitter emitter + ) + { + this.emitter = emitter; + } + + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response postEvents( + List events + ) + { + final Set services = new HashSet<>(); + final Set hosts = new HashSet<>(); + + int collectedEvents = 0; + + for (EventMap map : events) { + try { + final EventMapDeserializer deserializer = new EventMapDeserializer(map); + emitter.emit(deserializer.asEvent()); + ++collectedEvents; + + final String service = deserializer.getService(); + if (service != null) { + services.add(service); + } + + final String host = deserializer.getHost(); + if (host != null) { + hosts.add(host); + } + } + catch (Exception e) { + log.noStackTrace().error(e, "Could not collect event[%s]", map); + } + } + + log.debug("Collected [%d] events from service[%s], host[%s].", collectedEvents, services, hosts); + return Response.ok("{}").build(); + } +} diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventMapDeserializer.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventMapDeserializer.java new file mode 100644 index 000000000000..66f80895eea5 --- /dev/null +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/metrics/EventMapDeserializer.java @@ -0,0 +1,121 @@ +/* + * 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.metrics; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.java.util.emitter.service.AlertEvent; +import org.apache.druid.java.util.emitter.service.SegmentMetadataEvent; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + +import java.util.Map; +import java.util.Set; + +/** + * Deserializer used to convert {@link EventMap} to an {@link Event}. + */ +public class EventMapDeserializer +{ + private static final Set SYSTEM_DIMENSIONS = Set.of( + Event.FEED, + Event.TIMESTAMP, + Event.SERVICE, + Event.HOST, + Event.METRIC, + Event.VALUE + ); + + private final EventMap map; + + EventMapDeserializer(EventMap map) + { + this.map = map; + } + + Event asEvent() + { + final String feed = getStringValue(Event.FEED); + switch (feed) { + case "alerts": + return asAlertEvent(); + case "segment_metadata": + return asSegmentMetadataEvent(); + default: + return asMetricEvent(); + } + } + + String getService() + { + return getStringValue(Event.SERVICE); + } + + String getHost() + { + return getStringValue(Event.HOST); + } + + private String getStringValue(String key) + { + return (String) map.get(key); + } + + @SuppressWarnings("unchecked") + private AlertEvent asAlertEvent() + { + return new AlertEvent( + DateTimes.of(getStringValue(Event.TIMESTAMP)), + getService(), + getHost(), + AlertEvent.Severity.fromString(getStringValue("severity")), + getStringValue("description"), + (Map) map.get("data") + ); + } + + private ServiceMetricEvent asMetricEvent() + { + final ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder(); + map.forEach((key, value) -> { + if (!SYSTEM_DIMENSIONS.contains(key)) { + builder.setDimension(key, map.get(key)); + } + }); + + return builder + .setFeed(getStringValue(Event.FEED)) + .setCreatedTime(DateTimes.of(getStringValue(Event.TIMESTAMP))) + .setMetric(getStringValue(Event.METRIC), (Number) map.get(Event.VALUE)) + .build(getService(), getHost()); + } + + private SegmentMetadataEvent asSegmentMetadataEvent() + { + return new SegmentMetadataEvent( + getStringValue(SegmentMetadataEvent.DATASOURCE), + DateTimes.of(getStringValue(SegmentMetadataEvent.CREATED_TIME)), + DateTimes.of(getStringValue(SegmentMetadataEvent.START_TIME)), + DateTimes.of(getStringValue(SegmentMetadataEvent.END_TIME)), + getStringValue(SegmentMetadataEvent.VERSION), + (Boolean) map.get(SegmentMetadataEvent.IS_COMPACTED) + ); + } +} diff --git a/.github/workflows/distribution-checks.yml b/extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator similarity index 53% rename from .github/workflows/distribution-checks.yml rename to extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator index caadb16b608e..57b02c35b899 100644 --- a/.github/workflows/distribution-checks.yml +++ b/extensions-core/testing-tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator @@ -13,29 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: "Distribution Checks" -on: - push: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - paths: - - 'distribution/**' - - '**/pom.xml' - pull_request: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - paths: - - 'distribution/**' - - '**/pom.xml' - -jobs: - docker-build: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Build the Docker image - run: DOCKER_BUILDKIT=1 docker build -t apache/druid:tag -f distribution/docker/Dockerfile . +org.apache.druid.testing.cli.CustomNodeRoleCommandCreator diff --git a/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.py b/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.py deleted file mode 100644 index c958cbcaf15f..000000000000 --- a/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.py +++ /dev/null @@ -1,89 +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. - -from template import BaseTemplate, generate -from template import COORDINATOR, ZOO_KEEPER, METADATA, OVERLORD - -# The second Coordinator (and Overlord) cannot extend -# The base service: they need distinct ports. -class Template(BaseTemplate): - - def define_coordinator(self): - self.define_coordinator_one() - self.define_coordinator_two() - - def define_coordinator_one(self): - service_name = COORDINATOR + '-one' - service = self.define_master_service(service_name, COORDINATOR) - service['container_name'] = service_name - self.add_env(service, 'DRUID_INSTANCE', 'one') - self.add_env(service, 'druid_host', service_name) - service['container_name'] = service_name - - def define_coordinator_two(self): - service_name = COORDINATOR + '-two' - service = self.define_full_service(service_name, COORDINATOR, 120) - service['container_name'] = service_name - self.add_env(service, 'DRUID_INSTANCE', 'two') - self.add_env(service, 'druid_host', service_name) - service['ports'] = [ '18081:8081', '18281:8281', '15006:8000' ] - self.add_depends(service, [ZOO_KEEPER, METADATA]) - - def define_overlord(self): - self.define_overlord_one() - self.define_overlord_two() - - def define_overlord_one(self): - service_name = OVERLORD + '-one' - service = self.define_master_service(service_name, OVERLORD) - service['container_name'] = service_name - self.add_env(service, 'DRUID_INSTANCE', 'one') - self.add_env(service, 'druid_host', service_name) - - def define_overlord_two(self): - service_name = OVERLORD + '-two' - service = self.define_full_service(service_name, OVERLORD, 110) - service['container_name'] = service_name - self.add_env(service, 'DRUID_INSTANCE', 'two') - self.add_env(service, 'druid_host', service_name) - service['ports'] = [ '18090:8090', '18290:8290', '15009:8000' ] - self.add_depends(service, [ZOO_KEEPER, METADATA]) - - # No indexer in this cluster - def define_indexer(self): - pass - - # No historical in this cluster - def define_historical(self): - pass - - # No kafka dependency in this cluster - def define_kafka(self): - pass - - # The custom node role has no base definition. Also, there is - # no environment file: the needed environment settings are - # given here. - def define_custom_services(self): - service_name = 'custom-node-role' - service = self.define_full_service(service_name, None, 90) - service['container_name'] = service_name - self.add_env(service, 'DRUID_SERVICE', service_name) - self.add_env(service, 'SERVICE_DRUID_JAVA_OPTS', '-Xmx64m -Xms64m') - self.add_env(service, 'druid_host', service_name) - service['ports'] = [ '50011:50011', '9301:9301', '9501:9501', '5010:8000' ] - self.add_depends(service, [ZOO_KEEPER]) - -generate(__file__, Template()) diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 9c493ec2b3e0..2c475ca7ebf5 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -358,15 +358,6 @@ - - IT-HighAvailability - - false - - - HighAvailability - - IT-BatchIndex diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java deleted file mode 100644 index b12ba7f23e17..000000000000 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.testsEx.categories; - -public class HighAvailability -{ - -} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java deleted file mode 100644 index 57c1aed1b253..000000000000 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java +++ /dev/null @@ -1,244 +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.testsEx.leadership; - -import com.google.inject.Inject; -import org.apache.druid.cli.CliCustomNodeRole; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.DruidNodeDiscovery; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeRole; -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.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.TestClient; -import org.apache.druid.testing.utils.SqlTestQueryHelper; -import org.apache.druid.testsEx.categories.HighAvailability; -import org.apache.druid.testsEx.cluster.DruidClusterClient; -import org.apache.druid.testsEx.config.DruidTestRunner; -import org.apache.druid.testsEx.config.Initializer; -import org.apache.druid.testsEx.indexer.AbstractIndexerTest; -import org.apache.druid.testsEx.utils.DruidClusterAdminClient; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -@RunWith(DruidTestRunner.class) -@Category(HighAvailability.class) -public class ITHighAvailabilityTest -{ - private static final Logger LOG = new Logger(ITHighAvailabilityTest.class); - private static final String SYSTEM_QUERIES_RESOURCE = Initializer.queryFile(HighAvailability.class, "sys.json"); - private static final int NUM_LEADERSHIP_SWAPS = 3; - - @Inject - private IntegrationTestingConfig config; - - @Inject - private DruidClusterAdminClient druidClusterAdminClient; - - @Inject - private DruidNodeDiscoveryProvider druidNodeDiscovery; - - @Inject - private SqlTestQueryHelper queryHelper; - - @Inject - @TestClient - private HttpClient httpClient; - - @Inject - private DruidClusterClient clusterClient; - - @Test - public void testLeadershipChanges() throws Exception - { - int runCount = 0; - String previousCoordinatorLeader = null; - String previousOverlordLeader = null; - // fetch current leaders, make sure queries work, then swap leaders and do it again - do { - String coordinatorLeader = getLeader("coordinator"); - String overlordLeader = getLeader("indexer"); - - // we expect leadership swap to happen - assertNotEquals(previousCoordinatorLeader, coordinatorLeader); - assertNotEquals(previousOverlordLeader, overlordLeader); - - previousCoordinatorLeader = coordinatorLeader; - previousOverlordLeader = overlordLeader; - - String queries = fillTemplate( - AbstractIndexerTest.getResourceAsString(SYSTEM_QUERIES_RESOURCE), - overlordLeader, - coordinatorLeader - ); - queryHelper.testQueriesFromString(queries); - - swapLeadersAndWait(coordinatorLeader, overlordLeader); - } while (runCount++ < NUM_LEADERSHIP_SWAPS); - } - - @Test - public void testDiscoveryAndSelfDiscovery() - { - // The cluster used here has an abbreviated set of services. - verifyRoleDiscovery(NodeRole.BROKER, 1); - verifyRoleDiscovery(NodeRole.COORDINATOR, 2); - verifyRoleDiscovery(NodeRole.OVERLORD, 2); - verifyRoleDiscovery(NodeRole.ROUTER, 1); - } - - public void verifyRoleDiscovery(NodeRole role, int expectedCount) - { - DruidNodeDiscovery discovered = druidNodeDiscovery.getForNodeRole(role); - try { - int count = 0; - for (DiscoveryDruidNode node : discovered.getAllNodes()) { - if (clusterClient.selfDiscovered(clusterClient.nodeUrl(node.getDruidNode()))) { - count++; - } - } - assertEquals(expectedCount, count); - } - catch (Exception e) { - LOG.error(e, "node discovery failed"); - fail(); - } - } - - @Test - public void testCustomDiscovery() - { - verifyRoleDiscovery(CliCustomNodeRole.NODE_ROLE, 1); - verifyCoordinatorCluster(); - } - - private void swapLeadersAndWait(String coordinatorLeader, String overlordLeader) - { - String coordUrl; - String coordLabel; - if (isCoordinatorOneLeader(coordinatorLeader)) { - druidClusterAdminClient.restartCoordinatorContainer(); - coordUrl = config.getCoordinatorUrl(); - coordLabel = "coordinator one"; - } else { - druidClusterAdminClient.restartCoordinatorTwoContainer(); - coordUrl = config.getCoordinatorTwoUrl(); - coordLabel = "coordinator two"; - } - - String overlordUrl; - String overlordLabel; - if (isOverlordOneLeader(overlordLeader)) { - druidClusterAdminClient.restartOverlordContainer(); - overlordUrl = config.getOverlordUrl(); - overlordLabel = "overlord one"; - } else { - druidClusterAdminClient.restartOverlordTwoContainer(); - overlordUrl = config.getOverlordTwoUrl(); - overlordLabel = "overlord two"; - } - clusterClient.waitForNodeReady(coordLabel, coordUrl); - clusterClient.waitForNodeReady(overlordLabel, overlordUrl); - } - - private String getLeader(String service) - { - return clusterClient.getLeader(service); - } - - private String fillTemplate(String template, String overlordLeader, String coordinatorLeader) - { - /* - {"host":"%%BROKER%%","server_type":"broker", "is_leader": null}, - {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, - {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, - {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, - {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, - {"host":"%%ROUTER%%","server_type":"router", "is_leader": null}, - */ - String working = template; - working = StringUtils.replace(working, "%%OVERLORD_ONE%%", config.getOverlordInternalHost()); - working = StringUtils.replace(working, "%%OVERLORD_TWO%%", config.getOverlordTwoInternalHost()); - working = StringUtils.replace(working, "%%COORDINATOR_ONE%%", config.getCoordinatorInternalHost()); - working = StringUtils.replace(working, "%%COORDINATOR_TWO%%", config.getCoordinatorTwoInternalHost()); - working = StringUtils.replace(working, "%%BROKER%%", config.getBrokerInternalHost()); - working = StringUtils.replace(working, "%%ROUTER%%", config.getRouterInternalHost()); - if (isOverlordOneLeader(overlordLeader)) { - working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "1"); - working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "0"); - } else { - working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "0"); - working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "1"); - } - if (isCoordinatorOneLeader(coordinatorLeader)) { - working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "1"); - working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "0"); - } else { - working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "0"); - working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "1"); - } - return working; - } - - private boolean isCoordinatorOneLeader(String coordinatorLeader) - { - return coordinatorLeader.contains(transformHost(config.getCoordinatorInternalHost())); - } - - private boolean isOverlordOneLeader(String overlordLeader) - { - return overlordLeader.contains(transformHost(config.getOverlordInternalHost())); - } - - /** - * host + ':' which should be enough to distinguish subsets, e.g. 'druid-coordinator:8081' from - * 'druid-coordinator-two:8081' for example - */ - private static String transformHost(String host) - { - return StringUtils.format("%s:", host); - } - - private void verifyCoordinatorCluster() - { - // Verify the basics: 4 service types, excluding the custom node role. - // One of the two-node services has a size of 2. - // This endpoint includes an entry for historicals, even if none are running. - Map results = clusterClient.coordinatorCluster(); - assertEquals(5, results.size()); - @SuppressWarnings("unchecked") - List coordNodes = (List) results.get(NodeRole.COORDINATOR.getJsonName()); - assertEquals(2, coordNodes.size()); - @SuppressWarnings("unchecked") - List histNodes = (List) results.get(NodeRole.HISTORICAL.getJsonName()); - assertTrue(histNodes.isEmpty()); - } -} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml deleted file mode 100644 index 5d99ffcbc520..000000000000 --- a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml +++ /dev/null @@ -1,204 +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. -#------------------------------------------------------------------------- - -# Definition of the high-availability test cluster. -# See https://yaml.org/spec/1.2.2 for more about YAML -include: - - /cluster/Common/zk-metastore.yaml - -# This test does not use ingest or query workers. -druid: - coordinator: - instances: - - tag: one - port: 8081 - - tag: two - port: 8081 - proxyPort: 18081 - overlord: - instances: - - tag: one - port: 8090 - - tag: two - port: 8090 - proxyPort: 18090 - broker: - instances: - - port: 8082 - router: - instances: - - port: 8888 - custom-node-role: - instances: - - port: 9301 - -# Metastore initialization queries. -# REPLACE is used so that the statements are idempotent -# The fancy formatting is for human consumption, it is compacted internally -metastoreInit: - - sql: | - REPLACE INTO druid_segments ( - id, dataSource, created_date, start, end, partitioned, version, used, payload - ) - VALUES ( - 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', - 'twitterstream', - '2013-05-13T01:08:18.192Z', - '2013-01-01T00:00:00.000Z', - '2013-01-02T00:00:00.000Z', - 0, - '2013-01-02T04:13:41.980Z_v9', - 1, - '{"dataSource": "twitterstream", - "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", - "version": "2013-01-02T04:13:41.980Z_v9", - "loadSpec": { - "type": "s3_zip", - "bucket": "static.druid.io", - "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" - }, - "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, - rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", - "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", - "shardSpec": {"type": "none"}, - "binaryVersion": 9, - "size": 445235220, - "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" - }' - ) - - - sql: | - REPLACE INTO druid_segments ( - id, dataSource, created_date, start, end, partitioned, version, used, payload - ) - VALUES ( - 'twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9', - 'twitterstream', - '2013-05-13T00:03:28.640Z', - '2013-01-02T00:00:00.000Z', - '2013-01-03T00:00:00.000Z', - 0, - '2013-01-03T03:44:58.791Z_v9', - 1, - '{"dataSource": "twitterstream", - "interval": "2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z", - "version": "2013-01-03T03:44:58.791Z_v9", - "loadSpec": { - "type": "s3_zip", - "bucket": "static.druid.io", - "key": "data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip" - }, - "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, - rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", - "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", - "shardSpec": {"type": "none"}, - "binaryVersion": 9, - "size": 435325540, - "identifier": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9" - }' - ) - - - sql: | - REPLACE INTO druid_segments ( - id, dataSource, created_date, start, end, partitioned, version, used, payload - ) - VALUES ( - 'twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9', - 'twitterstream', - '2013-05-13T00:03:48.807Z', - '2013-01-03T00:00:00.000Z', - '2013-01-04T00:00:00.000Z', - 0, - '2013-01-04T04:09:13.590Z_v9', - 1, - '{"dataSource": "twitterstream", - "interval": "2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z", - "version": "2013-01-04T04:09:13.590Z_v9", - "loadSpec": { - "type": "s3_zip", - "bucket": "static.druid.io", - "key": "data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip" - }, - "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, - rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", - "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", - "shardSpec": {"type": "none"}, - "binaryVersion": 9, - "size": 411651320, - "identifier": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9" - }' - ) - - - sql: | - REPLACE INTO druid_segments ( - id, dataSource, created_date, start, end, partitioned, version, used, payload - ) - VALUES ( - 'wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9', - 'wikipedia_editstream', - '2013-03-15T20:49:52.348Z', - '2012-12-29T00:00:00.000Z', - '2013-01-10T08:00:00.000Z', - 0, - '2013-01-10T08:13:47.830Z_v9', - 1, - '{"dataSource": "wikipedia_editstream", - "interval": "2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z", - "version": "2013-01-10T08:13:47.830Z_v9", - "loadSpec": { - "type": "s3_zip", - "bucket": "static.druid.io", - "key": "data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip"}, - "dimensions": "anonymous,area_code,city,continent_code,country_name,dma_code,geo,language, - namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user", - "metrics": "added,count,deleted,delta,delta_hist,unique_users,variation", - "shardSpec": {"type": "none"}, - "binaryVersion": 9, - "size": 446027801, - "identifier": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" - }' - ) - - - sql: | - REPLACE INTO druid_segments ( - id, dataSource, created_date, start, end, partitioned, version, used, payload - ) - VALUES ( - 'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', - 'wikipedia', - '2013-08-08T21:26:23.799Z', - '2013-08-01T00:00:00.000Z', - '2013-08-02T00:00:00.000Z', - 0, - '2013-08-08T21:22:48.989Z', - 1, - '{"dataSource": "wikipedia", - "interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z", - "version": "2013-08-08T21:22:48.989Z", - "loadSpec": { - "type": "s3_zip", - "bucket": "static.druid.io", - "key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip" - }, - "dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace, - anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup", - "metrics": "count,delta,variation,added,deleted", - "shardSpec": {"type": "none"}, - "binaryVersion": 9, - "size": 24664730, - "identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z" - }' - ) diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json deleted file mode 100644 index c506352a4221..000000000000 --- a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json +++ /dev/null @@ -1,39 +0,0 @@ -[ - { - "description": "query sys.servers to make sure all expected servers are available", - "query": { - "query": "SELECT host, server_type, is_leader FROM sys.servers ORDER BY host" - }, - "expectedResults": [ - {"host":"%%BROKER%%","server_type":"broker", "is_leader": null}, - {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, - {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, - {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, - {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, - {"host":"%%ROUTER%%","server_type":"router", "is_leader": null} - ] - }, - { - "description": "query sys.segments which is fed via coordinator data", - "query": { - "query": "SELECT datasource, count(*) FROM sys.segments WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " - }, - "expectedResults": [ - { - "datasource": "wikipedia_editstream", - "EXPR$1": 1 - }, - { - "datasource": "twitterstream", - "EXPR$1": 3 - } - ] - }, - { - "description": "query sys.tasks which is fed via overlord", - "query": { - "query": "SELECT datasource, count(*) FROM sys.tasks WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " - }, - "expectedResults": [] - } -] \ No newline at end of file diff --git a/integration-tests-ex/docs/tests.md b/integration-tests-ex/docs/tests.md index 1ef74a773381..6351192edf1c 100644 --- a/integration-tests-ex/docs/tests.md +++ b/integration-tests-ex/docs/tests.md @@ -142,7 +142,6 @@ At present, the following test categories are fully or partly converted: | Category | Test NG Group | Description | | -------- | ------------- | ----------- | -| HighAvailability | high-availability | Cluster failover tests | | BatchIndex | batch-index | Batch indexing tsets | | InputFormat | input-format | Input format tests | diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CliCustomNodeRole.java b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CliCustomNodeRole.java deleted file mode 100644 index 087cb24bec5d..000000000000 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/CliCustomNodeRole.java +++ /dev/null @@ -1,173 +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.tools; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.rvesse.airline.annotations.Command; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.Module; -import com.google.inject.name.Names; -import com.google.inject.servlet.GuiceFilter; -import org.apache.druid.cli.ServerRunnable; -import org.apache.druid.discovery.NodeRole; -import org.apache.druid.guice.Jerseys; -import org.apache.druid.guice.LazySingleton; -import org.apache.druid.guice.LifecycleModule; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.http.SelfDiscoveryResource; -import org.apache.druid.server.initialization.ServerConfig; -import org.apache.druid.server.initialization.jetty.JettyServerInitUtils; -import org.apache.druid.server.initialization.jetty.JettyServerInitializer; -import org.apache.druid.server.security.AuthenticationUtils; -import org.apache.druid.server.security.Authenticator; -import org.apache.druid.server.security.AuthenticatorMapper; -import org.eclipse.jetty.server.Handler; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.HandlerList; -import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; - -import java.util.List; -import java.util.Properties; -import java.util.Set; - -@Command( - name = CliCustomNodeRole.SERVICE_NAME, - description = "Some custom druid node role defined in an extension" -) -public class CliCustomNodeRole extends ServerRunnable -{ - private static final Logger LOG = new Logger(CliCustomNodeRole.class); - - public static final String SERVICE_NAME = "custom-node-role"; - public static final int PORT = 9301; - public static final int TLS_PORT = 9501; - public static final NodeRole NODE_ROLE = new NodeRole(CliCustomNodeRole.SERVICE_NAME); - - public CliCustomNodeRole() - { - super(LOG); - } - - @Override - protected Set getNodeRoles(Properties properties) - { - return ImmutableSet.of(NODE_ROLE); - } - - @Override - protected List getModules() - { - return ImmutableList.of( - binder -> { - LOG.info("starting up custom node role"); - binder.bindConstant().annotatedWith(Names.named("serviceName")).to(CliCustomNodeRole.SERVICE_NAME); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(CliCustomNodeRole.PORT); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(CliCustomNodeRole.TLS_PORT); - - binder.bind(JettyServerInitializer.class).to(CustomJettyServiceInitializer.class).in(LazySingleton.class); - LifecycleModule.register(binder, Server.class); - - bindAnnouncer( - binder, - DiscoverySideEffectsProvider.create() - ); - Jerseys.addResource(binder, SelfDiscoveryResource.class); - LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); - } - ); - } - - // ugly mimic of other Jetty initializers - private static class CustomJettyServiceInitializer implements JettyServerInitializer - { - private static List UNSECURED_PATHS = ImmutableList.of( - "/status/health" - ); - - private final ServerConfig serverConfig; - - @Inject - public CustomJettyServiceInitializer(ServerConfig serverConfig) - { - this.serverConfig = serverConfig; - } - - @Override - public void initialize(Server server, Injector injector) - { - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - - final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); - final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); - - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - - // perform no-op authorization for these resources - AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS); - - List authenticators = authenticatorMapper.getAuthenticatorChain(); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); - - JettyServerInitUtils.addAllowHttpMethodsFilter(root, serverConfig.getAllowedHttpMethods()); - - JettyServerInitUtils.addExtensionFilters(root, injector); - - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper - ); - - root.addFilter(GuiceFilter.class, "/*", null); - - final HandlerList handlerList = new HandlerList(); - // Do not change the order of the handlers that have already been added - for (Handler handler : server.getHandlers()) { - handlerList.addHandler(handler); - } - - handlerList.addHandler(JettyServerInitUtils.getJettyRequestLogHandler()); - - // Add Gzip handler at the very end - handlerList.addHandler( - JettyServerInitUtils.wrapWithDefaultGzipHandler( - root, - serverConfig.getInflateBufferSize(), - serverConfig.getCompressionLevel() - ) - ); - - final StatisticsHandler statisticsHandler = new StatisticsHandler(); - statisticsHandler.setHandler(handlerList); - - server.setHandler(statisticsHandler); - } - } -} diff --git a/integration-tests-ex/tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator b/integration-tests-ex/tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator index 8a82b0a34812..c9b692c39c62 100644 --- a/integration-tests-ex/tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator +++ b/integration-tests-ex/tools/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator @@ -14,4 +14,3 @@ # limitations under the License. org.apache.druid.testing.tools.QueryRetryTestCommandCreator -org.apache.druid.testing.tools.CustomNodeRoleCommandCreator diff --git a/integration-tests/docker/docker-compose.high-availability.yml b/integration-tests/docker/docker-compose.high-availability.yml deleted file mode 100644 index 0b38b90efaf0..000000000000 --- a/integration-tests/docker/docker-compose.high-availability.yml +++ /dev/null @@ -1,152 +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. - -version: "2.2" -services: - druid-zookeeper-kafka: - extends: - file: docker-compose.base.yml - service: druid-zookeeper-kafka - - druid-metadata-storage: - extends: - file: docker-compose.base.yml - service: druid-metadata-storage - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - depends_on: - - druid-zookeeper-kafka - - druid-coordinator: - extends: - file: docker-compose.base.yml - service: druid-coordinator - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - DRUID_LOG_PATH=/shared/logs/ha-coordinator-one.log - - druid_manager_config_pollDuration=PT10S - - druid_manager_rules_pollDuration=PT10S - - druid_manager_segments_pollDuration=PT10S - - druid_coordinator_period=PT10S - depends_on: - - druid-metadata-storage - - druid-zookeeper-kafka - - druid-coordinator-two: - extends: - file: docker-compose.base.yml - service: druid-coordinator-two - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - DRUID_LOG_PATH=/shared/logs/ha-coordinator-two.log - - druid_host=druid-coordinator-two - - druid_manager_config_pollDuration=PT10S - - druid_manager_rules_pollDuration=PT10S - - druid_manager_segments_pollDuration=PT10S - - druid_coordinator_period=PT10S - depends_on: - - druid-coordinator - - druid-metadata-storage - - druid-zookeeper-kafka - - druid-overlord: - extends: - file: docker-compose.base.yml - service: druid-overlord - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - DRUID_LOG_PATH=/shared/logs/ha-overlord-one.log - depends_on: - - druid-coordinator - - druid-coordinator-two - - druid-metadata-storage - - druid-zookeeper-kafka - - druid-overlord-two: - extends: - file: docker-compose.base.yml - service: druid-overlord-two - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - DRUID_LOG_PATH=/shared/logs/ha-overlord-two.log - - druid_host=druid-overlord-two - depends_on: - - druid-coordinator - - druid-coordinator-two - - druid-metadata-storage - - druid-zookeeper-kafka - - druid-broker: - extends: - file: docker-compose.base.yml - service: druid-broker - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - depends_on: - - druid-coordinator - - druid-coordinator-two - - druid-overlord - - druid-overlord-two - - druid-zookeeper-kafka - - druid-router: - extends: - file: docker-compose.base.yml - service: druid-router - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - depends_on: - - druid-coordinator - - druid-coordinator-two - - druid-overlord - - druid-overlord-two - - druid-broker - - druid-custom-node-role: - image: druid/cluster - container_name: druid-custom-node-role - networks: - druid-it-net: - ipv4_address: 172.172.172.90 - ports: - - 50011:50011 - - 9301:9301 - - 9501:9501 - privileged: true - volumes: - - ${HOME}/shared:/shared - - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf - environment: - - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - DRUID_SERVICE=custom-node-role - - DRUID_LOG_PATH=/shared/logs/custom-node-role.log - - SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -Xms128m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5011 - - druid_host=druid-custom-node-role - - druid_auth_basic_common_cacheDirectory=/tmp/authCache/custom_node_role - - druid_server_https_crlPath=/tls/revocations.crl - env_file: - - ./environment-configs/common - - ${OVERRIDE_ENV} - depends_on: - - druid-zookeeper-kafka - - druid-coordinator - - druid-coordinator-two - -networks: - druid-it-net: - name: druid-it-net - ipam: - config: - - subnet: 172.172.172.0/24 \ No newline at end of file diff --git a/integration-tests/docker/druid.sh b/integration-tests/docker/druid.sh index 11c1ea2d6ce8..65210c86caa4 100755 --- a/integration-tests/docker/druid.sh +++ b/integration-tests/docker/druid.sh @@ -103,7 +103,7 @@ setupData() # The "query" and "security" test groups require data to be setup before running the tests. # In particular, they requires segments to be download from a pre-existing s3 bucket. # This is done by using the loadSpec put into metadatastore and s3 credientials set below. - if [ "$DRUID_INTEGRATION_TEST_GROUP" = "query" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "high-availability" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "upgrade" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "centralized-datasource-schema" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-task-schema-publish-disabled" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-coordinator-metadata-query-disabled" ]; then + if [ "$DRUID_INTEGRATION_TEST_GROUP" = "query" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "upgrade" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "centralized-datasource-schema" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-task-schema-publish-disabled" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-coordinator-metadata-query-disabled" ]; then cat /test-data/${DRUID_INTEGRATION_TEST_GROUP}-sample-data.sql | mysql -u root druid fi diff --git a/integration-tests/docker/test-data/high-availability-sample-data.sql b/integration-tests/docker/test-data/high-availability-sample-data.sql deleted file mode 100644 index abe0f115189b..000000000000 --- a/integration-tests/docker/test-data/high-availability-sample-data.sql +++ /dev/null @@ -1,20 +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. - -INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9','twitterstream','2013-05-13T01:08:18.192Z','2013-01-01T00:00:00.000Z','2013-01-02T00:00:00.000Z',0,'2013-01-02T04:13:41.980Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z\",\"version\":\"2013-01-02T04:13:41.980Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":445235220,\"identifier\":\"twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9\"}','1970-01-01T00:00:00.000Z'); -INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9','twitterstream','2013-05-13T00:03:28.640Z','2013-01-02T00:00:00.000Z','2013-01-03T00:00:00.000Z',0,'2013-01-03T03:44:58.791Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z\",\"version\":\"2013-01-03T03:44:58.791Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":435325540,\"identifier\":\"twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9\"}','1970-01-01T00:00:00.000Z'); -INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9','twitterstream','2013-05-13T00:03:48.807Z','2013-01-03T00:00:00.000Z','2013-01-04T00:00:00.000Z',0,'2013-01-04T04:09:13.590Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z\",\"version\":\"2013-01-04T04:09:13.590Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":411651320,\"identifier\":\"twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9\"}','1970-01-01T00:00:00.000Z'); -INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9','wikipedia_editstream','2013-03-15T20:49:52.348Z','2012-12-29T00:00:00.000Z','2013-01-10T08:00:00.000Z',0,'2013-01-10T08:13:47.830Z_v9',1,'{\"dataSource\":\"wikipedia_editstream\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"}','1970-01-01T00:00:00.000Z'); -INSERT INTO druid_segments (id, dataSource, created_date, start, end, partitioned, version, used, payload,used_status_last_updated) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}','1970-01-01T00:00:00.000Z'); diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 3ca3d41370c4..8435478ec104 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -346,20 +346,6 @@ osx-aarch_64 runtime - - org.eclipse.jetty - jetty-servlet - ${jetty.version} - - - org.eclipse.jetty - jetty-server - ${jetty.version} - - - com.google.inject.extensions - guice-servlet - io.confluent kafka-schema-registry-client diff --git a/integration-tests/script/docker_compose_args.sh b/integration-tests/script/docker_compose_args.sh index 477627d300be..d40f2c2f7944 100644 --- a/integration-tests/script/docker_compose_args.sh +++ b/integration-tests/script/docker_compose_args.sh @@ -29,7 +29,7 @@ getComposeArgs() if [ "$DRUID_INTEGRATION_TEST_INDEXER" = "indexer" ] then # Sanity check: cannot combine CliIndexer tests with security, query-retry tests - if [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "high-availability" ] + if [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] then echo "Cannot run test group '$DRUID_INTEGRATION_TEST_GROUP' with CliIndexer" exit 1 @@ -59,10 +59,6 @@ getComposeArgs() # default + additional historical modified for query error test # See CliHistoricalForQueryRetryTest. echo "-f ${DOCKERDIR}/docker-compose.query-error-test.yml" - elif [ "$DRUID_INTEGRATION_TEST_GROUP" = "high-availability" ] - then - # the 'high availability' test cluster with multiple coordinators and overlords - echo "-f ${DOCKERDIR}/docker-compose.high-availability.yml" elif [ "$DRUID_INTEGRATION_TEST_GROUP" = "kafka-data-format" ] then # default + schema registry container diff --git a/integration-tests/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator b/integration-tests/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator index 4953dc27b7ee..b2467a1be067 100644 --- a/integration-tests/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator +++ b/integration-tests/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator @@ -14,4 +14,3 @@ # limitations under the License. org.apache.druid.cli.QueryRetryTestCommandCreator -org.apache.druid.cli.CustomNodeRoleCommandCreator 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 601754a27007..b27feebbfbd6 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 @@ -144,8 +144,6 @@ public class TestNGGroup */ public static final String KINESIS_DATA_FORMAT = "kinesis-data-format"; - public static final String HIGH_AVAILABILTY = "high-availability"; - public static final String CUSTOM_COORDINATOR_DUTIES = "custom-coordinator-duties"; public static final String CENTRALIZED_DATASOURCE_SCHEMA = "centralized-datasource-schema"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java b/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java deleted file mode 100644 index eb7f9f54dfee..000000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java +++ /dev/null @@ -1,311 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.tests.leadership; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.inject.Inject; -import org.apache.druid.cli.CliCustomNodeRole; -import org.apache.druid.curator.discovery.ServerDiscoveryFactory; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.DruidNodeDiscovery; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeRole; -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.clients.CoordinatorResourceTestClient; -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.apache.druid.testing.utils.SqlTestQueryHelper; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractIndexerTest; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.testng.Assert; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.net.MalformedURLException; -import java.net.URL; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - -@Test(groups = TestNGGroup.HIGH_AVAILABILTY) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITHighAvailabilityTest -{ - private static final Logger LOG = new Logger(ITHighAvailabilityTest.class); - private static final String SYSTEM_QUERIES_RESOURCE = "/queries/high_availability_sys.json"; - private static final int NUM_LEADERSHIP_SWAPS = 3; - - private static final int NUM_RETRIES = 120; - private static final long RETRY_DELAY = TimeUnit.SECONDS.toMillis(5); - - @Inject - private IntegrationTestingConfig config; - - @Inject - private DruidClusterAdminClient druidClusterAdminClient; - - @Inject - ServerDiscoveryFactory factory; - - @Inject - DruidNodeDiscoveryProvider druidNodeDiscovery; - - @Inject - CoordinatorResourceTestClient coordinatorClient; - - @Inject - SqlTestQueryHelper queryHelper; - - @Inject - ObjectMapper jsonMapper; - - @Inject - @TestClient - HttpClient httpClient; - - @Test - public void testLeadershipChanges() throws Exception - { - int runCount = 0; - String previousCoordinatorLeader = null; - String previousOverlordLeader = null; - // fetch current leaders, make sure queries work, then swap leaders and do it again - do { - String coordinatorLeader = getLeader("coordinator"); - String overlordLeader = getLeader("indexer"); - - // we expect leadership swap to happen - Assert.assertNotEquals(previousCoordinatorLeader, coordinatorLeader); - Assert.assertNotEquals(previousOverlordLeader, overlordLeader); - - previousCoordinatorLeader = coordinatorLeader; - previousOverlordLeader = overlordLeader; - - String queries = fillTemplate( - config, - AbstractIndexerTest.getResourceAsString(SYSTEM_QUERIES_RESOURCE), - overlordLeader, - coordinatorLeader - ); - queryHelper.testQueriesFromString(queries); - - swapLeadersAndWait(coordinatorLeader, overlordLeader); - } while (runCount++ < NUM_LEADERSHIP_SWAPS); - } - - @Test - public void testDiscoveryAndSelfDiscovery() - { - ITRetryUtil.retryUntil( - () -> { - try { - List disco = ImmutableList.of( - druidNodeDiscovery.getForNodeRole(NodeRole.COORDINATOR), - druidNodeDiscovery.getForNodeRole(NodeRole.OVERLORD), - druidNodeDiscovery.getForNodeRole(NodeRole.HISTORICAL), - druidNodeDiscovery.getForNodeRole(NodeRole.MIDDLE_MANAGER), - druidNodeDiscovery.getForNodeRole(NodeRole.INDEXER), - druidNodeDiscovery.getForNodeRole(NodeRole.BROKER), - druidNodeDiscovery.getForNodeRole(NodeRole.ROUTER) - ); - - int servicesDiscovered = 0; - for (DruidNodeDiscovery nodeRole : disco) { - Collection nodes = nodeRole.getAllNodes(); - servicesDiscovered += testSelfDiscovery(nodes); - } - return servicesDiscovered > 5; - } - catch (Throwable t) { - return false; - } - }, - true, - RETRY_DELAY, - NUM_RETRIES, - "Standard services discovered" - ); - } - - @Test - public void testCustomDiscovery() - { - ITRetryUtil.retryUntil( - () -> { - try { - DruidNodeDiscovery customDisco = - druidNodeDiscovery.getForNodeRole(new NodeRole(CliCustomNodeRole.SERVICE_NAME)); - int count = testSelfDiscovery(customDisco.getAllNodes()); - return count > 0; - } - catch (Throwable t) { - return false; - } - }, - true, - RETRY_DELAY, - NUM_RETRIES, - "Custom service discovered" - ); - } - - private int testSelfDiscovery(Collection nodes) - throws MalformedURLException, ExecutionException, InterruptedException - { - int count = 0; - - for (DiscoveryDruidNode node : nodes) { - final String location = StringUtils.format( - "http://%s:%s/status/selfDiscovered", - config.isDocker() ? config.getDockerHost() : node.getDruidNode().getHost(), - node.getDruidNode().getPlaintextPort() - ); - LOG.info("testing self discovery %s", location); - StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.GET, new URL(location)), - StatusResponseHandler.getInstance() - ).get(); - LOG.info("%s responded with %s", location, response.getStatus().getCode()); - Assert.assertEquals(response.getStatus(), HttpResponseStatus.OK); - count++; - } - return count; - } - - private void swapLeadersAndWait(String coordinatorLeader, String overlordLeader) - { - Runnable waitUntilCoordinatorSupplier; - if (isCoordinatorOneLeader(config, coordinatorLeader)) { - druidClusterAdminClient.restartCoordinatorContainer(); - waitUntilCoordinatorSupplier = () -> druidClusterAdminClient.waitUntilCoordinatorReady(); - } else { - druidClusterAdminClient.restartCoordinatorTwoContainer(); - waitUntilCoordinatorSupplier = () -> druidClusterAdminClient.waitUntilCoordinatorTwoReady(); - } - - Runnable waitUntilOverlordSupplier; - if (isOverlordOneLeader(config, overlordLeader)) { - druidClusterAdminClient.restartOverlordContainer(); - waitUntilOverlordSupplier = () -> druidClusterAdminClient.waitUntilIndexerReady(); - } else { - druidClusterAdminClient.restartOverlordTwoContainer(); - waitUntilOverlordSupplier = () -> druidClusterAdminClient.waitUntilOverlordTwoReady(); - } - waitUntilCoordinatorSupplier.run(); - waitUntilOverlordSupplier.run(); - } - - private String getLeader(String service) - { - try { - StatusResponseHolder response = httpClient.go( - new Request( - HttpMethod.GET, - new URL(StringUtils.format( - "%s/druid/%s/v1/leader", - config.getRouterUrl(), - service - )) - ), - StatusResponseHandler.getInstance() - ).get(); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while fetching leader from[%s] status[%s] content[%s]", - config.getRouterUrl(), - response.getStatus(), - response.getContent() - ); - } - return response.getContent(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private static String fillTemplate(IntegrationTestingConfig config, String template, String overlordLeader, String coordinatorLeader) - { - /* - {"host":"%%BROKER%%","server_type":"broker", "is_leader": null}, - {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, - {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, - {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, - {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, - {"host":"%%ROUTER%%","server_type":"router", "is_leader": null} - */ - String working = template; - - working = StringUtils.replace(working, "%%OVERLORD_ONE%%", config.getOverlordInternalHost()); - working = StringUtils.replace(working, "%%OVERLORD_TWO%%", config.getOverlordTwoInternalHost()); - working = StringUtils.replace(working, "%%COORDINATOR_ONE%%", config.getCoordinatorInternalHost()); - working = StringUtils.replace(working, "%%COORDINATOR_TWO%%", config.getCoordinatorTwoInternalHost()); - working = StringUtils.replace(working, "%%BROKER%%", config.getBrokerInternalHost()); - working = StringUtils.replace(working, "%%ROUTER%%", config.getRouterInternalHost()); - if (isOverlordOneLeader(config, overlordLeader)) { - working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "1"); - working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "0"); - } else { - working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "0"); - working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "1"); - } - if (isCoordinatorOneLeader(config, coordinatorLeader)) { - working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "1"); - working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "0"); - } else { - working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "0"); - working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "1"); - } - return working; - } - - private static boolean isCoordinatorOneLeader(IntegrationTestingConfig config, String coordinatorLeader) - { - return coordinatorLeader.contains(transformHost(config.getCoordinatorInternalHost())); - } - - private static boolean isOverlordOneLeader(IntegrationTestingConfig config, String overlordLeader) - { - return overlordLeader.contains(transformHost(config.getOverlordInternalHost())); - } - - /** - * host + ':' which should be enough to distinguish subsets, e.g. 'druid-coordinator:8081' from - * 'druid-coordinator-two:8081' for example - */ - private static String transformHost(String host) - { - return StringUtils.format("%s:", host); - } -} diff --git a/integration-tests/src/test/resources/queries/high_availability_sys.json b/integration-tests/src/test/resources/queries/high_availability_sys.json deleted file mode 100644 index c506352a4221..000000000000 --- a/integration-tests/src/test/resources/queries/high_availability_sys.json +++ /dev/null @@ -1,39 +0,0 @@ -[ - { - "description": "query sys.servers to make sure all expected servers are available", - "query": { - "query": "SELECT host, server_type, is_leader FROM sys.servers ORDER BY host" - }, - "expectedResults": [ - {"host":"%%BROKER%%","server_type":"broker", "is_leader": null}, - {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, - {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, - {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, - {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, - {"host":"%%ROUTER%%","server_type":"router", "is_leader": null} - ] - }, - { - "description": "query sys.segments which is fed via coordinator data", - "query": { - "query": "SELECT datasource, count(*) FROM sys.segments WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " - }, - "expectedResults": [ - { - "datasource": "wikipedia_editstream", - "EXPR$1": 1 - }, - { - "datasource": "twitterstream", - "EXPR$1": 3 - } - ] - }, - { - "description": "query sys.tasks which is fed via overlord", - "query": { - "query": "SELECT datasource, count(*) FROM sys.tasks WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " - }, - "expectedResults": [] - } -] \ No newline at end of file diff --git a/licenses.yaml b/licenses.yaml index ffe0da59725a..777e13552617 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -5101,6 +5101,62 @@ libraries: --- +name: Testcontainers +license_category: binary +module: extensions-core/druid-testcontainers +license_name: MIT License +version: 1.21.3 +libraries: + - org.testcontainers: testcontainers + +--- + +name: Junit +license_category: binary +module: extensions-core/druid-testcontainers +license_name: Eclipse Public License 1.0 +version: 4.13.2 +license_file_path: licenses/bin/junit.EPL +libraries: + - junit: junit + +--- + +# Dependency of Testcontainers +name: Docker Java +license_category: binary +module: extensions-core/druid-testcontainers +license_name: Apache License version 2.0 +version: 3.3.4 +libraries: + - com.github.docker-java: docker-java-api + - com.github.docker-java: docker-java-transport + - com.github.docker-java: docker-java-transport-zerodep + +--- + +# Dependency of Testcontainers +name: Duct Tape +license_category: binary +module: extensions-core/druid-testcontainers +license_name: MIT License +version: 1.0.8 +libraries: + - org.rnorth.duct-tape: duct-tape + +--- + +# Dependency of Testcontainers +name: jakarta.activation +license_category: binary +module: extensions-core/druid-testcontainers +license_name: Eclipse Distribution License 1.0 +version: 2.0.1 +libraries: + - com.sun.activation: jakarta.activation + +--- + # Web console modules start name: "@babel/code-frame" license_category: binary diff --git a/pom.xml b/pom.xml index 214dadd4bd1e..70e2d89a2359 100644 --- a/pom.xml +++ b/pom.xml @@ -227,6 +227,7 @@ extensions-core/google-extensions extensions-core/druid-catalog extensions-core/testing-tools + extensions-core/druid-testcontainers extensions-contrib/compressed-bigdecimal extensions-contrib/influx-extensions diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Event.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Event.java index 1f5a20c6188f..2c37c25dd79f 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Event.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Event.java @@ -23,6 +23,13 @@ */ public interface Event { + String TIMESTAMP = "timestamp"; + String FEED = "feed"; + String SERVICE = "service"; + String HOST = "host"; + String METRIC = "metric"; + String VALUE = "value"; + EventMap toMap(); String getFeed(); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/service/AlertEvent.java b/processing/src/main/java/org/apache/druid/java/util/emitter/service/AlertEvent.java index 0c960a86860f..2a5fa9657df6 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/service/AlertEvent.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/service/AlertEvent.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.InvalidInput; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.core.Event; @@ -66,7 +67,7 @@ public AlertEvent( Map dataMap ) { - this(createdTime, ImmutableMap.of("service", service, "host", host), severity, description, dataMap); + this(createdTime, ImmutableMap.of(SERVICE, service, HOST, host), severity, description, dataMap); } public AlertEvent( @@ -116,7 +117,7 @@ public String getFeed() @SuppressWarnings("unused") public String getService() { - return serviceDimensions.get("service"); + return serviceDimensions.get(SERVICE); } /* @@ -125,7 +126,7 @@ public String getService() @SuppressWarnings("unused") public String getHost() { - return serviceDimensions.get("host"); + return serviceDimensions.get(HOST); } public Severity getSeverity() @@ -153,8 +154,8 @@ public EventMap toMap() { return EventMap .builder() - .put("feed", getFeed()) - .put("timestamp", createdTime.toString()) + .put(FEED, getFeed()) + .put(TIMESTAMP, createdTime.toString()) .putAll(serviceDimensions) .put("severity", severity.toString()) .put("description", description) @@ -164,37 +165,34 @@ public EventMap toMap() public enum Severity { - ANOMALY { - @Override - public String toString() - { - return "anomaly"; - } - }, + ANOMALY("anomaly"), + COMPONENT_FAILURE("component-failure"), + SERVICE_FAILURE("service-failure"), + DEPRECATED("deprecated"); - COMPONENT_FAILURE { - @Override - public String toString() - { - return "component-failure"; - } - }, + public static final Severity DEFAULT = COMPONENT_FAILURE; - SERVICE_FAILURE { - @Override - public String toString() - { - return "service-failure"; - } - }, - DEPRECATED { - @Override - public String toString() - { - return "deprecated"; - } - }; + private final String name; - public static final Severity DEFAULT = COMPONENT_FAILURE; + Severity(String name) + { + this.name = name; + } + + public static Severity fromString(String value) + { + for (Severity severity : Severity.values()) { + if (severity.toString().equals(value)) { + return severity; + } + } + throw InvalidInput.exception("No such severity[%s]", value); + } + + @Override + public String toString() + { + return name; + } } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java index 7e249f72d0a6..ed6a7ff3c0f4 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java @@ -30,7 +30,6 @@ */ public class SegmentMetadataEvent implements Event { - public static final String FEED = "feed"; public static final String DATASOURCE = "dataSource"; public static final String CREATED_TIME = "createdTime"; public static final String START_TIME = "startTime"; diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/service/ServiceMetricEvent.java b/processing/src/main/java/org/apache/druid/java/util/emitter/service/ServiceMetricEvent.java index 6644886497df..41395ffea58d 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/service/ServiceMetricEvent.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/service/ServiceMetricEvent.java @@ -85,12 +85,12 @@ public String getFeed() public String getService() { - return serviceDims.get("service"); + return serviceDims.get(SERVICE); } public String getHost() { - return serviceDims.get("host"); + return serviceDims.get(HOST); } public Map getUserDims() @@ -114,11 +114,11 @@ public EventMap toMap() { return EventMap .builder() - .put("feed", getFeed()) - .put("timestamp", createdTime.toString()) + .put(FEED, getFeed()) + .put(TIMESTAMP, createdTime.toString()) .putAll(serviceDims) - .put("metric", metric) - .put("value", value) + .put(METRIC, metric) + .put(VALUE, value) .putAll(userDims) .build(); } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/service/AlertEventTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/service/AlertEventTest.java index 68bebb61f02b..28367b0301b6 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/service/AlertEventTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/service/AlertEventTest.java @@ -19,9 +19,9 @@ package org.apache.druid.java.util.emitter.service; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.service.AlertEvent.Severity; import org.junit.Assert; import org.junit.Test; @@ -167,15 +167,21 @@ public void testDefaulting() } } + @Test + public void test_deserializeSeverityFromString() + { + Assert.assertEquals(AlertEvent.Severity.ANOMALY, AlertEvent.Severity.fromString("anomaly")); + Assert.assertEquals(AlertEvent.Severity.DEPRECATED, AlertEvent.Severity.fromString("deprecated")); + Assert.assertEquals(AlertEvent.Severity.COMPONENT_FAILURE, AlertEvent.Severity.fromString("component-failure")); + Assert.assertEquals(AlertEvent.Severity.SERVICE_FAILURE, AlertEvent.Severity.fromString("service-failure")); + Assert.assertThrows( + DruidException.class, + () -> AlertEvent.Severity.fromString("unknown") + ); + } + public Map contents(AlertEvent a) { - return Maps.filterKeys(a.toMap(), new Predicate<>() - { - @Override - public boolean apply(String k) - { - return !"timestamp".equals(k); - } - }); + return Maps.filterKeys(a.toMap(), k -> !"timestamp".equals(k)); } } diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java index ef78930f65a9..bd0b486f6dac 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java @@ -235,6 +235,8 @@ private WaitCondition(Predicate predicate) */ public static class EventMatcher implements Predicate { + private String host; + private String service; private String metricName; private Long metricValue; private final Map dimensions = new HashMap<>(); @@ -251,9 +253,10 @@ public EventMatcher hasMetricName(String metricName) } /** - * Matches an event only if it has the given metric value. + * Matches an event only if it has a metric value equal to or greater than + * the given value. */ - public EventMatcher hasValue(long metricValue) + public EventMatcher hasValueAtLeast(long metricValue) { this.metricValue = metricValue; return this; @@ -268,12 +271,34 @@ public EventMatcher hasDimension(String dimension, Object value) return this; } + /** + * Matches an event only if it has the given service name. + */ + public EventMatcher hasService(String service) + { + this.service = service; + return this; + } + + /** + * Matches an event only if it has the given host. + */ + public EventMatcher hasHost(String host) + { + this.host = host; + return this; + } + @Override public boolean test(ServiceMetricEvent event) { if (metricName != null && !event.getMetric().equals(metricName)) { return false; - } else if (metricValue != null && event.getValue().longValue() != metricValue) { + } else if (metricValue != null && event.getValue().longValue() < metricValue) { + return false; + } else if (service != null && !service.equals(event.getService())) { + return false; + } else if (host != null && !host.equals(event.getHost())) { return false; } diff --git a/services/pom.xml b/services/pom.xml index 7fbdc3ac952b..8a5a7938c6f7 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -267,6 +267,11 @@ test-jar test + + org.apache.httpcomponents + httpclient + test + junit junit diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedBroker.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedBroker.java index 6516afae8dad..3ee761199b0a 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedBroker.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedBroker.java @@ -37,7 +37,7 @@ public class EmbeddedBroker extends EmbeddedDruidServer { @Override - ServerRunnable createRunnable(LifecycleInitHandler handler) + protected ServerRunnable createRunnable(LifecycleInitHandler handler) { return new Broker(handler); } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 878cc055ffdb..ce1e613b34d8 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -43,6 +43,7 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; +import org.apache.druid.server.metrics.LatchableEmitter; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -194,7 +195,7 @@ public void submitTask(Task task) */ public void waitForTaskToSucceed(String taskId, EmbeddedOverlord overlord) { - TaskStatus taskStatus = waitForTaskToFinish(taskId, overlord); + TaskStatus taskStatus = waitForTaskToFinish(taskId, overlord.latchableEmitter()); Assertions.assertEquals( TaskState.SUCCESS, taskStatus.getStatusCode(), @@ -203,13 +204,26 @@ public void waitForTaskToSucceed(String taskId, EmbeddedOverlord overlord) } /** - * Waits for the given task to finish (either successfully or unsuccessfully). If the given - * {@link EmbeddedOverlord} is not the leader, this method can only return by - * throwing an exception upon timeout. + * Waits for the given task to finish successfully. + * If the given {@link LatchableEmitter} does not receive the task completion + * event, this method can only return by throwing an exception upon timeout. + */ + public void waitForTaskToSucceed(String taskId, LatchableEmitter emitter) + { + Assertions.assertEquals( + TaskState.SUCCESS, + waitForTaskToFinish(taskId, emitter).getStatusCode() + ); + } + + /** + * Waits for the given task to finish (either successfully or unsuccessfully). + * If the given {@link LatchableEmitter} does not receive the task completion + * event, this method can only return by throwing an exception upon timeout. */ - public TaskStatus waitForTaskToFinish(String taskId, EmbeddedOverlord overlord) + public TaskStatus waitForTaskToFinish(String taskId, LatchableEmitter emitter) { - overlord.latchableEmitter().waitForEvent( + emitter.waitForEvent( event -> event.hasMetricName(TaskMetrics.RUN_DURATION) .hasDimension(DruidMetrics.TASK_ID, taskId) ); @@ -249,7 +263,7 @@ public List getSortedSegmentIntervals(String dataSource, EmbeddedOverl */ public void verifyNumVisibleSegmentsIs(int numExpectedSegments, String dataSource, EmbeddedOverlord overlord) { - int segmentCount = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).size(); + int segmentCount = getVisibleUsedSegments(dataSource, overlord).size(); Assertions.assertEquals( numExpectedSegments, segmentCount, @@ -257,7 +271,7 @@ public void verifyNumVisibleSegmentsIs(int numExpectedSegments, String dataSourc ); Assertions.assertEquals( String.valueOf(segmentCount), - cluster.runSql( + runSql( "SELECT COUNT(*) FROM sys.segments WHERE datasource='%s'" + " AND is_overshadowed = 0 AND is_available = 1", dataSource diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedCoordinator.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedCoordinator.java index bda265f98860..16c087f27e1c 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedCoordinator.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedCoordinator.java @@ -43,7 +43,7 @@ public EmbeddedCoordinator() } @Override - ServerRunnable createRunnable(LifecycleInitHandler handler) + protected ServerRunnable createRunnable(LifecycleInitHandler handler) { return new Coordinator(handler); } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidCluster.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidCluster.java index dabceaff4b98..ee6885f030b9 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidCluster.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidCluster.java @@ -80,8 +80,8 @@ public class EmbeddedDruidCluster implements EmbeddedResource private final List> extensionModules = new ArrayList<>(); private final Properties commonProperties = new Properties(); + private EmbeddedHostname embeddedHostname = EmbeddedHostname.localhost(); private boolean startedFirstDruidServer = false; - private EmbeddedZookeeper zookeeper; private EmbeddedDruidCluster() { @@ -98,9 +98,7 @@ private EmbeddedDruidCluster() */ public static EmbeddedDruidCluster withZookeeper() { - final EmbeddedDruidCluster cluster = new EmbeddedDruidCluster(); - cluster.addEmbeddedZookeeper(); - return cluster; + return new EmbeddedDruidCluster().addResource(new EmbeddedZookeeper()); } /** @@ -129,12 +127,6 @@ public static EmbeddedDruidCluster empty() return new EmbeddedDruidCluster(); } - private void addEmbeddedZookeeper() - { - this.zookeeper = new EmbeddedZookeeper(); - resources.add(zookeeper); - } - /** * Configures this cluster to use a {@link LatchableEmitter}. This method is a * shorthand for the following: @@ -184,7 +176,6 @@ public final EmbeddedDruidCluster addExtensions(Class... */ public EmbeddedDruidCluster addServer(EmbeddedDruidServer server) { - server.onAddedToCluster(commonProperties); servers.add(server); resources.add(server); if (startedFirstDruidServer) { @@ -219,6 +210,11 @@ public EmbeddedDruidCluster addCommonProperty(String key, String value) return this; } + public Properties getCommonProperties() + { + return commonProperties; + } + /** * The test directory used by this cluster. Each Druid service creates a * sub-folder inside this directory to write out task logs or segments. @@ -229,13 +225,21 @@ public TestFolder getTestFolder() } /** - * The embedded Zookeeper server used by this cluster, if any. - * - * @throws NullPointerException if this cluster has no embedded zookeeper. + * Uses a container-friendly hostname for all embedded services, Druid as well + * as external. + */ + public EmbeddedDruidCluster useContainerFriendlyHostname() + { + this.embeddedHostname = EmbeddedHostname.containerFriendly(); + return this; + } + + /** + * Hostname to be used for embedded services (both Druid or external). */ - public EmbeddedZookeeper getZookeeper() + public EmbeddedHostname getEmbeddedHostname() { - return Objects.requireNonNull(zookeeper, "No embedded zookeeper configured for this cluster"); + return embeddedHostname; } /** diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidServer.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidServer.java index cdc7247b17f8..c3bdac04569a 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidServer.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedDruidServer.java @@ -63,7 +63,7 @@ public abstract class EmbeddedDruidServer> impl private final List beforeStartHooks = new ArrayList<>(); private final ServerReferenceHolder referenceHolder = new ServerReferenceHolder(); - EmbeddedDruidServer() + protected EmbeddedDruidServer() { this.name = StringUtils.format( "%s-%d", @@ -75,7 +75,7 @@ public abstract class EmbeddedDruidServer> impl // Add properties for temporary directories used by the servers final String logsDirectory = cluster.getTestFolder().getOrCreateFolder("indexer-logs").getAbsolutePath(); final String taskDirectory = cluster.getTestFolder().newFolder().getAbsolutePath(); - final String storageDirectory = cluster.getTestFolder().newFolder().getAbsolutePath(); + final String storageDirectory = cluster.getTestFolder().getOrCreateFolder("deep-store").getAbsolutePath(); log.info( "Server[%s] using directories: task directory[%s], logs directory[%s], storage directory[%s].", self.getName(), @@ -83,16 +83,13 @@ public abstract class EmbeddedDruidServer> impl logsDirectory, storageDirectory ); - self.addProperty("druid.host", "localhost"); + + self.addProperty("druid.extensions.loadList", "[]"); + self.addProperty("druid.host", cluster.getEmbeddedHostname().toString()); self.addProperty("druid.indexer.task.baseDir", taskDirectory); self.addProperty("druid.indexer.logs.directory", logsDirectory); self.addProperty("druid.storage.storageDirectory", storageDirectory); - // Add properties for Zookeeper - if (cluster.getZookeeper() != null) { - self.addProperty("druid.zk.service.host", cluster.getZookeeper().getConnectString()); - } - // Add properties for RuntimeInfoModule self.addProperty(RuntimeInfoModule.SERVER_MEMORY_PROPERTY, String.valueOf(serverMemory)); self.addProperty(RuntimeInfoModule.SERVER_DIRECT_MEMORY_PROPERTY, String.valueOf(serverDirectMemory)); @@ -125,6 +122,7 @@ public void stop() throws Exception @Override public void beforeStart(EmbeddedDruidCluster cluster) { + initServerLifecycle(cluster.getCommonProperties()); for (BeforeStart hook : beforeStartHooks) { hook.run(cluster, this); } @@ -153,7 +151,7 @@ public final T addProperty(String key, String value) /** * Adds a {@link BeforeStart} to run as part of {@link #beforeStart(EmbeddedDruidCluster)} */ - @SuppressWarnings("UnusedReturnValue") + @SuppressWarnings({"UnusedReturnValue", "unchecked"}) public final T addBeforeStartHook(BeforeStart hook) { beforeStartHooks.add(hook); @@ -182,9 +180,11 @@ public final EmbeddedDruidServer setServerDirectMemory(long serverDirectMemor * Called from {@link EmbeddedDruidCluster#addServer(EmbeddedDruidServer)} to * tie the lifecycle of this server to the cluster. */ - final void onAddedToCluster(Properties commonProperties) + private void initServerLifecycle(Properties commonProperties) { - this.lifecycle.set(new EmbeddedServerLifecycle(this, commonProperties)); + if (lifecycle.get() == null) { + lifecycle.set(new EmbeddedServerLifecycle(this, commonProperties)); + } } /** @@ -199,7 +199,7 @@ final void onAddedToCluster(Properties commonProperties) * @see EmbeddedDruidCluster#addCommonProperty * @see EmbeddedDruidServer#addProperty */ - abstract ServerRunnable createRunnable( + protected abstract ServerRunnable createRunnable( LifecycleInitHandler handler ); @@ -220,7 +220,7 @@ final Properties getStartupProperties() * All implementations of {@link EmbeddedDruidServer} must use this binding in * {@link ServerRunnable#getModules()}. */ - final void bindReferenceHolder(Binder binder) + protected final void bindReferenceHolder(Binder binder) { binder.bind(ServerReferenceHolder.class).toInstance(referenceHolder); } @@ -255,7 +255,7 @@ public String toString() /** * Handler used to register the lifecycle of an embedded server. */ - interface LifecycleInitHandler + protected interface LifecycleInitHandler { /** * Registers the lifecycle of this server so that it can be stopped later. diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHistorical.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHistorical.java index 755c8833858d..b9df19acca12 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHistorical.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHistorical.java @@ -51,7 +51,7 @@ public EmbeddedHistorical() } @Override - ServerRunnable createRunnable(LifecycleInitHandler handler) + protected ServerRunnable createRunnable(LifecycleInitHandler handler) { return new Historical(handler); } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHostname.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHostname.java new file mode 100644 index 000000000000..0b4a89955db1 --- /dev/null +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedHostname.java @@ -0,0 +1,115 @@ +/* + * 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.embedded; + +import com.google.common.net.HostAndPort; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.http.client.utils.URIBuilder; + +import java.net.InetAddress; +import java.net.URISyntaxException; +import java.net.UnknownHostException; + +/** + * Hostname to be used by embedded services, both Druid and external. + * The default value is {@code localhost}. + *

+ * When running Druid containers, {@code localhost} cannot be used to connect to + * services since {@code localhost} refers to the container and not the host + * machine. + */ +public class EmbeddedHostname +{ + private static final EmbeddedHostname LOCALHOST = new EmbeddedHostname("localhost"); + + private final String hostname; + + private EmbeddedHostname(String hostname) + { + this.hostname = hostname; + } + + public static EmbeddedHostname localhost() + { + return LOCALHOST; + } + + /** + * Hostname for the host machine running the containers. When a service uses + * this hostname instead of {@link #localhost}, it is reachable by Druid + * containers and EmbeddedDruidServers alike. + */ + public static EmbeddedHostname containerFriendly() + { + try { + return new EmbeddedHostname(InetAddress.getLocalHost().getHostAddress()); + } + catch (UnknownHostException e) { + throw new ISE(e, "Unable to determine host name"); + } + } + + /** + * Replaces {@code localhost} or {@code 127.0.0.1} in the given connectUri + * with {@link #hostname}. + * + * @param connectUri Syntactically valid connect URI, complete with a scheme, host and port. + */ + public String useInUri(String connectUri) + { + try { + final URIBuilder uri = new URIBuilder(connectUri); + validateLocalhost(uri.getHost(), connectUri); + uri.setHost(hostname); + return uri.build().toString(); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + /** + * Replaces {@code localhost} or {@code 127.0.0.1} in the given hostAndPort + * with {@link #hostname}. + */ + public String useInHostAndPort(String hostAndPort) + { + final HostAndPort parsedHostAndPort = HostAndPort.fromString(hostAndPort); + validateLocalhost(parsedHostAndPort.getHost(), hostAndPort); + return HostAndPort.fromParts(hostname, parsedHostAndPort.getPort()).toString(); + } + + private static void validateLocalhost(String host, String connectUri) + { + if (!"localhost".equals(host) && !"127.0.0.1".equals(host)) { + throw new IAE( + "Connect URI[%s] must contain 'localhost' or '127.0.0.1' to be reachable.", + connectUri + ); + } + } + + @Override + public String toString() + { + return hostname; + } +} diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedIndexer.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedIndexer.java index 0f8a7e9d6c0e..16ec4f53809a 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedIndexer.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedIndexer.java @@ -42,7 +42,7 @@ public EmbeddedIndexer() } @Override - ServerRunnable createRunnable(LifecycleInitHandler handler) + protected ServerRunnable createRunnable(LifecycleInitHandler handler) { return new Indexer(handler); } diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedMiddleManager.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedMiddleManager.java index 898bddb1d478..2f5b1404c199 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedMiddleManager.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedMiddleManager.java @@ -42,7 +42,7 @@ public class EmbeddedMiddleManager extends EmbeddedDruidServer