props = resolved.toProperties();
+ // Added from ZK section
+ assertEquals("localhost:2181", props.get("druid.zk.service.zkHosts"));
+ // Generic property
+ assertEquals("howdy", props.get("my.test.property"));
+ // Mapped from settings
+ assertEquals("myBucket", props.get("druid.test.config.cloudBucket"));
+ assertEquals("myPath", props.get("druid.test.config.cloudPath"));
+ assertEquals("secret", props.get("druid.test.config.s3AccessKey"));
+ // From settings, overridden in properties
+ assertEquals("myRegion", props.get("druid.test.config.cloudRegion"));
+ // System property
+ assertEquals("sys", props.get("druid.test.config.sys_prop"));
+ // From user override
+ assertEquals("user", props.get("druid.test.config.user_var"));
+
+ // Test plumbing through the test config
+ Properties properties = new Properties();
+ properties.putAll(props);
+ IntegrationTestingConfig testingConfig = new IntegrationTestingConfigEx(resolved, properties);
+ assertEquals("myBucket", testingConfig.getCloudBucket());
+ assertEquals("myPath", testingConfig.getCloudPath());
+ // From settings, overridden in properties
+ assertEquals("myRegion", testingConfig.getCloudRegion());
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java
new file mode 100644
index 00000000000..99e8b7e3cbb
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java
@@ -0,0 +1,42 @@
+/*
+ * 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.config;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Indicates a method to use to configure the {@link Initializer.Builder}. Must be of the
+ * form:
+ *
+ * @Configure
+ * public static void configure(Initializer.Builder builder)
+ * {
+ * builder.something(arg);
+ * }
+ *
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface Configure
+{
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
new file mode 100644
index 00000000000..5e65ec7f9d4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
@@ -0,0 +1,208 @@
+/*
+ * 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.config;
+
+import junitparams.JUnitParamsRunner;
+import org.apache.druid.java.util.common.UOE;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.Statement;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+
+/**
+ * Test runner for Druid integration tests. Loads test configuration. Initializes
+ * Guice. Injects dependencies into the test. Runs the tests.
+ * Shuts down afterwards. Since cluster configuration and health checks are a bit
+ * expensive, does that work once per test class rather than once per test method.
+ *
+ * Note that our Guice usage is a bit awkward for tests. Druid modules define
+ * objects that must be lifecycle managed, but as lazy singletons, which means
+ * that they might be created after the lifecycle starts, which causes the dreaded
+ * "it doesn't work that way" message. The awkward workaround is to ask to inject
+ * test members before starting the lifecycle, so that the injection creates
+ * a reference, which creates the object, which registers it in the lifecycle. We
+ * should fix this issue. Until then, the awkwardness is hidden in this test runner.
+ *
+ * Extends the parameterize test runner, so your Druid ITs can also use parameters.
+ */
+public class DruidTestRunner extends JUnitParamsRunner
+{
+ private class CloseInitializer extends Statement
+ {
+ private final Statement next;
+
+ public CloseInitializer(Statement next)
+ {
+ this.next = next;
+ }
+
+ @Override
+ public void evaluate() throws Throwable
+ {
+ next.evaluate();
+ if (initializer != null) {
+ initializer.close();
+ initializer = null;
+ }
+ }
+ }
+
+ private Initializer initializer;
+
+ public DruidTestRunner(Class> testClass) throws InitializationError
+ {
+ super(testClass);
+ }
+
+ @Override
+ protected Object createTest() throws Exception
+ {
+ Object test = super.createTest();
+ if (initializer == null) {
+ initializer = buildInitializer(test);
+ } else {
+ initializer.injector().injectMembers(test);
+ }
+ return test;
+ }
+
+ private Initializer buildInitializer(Object test)
+ {
+ Class> testClass = test.getClass();
+ Category[] annotations = testClass.getAnnotationsByType(Category.class);
+ if (annotations.length == 0) {
+ throw new UOE(
+ "Class % must have a @Category annotation",
+ testClass.getSimpleName()
+ );
+ }
+ if (annotations.length != 1) {
+ throw new UOE(
+ "Class % must have exactly one @Category annotation",
+ testClass.getSimpleName()
+ );
+ }
+ Class>[] categories = annotations[0].value();
+ if (categories.length == 0) {
+ throw new UOE(
+ "Class % must have a @Category value",
+ testClass.getSimpleName()
+ );
+ }
+ if (categories.length != 1) {
+ throw new UOE(
+ "Class % must have exactly one @Category value",
+ testClass.getSimpleName()
+ );
+ }
+ Class> category = category(testClass);
+ String clusterName = inferCluster(category);
+ Initializer.Builder builder = Initializer.builder(clusterName)
+ .test(test)
+ .validateCluster();
+ for (Method method : testClass.getMethods()) {
+ if (method.getAnnotation(Configure.class) == null) {
+ continue;
+ }
+ final int requiredMods = Modifier.STATIC | Modifier.PUBLIC;
+ if ((method.getModifiers() & requiredMods) != requiredMods) {
+ throw new UOE(
+ "Method %s annotated with @Configure must be public static",
+ method.getName()
+ );
+ }
+ try {
+ method.invoke(null, builder);
+ }
+ catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+ throw new UOE(
+ "Call to Method %s annotated with @Configure failed: %s",
+ method.getName(),
+ e.getMessage()
+ );
+ }
+ }
+ return builder.build();
+ }
+
+ /**
+ * Resolve the {@code @Category} annotation for the test class.
+ */
+ private Class> category(Class> testClass)
+ {
+ Category[] annotations = testClass.getAnnotationsByType(Category.class);
+ if (annotations.length == 0) {
+ throw new UOE(
+ "Class % must have a @Category annotation",
+ testClass.getSimpleName()
+ );
+ }
+ if (annotations.length != 1) {
+ throw new UOE(
+ "Class % must have exactly one @Category annotation",
+ testClass.getSimpleName()
+ );
+ }
+ Class>[] categories = annotations[0].value();
+ if (categories.length == 0) {
+ throw new UOE(
+ "Class % must have a @Category value",
+ testClass.getSimpleName()
+ );
+ }
+ if (categories.length != 1) {
+ throw new UOE(
+ "Class % must have exactly one @Category value",
+ testClass.getSimpleName()
+ );
+ }
+ return categories[0];
+ }
+
+ /**
+ * Resolve the optional {@code @Cluster} annotation on the test category.
+ * If omitted, the category itself is the cluster name.
+ */
+ private String inferCluster(Class> category)
+ {
+ String categoryName = category.getSimpleName();
+ Cluster[] annotations = category.getAnnotationsByType(Cluster.class);
+ if (annotations.length == 0) {
+ return categoryName;
+ }
+ if (annotations.length != 1) {
+ throw new UOE(
+ "Category % must have no more than one @Cluster annotation",
+ category.getSimpleName()
+ );
+ }
+ Class> clusterClass = annotations[0].value();
+ return clusterClass.getSimpleName();
+ }
+
+ @Override
+ protected Statement withAfterClasses(Statement statement)
+ {
+ return new CloseInitializer(super.withAfterClasses(statement));
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java
new file mode 100644
index 00000000000..a2899a08448
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java
@@ -0,0 +1,590 @@
+/*
+ * 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.config;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Binder;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import org.apache.druid.cli.GuiceRunnable;
+import org.apache.druid.curator.CuratorModule;
+import org.apache.druid.curator.discovery.DiscoveryModule;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.guice.AnnouncerModule;
+import org.apache.druid.guice.DruidProcessingConfigModule;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.guice.PolyBind;
+import org.apache.druid.guice.SQLMetadataStorageDruidModule;
+import org.apache.druid.guice.StartupInjectorBuilder;
+import org.apache.druid.guice.StorageNodeModule;
+import org.apache.druid.guice.annotations.Client;
+import org.apache.druid.guice.annotations.EscalatedClient;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.guice.http.HttpClientModule;
+import org.apache.druid.guice.security.EscalatorModule;
+import org.apache.druid.initialization.CoreInjectorBuilder;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.jackson.DruidServiceSerializerModifier;
+import org.apache.druid.jackson.StringObjectPairList;
+import org.apache.druid.jackson.ToStringObjectPairListDeserializer;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.core.LoggingEmitter;
+import org.apache.druid.java.util.emitter.core.LoggingEmitterConfig;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.metadata.MetadataStorageConnector;
+import org.apache.druid.metadata.MetadataStorageConnectorConfig;
+import org.apache.druid.metadata.MetadataStorageProvider;
+import org.apache.druid.metadata.MetadataStorageTablesConfig;
+import org.apache.druid.metadata.NoopMetadataStorageProvider;
+import org.apache.druid.metadata.SQLMetadataConnector;
+import org.apache.druid.metadata.storage.mysql.MySQLConnector;
+import org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig;
+import org.apache.druid.metadata.storage.mysql.MySQLConnectorSslConfig;
+import org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.druid.testing.IntegrationTestingConfigProvider;
+import org.apache.druid.testing.guice.TestClient;
+import org.apache.druid.testsEx.cluster.DruidClusterClient;
+import org.apache.druid.testsEx.cluster.MetastoreClient;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+/**
+ * The magic needed to piece together enough of Druid to allow clients to
+ * run without server dependencies being pulled in. Used to set up the
+ * Guice injector used to inject members into integration tests, while
+ * reading configuration from the docker.yaml or similar test
+ * configuration file.
+ *
+ * Much of the work here deals the tedious task of assembling Druid
+ * modules, sometimes using copy/past to grab the part that a client
+ * wants (such as object deserialization) without the parts that the
+ * server needs (and which would introduce the need for unused configuration
+ * just to make dependencies work.)
+ *
+ * See the documentation for these test for the "user view" of this
+ * class and its configuration.
+ */
+public class Initializer
+{
+ public static final String TEST_CONFIG_PROPERTY = "testConfig";
+ public static final String TEST_CONFIG_VAR = "TEST_CONFIG";
+ public static final String CLUSTER_RESOURCES = "/cluster/";
+ public static final String CLUSTER_CONFIG_RESOURCE = CLUSTER_RESOURCES + "%s/%s.yaml";
+ public static final String CLUSTER_CONFIG_DEFAULT = "docker";
+ public static final String METASTORE_CONFIG_PROPERTY = "sqlConfig";
+ public static final String METASTORE_CONFIG_RESOURCE = "/metastore/%s.sql";
+ public static final String METASTORE_CONFIG_DEFAULT = "init";
+
+ private static final Logger log = new Logger(Initializer.class);
+
+ public static String queryFile(Class> category, String fileName)
+ {
+ return CLUSTER_RESOURCES + category.getSimpleName() + "/queries/" + fileName;
+ }
+
+ private static class TestModule implements DruidModule
+ {
+ ResolvedConfig config;
+
+ public TestModule(ResolvedConfig config)
+ {
+ this.config = config;
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ binder
+ .bind(ResolvedConfig.class)
+ .toInstance(config);
+ binder
+ .bind(IntegrationTestingConfig.class)
+ .to(IntegrationTestingConfigEx.class)
+ .in(LazySingleton.class);
+ binder
+ .bind(MetastoreClient.class)
+ .in(LazySingleton.class);
+
+ // Dummy DruidNode instance to make Guice happy. This instance is unused.
+ binder
+ .bind(DruidNode.class)
+ .annotatedWith(Self.class)
+ .toInstance(
+ new DruidNode("integration-tests", "localhost", false, 9191, null, null, true, false));
+
+ // Reduced form of SQLMetadataStorageDruidModule
+ String prop = SQLMetadataStorageDruidModule.PROPERTY;
+ String defaultValue = MySQLMetadataStorageModule.TYPE;
+ PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageConnector.class), defaultValue);
+ PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageProvider.class), defaultValue);
+ PolyBind.createChoiceWithDefault(binder, prop, Key.get(SQLMetadataConnector.class), defaultValue);
+
+ // Reduced form of MetadataConfigModule
+ // Not actually used here (tests don't create tables), but needed by MySQLConnector constructor
+ JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class);
+
+ // Build from properties provided in the config
+ JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class);
+ }
+
+ @Provides
+ @TestClient
+ public HttpClient getHttpClient(
+ IntegrationTestingConfig config,
+ Lifecycle lifecycle,
+ @Client HttpClient delegate
+ )
+ {
+ return delegate;
+ }
+
+ @Provides
+ @ManageLifecycle
+ public ServiceEmitter getServiceEmitter(ObjectMapper jsonMapper)
+ {
+ return new ServiceEmitter("", "", new LoggingEmitter(new LoggingEmitterConfig(), jsonMapper));
+ }
+
+ // From ServerModule to allow deserialization of DiscoveryDruidNode objects from ZK.
+ // We don't want the other dependencies of that module.
+ @Override
+ public List extends com.fasterxml.jackson.databind.Module> getJacksonModules()
+ {
+ return ImmutableList.of(
+ new SimpleModule()
+ .addDeserializer(StringObjectPairList.class, new ToStringObjectPairListDeserializer())
+ .setSerializerModifier(new DruidServiceSerializerModifier())
+ );
+ }
+ }
+
+ /**
+ * Reduced form of MySQLMetadataStorageModule.
+ */
+ private static class TestMySqlModule implements DruidModule
+ {
+ @Override
+ public void configure(Binder binder)
+ {
+ JsonConfigProvider.bind(binder, "druid.metadata.mysql.ssl", MySQLConnectorSslConfig.class);
+ JsonConfigProvider.bind(binder, "druid.metadata.mysql.driver", MySQLConnectorDriverConfig.class);
+ String type = MySQLMetadataStorageModule.TYPE;
+ PolyBind
+ .optionBinder(binder, Key.get(MetadataStorageProvider.class))
+ .addBinding(type)
+ .to(NoopMetadataStorageProvider.class)
+ .in(LazySingleton.class);
+
+ PolyBind
+ .optionBinder(binder, Key.get(MetadataStorageConnector.class))
+ .addBinding(type)
+ .to(MySQLConnector.class)
+ .in(LazySingleton.class);
+
+ PolyBind
+ .optionBinder(binder, Key.get(SQLMetadataConnector.class))
+ .addBinding(type)
+ .to(MySQLConnector.class)
+ .in(LazySingleton.class);
+ }
+
+ @Override
+ public List extends com.fasterxml.jackson.databind.Module> getJacksonModules()
+ {
+ return new MySQLMetadataStorageModule().getJacksonModules();
+ }
+ }
+
+ /**
+ * Class used by test to identify test-specific options, load configuration
+ * and "inject themselves" with dependencies.
+ */
+ public static class Builder
+ {
+ private final String clusterName;
+ private String configFile;
+ private Object test;
+ private List modules = new ArrayList<>();
+ private boolean validateCluster;
+ private List> eagerCreation = new ArrayList<>();
+ private Map envVarBindings = new HashMap<>();
+ private Properties testProperties = new Properties();
+
+ public Builder(String clusterName)
+ {
+ this.clusterName = clusterName;
+
+ // Node discovery is lifecycle managed. If we're using it, we have to
+ // create the instance within Guice during setup so it can be lifecycle
+ // managed. Using LazySingleon works in a server, but not in test clients,
+ // because test clients declare their need of node discovery after the
+ // the lifecycle starts.
+ eagerInstance(DruidNodeDiscoveryProvider.class);
+
+ // Set properties from environment variables, or hard-coded values
+ // previously set in Maven.
+ propertyEnvVarBinding("druid.test.config.dockerIp", "DOCKER_IP");
+ propertyEnvVarBinding("druid.zk.service.host", "DOCKER_IP");
+ propertyEnvVarBinding("druid.test.config.hadoopDir", "HADOOP_DIR");
+ property("druid.client.https.trustStorePath", "client_tls/truststore.jks");
+ property("druid.client.https.trustStorePassword", "druid123");
+ property("druid.client.https.keyStorePath", "client_tls/client.jks");
+ property("druid.client.https.certAlias", "druid");
+ property("druid.client.https.keyManagerPassword", "druid123");
+ property("druid.client.https.keyStorePassword", "druid123");
+
+ // More env var bindings for properties formerly passed in via
+ // a generated config file.
+ final String base = IntegrationTestingConfigProvider.PROPERTY_BASE + ".";
+ propertyEnvVarBinding(base + "cloudBucket", "DRUID_CLOUD_BUCKET");
+ propertyEnvVarBinding(base + "cloudPath", "DRUID_CLOUD_PATH");
+ propertyEnvVarBinding(base + "s3AccessKey", "AWS_ACCESS_KEY_ID");
+ propertyEnvVarBinding(base + "s3SecretKey", "AWS_SECRET_ACCESS_KEY");
+ propertyEnvVarBinding(base + "azureContainer", "AZURE_CONTAINER");
+ propertyEnvVarBinding(base + "azureAccount", "AZURE_ACCOUNT");
+ propertyEnvVarBinding(base + "azureKey", "AZURE_KEY");
+ propertyEnvVarBinding(base + "googleBucket", "GOOGLE_BUCKET");
+ propertyEnvVarBinding(base + "googlePrefix", "GOOGLE_PREFIX");
+
+ // Other defaults
+ // druid.global.http.numMaxThreads avoids creating 40+ Netty threads.
+ // We only ever use 1.
+ property("druid.global.http.numMaxThreads", 3);
+ property("druid.broker.http.numMaxThreads", 3);
+ }
+
+ /**
+ * Load a configuration from the named file. Primarily for
+ * debugging to use a one-off, custom configuration file.
+ */
+ public Builder configFile(String configFile)
+ {
+ this.configFile = configFile;
+ return this;
+ }
+
+ /**
+ * The test class with members to be injected.
+ */
+ public Builder test(Object test)
+ {
+ this.test = test;
+ return this;
+ }
+
+ /**
+ * Druid provides the {@code PolyBind} abstraction and the {@code Lifecycle}
+ * abstraction. When used together, we can encounter initialization issues. We won't create
+ * and instance of a polymorphic binding until it is first needed, and only then does
+ * the instance add itself to the lifecycle. However, if it is a test that acks for
+ * the instance, that is too late: the lifecycle has started. A test should call this
+ * method to "register" polymorphic lifecycle classes that will be injected later.
+ *
+ * The builder registers {@code DruidNodeDiscoveryProvider} by default: add any
+ * test-specific instances as needed.
+ */
+ public Builder eagerInstance(Class> theClass)
+ {
+ this.eagerCreation.add(theClass);
+ return this;
+ }
+
+ /**
+ * Optional test-specific modules to load.
+ */
+ public Builder modules(List modules)
+ {
+ this.modules.addAll(modules);
+ return this;
+ }
+
+ public Builder modules(Module...modules)
+ {
+ return modules(Arrays.asList(modules));
+ }
+
+ /**
+ * Validates the cluster before running tests. Ensures that each
+ * Druid service reports itself as healthy. Since Druid services
+ * depend on ZK and the metadata DB, this indirectly checks their
+ * health as well.
+ */
+ public Builder validateCluster()
+ {
+ this.validateCluster = true;
+ return this;
+ }
+
+ /**
+ * Set a property value in code. Such values go into the {@link Properties}
+ * object in Guice, and act as defaults to properties defined in the config
+ * file or via system properties. These properties can also "hard code" items
+ * that would normally be user-settable in a server. The value can be of any
+ * type: it is converted to a String internally.
+ */
+ public Builder property(String key, Object value)
+ {
+ if (value == null) {
+ testProperties.remove(key);
+ } else {
+ testProperties.put(key, value.toString());
+ }
+ return this;
+ }
+
+ /**
+ * Bind a property value to an environment variable. Useful if the property
+ * is set in the environment via the build system, Maven or other means.
+ * Avoids the need to add command-line arguments of the form
+ * {@code -Dproperty.name=$ENV_VAR}. Environment variable bindings take
+ * precedence over values set via {@link #property(String, Object)}, or
+ * the config file, but are lower priority than system properties. The
+ * environment variable is used only if set, else it is ignored.
+ */
+ public Builder propertyEnvVarBinding(String property, String envVar)
+ {
+ this.envVarBindings.put(property, envVar);
+ return this;
+ }
+
+ public synchronized Initializer build()
+ {
+ return new Initializer(this);
+ }
+ }
+
+ private final ResolvedConfig clusterConfig;
+ private final Injector injector;
+ private final Lifecycle lifecycle;
+ private MetastoreClient metastoreClient;
+ private DruidClusterClient clusterClient;
+
+ private Initializer(Builder builder)
+ {
+ if (builder.configFile != null) {
+ this.clusterConfig = loadConfigFile(builder.clusterName, builder.configFile);
+ } else {
+ this.clusterConfig = loadConfig(builder.clusterName, builder.configFile);
+ }
+ this.injector = makeInjector(builder, clusterConfig);
+
+ // Do the injection of test members early, for force lazy singleton
+ // instance creation to avoid problems when lifecycle-managed objects
+ // are combined with PolyBind.
+ if (builder.test != null) {
+ this.injector.injectMembers(builder.test);
+ }
+
+ // Backup: instantiate any additional instances that might be referenced
+ // later outside of injection.
+ for (Class> eagerClass : builder.eagerCreation) {
+ this.injector.getInstance(eagerClass);
+ }
+
+ // Now that we've created lifecycle-managed instances, start the lifecycle.
+ log.info("Starting lifecycle");
+ this.lifecycle = GuiceRunnable.initLifecycle(injector, log);
+
+ // Verify the cluster to ensure it is ready.
+ log.info("Creating cluster client");
+ this.clusterClient = this.injector.getInstance(DruidClusterClient.class);
+ if (builder.validateCluster) {
+ clusterClient.validate();
+ }
+
+ // Now that the cluster is ready (which implies that the metastore is ready),
+ // load any "starter data" into the metastore. Warning: there is a time-lag between
+ // when the DB is updated and when Coordinator or Overlord learns about the updates.
+ // At present, there is no API to force a cache flush. Caveat emptor.
+ prepareDB();
+ }
+
+ public static Builder builder(String clusterName)
+ {
+ return new Builder(clusterName);
+ }
+
+ private static ResolvedConfig loadConfig(String category, String configName)
+ {
+ if (configName == null) {
+ configName = System.getProperty(TEST_CONFIG_PROPERTY);
+ }
+ if (configName == null) {
+ configName = System.getenv(TEST_CONFIG_VAR);
+ }
+ if (configName == null) {
+ configName = CLUSTER_CONFIG_DEFAULT;
+ }
+ String loadName = StringUtils.format(CLUSTER_CONFIG_RESOURCE, category, configName);
+ ClusterConfig config = ClusterConfig.loadFromResource(loadName);
+ return config.resolve(category);
+ }
+
+ private static ResolvedConfig loadConfigFile(String category, String path)
+ {
+ ClusterConfig config = ClusterConfig.loadFromFile(path);
+ return config.resolve(category);
+ }
+
+ private static Injector makeInjector(
+ Builder builder,
+ ResolvedConfig clusterConfig
+ )
+ {
+ Injector startupInjector = new StartupInjectorBuilder()
+ .withProperties(properties(builder, clusterConfig))
+ .build();
+ return new CoreInjectorBuilder(startupInjector)
+ .withLifecycle()
+ .add(
+ // Required by clients
+ new EscalatorModule(),
+ HttpClientModule.global(),
+ HttpClientModule.escalatedGlobal(),
+ new HttpClientModule("druid.broker.http", Client.class),
+ new HttpClientModule("druid.broker.http", EscalatedClient.class),
+ // For ZK discovery
+ new CuratorModule(),
+ new AnnouncerModule(),
+ new DiscoveryModule(),
+ // Dependencies from other modules
+ new DruidProcessingConfigModule(),
+ // Dependencies from other modules
+ new StorageNodeModule(),
+
+ // Test-specific items, including bits copy/pasted
+ // from modules that don't play well in a client setting.
+ new TestModule(clusterConfig),
+ new TestMySqlModule()
+ )
+ .addAll(builder.modules)
+ .build();
+ }
+
+ /**
+ * Define test properties similar to how the server does. Property precedence
+ * is:
+ *
+ * System properties (highest)
+ * Environment variable bindings
+ * Configuration file
+ * Hard-coded values (lowest>
+ *
+ */
+ private static Properties properties(
+ Builder builder,
+ ResolvedConfig clusterConfig
+ )
+ {
+ Properties finalProperties = new Properties();
+ finalProperties.putAll(builder.testProperties);
+ finalProperties.putAll(clusterConfig.toProperties());
+ for (Entry entry : builder.envVarBindings.entrySet()) {
+ String value = System.getenv(entry.getValue());
+ if (value != null) {
+ finalProperties.put(entry.getKey(), value);
+ }
+ }
+ finalProperties.putAll(System.getProperties());
+ log.info("Properties:");
+ log.info(finalProperties.toString());
+ return finalProperties;
+ }
+
+ /**
+ * Some tests need a known set of metadata in the metadata DB. To avoid the
+ * complexity of do the actual actions (such as creating segments), the tests
+ * "seed" the database directly. The result is not entirely valid and consistent,
+ * but is good enough for the test at hand.
+ *
+ * WARNING : At present, there is no way to force the Coordinator or
+ * Overlord to flush its cache to learn about these new entries. Instead, we have
+ * to sleep for the cache timeout period. This solution is unsatisfying, and error-prone.
+ */
+ private void prepareDB()
+ {
+ ResolvedMetastore metastoreConfig = clusterConfig.metastore();
+ if (metastoreConfig == null) {
+ return;
+ }
+ List stmts = metastoreConfig.initStmts();
+ if (stmts == null || stmts.isEmpty()) {
+ return;
+ }
+ log.info("Preparing database");
+ MetastoreClient client = injector.getInstance(MetastoreClient.class);
+ for (MetastoreStmt stmt : stmts) {
+ client.execute(stmt.toSQL());
+ }
+ try {
+ Thread.sleep(metastoreConfig.initDelaySec() * 1000);
+ }
+ catch (InterruptedException e) {
+ throw new RuntimeException("Interrupted while waiting for coordinator to notice DB changes");
+ }
+ log.info("Database prepared");
+ }
+
+ public Injector injector()
+ {
+ return injector;
+ }
+
+ public ResolvedConfig clusterConfig()
+ {
+ return clusterConfig;
+ }
+
+ public MetastoreClient metastoreClient()
+ {
+ if (clusterConfig.metastore() == null) {
+ throw new IAE("Please provide a metastore section in docker.yaml");
+ }
+ return injector.getInstance(MetastoreClient.class);
+ }
+
+ public DruidClusterClient clusterClient()
+ {
+ return clusterClient;
+ }
+
+ public void close()
+ {
+ lifecycle.stop();
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java
new file mode 100644
index 00000000000..c14ea745aa8
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java
@@ -0,0 +1,421 @@
+/*
+ * 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.config;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.druid.testing.IntegrationTestingConfigProvider;
+
+import javax.inject.Inject;
+
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Adapter to the "legacy" cluster configuration used by tests.
+ */
+class IntegrationTestingConfigEx implements IntegrationTestingConfig
+{
+ private final ResolvedConfig config;
+ private final Map properties;
+
+ @Inject
+ public IntegrationTestingConfigEx(
+ final ResolvedConfig config,
+ final Properties properties)
+ {
+ this.config = config;
+ ImmutableMap.Builder builder = ImmutableMap.builder();
+ for (Map.Entry entry : properties.entrySet()) {
+ String key = (String) entry.getKey();
+ if (key.startsWith(IntegrationTestingConfigProvider.PROPERTY_BASE)) {
+ key = key.substring(IntegrationTestingConfigProvider.PROPERTY_BASE.length() + 1);
+ builder.put(key, (String) entry.getValue());
+ }
+ }
+ this.properties = builder.build();
+ }
+
+ @Override
+ public String getZookeeperHosts()
+ {
+ return config.requireZk().clientHosts();
+ }
+
+ @Override
+ public String getKafkaHost()
+ {
+ return config.requireKafka().instance().clientHost();
+ }
+
+ @Override
+ public String getKafkaInternalHost()
+ {
+ return config.requireKafka().instance().host();
+ }
+
+ @Override
+ public String getBrokerHost()
+ {
+ return config.requireBroker().instance().clientHost();
+ }
+
+ @Override
+ public String getBrokerInternalHost()
+ {
+ return config.requireBroker().instance().host();
+ }
+
+ @Override
+ public String getRouterHost()
+ {
+ return config.requireRouter().instance().clientHost();
+ }
+
+ @Override
+ public String getRouterInternalHost()
+ {
+ return config.requireRouter().instance().host();
+ }
+
+ @Override
+ public String getCoordinatorHost()
+ {
+ return config.requireCoordinator().tagOrDefault("one").clientHost();
+ }
+
+ @Override
+ public String getCoordinatorInternalHost()
+ {
+ return config.requireCoordinator().tagOrDefault("one").host();
+ }
+
+ @Override
+ public String getCoordinatorTwoInternalHost()
+ {
+ return config.requireCoordinator().requireInstance("two").host();
+ }
+
+ @Override
+ public String getCoordinatorTwoHost()
+ {
+ return config.requireCoordinator().tagOrDefault("one").clientHost();
+ }
+
+ @Override
+ public String getOverlordHost()
+ {
+ return config.requireOverlord().tagOrDefault("one").clientHost();
+ }
+
+ @Override
+ public String getOverlordTwoHost()
+ {
+ return config.requireOverlord().tagOrDefault("two").clientHost();
+ }
+
+ @Override
+ public String getOverlordInternalHost()
+ {
+ return config.requireOverlord().tagOrDefault("one").host();
+ }
+
+ @Override
+ public String getOverlordTwoInternalHost()
+ {
+ return config.requireOverlord().requireInstance("two").host();
+ }
+
+ @Override
+ public String getMiddleManagerHost()
+ {
+ return config.requireMiddleManager().instance().clientHost();
+ }
+
+ @Override
+ public String getMiddleManagerInternalHost()
+ {
+ return config.requireMiddleManager().instance().host();
+ }
+
+ @Override
+ public String getHistoricalHost()
+ {
+ return config.requireHistorical().instance().clientHost();
+ }
+
+ @Override
+ public String getHistoricalInternalHost()
+ {
+ return config.requireHistorical().instance().host();
+ }
+
+ @Override
+ public String getCoordinatorUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireCoordinator();
+ return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one"));
+ }
+
+ @Override
+ public String getCoordinatorTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getCoordinatorTwoUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireCoordinator();
+ return serviceConfig.resolveUrl(serviceConfig.requireInstance("two"));
+ }
+
+ @Override
+ public String getCoordinatorTwoTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getOverlordUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireOverlord();
+ return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one"));
+ }
+
+ @Override
+ public String getOverlordTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getOverlordTwoUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireOverlord();
+ return serviceConfig.resolveUrl(serviceConfig.requireInstance("two"));
+ }
+
+ @Override
+ public String getOverlordTwoTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getIndexerUrl()
+ {
+ ResolvedDruidService indexer = config.druidService(ResolvedConfig.INDEXER);
+ if (indexer == null) {
+ indexer = config.requireMiddleManager();
+ }
+ return indexer.resolveUrl(indexer.instance());
+ }
+
+ @Override
+ public String getIndexerTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getRouterUrl()
+ {
+ return config.routerUrl();
+ }
+
+ @Override
+ public String getRouterTLSUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireRouter();
+ return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls"));
+ }
+
+ @Override
+ public String getPermissiveRouterUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getPermissiveRouterTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getNoClientAuthRouterUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getNoClientAuthRouterTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getCustomCertCheckRouterUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getCustomCertCheckRouterTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getBrokerUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireBroker();
+ return serviceConfig.resolveUrl(serviceConfig.instance());
+ }
+
+ @Override
+ public String getBrokerTLSUrl()
+ {
+ ResolvedDruidService serviceConfig = config.requireBroker();
+ return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls"));
+ }
+
+ @Override
+ public String getHistoricalUrl()
+ {
+ return config.requireHistorical().resolveUrl();
+ }
+
+ @Override
+ public String getHistoricalTLSUrl()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getProperty(String prop)
+ {
+ return properties.get(prop);
+ }
+
+ @Override
+ public String getUsername()
+ {
+ return getProperty("username");
+ }
+
+ @Override
+ public String getPassword()
+ {
+ return getProperty("password");
+ }
+
+ @Override
+ public Map getProperties()
+ {
+ return properties;
+ }
+
+ @Override
+ public boolean manageKafkaTopic()
+ {
+ throw new ISE("Not implemented");
+ }
+
+ @Override
+ public String getExtraDatasourceNameSuffix()
+ {
+ return config.datasourceNameSuffix;
+ }
+
+ @Override
+ public String getCloudBucket()
+ {
+ return getProperty("cloudBucket");
+ }
+
+ @Override
+ public String getCloudPath()
+ {
+ return getProperty("cloudPath");
+ }
+
+ @Override
+ public String getCloudRegion()
+ {
+ return getProperty("cloudRegion");
+ }
+
+ @Override
+ public String getS3AssumeRoleWithExternalId()
+ {
+ return getProperty("s3AssumeRoleWithExternalId");
+ }
+
+ @Override
+ public String getS3AssumeRoleExternalId()
+ {
+ return getProperty("s3AssumeRoleExternalId");
+ }
+
+ @Override
+ public String getS3AssumeRoleWithoutExternalId()
+ {
+ return getProperty("s3AssumeRoleWithoutExternalId");
+ }
+
+ @Override
+ public String getAzureKey()
+ {
+ return getProperty("azureKey");
+ }
+
+ @Override
+ public String getHadoopGcsCredentialsPath()
+ {
+ return getProperty("hadoopGcsCredentialsPath");
+ }
+
+ @Override
+ public String getStreamEndpoint()
+ {
+ return getProperty("streamEndpoint");
+ }
+
+ @Override
+ public String getSchemaRegistryHost()
+ {
+ return getProperty("schemaRegistryHost");
+ }
+
+ @Override
+ public boolean isDocker()
+ {
+ return config.isDocker();
+ }
+
+ @Override
+ public String getDockerHost()
+ {
+ return config.proxyHost();
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java
new file mode 100644
index 00000000000..00e785d940d
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.testsEx.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+public class KafkaConfig extends ServiceConfig
+{
+ @JsonCreator
+ public KafkaConfig(
+ @JsonProperty("service") String service,
+ @JsonProperty("instances") List instances
+ )
+ {
+ super(service, instances);
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java
new file mode 100644
index 00000000000..cad2fd5293e
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.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.testsEx.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Strings;
+
+import java.util.List;
+import java.util.Map;
+
+public class MetastoreConfig extends ServiceConfig
+{
+ /**
+ * Driver. Defaults to the MySQL Driver.
+ * @see {@link org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig}
+ */
+ private final String driver;
+
+ /**
+ * JDBC connection URI. Required.
+ */
+ private final String connectURI;
+
+ /**
+ * User for the metastore DB.
+ */
+ private final String user;
+
+ /**
+ * Password for the metastore DB.
+ */
+ private final String password;
+
+ /**
+ * Optional connection properties.
+ */
+ private final Map properties;
+
+ @JsonCreator
+ public MetastoreConfig(
+ @JsonProperty("service") String service,
+ @JsonProperty("driver") String driver,
+ @JsonProperty("connectURI") String connectURI,
+ @JsonProperty("user") String user,
+ @JsonProperty("password") String password,
+ @JsonProperty("properties") Map properties,
+ @JsonProperty("instances") List instances
+ )
+ {
+ super(service, instances);
+ this.driver = driver;
+ this.connectURI = connectURI;
+ this.user = user;
+ this.password = password;
+ this.properties = properties;
+ }
+
+ @JsonProperty("driver")
+ @JsonInclude(Include.NON_NULL)
+ public String driver()
+ {
+ return driver;
+ }
+
+ @JsonProperty("connectURI")
+ @JsonInclude(Include.NON_NULL)
+ public String connectURI()
+ {
+ return connectURI;
+ }
+
+ @JsonProperty("user")
+ @JsonInclude(Include.NON_NULL)
+ public String user()
+ {
+ return user;
+ }
+
+ @JsonProperty("password")
+ @JsonInclude(Include.NON_NULL)
+ public String password()
+ {
+ return password;
+ }
+
+ @JsonProperty("properties")
+ @JsonInclude(Include.NON_NULL)
+ public Map properties()
+ {
+ return properties;
+ }
+
+ public boolean validate(List errs)
+ {
+ if (Strings.isNullOrEmpty(connectURI)) {
+ errs.add("Metastore connect URI is required");
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java
new file mode 100644
index 00000000000..5213678f562
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java
@@ -0,0 +1,78 @@
+/*
+ * 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.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.lang3.RegExUtils;
+
+/**
+ * Represents a statement (query) to send to the Druid metadata
+ * storage database (metastore) before running tests. At present,
+ * each query is only a SQL statement. The statements are represented
+ * as objects to allow for other options (such as ignoring failures,
+ * etc.)
+ *
+ * Metastore queries often include a JSON payload. The metastore wants
+ * to store the payload in compact form without spaces. However, such
+ * JSON is hard for humans to understand. So, the configuration file
+ * should format the SQL statement and JSON for readability. This class
+ * will "compactify" the statement prior to execution.
+ */
+public class MetastoreStmt
+{
+ private final String sql;
+
+ @JsonCreator
+ public MetastoreStmt(
+ @JsonProperty("sql") String sql
+ )
+ {
+ this.sql = sql;
+ }
+
+ @JsonProperty("sql")
+ public String sql()
+ {
+ return sql;
+ }
+
+ @Override
+ public String toString()
+ {
+ return TestConfigs.toYaml(this);
+ }
+
+ /**
+ * Convert the human-readable form of the statement in YAML
+ * into the compact JSON form preferred in the DB. Also
+ * compacts the SQL, but that's OK.
+ */
+ public String toSQL()
+ {
+ String stmt = RegExUtils.replaceAll(sql, "\n", " ");
+ stmt = RegExUtils.replaceAll(stmt, " +", " ");
+ stmt = RegExUtils.replaceAll(stmt, ": ", ":");
+ stmt = RegExUtils.replaceAll(stmt, ", ", ",");
+ stmt = RegExUtils.replaceAll(stmt, " }", "}");
+ stmt = RegExUtils.replaceAll(stmt, "\\{ ", "{");
+ return stmt;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java
new file mode 100644
index 00000000000..6bdfe96b2f9
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java
@@ -0,0 +1,406 @@
+/*
+ * 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.config;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.curator.CuratorConfig;
+import org.apache.druid.curator.ExhibitorConfig;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.testing.IntegrationTestingConfigProvider;
+import org.apache.druid.testsEx.config.ClusterConfig.ClusterType;
+import org.apache.druid.testsEx.config.ResolvedService.ResolvedKafka;
+import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk;
+import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class ResolvedConfig
+{
+ public static final String COORDINATOR = "coordinator";
+ public static final String HISTORICAL = "historical";
+ public static final String OVERLORD = "overlord";
+ public static final String BROKER = "broker";
+ public static final String ROUTER = "router";
+ public static final String MIDDLEMANAGER = "middlemanager";
+ public static final String INDEXER = "indexer";
+
+ public static final int DEFAULT_READY_TIMEOUT_SEC = 120;
+ public static final int DEFAULT_READY_POLL_MS = 2000;
+
+ private final String category;
+ private final ClusterType type;
+ private final String proxyHost;
+ private final int readyTimeoutSec;
+ private final int readyPollMs;
+ final String datasourceNameSuffix;
+ private Map properties;
+ private Map settings;
+
+ private final ResolvedZk zk;
+ private final ResolvedKafka kafka;
+ private final ResolvedMetastore metastore;
+ private final Map druidServices = new HashMap<>();
+
+ public ResolvedConfig(String category, ClusterConfig config)
+ {
+ this.category = category;
+ type = config.type() == null ? ClusterType.docker : config.type();
+ if (!hasProxy()) {
+ proxyHost = null;
+ } else if (Strings.isNullOrEmpty(config.proxyHost())) {
+ proxyHost = "localhost";
+ } else {
+ proxyHost = config.proxyHost();
+ }
+ readyTimeoutSec = config.readyTimeoutSec() > 0 ?
+ config.readyTimeoutSec() : DEFAULT_READY_TIMEOUT_SEC;
+ readyPollMs = config.readyPollMs() > 0 ? config.readyPollMs() : DEFAULT_READY_POLL_MS;
+ if (config.properties() == null) {
+ this.properties = ImmutableMap.of();
+ } else {
+ this.properties = config.properties();
+ }
+ if (config.settings() == null) {
+ this.settings = ImmutableMap.of();
+ } else {
+ this.settings = config.settings();
+ }
+ if (config.datasourceSuffix() == null) {
+ this.datasourceNameSuffix = "";
+ } else {
+ this.datasourceNameSuffix = config.datasourceSuffix();
+ }
+
+ if (config.zk() == null) {
+ this.zk = null;
+ } else {
+ this.zk = new ResolvedZk(this, config.zk());
+ }
+ if (config.kafka() == null) {
+ this.kafka = null;
+ } else {
+ this.kafka = new ResolvedKafka(this, config.kafka());
+ }
+ if (config.metastore() == null) {
+ this.metastore = null;
+ } else {
+ this.metastore = new ResolvedMetastore(this, config.metastore(), config);
+ }
+
+ if (config.druid() != null) {
+ for (Entry entry : config.druid().entrySet()) {
+ druidServices.put(entry.getKey(),
+ new ResolvedDruidService(this, entry.getValue(), entry.getKey()));
+ }
+ }
+ }
+
+ public ClusterType type()
+ {
+ return type;
+ }
+
+ public String proxyHost()
+ {
+ return proxyHost;
+ }
+
+ public int readyTimeoutSec()
+ {
+ return readyTimeoutSec;
+ }
+
+ public int readyPollMs()
+ {
+ return readyPollMs;
+ }
+
+ public boolean isDocker()
+ {
+ return type == ClusterType.docker;
+ }
+
+ public boolean hasProxy()
+ {
+ switch (type) {
+ case docker:
+ case k8s:
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ public ResolvedZk zk()
+ {
+ return zk;
+ }
+
+ public ResolvedMetastore metastore()
+ {
+ return metastore;
+ }
+
+ public ResolvedKafka kafka()
+ {
+ return kafka;
+ }
+
+ public Map settings()
+ {
+ return settings;
+ }
+
+ public Map properties()
+ {
+ return properties;
+ }
+
+ public Map requireDruid()
+ {
+ if (druidServices == null) {
+ throw new ISE("Please configure Druid services");
+ }
+ return druidServices;
+ }
+
+ public ResolvedZk requireZk()
+ {
+ if (zk == null) {
+ throw new ISE("Please specify the ZooKeeper configuration");
+ }
+ return zk;
+ }
+
+ public ResolvedMetastore requireMetastore()
+ {
+ if (metastore == null) {
+ throw new ISE("Please specify the Metastore configuration");
+ }
+ return metastore;
+ }
+
+ public ResolvedKafka requireKafka()
+ {
+ if (kafka == null) {
+ throw new ISE("Please specify the Kafka configuration");
+ }
+ return kafka;
+ }
+
+ public ResolvedDruidService druidService(String serviceKey)
+ {
+ return requireDruid().get(serviceKey);
+ }
+
+ public ResolvedDruidService requireService(String serviceKey)
+ {
+ ResolvedDruidService service = druidService(serviceKey);
+ if (service == null) {
+ throw new ISE("Please configure Druid service " + serviceKey);
+ }
+ return service;
+ }
+
+ public ResolvedDruidService requireCoordinator()
+ {
+ return requireService(COORDINATOR);
+ }
+
+ public ResolvedDruidService requireOverlord()
+ {
+ return requireService(OVERLORD);
+ }
+
+ public ResolvedDruidService requireBroker()
+ {
+ return requireService(BROKER);
+ }
+
+ public ResolvedDruidService requireRouter()
+ {
+ return requireService(ROUTER);
+ }
+
+ public ResolvedDruidService requireMiddleManager()
+ {
+ return requireService(MIDDLEMANAGER);
+ }
+
+ public ResolvedDruidService requireHistorical()
+ {
+ return requireService(HISTORICAL);
+ }
+
+ public String routerUrl()
+ {
+ return requireRouter().clientUrl();
+ }
+
+ public CuratorConfig toCuratorConfig()
+ {
+ if (zk == null) {
+ throw new ISE("ZooKeeper not configured");
+ }
+ // TODO: Add a builder for other properties
+ return CuratorConfig.create(zk.clientHosts());
+ }
+
+ public ExhibitorConfig toExhibitorConfig()
+ {
+ // Does not yet support exhibitors
+ return ExhibitorConfig.create(Collections.emptyList());
+ }
+
+ /**
+ * Map from old-style config file (and settings) name to the
+ * corresponding property.
+ */
+ private static final Map SETTINGS_MAP =
+ ImmutableMap.builder()
+ .put("cloud_bucket", "cloudBucket")
+ .put("cloud_path", "cloudPath")
+ .put("cloud_region", "cloudRegion")
+ .put("s3_assume_role_with_external_id", "s3AssumeRoleWithExternalId")
+ .put("s3_assume_role_external_id", "s3AssumeRoleExternalId")
+ .put("s3_assume_role_without_external_id", "s3AssumeRoleWithoutExternalId")
+ .put("stream_endpoint", "streamEndpoint")
+ .put("s3_accessKey", "s3AccessKey")
+ .put("s3_secretKey", "s3SecretKey")
+ .put("azure_account", "azureAccount")
+ .put("azure_key", "azureKey")
+ .put("azure_container", "azureContainer")
+ .put("google_bucket", "googleBucket")
+ .put("google_prefix", "googlePrefix")
+ .build();
+
+ private static void setDruidProperyVar(Map properties, String key, Object value)
+ {
+ if (value == null) {
+ return;
+ }
+ if (key.startsWith("druid_")) {
+ key = key.substring("druid_".length());
+ }
+ String mapped = SETTINGS_MAP.get(key);
+ key = mapped == null ? key : mapped;
+ TestConfigs.putProperty(properties, IntegrationTestingConfigProvider.PROPERTY_BASE, key, value.toString());
+ }
+
+ private void loadPropertyFile(Map properties, File file)
+ {
+ try (BufferedReader in = new BufferedReader(
+ new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))) {
+ String line;
+ while ((line = in.readLine()) != null) {
+ if (Strings.isNullOrEmpty(line) || line.startsWith("#")) {
+ continue;
+ }
+ String[] parts = line.split("=");
+ if (parts.length != 2) {
+ continue;
+ }
+ setDruidProperyVar(properties, parts[0], parts[1]);
+ }
+ }
+ catch (IOException e) {
+ throw new IAE(e, "Cannot read file %s", file.getAbsolutePath());
+ }
+ }
+
+ /**
+ * Convert the config in this structure the the properties
+ * used to configure Guice.
+ */
+ public Map toProperties()
+ {
+ Map properties = new HashMap<>();
+ // druid.test.config.dockerIp is used by some older test code. Remove
+ // it when that code is updated.
+ TestConfigs.putProperty(properties, "druid.test.config.dockerIp", proxyHost);
+
+ // Start with implicit properties from various sections.
+ if (zk != null) {
+ properties.putAll(zk.toProperties());
+ }
+ if (metastore != null) {
+ properties.putAll(metastore.toProperties());
+ }
+
+ // Add settings, converted to properties. Map both old and
+ // "property-style" settings to the full property path.
+ // Settings are converted to properties so they can be overridden
+ // by environment variables and -D command-line settings.
+ for (Map.Entry entry : settings.entrySet()) {
+ setDruidProperyVar(properties, entry.getKey(), entry.getValue());
+ }
+
+ // Add explicit properties
+ if (this.properties != null) {
+ properties.putAll(this.properties);
+ }
+
+ // Override with a user-specific config file.
+ File userEnv = new File(
+ new File(
+ System.getProperty("user.home"),
+ "druid-it"),
+ category + ".env");
+ if (userEnv.exists()) {
+ loadPropertyFile(properties, userEnv);
+ }
+
+ // Override with a user-specific config file.
+ String overrideEnv = System.getenv("OVERRIDE_ENV");
+ if (overrideEnv != null) {
+ loadPropertyFile(properties, new File(overrideEnv));
+ }
+
+ // Override with any environment variables of the form "druid_"
+ for (Map.Entry entry : System.getenv().entrySet()) {
+ String key = entry.getKey();
+ if (!key.startsWith("druid_")) {
+ continue;
+ }
+ setDruidProperyVar(properties, key, entry.getValue());
+ }
+
+ // Override with any system properties of the form "druid_"
+ for (Map.Entry entry : System.getProperties().entrySet()) {
+ String key = (String) entry.getKey();
+ if (!key.startsWith("druid_")) {
+ continue;
+ }
+ setDruidProperyVar(properties, key, entry.getValue());
+ }
+ return properties;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java
new file mode 100644
index 00000000000..58cbdcdc33c
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java
@@ -0,0 +1,145 @@
+/*
+ * 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.config;
+
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ResolvedDruidService extends ResolvedService
+{
+ public ResolvedDruidService(ResolvedConfig root, DruidConfig config, String serviceKey)
+ {
+ super(root, config, serviceKey);
+ }
+
+ /**
+ * Get the URL (visible to the test) of the service.
+ */
+ public String clientUrl()
+ {
+ return resolveUrl(instance());
+ }
+
+ /**
+ * Find an instance given the instance name (tag).
+ */
+ public ResolvedInstance findInstance(String instanceName)
+ {
+ for (ResolvedInstance instance : requireInstances()) {
+ if (instance.tag() != null && instance.tag().equals(instanceName)) {
+ return instance;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Find an instance given the instance name (tag). Raises
+ * an error (which fails the test) if the tag is not defined.
+ */
+ public ResolvedInstance requireInstance(String instanceName)
+ {
+ ResolvedInstance instance = findInstance(instanceName);
+ if (instance != null) {
+ return instance;
+ }
+ throw new ISE(
+ StringUtils.format(
+ "No Druid instance of service %s with name %s is defined",
+ service,
+ instanceName));
+ }
+
+ public String resolveUrl()
+ {
+ return resolveUrl(instance());
+ }
+
+ /**
+ * Return the URL for the given instance name (tag) of this service
+ * as visible to the test.
+ */
+ public String resolveUrl(String instanceName)
+ {
+ return resolveUrl(requireInstance(instanceName));
+ }
+
+ /**
+ * Return the URL, known to the test, of the given service instance.
+ */
+ public String resolveUrl(ResolvedInstance instance)
+ {
+ return StringUtils.format(
+ "http://%s:%d",
+ instance.clientHost(),
+ instance.clientPort());
+ }
+
+ /**
+ * Return the named service instance. If not found, return the
+ * "default" instance. This is used by the somewhat awkward test
+ * config object so that if a test asks for "Coordinator one" in
+ * a cluster with a single Coordinator, it will get that Coordinator.
+ * Same for Overlord.
+ */
+ public ResolvedInstance tagOrDefault(String tag)
+ {
+ ResolvedInstance taggedInstance = findInstance(tag);
+ return taggedInstance == null ? instance() : taggedInstance;
+ }
+
+ /**
+ * Returns the "default" host for this service as known to the
+ * cluster. The host is that of the only instance and is undefined
+ * if there are multiple instances.
+ */
+ public String resolveHost()
+ {
+ ResolvedInstance instance = instance();
+ if (instances.size() > 1) {
+ throw new ISE(
+ StringUtils.format("Service %s has %d hosts, default is ambiguous",
+ service,
+ instances.size()));
+ }
+ return instance.host();
+ }
+
+ public ResolvedInstance findHost(String host)
+ {
+ Pattern p = Pattern.compile("https?://(.*):(\\d+)");
+ Matcher m = p.matcher(host);
+ if (!m.matches()) {
+ return null;
+ }
+ String hostName = m.group(1);
+ int port = Integer.parseInt(m.group(2));
+ for (ResolvedInstance instance : instances) {
+ if (instance.host().equals(hostName) && instance.port() == port) {
+ return instance;
+ }
+ }
+ return null;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java
new file mode 100644
index 00000000000..f65790eb1c9
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java
@@ -0,0 +1,128 @@
+/*
+ * 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.config;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.druid.metadata.MetadataStorageConnectorConfig;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ResolvedMetastore extends ResolvedService
+{
+ // Set to be 1 sec. longer than the setting in the
+ // docker-compose.yaml file:
+ // druid_manager_segments_pollDuration=PT5S
+ public static final int DEFAULT_METASTORE_INIT_DELAY_SEC = 6;
+
+ private final String driver;
+ private final String connectURI;
+ private final String user;
+ private final String password;
+ private final Map properties;
+ private final int initDelaySec;
+ private List initStmts;
+
+ public ResolvedMetastore(ResolvedConfig root, MetastoreConfig config, ClusterConfig clusterConfig)
+ {
+ super(root, config, "metastore");
+ this.driver = config.driver();
+ if (config.connectURI() != null) {
+ ResolvedInstance instance = instance();
+ this.connectURI = RegExUtils.replaceAll(
+ RegExUtils.replaceAll(
+ config.connectURI(),
+ "",
+ Integer.toString(instance.clientPort())),
+ "",
+ instance.clientHost());
+ } else {
+ this.connectURI = null;
+ }
+ this.user = config.user();
+ this.password = config.password();
+ if (config.properties() == null) {
+ this.properties = ImmutableMap.of();
+ } else {
+ this.properties = config.properties();
+ }
+
+ this.initDelaySec = clusterConfig.metastoreInitDelaySec() > 0
+ ? clusterConfig.metastoreInitDelaySec()
+ : DEFAULT_METASTORE_INIT_DELAY_SEC;
+ this.initStmts = clusterConfig.metastoreInit();
+ }
+
+ public String driver()
+ {
+ return driver;
+ }
+
+ public String connectURI()
+ {
+ return connectURI;
+ }
+
+ public String user()
+ {
+ return user;
+ }
+
+ public String password()
+ {
+ return password;
+ }
+
+ public Map properties()
+ {
+ return properties;
+ }
+
+ /**
+ * Create the properties Guice needs to create the connector config.
+ *
+ * @see
+ * Setting up MySQL
+ */
+ public Map toProperties()
+ {
+ final String base = MetadataStorageConnectorConfig.PROPERTY_BASE;
+ Map properties = new HashMap<>();
+ TestConfigs.putProperty(properties, "druid.metadata.mysql.driver.driverClassName", driver);
+ TestConfigs.putProperty(properties, "druid.metadata.storage.type", "mysql");
+ TestConfigs.putProperty(properties, base, "connectURI", connectURI);
+ TestConfigs.putProperty(properties, base, "user", user);
+ TestConfigs.putProperty(properties, base, "password", password);
+ TestConfigs.putProperty(properties, base, "dbcp", this.properties);
+ return properties;
+ }
+
+ public List initStmts()
+ {
+ return initStmts;
+ }
+
+ public int initDelaySec()
+ {
+ return initDelaySec;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java
new file mode 100644
index 00000000000..5bd377f7079
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java
@@ -0,0 +1,219 @@
+/*
+ * 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.config;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.curator.CuratorConfig;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testsEx.config.ServiceConfig.ZKConfig;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class ResolvedService
+{
+ protected final String service;
+ protected final List instances = new ArrayList<>();
+
+ public ResolvedService(ResolvedConfig root, ServiceConfig config, String name)
+ {
+ this.service = config.service() == null ? name : config.service();
+ for (ServiceInstance instanceConfig : config.instances()) {
+ this.instances.add(new ResolvedInstance(root, instanceConfig, this));
+ }
+ }
+
+ public String service()
+ {
+ return service;
+ }
+
+ public List requireInstances()
+ {
+ if (instances.isEmpty()) {
+ throw new ISE("Please specify a " + service + " instance");
+ }
+ return instances;
+ }
+
+ public ResolvedInstance instance()
+ {
+ return requireInstances().get(0);
+ }
+
+ public class ResolvedInstance
+ {
+ private final String container;
+ private final String host;
+ private final String clientHost;
+ private final String tag;
+ private final int port;
+ private final int clientPort;
+
+ public ResolvedInstance(ResolvedConfig root, ServiceInstance config, ResolvedService service)
+ {
+ this.tag = config.tag();
+
+ // The actual (cluster) host is...
+ if (config.host() != null) {
+ // The specified host, if provided
+ this.host = config.host();
+ } else {
+ String baseHost;
+ if (root.hasProxy()) {
+ // The same as the service, if there is a proxy
+ baseHost = service.service;
+ // with the tag appended
+ if (tag != null) {
+ baseHost += "-" + config.tag();
+ }
+ this.host = baseHost;
+ } else {
+ // The local host otherwise
+ this.host = "localhost";
+ }
+ }
+
+ if (root.hasProxy()) {
+ this.clientHost = root.proxyHost();
+ } else {
+ this.clientHost = this.host;
+ }
+
+ this.container = config.container() != null ? config.container() : service.service;
+ if (config.port() == 0) {
+ throw new ISE("Must provide port");
+ }
+ this.port = config.port();
+ if (config.proxyPort() != 0) {
+ this.clientPort = config.proxyPort();
+ } else {
+ this.clientPort = this.port;
+ }
+ }
+
+ public ResolvedService service()
+ {
+ return ResolvedService.this;
+ }
+
+ public String container()
+ {
+ return container;
+ }
+
+ public String host()
+ {
+ return host;
+ }
+
+ public String clientHost()
+ {
+ return clientHost;
+ }
+
+ public String tag()
+ {
+ return tag;
+ }
+
+ public int port()
+ {
+ return port;
+ }
+
+ public int clientPort()
+ {
+ return clientPort;
+ }
+ }
+
+ public static class ResolvedZk extends ResolvedService
+ {
+ private final int startTimeoutSecs;
+
+ public ResolvedZk(ResolvedConfig root, ZKConfig config)
+ {
+ super(root, config, "zookeeper");
+ this.startTimeoutSecs = config.startTimeoutSecs();
+ }
+
+ public int startTimeoutSecs()
+ {
+ return startTimeoutSecs;
+ }
+
+ public String clientHosts()
+ {
+ List hosts = new ArrayList<>();
+ for (ResolvedInstance instance : instances) {
+ hosts.add(formatHost(instance.clientHost(), instance.clientPort()));
+ }
+ return String.join(",", hosts);
+ }
+
+ public String clusterHosts()
+ {
+ List hosts = new ArrayList<>();
+ for (ResolvedInstance instance : instances) {
+ hosts.add(formatHost(instance.host(), instance.port()));
+ }
+ return String.join(",", hosts);
+ }
+
+ private String formatHost(String host, int port)
+ {
+ return StringUtils.format("%s:%d", host, port);
+ }
+
+ public Map extends String, ? extends Object> toProperties()
+ {
+ /*
+ * We will use this instead of druid server's CuratorConfig, because CuratorConfig in
+ * a test cluster environment sees Zookeeper at localhost even if Zookeeper is elsewhere.
+ * We'll take the Zookeeper host from the configuration file instead.
+ */
+ return ImmutableMap.of(
+ CuratorConfig.CONFIG_PREFIX + ".zkHosts",
+ clientHosts());
+ }
+ }
+
+ public static class ResolvedKafka extends ResolvedService
+ {
+ public ResolvedKafka(ResolvedConfig root, KafkaConfig config)
+ {
+ super(root, config, "kafka");
+ }
+
+ public String clientHost()
+ {
+ return instance().clientHost();
+ }
+
+ public String bootstrap()
+ {
+ ResolvedInstance instance = instance();
+ return StringUtils.format("%s:%d", instance.clientHost(), instance.clientPort());
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java
new file mode 100644
index 00000000000..08b07bc4ed8
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.testsEx.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+public class ServiceConfig
+{
+ protected final String service;
+ protected List instances;
+
+ public ServiceConfig(
+ String service,
+ List instances
+ )
+ {
+ this.service = service;
+ this.instances = instances;
+ }
+
+ @JsonProperty("service")
+ @JsonInclude(Include.NON_NULL)
+ public String service()
+ {
+ return service;
+ }
+
+ @JsonProperty("instances")
+ @JsonInclude(Include.NON_NULL)
+ public List instances()
+ {
+ return instances;
+ }
+
+ @Override
+ public String toString()
+ {
+ return TestConfigs.toYaml(this);
+ }
+
+ /**
+ * YAML description of a ZK cluster. Converted to
+ * {@link org.apache.druid.curator.CuratorConfig}
+ */
+ public static class ZKConfig extends ServiceConfig
+ {
+ /**
+ * Amount of time to wait for ZK to become ready.
+ * Defaults to 5 seconds.
+ */
+ private final int startTimeoutSecs;
+
+ @JsonCreator
+ public ZKConfig(
+ @JsonProperty("service") String service,
+ @JsonProperty("startTimeoutSecs") int startTimeoutSecs,
+ @JsonProperty("instances") List instances
+ )
+ {
+ super(service, instances);
+ this.startTimeoutSecs = startTimeoutSecs;
+ }
+
+ @JsonProperty("startTimeoutSecs")
+ public int startTimeoutSecs()
+ {
+ return startTimeoutSecs;
+ }
+ }
+
+ /**
+ * Represents a Druid service (of one or more instances) running
+ * in the test cluster. The service name comes from the key used
+ * in the {@code druid} map:
+ * druid:
+ * broker: # <-- key (service name)
+ * instances:
+ * ...
+ *
+ */
+ public static class DruidConfig extends ServiceConfig
+ {
+ @JsonCreator
+ public DruidConfig(
+ // Note: service is not actually used.
+ @JsonProperty("service") String service,
+ @JsonProperty("instances") List instances
+ )
+ {
+ super(service, instances);
+ }
+ }
+
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java
new file mode 100644
index 00000000000..c9a7e0e3a87
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java
@@ -0,0 +1,124 @@
+/*
+ * 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.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * One instance of a Druid or third-party service running on
+ * a host or in a container.
+ */
+public class ServiceInstance
+{
+ /**
+ * Name of the Docker container. Used in Docker commands against
+ * the container, such as starting and stopping.
+ */
+ private final String container;
+
+ /**
+ * Name of the host running the service as known to the cluster
+ * (which many not be visible to the host running the test.)
+ * Assumed to be {@code } or @{code -}
+ * if not explicitly set.
+ */
+ private final String host;
+
+ /**
+ * Tag used to identify a service when there are multiple
+ * instances. The host is assumed to be @{code-} if
+ * not explicitly set.
+ */
+ private final String tag;
+
+ /**
+ * The port exposed by the service on its host. May not be
+ * visible to the test. Required.
+ */
+
+ private final int port;
+
+ /**
+ * The proxy port visible for the test for this service. Defaults
+ * to the same as the @{code port}. Define only if Docker is configured
+ * for port mapping other than identity.
+ */
+ private final int proxyPort;
+
+ @JsonCreator
+ public ServiceInstance(
+ @JsonProperty("container") String container,
+ @JsonProperty("host") String host,
+ @JsonProperty("tag") String tag,
+ @JsonProperty("port") int port,
+ @JsonProperty("proxyPort") int proxyPort
+ )
+ {
+ this.container = container;
+ this.host = host;
+ this.tag = tag;
+ this.port = port;
+ this.proxyPort = proxyPort;
+ }
+
+ @JsonProperty("container")
+ @JsonInclude(Include.NON_NULL)
+ public String container()
+ {
+ return container;
+ }
+
+ @JsonProperty("host")
+ @JsonInclude(Include.NON_NULL)
+ public String host()
+ {
+ return host;
+ }
+
+ @JsonProperty("tag")
+ @JsonInclude(Include.NON_NULL)
+ public String tag()
+ {
+ return tag;
+ }
+
+ @JsonProperty("port")
+ @JsonInclude(Include.NON_DEFAULT)
+ public int port()
+ {
+ return port;
+ }
+
+ @JsonProperty("proxyPort")
+ @JsonInclude(Include.NON_DEFAULT)
+ public int proxyPort()
+ {
+ return proxyPort;
+ }
+
+ @Override
+ public String toString()
+ {
+ return TestConfigs.toYaml(this);
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java
new file mode 100644
index 00000000000..8b909487c23
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java
@@ -0,0 +1,66 @@
+/*
+ * 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.config;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator;
+
+import java.util.Map;
+
+/**
+ * Utility functions related to test configuration.
+ */
+public class TestConfigs
+{
+ /**
+ * Converts a YAML-aware object to a YAML string, primarily
+ * for use in @{code toString()} methods.
+ */
+ public static String toYaml(Object obj)
+ {
+ ObjectMapper mapper = new ObjectMapper(
+ new YAMLFactory()
+ .enable(YAMLGenerator.Feature.MINIMIZE_QUOTES));
+ try {
+ return mapper.writeValueAsString(obj);
+ }
+ catch (JsonProcessingException e) {
+ return "";
+ }
+ }
+
+ public static void putProperty(Map properties, String key, Object value)
+ {
+ if (value == null) {
+ return;
+ }
+ properties.put(key, value);
+ }
+
+ public static void putProperty(Map properties, String base, String key, Object value)
+ {
+ if (value == null) {
+ return;
+ }
+ properties.put(base + "." + key, value);
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java
new file mode 100644
index 00000000000..5b080cd7448
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java
@@ -0,0 +1,502 @@
+/*
+ * 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.indexer;
+
+import com.google.common.collect.FluentIterable;
+import com.google.inject.Inject;
+import org.apache.commons.io.IOUtils;
+import org.apache.druid.indexer.partitions.SecondaryPartitionType;
+import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
+import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionCardinalityTask;
+import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask;
+import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask;
+import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask;
+import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask;
+import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.druid.testing.clients.ClientInfoResourceTestClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.SqlTestQueryHelper;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.TimelineObjectHolder;
+import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
+{
+ public enum InputFormatDetails
+ {
+ AVRO("avro_ocf", ".avro", "/avro"),
+ CSV("csv", ".csv", "/csv"),
+ TSV("tsv", ".tsv", "/tsv"),
+ ORC("orc", ".orc", "/orc"),
+ JSON("json", ".json", "/json"),
+ PARQUET("parquet", ".parquet", "/parquet");
+
+ private final String inputFormatType;
+ private final String fileExtension;
+ private final String folderSuffix;
+
+ InputFormatDetails(String inputFormatType, String fileExtension, String folderSuffix)
+ {
+ this.inputFormatType = inputFormatType;
+ this.fileExtension = fileExtension;
+ this.folderSuffix = folderSuffix;
+ }
+
+ public String getInputFormatType()
+ {
+ return inputFormatType;
+ }
+
+ public String getFileExtension()
+ {
+ return fileExtension;
+ }
+
+ public String getFolderSuffix()
+ {
+ return folderSuffix;
+ }
+ }
+
+ private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class);
+
+ @Inject
+ protected IntegrationTestingConfig config;
+ @Inject
+ protected SqlTestQueryHelper sqlQueryHelper;
+
+ @Inject
+ ClientInfoResourceTestClient clientInfoResourceTestClient;
+
+ protected void doIndexTest(
+ String dataSource,
+ String indexTaskFilePath,
+ String queryFilePath,
+ boolean waitForNewVersion,
+ boolean runTestQueries,
+ boolean waitForSegmentsToLoad,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws IOException
+ {
+ doIndexTest(
+ dataSource,
+ indexTaskFilePath,
+ Function.identity(),
+ queryFilePath,
+ waitForNewVersion,
+ runTestQueries,
+ waitForSegmentsToLoad,
+ segmentAvailabilityConfirmationPair
+ );
+ }
+
+ protected void doIndexTest(
+ String dataSource,
+ String indexTaskFilePath,
+ Function taskSpecTransform,
+ String queryFilePath,
+ boolean waitForNewVersion,
+ boolean runTestQueries,
+ boolean waitForSegmentsToLoad,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws IOException
+ {
+ final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
+ final String taskSpec = taskSpecTransform.apply(
+ StringUtils.replace(
+ getResourceAsString(indexTaskFilePath),
+ "%%DATASOURCE%%",
+ fullDatasourceName
+ )
+ );
+
+ submitTaskAndWait(
+ taskSpec,
+ fullDatasourceName,
+ waitForNewVersion,
+ waitForSegmentsToLoad,
+ segmentAvailabilityConfirmationPair
+ );
+ if (runTestQueries) {
+ doTestQuery(dataSource, queryFilePath);
+ }
+ }
+
+ protected void doTestQuery(String dataSource, String queryFilePath)
+ {
+ try {
+ String queryResponseTemplate;
+ try {
+ InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath);
+ queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8);
+ }
+ catch (IOException e) {
+ throw new ISE(e, "could not read query file: %s", queryFilePath);
+ }
+
+ queryResponseTemplate = StringUtils.replace(
+ queryResponseTemplate,
+ "%%DATASOURCE%%",
+ dataSource + config.getExtraDatasourceNameSuffix()
+ );
+ queryHelper.testQueriesFromString(queryResponseTemplate);
+
+ }
+ catch (Exception e) {
+ LOG.error(e, "Error while testing");
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected void doReindexTest(
+ String baseDataSource,
+ String reindexDataSource,
+ String reindexTaskFilePath,
+ String queryFilePath,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws IOException
+ {
+ doReindexTest(
+ baseDataSource,
+ reindexDataSource,
+ Function.identity(),
+ reindexTaskFilePath,
+ queryFilePath,
+ segmentAvailabilityConfirmationPair
+ );
+ }
+
+ void doReindexTest(
+ String baseDataSource,
+ String reindexDataSource,
+ Function taskSpecTransform,
+ String reindexTaskFilePath,
+ String queryFilePath,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws IOException
+ {
+ final String fullBaseDatasourceName = baseDataSource + config.getExtraDatasourceNameSuffix();
+ final String fullReindexDatasourceName = reindexDataSource + config.getExtraDatasourceNameSuffix();
+
+ String taskSpec = StringUtils.replace(
+ getResourceAsString(reindexTaskFilePath),
+ "%%DATASOURCE%%",
+ fullBaseDatasourceName
+ );
+
+ taskSpec = StringUtils.replace(
+ taskSpec,
+ "%%REINDEX_DATASOURCE%%",
+ fullReindexDatasourceName
+ );
+
+ taskSpec = taskSpecTransform.apply(taskSpec);
+
+ submitTaskAndWait(
+ taskSpec,
+ fullReindexDatasourceName,
+ false,
+ true,
+ segmentAvailabilityConfirmationPair
+ );
+ try {
+ String queryResponseTemplate;
+ try {
+ InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath);
+ queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8);
+ }
+ catch (IOException e) {
+ throw new ISE(e, "could not read query file: %s", queryFilePath);
+ }
+
+ queryResponseTemplate = StringUtils.replace(
+ queryResponseTemplate,
+ "%%DATASOURCE%%",
+ fullReindexDatasourceName
+ );
+
+ queryHelper.testQueriesFromString(queryResponseTemplate);
+ // verify excluded dimension is not reIndexed
+ final List dimensions = clientInfoResourceTestClient.getDimensions(
+ fullReindexDatasourceName,
+ "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z"
+ );
+ Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot"));
+ }
+ catch (Exception e) {
+ LOG.error(e, "Error while testing");
+ throw new RuntimeException(e);
+ }
+ }
+
+ void doIndexTestSqlTest(
+ String dataSource,
+ String indexTaskFilePath,
+ String queryFilePath
+ ) throws IOException
+ {
+ doIndexTestSqlTest(
+ dataSource,
+ indexTaskFilePath,
+ queryFilePath,
+ Function.identity()
+ );
+ }
+ void doIndexTestSqlTest(
+ String dataSource,
+ String indexTaskFilePath,
+ String queryFilePath,
+ Function taskSpecTransform
+ ) throws IOException
+ {
+ final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
+ final String taskSpec = taskSpecTransform.apply(
+ StringUtils.replace(
+ getResourceAsString(indexTaskFilePath),
+ "%%DATASOURCE%%",
+ fullDatasourceName
+ )
+ );
+
+ Pair dummyPair = new Pair<>(false, false);
+ submitTaskAndWait(taskSpec, fullDatasourceName, false, true, dummyPair);
+ try {
+ sqlQueryHelper.testQueriesFromFile(queryFilePath);
+ }
+ catch (Exception e) {
+ LOG.error(e, "Error while testing");
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected void submitTaskAndWait(
+ String taskSpec,
+ String dataSourceName,
+ boolean waitForNewVersion,
+ boolean waitForSegmentsToLoad,
+ Pair segmentAvailabilityConfirmationPair
+ )
+ {
+ final List oldVersions = waitForNewVersion ? coordinator.getAvailableSegments(dataSourceName) : null;
+
+ long startSubTaskCount = -1;
+ final boolean assertRunsSubTasks = taskSpec.contains("index_parallel");
+ if (assertRunsSubTasks) {
+ startSubTaskCount = countCompleteSubTasks(dataSourceName, !taskSpec.contains("dynamic"));
+ }
+
+ final String taskID = indexer.submitTask(taskSpec);
+ LOG.info("TaskID for loading index task %s", taskID);
+ indexer.waitUntilTaskCompletes(taskID);
+
+ if (assertRunsSubTasks) {
+ final boolean perfectRollup = !taskSpec.contains("dynamic");
+ final long newSubTasks = countCompleteSubTasks(dataSourceName, perfectRollup) - startSubTaskCount;
+ Assert.assertTrue(
+ StringUtils.format(
+ "The supervisor task [%s] didn't create any sub tasks. Was it executed in the parallel mode?",
+ taskID
+ ),
+ newSubTasks > 0
+ );
+ }
+
+ if (segmentAvailabilityConfirmationPair.lhs != null && segmentAvailabilityConfirmationPair.lhs) {
+ TaskReport reportRaw = indexer.getTaskReport(taskID).get("ingestionStatsAndErrors");
+ IngestionStatsAndErrorsTaskReport report = (IngestionStatsAndErrorsTaskReport) reportRaw;
+ IngestionStatsAndErrorsTaskReportData reportData = (IngestionStatsAndErrorsTaskReportData) report.getPayload();
+
+ // Confirm that the task waited longer than 0ms for the task to complete.
+ Assert.assertTrue(reportData.getSegmentAvailabilityWaitTimeMs() > 0);
+
+ // Make sure that the result of waiting for segments to load matches the expected result
+ if (segmentAvailabilityConfirmationPair.rhs != null) {
+ Assert.assertEquals(
+ Boolean.valueOf(reportData.isSegmentAvailabilityConfirmed()),
+ segmentAvailabilityConfirmationPair.rhs
+ );
+ }
+ }
+
+ // IT*ParallelIndexTest do a second round of ingestion to replace segments in an existing
+ // data source. For that second round we need to make sure the coordinator actually learned
+ // about the new segments before waiting for it to report that all segments are loaded; otherwise
+ // this method could return too early because the coordinator is merely reporting that all the
+ // original segments have loaded.
+ if (waitForNewVersion) {
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(
+ coordinator.getAvailableSegments(dataSourceName)
+ );
+
+ final List> holders = timeline.lookup(Intervals.ETERNITY);
+ return FluentIterable
+ .from(holders)
+ .transformAndConcat(TimelineObjectHolder::getObject)
+ .anyMatch(
+ chunk -> FluentIterable.from(oldVersions)
+ .anyMatch(oldSegment -> chunk.getObject().overshadows(oldSegment))
+ );
+ },
+ "See a new version"
+ );
+ }
+
+ if (waitForSegmentsToLoad) {
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load"
+ );
+ }
+ }
+
+ private long countCompleteSubTasks(final String dataSource, final boolean perfectRollup)
+ {
+ return indexer.getCompleteTasksForDataSource(dataSource)
+ .stream()
+ .filter(t -> {
+ if (!perfectRollup) {
+ return t.getType().equals(SinglePhaseSubTask.TYPE);
+ } else {
+ return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE)
+ || t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE)
+ || t.getType().equalsIgnoreCase(PartialDimensionCardinalityTask.TYPE)
+ || t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE)
+ || t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE);
+ }
+ })
+ .count();
+ }
+
+ void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments)
+ {
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()),
+ "Segment load check"
+ );
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ List segments = coordinator.getAvailableSegments(
+ dataSource + config.getExtraDatasourceNameSuffix()
+ );
+ int segmentCount = segments.size();
+ LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments);
+
+ return segmentCount == numExpectedSegments;
+ },
+ "Segment count check"
+ );
+ }
+
+ void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments, int numExpectedTombstones)
+ {
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()),
+ "Segment load check"
+ );
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ List segments = coordinator.getAvailableSegments(
+ dataSource + config.getExtraDatasourceNameSuffix()
+ );
+ int segmentCount = segments.size();
+ LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments);
+
+ int tombstoneCount = 0;
+ for (DataSegment segment : segments) {
+ if (segment.isTombstone()) {
+ tombstoneCount++;
+ }
+ }
+
+ LOG.info("Current tombstone count: %d, expected: %d", tombstoneCount, numExpectedTombstones);
+
+ return segmentCount == numExpectedSegments && tombstoneCount == numExpectedTombstones;
+ },
+ "Segment count check"
+ );
+ }
+
+ void compactData(String dataSource, String compactionTask) throws Exception
+ {
+ final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
+ final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName);
+ intervalsBeforeCompaction.sort(null);
+ final String template = getResourceAsString(compactionTask);
+ String taskSpec = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName);
+
+ final String taskID = indexer.submitTask(taskSpec);
+ LOG.info("TaskID for compaction task %s", taskID);
+ indexer.waitUntilTaskCompletes(taskID);
+
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(fullDatasourceName),
+ "Segment Compaction"
+ );
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ final List actualIntervals = coordinator.getSegmentIntervals(
+ dataSource + config.getExtraDatasourceNameSuffix()
+ );
+ actualIntervals.sort(null);
+ return actualIntervals.equals(intervalsBeforeCompaction);
+ },
+ "Compaction interval check"
+ );
+ }
+
+ void verifySegmentsCompacted(String dataSource, int expectedCompactedSegmentCount)
+ {
+ List segments = coordinator.getFullSegmentsMetadata(
+ dataSource + config.getExtraDatasourceNameSuffix()
+ );
+ List foundCompactedSegments = new ArrayList<>();
+ for (DataSegment segment : segments) {
+ if (segment.getLastCompactionState() != null) {
+ foundCompactedSegments.add(segment);
+ }
+ }
+ Assert.assertEquals(foundCompactedSegments.size(), expectedCompactedSegmentCount);
+ for (DataSegment compactedSegment : foundCompactedSegments) {
+ Assert.assertNotNull(compactedSegment.getLastCompactionState());
+ Assert.assertNotNull(compactedSegment.getLastCompactionState().getPartitionsSpec());
+ Assert.assertEquals(
+ compactedSegment.getLastCompactionState().getPartitionsSpec().getType(),
+ SecondaryPartitionType.LINEAR
+ );
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java
new file mode 100644
index 00000000000..5e53330e705
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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.indexer;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Inject;
+import org.apache.commons.io.IOUtils;
+import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.guice.annotations.Smile;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
+import org.apache.druid.testing.clients.OverlordResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.TestQueryHelper;
+import org.joda.time.Interval;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+public abstract class AbstractIndexerTest
+{
+ private static final Logger LOG = new Logger(AbstractIndexerTest.class);
+
+ @Inject
+ protected CoordinatorResourceTestClient coordinator;
+ @Inject
+ protected OverlordResourceTestClient indexer;
+ @Inject
+ @Json
+ protected ObjectMapper jsonMapper;
+ @Inject
+ @Smile
+ protected ObjectMapper smileMapper;
+ @Inject
+ protected TestQueryHelper queryHelper;
+
+ @Inject
+ protected IntegrationTestingConfig config;
+
+ protected Closeable unloader(final String dataSource)
+ {
+ return () -> unloadAndKillData(dataSource);
+ }
+
+ protected void unloadAndKillData(final String dataSource)
+ {
+ // Get all failed task logs
+ List allTasks = indexer.getCompleteTasksForDataSource(dataSource);
+ for (TaskResponseObject task : allTasks) {
+ if (task.getStatus().isFailure()) {
+ LOG.info("------- START Found failed task logging for taskId=" + task.getId() + " -------");
+ LOG.info("Start failed task log:");
+ LOG.info(indexer.getTaskLog(task.getId()));
+ LOG.info("End failed task log.");
+ LOG.info("Start failed task errorMsg:");
+ LOG.info(indexer.getTaskErrorMessage(task.getId()));
+ LOG.info("End failed task errorMsg.");
+ LOG.info("------- END Found failed task logging for taskId=" + task.getId() + " -------");
+ }
+ }
+
+ List intervals = coordinator.getSegmentIntervals(dataSource);
+
+ // each element in intervals has this form:
+ // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z
+ // we'll sort the list (ISO dates have lexicographic order)
+ // then delete segments from the 1st date in the first string
+ // to the 2nd date in the last string
+ Collections.sort(intervals);
+ String first = intervals.get(0).split("/")[0];
+ String last = intervals.get(intervals.size() - 1).split("/")[1];
+ unloadAndKillData(dataSource, first, last);
+ }
+
+ protected String submitIndexTask(String indexTask, final String fullDatasourceName) throws Exception
+ {
+ String taskSpec = getResourceAsString(indexTask);
+ taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
+ taskSpec = StringUtils.replace(
+ taskSpec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("0")
+ );
+ final String taskID = indexer.submitTask(taskSpec);
+ LOG.info("TaskID for loading index task %s", taskID);
+
+ return taskID;
+ }
+
+ protected void loadData(String indexTask, final String fullDatasourceName) throws Exception
+ {
+ final String taskID = submitIndexTask(indexTask, fullDatasourceName);
+ indexer.waitUntilTaskCompletes(taskID);
+
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(fullDatasourceName),
+ "Segment Load"
+ );
+ }
+
+ private void unloadAndKillData(final String dataSource, String start, String end)
+ {
+ // Wait for any existing index tasks to complete before disabling the datasource otherwise
+ // realtime tasks can get stuck waiting for handoff. https://github.com/apache/druid/issues/1729
+ waitForAllTasksToCompleteForDataSource(dataSource);
+ Interval interval = Intervals.of(start + "/" + end);
+ coordinator.unloadSegmentsForDataSource(dataSource);
+ ITRetryUtil.retryUntilFalse(
+ new Callable()
+ {
+ @Override
+ public Boolean call()
+ {
+ return coordinator.areSegmentsLoaded(dataSource);
+ }
+ }, "Segment Unloading"
+ );
+ coordinator.deleteSegmentsDataSource(dataSource, interval);
+ waitForAllTasksToCompleteForDataSource(dataSource);
+ }
+
+ protected void waitForAllTasksToCompleteForDataSource(final String dataSource)
+ {
+ ITRetryUtil.retryUntilTrue(
+ () -> (indexer.getUncompletedTasksForDataSource(dataSource).size() == 0),
+ StringUtils.format("Waiting for all tasks of [%s] to complete", dataSource)
+ );
+ }
+
+ public static String getResourceAsString(String file) throws IOException
+ {
+ try (final InputStream inputStream = getResourceAsStream(file)) {
+ return IOUtils.toString(inputStream, StandardCharsets.UTF_8);
+ }
+ }
+
+ public static InputStream getResourceAsStream(String resource)
+ {
+ return AbstractIndexerTest.class.getResourceAsStream(resource);
+ }
+
+ public static List listResources(String dir) throws IOException
+ {
+ List resources = new ArrayList<>();
+
+ try (
+ InputStream in = getResourceAsStream(dir);
+ BufferedReader br = new BufferedReader(new InputStreamReader(in, StringUtils.UTF8_STRING))
+ ) {
+ String resource;
+
+ while ((resource = br.readLine()) != null) {
+ resources.add(resource);
+ }
+ }
+
+ return resources;
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java
new file mode 100644
index 00000000000..ce8a9f5c13c
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.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.testsEx.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.Nonnull;
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+public abstract class AbstractLocalInputSourceParallelIndexTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+
+ public void doIndexTest(
+ InputFormatDetails inputFormatDetails,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws Exception
+ {
+ doIndexTest(inputFormatDetails, ImmutableMap.of(), segmentAvailabilityConfirmationPair);
+ }
+
+ public void doIndexTest(
+ InputFormatDetails inputFormatDetails,
+ @Nonnull Map extraInputFormatMap,
+ Pair segmentAvailabilityConfirmationPair
+ ) throws Exception
+ {
+ final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
+ Map inputFormatMap = new ImmutableMap.Builder().putAll(extraInputFormatMap)
+ .put("type", inputFormatDetails.getInputFormatType())
+ .build();
+ try (
+ final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
+ ) {
+ final Function sqlInputSourcePropsTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null))
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_FILTER%%",
+ "*" + inputFormatDetails.getFileExtension()
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_BASE_DIR%%",
+ "/resources/data/batch_index" + inputFormatDetails.getFolderSuffix()
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_FORMAT%%",
+ jsonMapper.writeValueAsString(inputFormatMap)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%APPEND_TO_EXISTING%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%DROP_EXISTING%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ return spec;
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ indexDatasource,
+ INDEX_TASK,
+ sqlInputSourcePropsTransform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ segmentAvailabilityConfirmationPair
+ );
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java
new file mode 100644
index 00000000000..96e8d500fa6
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java
@@ -0,0 +1,250 @@
+/*
+ * 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.indexer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest
+{
+ // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle
+ // maxSplitSize of 1 properly.
+ private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json";
+ private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json";
+ private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_reindex_queries.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test";
+ private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test";
+ private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json";
+ private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test";
+ private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json";
+
+ private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED =
+ CoordinatorDynamicConfig.builder().withPauseCoordination(true).build();
+ private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT =
+ CoordinatorDynamicConfig.builder().build();
+
+ @Inject
+ CoordinatorResourceTestClient coordinatorClient;
+
+ @Test
+ public void testIndexData() throws Exception
+ {
+ PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null);
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix())
+ ) {
+ boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible();
+ Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup);
+
+ final Function rollupTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ Boolean.toString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("0")
+ );
+ return StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(partitionsSpec)
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ rollupTransform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+
+ // Index again, this time only choosing the second data file, and without explicit intervals chosen.
+ // The second datafile covers both day segments, so this should replace them, as reflected in the queries.
+ doIndexTest(
+ INDEX_DATASOURCE,
+ REINDEX_TASK,
+ rollupTransform,
+ REINDEX_QUERIES_RESOURCE,
+ true,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+
+ doReindexTest(
+ INDEX_DATASOURCE,
+ INDEX_INGEST_SEGMENT_DATASOURCE,
+ rollupTransform,
+ INDEX_INGEST_SEGMENT_TASK,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+
+ // with DruidInputSource instead of IngestSegmentFirehose
+ doReindexTest(
+ INDEX_DATASOURCE,
+ INDEX_DRUID_INPUT_SOURCE_DATASOURCE,
+ rollupTransform,
+ INDEX_DRUID_INPUT_SOURCE_TASK,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ /**
+ * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. This will confirm that the report for the task
+ * indicates segments were confirmed to be available on the cluster before finishing the ingestion job.
+ */
+ @Test
+ public void testIndexDataVerifySegmentAvailability() throws Exception
+ {
+ PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null);
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible();
+ Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup);
+
+ final Function rollupTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ Boolean.toString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("600000")
+ );
+ return StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(partitionsSpec)
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ rollupTransform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(true, true)
+ );
+ }
+ }
+
+ /**
+ * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. Setting the config value to 1 millis
+ * and pausing coordination to confirm that the task will still succeed even if the job was not able to confirm the
+ * segments were loaded by the time the timeout occurs.
+ */
+ @Test
+ public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception
+ {
+ PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null);
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED);
+ boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible();
+ Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup);
+
+ final Function rollupTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ Boolean.toString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("1")
+ );
+ return StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(partitionsSpec)
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ rollupTransform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ false,
+ false,
+ new Pair<>(true, false)
+ );
+ coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT);
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load"
+ );
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java
new file mode 100644
index 00000000000..8d394a0297e
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.indexer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_index_test";
+
+ private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json";
+ private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
+ private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
+
+ @Test
+ public void testIndexData() throws Exception
+ {
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ final Function combiningFirehoseSpecTransform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%COMBINING_DATASOURCE%%",
+ INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()
+ );
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ };
+ final Function transform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("0")
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ transform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doIndexTest(
+ COMBINING_INDEX_DATASOURCE,
+ COMBINING_INDEX_TASK,
+ combiningFirehoseSpecTransform,
+ COMBINING_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java
new file mode 100644
index 00000000000..5ec9ea0dbc4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.testsEx.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_index_test";
+
+ private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json";
+ private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
+ private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
+
+ @Test
+ public void testIndexData() throws Exception
+ {
+ Map inputFormatMap = new ImmutableMap
+ .Builder()
+ .put("type", "json")
+ .build();
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ final Function combiningInputSourceSpecTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null))
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_FILTER%%",
+ "wikipedia_index_data*"
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_BASE_DIR%%",
+ "/resources/data/batch_index/json"
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_FORMAT%%",
+ jsonMapper.writeValueAsString(inputFormatMap)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%APPEND_TO_EXISTING%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%DROP_EXISTING%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%COMBINING_DATASOURCE%%",
+ INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()
+ );
+ return spec;
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ combiningInputSourceSpecTransform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doIndexTest(
+ COMBINING_INDEX_DATASOURCE,
+ COMBINING_INDEX_TASK,
+ combiningInputSourceSpecTransform,
+ COMBINING_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java
new file mode 100644
index 00000000000..65b8dc0b1ac
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java
@@ -0,0 +1,386 @@
+/*
+ * 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.indexer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.inject.Inject;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.joda.time.Interval;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITIndexerTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_index_test";
+
+ private static final String INDEX_WITH_TIMESTAMP_TASK = "/indexer/wikipedia_with_timestamp_index_task.json";
+ // TODO: add queries that validate timestamp is different from the __time column since it is a dimension
+ // TODO: https://github.com/apache/druid/issues/9565
+ private static final String INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+ private static final String INDEX_WITH_TIMESTAMP_DATASOURCE = "wikipedia_with_timestamp_index_test";
+
+ private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json";
+ private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task.json";
+ private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json";
+ private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test";
+
+ private static final String MERGE_INDEX_TASK = "/indexer/wikipedia_merge_index_task.json";
+ private static final String MERGE_INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json";
+ private static final String MERGE_INDEX_DATASOURCE = "wikipedia_merge_index_test";
+
+ private static final String MERGE_REINDEX_TASK = "/indexer/wikipedia_merge_reindex_task.json";
+ private static final String MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_merge_reindex_druid_input_source_task.json";
+ private static final String MERGE_REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json";
+ private static final String MERGE_REINDEX_DATASOURCE = "wikipedia_merge_reindex_test";
+
+ private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_TASK = "/indexer/wikipedia_index_with_merge_column_limit_task.json";
+ private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE = "wikipedia_index_with_merge_column_limit_test";
+
+ private static final String GET_LOCKED_INTERVALS = "wikipedia_index_get_locked_intervals_test";
+
+ private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED =
+ CoordinatorDynamicConfig.builder().withPauseCoordination(true).build();
+ private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT =
+ CoordinatorDynamicConfig.builder().build();
+
+ @Inject
+ CoordinatorResourceTestClient coordinatorClient;
+
+ @Test
+ public void testIndexData() throws Exception
+ {
+ final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData";
+ final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testIndexData-druidInputSource";
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix())
+ ) {
+
+ final Function transform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("0")
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ transform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_DATASOURCE,
+ reindexDatasource,
+ REINDEX_TASK,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_DATASOURCE,
+ reindexDatasourceWithDruidInputSource,
+ REINDEX_TASK_WITH_DRUID_INPUT_SOURCE,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ @Test
+ public void testReIndexDataWithTimestamp() throws Exception
+ {
+ final String reindexDatasource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp";
+ final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp-druidInputSource";
+ try (
+ final Closeable ignored1 = unloader(INDEX_WITH_TIMESTAMP_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix())
+ ) {
+ doIndexTest(
+ INDEX_WITH_TIMESTAMP_DATASOURCE,
+ INDEX_WITH_TIMESTAMP_TASK,
+ INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_WITH_TIMESTAMP_DATASOURCE,
+ reindexDatasource,
+ REINDEX_TASK,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_WITH_TIMESTAMP_DATASOURCE,
+ reindexDatasourceWithDruidInputSource,
+ REINDEX_TASK_WITH_DRUID_INPUT_SOURCE,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ @Test
+ public void testReIndexWithNonExistingDatasource() throws Exception
+ {
+ Pair dummyPair = new Pair<>(false, false);
+ final String fullBaseDatasourceName = "nonExistingDatasource2904";
+ final String fullReindexDatasourceName = "newDatasource123";
+
+ String taskSpec = StringUtils.replace(
+ getResourceAsString(REINDEX_TASK_WITH_DRUID_INPUT_SOURCE),
+ "%%DATASOURCE%%",
+ fullBaseDatasourceName
+ );
+ taskSpec = StringUtils.replace(
+ taskSpec,
+ "%%REINDEX_DATASOURCE%%",
+ fullReindexDatasourceName
+ );
+
+ // This method will also verify task is successful after task finish running
+ // We expect task to be successful even if the datasource to reindex does not exist
+ submitTaskAndWait(
+ taskSpec,
+ fullReindexDatasourceName,
+ false,
+ false,
+ dummyPair
+ );
+ }
+
+ @Test
+ public void testMERGEIndexData() throws Exception
+ {
+ final String reindexDatasource = MERGE_REINDEX_DATASOURCE + "-testMergeIndexData";
+ final String reindexDatasourceWithDruidInputSource = MERGE_REINDEX_DATASOURCE + "-testMergeReIndexData-druidInputSource";
+ try (
+ final Closeable ignored1 = unloader(MERGE_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix())
+ ) {
+ doIndexTest(
+ MERGE_INDEX_DATASOURCE,
+ MERGE_INDEX_TASK,
+ MERGE_INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ MERGE_INDEX_DATASOURCE,
+ reindexDatasource,
+ MERGE_REINDEX_TASK,
+ MERGE_REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ MERGE_INDEX_DATASOURCE,
+ reindexDatasourceWithDruidInputSource,
+ MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE,
+ MERGE_INDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ /**
+ * Test that task reports indicate the ingested segments were loaded before the configured timeout expired.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testIndexDataAwaitSegmentAvailability() throws Exception
+ {
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ final Function transform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("600000")
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ transform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(true, true)
+ );
+ }
+ }
+
+ /**
+ * Test that the task still succeeds if the segments do not become available before the configured wait timeout
+ * expires.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception
+ {
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED);
+ final Function transform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("1")
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ transform,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ false,
+ false,
+ new Pair<>(true, false)
+ );
+ coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT);
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load"
+ );
+ }
+ }
+
+
+ @Test
+ public void testIndexWithMergeColumnLimitData() throws Exception
+ {
+ try (
+ final Closeable ignored1 = unloader(INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ ) {
+ doIndexTest(
+ INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE,
+ INDEX_WITH_MERGE_COLUMN_LIMIT_TASK,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ @Test
+ public void testGetLockedIntervals() throws Exception
+ {
+ final String datasourceName = GET_LOCKED_INTERVALS + config.getExtraDatasourceNameSuffix();
+ try (final Closeable ignored = unloader(datasourceName)) {
+ // Submit an Indexing Task
+ submitIndexTask(INDEX_TASK, datasourceName);
+
+ // Wait until it acquires a lock
+ final Map minTaskPriority = Collections.singletonMap(datasourceName, 0);
+ final Map> lockedIntervals = new HashMap<>();
+ ITRetryUtil.retryUntilFalse(
+ () -> {
+ lockedIntervals.clear();
+ lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority));
+ return lockedIntervals.isEmpty();
+ },
+ "Verify Intervals are Locked"
+ );
+
+ // Verify the locked intervals for this datasource
+ Assert.assertEquals(lockedIntervals.size(), 1);
+ Assert.assertEquals(
+ lockedIntervals.get(datasourceName),
+ Collections.singletonList(Intervals.of("2013-08-31/2013-09-02"))
+ );
+
+ ITRetryUtil.retryUntilTrue(
+ () -> coordinator.areSegmentsLoaded(datasourceName),
+ "Segment Load"
+ );
+ }
+ }
+
+ @Test
+ public void testJsonFunctions() throws Exception
+ {
+ final String taskSpec = getResourceAsString("/indexer/json_path_index_task.json");
+
+ submitTaskAndWait(
+ taskSpec,
+ "json_path_index_test",
+ false,
+ true,
+ new Pair<>(false, false)
+ );
+
+ doTestQuery("json_path_index_test", "/indexer/json_path_index_queries.json");
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java
new file mode 100644
index 00000000000..73fc73d42d8
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.indexer;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json";
+ private static final String INDEX_QUERIES_ALL_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries.json";
+ private static final String INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries_only_data3.json";
+ private static final InputFormatDetails INPUT_FORMAT_DETAILS = InputFormatDetails.JSON;
+ private static final String ALL_DATA = "*" + INPUT_FORMAT_DETAILS.getFileExtension();
+ private static final String ONE_DAY_DATA = "wikipedia_index_data3" + INPUT_FORMAT_DETAILS.getFileExtension();
+
+ @Test
+ public void doIndexTestWithOverwriteAndDrop() throws Exception
+ {
+ final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
+ try (
+ final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix())
+ ) {
+ // Submit initial ingestion task
+ // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity
+ // dropExisting true or false does not matter as there is no existing segments
+ // Even though there are only 10 rows in the three input files and that maxRowsPerSegment is the default
+ // there will be four segments generated due to the distribution of rows in the input files and the
+ // fact that there are three sub-tasks (two of the tasks generate one segment each and the other
+ // task generates two segments)
+ submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, true);
+ verifySegmentsCountAndLoaded(indexDatasource, 4, 0);
+ doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
+ // Submit overwrite ingestion task with drop existing
+ // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"),
+ // however, the input data only contains one file with one day of data, 2013-09-02
+ // with dropExisting flag set to true, after the second ingestion, we should expect to
+ // have two segments one for the sub-task that ingested the single data file containing one day
+ // and one segment for the tombstone to cover the other segment previously generated by the first ingestion
+ submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, true);
+ verifySegmentsCountAndLoaded(indexDatasource, 2, 1);
+ doTestQuery(indexDatasource, INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE);
+ }
+ }
+
+ @Test
+ public void doIndexTestWithOverwriteAndNoDrop() throws Exception
+ {
+ final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
+ try (
+ final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix())
+ ) {
+ // Submit initial ingestion task
+ // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity
+ // dropExisting true or false does not matter as there is no existing segments
+ submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, false);
+ verifySegmentsCountAndLoaded(indexDatasource, 4, 0);
+ doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
+ // Submit overwrite ingestion task without drop existing
+ // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"),
+ // however, the input data only contains one day of data, 2013-09-02 (instead of 2 days of data)
+ // with dropExisting flag set to false, after the second ingestion, we should expect to have
+ // data from 2013-08-31/2013-09-01 remains unchanged and data for 2013-09-01/2013-09-02 from
+ // the second overwrite ingestion task
+ submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, false);
+ verifySegmentsCountAndLoaded(indexDatasource, 3, 0);
+ doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
+ }
+ }
+
+ private void submitIngestionTaskAndVerify(
+ String indexDatasource,
+ String fileFilter,
+ boolean dropExisting
+ ) throws Exception
+ {
+ Map inputFormatMap = new ImmutableMap.Builder()
+ .put("type", INPUT_FORMAT_DETAILS.getInputFormatType())
+ .build();
+ final Function sqlInputSourcePropsTransform = spec -> {
+ try {
+ spec = StringUtils.replace(
+ spec,
+ "%%PARTITIONS_SPEC%%",
+ jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null))
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_FILTER%%",
+ fileFilter
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_SOURCE_BASE_DIR%%",
+ "/resources/data/batch_index" + INPUT_FORMAT_DETAILS.getFolderSuffix()
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%INPUT_FORMAT%%",
+ jsonMapper.writeValueAsString(inputFormatMap)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%APPEND_TO_EXISTING%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%DROP_EXISTING%%",
+ jsonMapper.writeValueAsString(dropExisting)
+ );
+ spec = StringUtils.replace(
+ spec,
+ "%%FORCE_GUARANTEED_ROLLUP%%",
+ jsonMapper.writeValueAsString(false)
+ );
+ return spec;
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTest(
+ indexDatasource,
+ INDEX_TASK,
+ sqlInputSourcePropsTransform,
+ null,
+ false,
+ false,
+ true,
+ new Pair<>(false, false)
+ );
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java
new file mode 100644
index 00000000000..a5e4edc9580
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.indexer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.util.function.Function;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITSystemTableBatchIndexTaskTest extends AbstractITBatchIndexTest
+{
+ private static final Logger LOG = new Logger(ITSystemTableBatchIndexTaskTest.class);
+ private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json";
+ private static final String SYSTEM_QUERIES_RESOURCE = "/indexer/sys_segment_batch_index_queries.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_index_test";
+
+ @Test
+ public void testIndexData() throws Exception
+ {
+ LOG.info("Starting batch index sys table queries");
+ try (
+ final Closeable ignored = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix())
+ ) {
+
+ final Function transform = spec -> {
+ try {
+ return StringUtils.replace(
+ spec,
+ "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
+ jsonMapper.writeValueAsString("0")
+ );
+ }
+ catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ doIndexTestSqlTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK,
+ SYSTEM_QUERIES_RESOURCE,
+ transform
+ );
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java
new file mode 100644
index 00000000000..61cb33fd2c6
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.indexer;
+
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.testsEx.categories.BatchIndex;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITTransformTest extends AbstractITBatchIndexTest
+{
+ private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json";
+ private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json";
+ private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json";
+ private static final String INDEX_DATASOURCE = "wikipedia_index_test";
+
+ private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json";
+ private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json";
+ private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json";
+ private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test";
+
+ @Test
+ public void testIndexAndReIndexWithTransformSpec() throws IOException
+ {
+ final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-druidInputSource";
+
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix())
+ ) {
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK_WITH_INPUT_SOURCE,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_DATASOURCE,
+ reindexDatasourceWithDruidInputSource,
+ REINDEX_TASK_WITH_DRUID_INPUT_SOURCE,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ @Test
+ @Ignore
+ public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException
+ {
+ // TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed
+ // Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests!
+ final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData";
+ try (
+ final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
+ final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix())
+ ) {
+ doIndexTest(
+ INDEX_DATASOURCE,
+ INDEX_TASK_WITH_INPUT_SOURCE,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ doReindexTest(
+ INDEX_DATASOURCE,
+ reindexDatasource,
+ REINDEX_TASK,
+ REINDEX_QUERIES_RESOURCE,
+ new Pair<>(false, false)
+ );
+ }
+ }
+
+ @Test
+ @Ignore
+ public void testIndexWithFirehoseAndTransforms() throws IOException
+ {
+ // TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed
+ final String indexDatasource = INDEX_DATASOURCE + "-firehose";
+ try (
+ final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
+ ) {
+ doIndexTest(
+ indexDatasource,
+ INDEX_TASK_WITH_FIREHOSE,
+ INDEX_QUERIES_RESOURCE,
+ false,
+ true,
+ true,
+ new Pair<>(false, false)
+ );
+ }
+ }
+}
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
new file mode 100644
index 00000000000..735bc0a50ff
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java
@@ -0,0 +1,246 @@
+/*
+ * 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.common.config.NullHandling;
+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.tests.indexer.AbstractIndexerTest;
+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.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": %%NON_LEADER%%},
+ {"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": %%NON_LEADER%%},
+ */
+ 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");
+ }
+ working = StringUtils.replace(working, "%%NON_LEADER%%", String.valueOf(NullHandling.defaultLongValue()));
+ 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/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java
new file mode 100644
index 00000000000..60bd64ef847
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java
@@ -0,0 +1,328 @@
+/*
+ * 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.utils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import com.github.dockerjava.api.model.Container;
+import com.github.dockerjava.core.DockerClientBuilder;
+import com.github.dockerjava.core.command.ExecStartResultCallback;
+import com.github.dockerjava.netty.NettyDockerCmdExecFactory;
+import com.google.inject.Inject;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
+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.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.testing.IntegrationTestingConfig;
+import org.apache.druid.testing.guice.TestClient;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import java.io.ByteArrayOutputStream;
+import java.net.URL;
+import java.nio.channels.ClosedChannelException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+public class DruidClusterAdminClient
+{
+ public static final String COORDINATOR_DOCKER_CONTAINER_NAME = "/coordinator-one";
+ public static final String COORDINATOR_TWO_DOCKER_CONTAINER_NAME = "/coordinator-two";
+ public static final String HISTORICAL_DOCKER_CONTAINER_NAME = "/historical";
+ public static final String OVERLORD_DOCKER_CONTAINER_NAME = "/overlord-one";
+ public static final String OVERLORD_TWO_DOCKER_CONTAINER_NAME = "/overlord-two";
+ public static final String BROKER_DOCKER_CONTAINER_NAME = "/broker";
+ public static final String ROUTER_DOCKER_CONTAINER_NAME = "/router";
+ public static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "/middlemanager";
+
+ private static final Logger LOG = new Logger(DruidClusterAdminClient.class);
+
+ private final ObjectMapper jsonMapper;
+ private final HttpClient httpClient;
+ private IntegrationTestingConfig config;
+
+ @Inject
+ DruidClusterAdminClient(
+ ObjectMapper jsonMapper,
+ @TestClient HttpClient httpClient,
+ IntegrationTestingConfig config
+ )
+ {
+ this.jsonMapper = jsonMapper;
+ this.httpClient = httpClient;
+ this.config = config;
+ }
+
+ public void restartCoordinatorContainer()
+ {
+ restartDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartCoordinatorTwoContainer()
+ {
+ restartDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartHistoricalContainer()
+ {
+ restartDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartOverlordContainer()
+ {
+ restartDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartOverlordTwoContainer()
+ {
+ restartDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartBrokerContainer()
+ {
+ restartDockerContainer(BROKER_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartRouterContainer()
+ {
+ restartDockerContainer(ROUTER_DOCKER_CONTAINER_NAME);
+ }
+
+ public void restartMiddleManagerContainer()
+ {
+ restartDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME);
+ }
+
+ public void waitUntilCoordinatorReady()
+ {
+ waitUntilInstanceReady(config.getCoordinatorUrl());
+ postDynamicConfig(CoordinatorDynamicConfig.builder()
+ .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
+ .build());
+ }
+
+ public void waitUntilCoordinatorTwoReady()
+ {
+ waitUntilInstanceReady(config.getCoordinatorTwoUrl());
+ postDynamicConfig(CoordinatorDynamicConfig.builder()
+ .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
+ .build());
+ }
+
+ public void waitUntilOverlordTwoReady()
+ {
+ waitUntilInstanceReady(config.getOverlordTwoUrl());
+ }
+
+ public void waitUntilHistoricalReady()
+ {
+ waitUntilInstanceReady(config.getHistoricalUrl());
+ }
+
+ public void waitUntilIndexerReady()
+ {
+ waitUntilInstanceReady(config.getOverlordUrl());
+ }
+
+ public void waitUntilBrokerReady()
+ {
+ waitUntilInstanceReady(config.getBrokerUrl());
+ }
+
+ public void waitUntilRouterReady()
+ {
+ waitUntilInstanceReady(config.getRouterUrl());
+ }
+
+ public Pair runCommandInCoordinatorContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInCoordinatorTwoContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInHistoricalContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInOverlordContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInOverlordTwoContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInBrokerContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(BROKER_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInRouterContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(ROUTER_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInMiddleManagerContainer(String... cmd) throws Exception
+ {
+ return runCommandInDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME, cmd);
+ }
+
+ public Pair runCommandInDockerContainer(String serviceName, String... cmd) throws Exception
+ {
+ DockerClient dockerClient = newClient();
+ ByteArrayOutputStream stdout = new ByteArrayOutputStream();
+ ByteArrayOutputStream stderr = new ByteArrayOutputStream();
+ ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(findDockerContainer(dockerClient, serviceName))
+ .withAttachStderr(true)
+ .withAttachStdout(true)
+ .withCmd(cmd)
+ .exec();
+ dockerClient.execStartCmd(execCreateCmdResponse.getId())
+ .exec(new ExecStartResultCallback(stdout, stderr))
+ .awaitCompletion();
+
+ return new Pair<>(stdout.toString(StandardCharsets.UTF_8.name()), stderr.toString(StandardCharsets.UTF_8.name()));
+ }
+
+ public void restartDockerContainer(String serviceName)
+ {
+ DockerClient dockerClient = newClient();
+ dockerClient.restartContainerCmd(findDockerContainer(dockerClient, serviceName)).exec();
+ }
+
+ public void killAndRestartDockerContainer(String serviceName)
+ {
+ final DockerClient dockerClient = newClient();
+ final String containerId = findDockerContainer(dockerClient, serviceName);
+
+ dockerClient.killContainerCmd(containerId).withSignal("SIGKILL").exec();
+ dockerClient.startContainerCmd(containerId).exec();
+ }
+
+ private static DockerClient newClient()
+ {
+ return DockerClientBuilder
+ .getInstance()
+ .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()).withConnectTimeout(10 * 1000))
+ .build();
+ }
+
+ private String findDockerContainer(DockerClient dockerClient, String serviceName)
+ {
+
+ List containers = dockerClient.listContainersCmd().exec();
+ Optional containerName = containers
+ .stream()
+ .filter(container -> Arrays.asList(container.getNames()).contains(serviceName))
+ .findFirst()
+ .map(Container::getId);
+
+ if (!containerName.isPresent()) {
+ LOG.error("Cannot find docker container for " + serviceName);
+ throw new ISE("Cannot find docker container for " + serviceName);
+ }
+ return containerName.get();
+ }
+
+ private void waitUntilInstanceReady(final String host)
+ {
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ try {
+ StatusResponseHolder response = httpClient.go(
+ new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status/health", host))),
+ StatusResponseHandler.getInstance()
+ ).get();
+
+ LOG.info("%s %s", response.getStatus(), response.getContent());
+ return response.getStatus().equals(HttpResponseStatus.OK);
+ }
+ catch (Throwable e) {
+ //
+ // suppress stack trace logging for some specific exceptions
+ // to reduce excessive stack trace messages when waiting druid nodes to start up
+ //
+ if (e.getCause() instanceof ChannelException) {
+ Throwable channelException = e.getCause();
+
+ if (channelException.getCause() instanceof ClosedChannelException) {
+ LOG.error("Channel Closed");
+ } else if ("Channel disconnected".equals(channelException.getMessage())) {
+ // log message only
+ LOG.error("Channel disconnected");
+ } else {
+ // log stack trace for unknown exception
+ LOG.error(e, "Error while waiting for [%s] to be ready", host);
+ }
+ } else {
+ // log stack trace for unknown exception
+ LOG.error(e, "Error while waiting for [%s] to be ready", host);
+ }
+
+ return false;
+ }
+ },
+ "Waiting for instance to be ready: [" + host + "]"
+ );
+ }
+
+ private void postDynamicConfig(CoordinatorDynamicConfig coordinatorDynamicConfig)
+ {
+ ITRetryUtil.retryUntilTrue(
+ () -> {
+ try {
+ String url = StringUtils.format("%s/druid/coordinator/v1/config", config.getCoordinatorUrl());
+ StatusResponseHolder response = httpClient.go(
+ new Request(HttpMethod.POST, new URL(url)).setContent(
+ "application/json",
+ jsonMapper.writeValueAsBytes(coordinatorDynamicConfig)
+ ), StatusResponseHandler.getInstance()
+ ).get();
+
+ LOG.info("%s %s", response.getStatus(), response.getContent());
+ // if coordinator is not leader then it will return 307 instead of 200
+ return response.getStatus().equals(HttpResponseStatus.OK) || response.getStatus().equals(HttpResponseStatus.TEMPORARY_REDIRECT);
+ }
+ catch (Throwable e) {
+ LOG.error(e, "Error while posting dynamic config");
+ return false;
+ }
+ },
+ "Posting dynamic config after startup"
+ );
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml
new file mode 100644
index 00000000000..ac07bab9843
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml
@@ -0,0 +1,58 @@
+# 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 batch index test cluster.
+# See https://yaml.org/spec/1.2.2 for more about YAML
+include:
+ - /cluster/Common/zk-metastore.yaml
+
+druid:
+ coordinator:
+ instances:
+ - port: 8081
+ overlord:
+ instances:
+ - port: 8090
+ broker:
+ instances:
+ - port: 8082
+ router:
+ instances:
+ - port: 8888
+ historical:
+ instances:
+ - port: 8083
+ indexer:
+ instances:
+ - port: 8091
+
+# Properties to be set in the Properties object used in
+# Guice configuration in lieu of the server-side runtime.properties
+# file.
+#
+# druid.global.http.numMaxThreads avoids creating 40+ Netty threads.
+# We only ever use 1.
+# druid.test.config.dockerIp is used by some older test code. Remove
+# it when that code is updated.
+properties:
+ druid.global.http.numMaxThreads: 3
+ druid.broker.http.numMaxThreads: 3
+ druid.test.config.dockerIp: localhost
+ druid.test.config.cloudBucket: "new-it-framework"
+ druid.test.config.cloudPath: ""
+ docker.build.hadoop: true
+ start.hadoop.docker: true
+ override.config.path: "/Users/abhishekagrawal/pr_druid_it/druid/integration-tests-ex/it-azure-deep-storage/azure-config"
diff --git a/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml
new file mode 100644
index 00000000000..d676f530e90
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml
@@ -0,0 +1,40 @@
+# 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 batch index test cluster.
+# See https://yaml.org/spec/1.2.2 for more about YAML
+include:
+ - /cluster/Common/zk-metastore.yaml
+
+druid:
+ coordinator:
+ instances:
+ - port: 8081
+ overlord:
+ instances:
+ - port: 8090
+ broker:
+ instances:
+ - port: 8082
+ router:
+ instances:
+ - port: 8888
+ historical:
+ instances:
+ - port: 8083
+ indexer:
+ instances:
+ - port: 8091
diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml
new file mode 100644
index 00000000000..769c80989a4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Definition of the Kafka node used by some integration tests.
+# Include this into test-specific configurations.
+
+kafka:
+ instances:
+ - port: 9092
+ proxyPort: 9093
diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml
new file mode 100644
index 00000000000..fded303c113
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml
@@ -0,0 +1,28 @@
+# 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.
+
+# Defines the standard ZooKeeper and metadata storage (metastore)
+# services used in all Druid test clusters. Include this in to
+# test-specific configuration files.
+
+zk:
+ instances:
+ - port: 2181
+metastore:
+ connectURI: jdbc:mysql://:/druid
+ user: druid
+ password: diurd
+ instances:
+ - port: 3306
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
new file mode 100644
index 00000000000..5d99ffcbc52
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml
@@ -0,0 +1,204 @@
+# 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
new file mode 100644
index 00000000000..d5d60d4f297
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json
@@ -0,0 +1,39 @@
+[
+ {
+ "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": %%NON_LEADER%%},
+ {"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": %%NON_LEADER%%}
+ ]
+ },
+ {
+ "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/cases/src/test/resources/config-test/test.yaml b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml
new file mode 100644
index 00000000000..92745caea11
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml
@@ -0,0 +1,85 @@
+# 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 a simple dummy "cluster" for use in
+# testing the configuration mechanism itself.
+# See https://yaml.org/spec/1.2.2 for more about YAML
+include:
+ - /cluster/Common/zk-metastore.yaml
+ - /cluster/Common/kafka.yaml
+
+druid:
+ overlord:
+ instances:
+ - port: 8090
+ coordinator:
+ instances:
+ - port: 8081
+ historical:
+ instances:
+ - port: 8083
+ middlemanager:
+ instances:
+ - port: 8091
+ broker:
+ instances:
+ - port: 8082
+ router:
+ instances:
+ - port: 8888
+
+properties:
+ druid.test.config.dockerIp: localhost
+ druid.test.config.cloudRegion: myRegion
+ my.test.property: howdy
+
+settings:
+ cloudBucket: myBucket
+ cloud_path: myPath
+ cloud_region: hidden
+ druid_s3_accessKey: secret
+
+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"
+ }'
+ )
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json
new file mode 100644
index 00000000000..20c3b162ea6
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json
@@ -0,0 +1,82 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%JOIN_DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city",
+ {
+ "type": "long",
+ "name": "added"
+ },
+ {
+ "type": "long",
+ "name": "deleted"
+ }
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "sum_added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "sum_deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "YEAR",
+ "queryGranularity": "second"
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "local",
+ "baseDir": "/resources/data/union_query/",
+ "filter": "wikipedia_index_data*"
+ },
+ "appendToExisting": false,
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "indexSpec": {
+ "segmentLoader": {
+ "type": "broadcastJoinableMMapSegmentFactory",
+ "keyColumns": ["user", "language", "added", "deleted"]
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json
new file mode 100644
index 00000000000..1940cc6ea02
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json
@@ -0,0 +1,49 @@
+[
+ {
+ "description": "timeseries",
+ "query": {
+ "queryType": "timeseries",
+ "dataSource": "json_path_index_test",
+ "intervals": [
+ "1000/3000"
+ ],
+ "aggregations": [
+ {
+ "type": "longSum",
+ "name": "len",
+ "fieldName": "len"
+ },
+ {
+ "type": "longSum",
+ "name": "max",
+ "fieldName": "max"
+ },
+ {
+ "type": "longSum",
+ "name": "min",
+ "fieldName": "min"
+ },
+ {
+ "type": "longSum",
+ "name": "sum",
+ "fieldName": "sum"
+ }
+ ],
+ "granularity": {
+ "type": "all"
+ }
+ },
+ "expectedResults": [
+ {
+ "timestamp": "2013-08-31T01:02:33.000Z",
+ "result": {
+ "sum": 10,
+ "min": 0,
+ "len": 5,
+ "max": 4
+ }
+ }
+ ]
+ }
+]
+
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json
new file mode 100644
index 00000000000..2fd6990b116
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json
@@ -0,0 +1,80 @@
+{
+ "type": "index",
+ "dataSource": "json_path_index_test",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "json_path_index_test",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ {
+ "type": "long",
+ "name": "len"
+ },
+ {
+ "type": "long",
+ "name": "min"
+ },
+ {
+ "type": "long",
+ "name": "max"
+ },
+ {
+ "type": "long",
+ "name": "sum"
+ }
+ ],
+ "dimensionExclusions": [
+ "__time",
+ "timestamp"
+ ]
+ },
+ "metricsSpec": [],
+ "granularitySpec": {
+ "type": "uniform",
+ "segmentGranularity": "HOUR",
+ "queryGranularity": {
+ "type": "none"
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "inline",
+ "data": "{\"timestamp\": \"2013-08-31T01:02:33Z\", \"values\": [0,1,2,3,4] }"
+ },
+ "inputFormat": {
+ "type": "json",
+ "flattenSpec": {
+ "useFieldDiscovery": true,
+ "fields": [
+ {
+ "type": "path",
+ "name": "len",
+ "expr": "$.values.length()"
+ },
+ {
+ "type": "path",
+ "name": "min",
+ "expr": "$.values.min()"
+ },
+ {
+ "type": "path",
+ "name": "max",
+ "expr": "$.values.max()"
+ },
+ {
+ "type": "path",
+ "name": "sum",
+ "expr": "$.values.sum()"
+ }
+ ]
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json
new file mode 100644
index 00000000000..1868c4596d0
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json
@@ -0,0 +1,16 @@
+[
+ {
+ "query": {
+ "queryType": "segmentMetadata",
+ "dataSource": "%%DATASOURCE%%",
+ "analysisTypes": [
+ "%%ANALYSIS_TYPE%%"
+ ],
+ "intervals": [
+ "%%INTERVALS%%"
+ ]
+ },
+ "expectedResults": %%EXPECTED_QUERY_GRANULARITY%%,
+ "fieldsToTest": ["queryGranularity"]
+ }
+]
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json
new file mode 100644
index 00000000000..193e69fbc9c
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json
@@ -0,0 +1,80 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2015-09-12T00:00:00.000Z",
+ "result" : {
+ "minTime" : "2015-09-12T00:00:00.000Z",
+ "maxTime" : "2015-09-12T00:00:00.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "scan, all",
+ "query": {
+ "queryType": "scan",
+ "dataSource": "%%DATASOURCE%%",
+ "intervals": [
+ "2013-01-01/2020-01-02"
+ ],
+ "resultFormat":"compactedList"
+ },
+ "expectedResults": %%EXPECTED_SCAN_RESULT%%,
+ "fieldsToTest": ["events"]
+ },
+ {
+ "description": "roll up ratio",
+ "query": {
+ "queryType":"timeseries",
+ "dataSource":{
+ "type":"table",
+ "name":"%%DATASOURCE%%"
+ },
+ "intervals":{
+ "type":"intervals",
+ "intervals":[
+ "2013-01-01/2020-01-02"
+ ]
+ },
+ "granularity":{
+ "type":"all"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"a0"
+ },
+ {
+ "type":"longSum",
+ "name":"a1",
+ "fieldName":"count",
+ "expression":null
+ }
+ ],
+ "postAggregations":[
+ {
+ "type":"expression",
+ "name":"p0",
+ "expression":"((\"a0\" * 1.00) / \"a1\")",
+ "ordering":null
+ }
+ ]
+ },
+ "expectedResults": [
+ {
+ "timestamp" : "2015-09-12T00:00:00.000Z",
+ "result" : {
+ "a1" : %%EXPECTED_SUM_COUNT%%,
+ "p0" : %%EXPECTED_ROLLUP_RATIO%%,
+ "a0" : %%EXPECTED_NUM_ROW%%
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json
new file mode 100644
index 00000000000..3a21a856ac6
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json
@@ -0,0 +1,57 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "inline",
+ "data": "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n"
+ },
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "partitionsSpec": {
+ "type": "dynamic",
+ "maxRowsPerSegment": 3,
+ "maxTotalRows": 3
+ },
+ "maxRowsInMemory": 3
+ },
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "time",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "dimB",
+ "dimA",
+ "dimC",
+ "dimD",
+ "dimE",
+ "dimF"
+ ]
+ },
+ "granularitySpec": {
+ "queryGranularity": "hour",
+ "rollup": true,
+ "segmentGranularity": "hour"
+ },
+ "metricsSpec": [
+ {
+ "name": "count",
+ "type": "count"
+ },
+ {
+ "name": "sum_metA",
+ "type": "longSum",
+ "fieldName": "metA"
+ }
+ ]
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json
new file mode 100644
index 00000000000..9416a3f6bda
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json
@@ -0,0 +1,19 @@
+{
+ "type": "compact",
+ "dataSource": "%%DATASOURCE%%",
+ "dimensionsSpec": {
+ "dimensions": %%DIMENSION_NAMES%%
+ },
+ "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z",
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxRowsPerSegment": 3,
+ "maxRowsInMemory": 3,
+ "maxNumConcurrentSubTasks": 2,
+ "partitionsSpec": {
+ "type": "hashed",
+ "numShards": 1
+ },
+ "forceGuaranteedRollup": true
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json
new file mode 100644
index 00000000000..a149d7a2512
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json
@@ -0,0 +1,16 @@
+{
+ "type": "compact",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z",
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxRowsPerSegment": 3,
+ "maxRowsInMemory": 3,
+ "maxNumConcurrentSubTasks": 2,
+ "partitionsSpec": {
+ "type": "hashed",
+ "numShards": 1
+ },
+ "forceGuaranteedRollup": true
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json
new file mode 100644
index 00000000000..931ad895778
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json
@@ -0,0 +1,35 @@
+[
+ {
+ "query": {
+ "query": "SELECT count(*) FROM sys.segments WHERE datasource LIKE 'wikipedia_index_test%'"
+ },
+ "expectedResults": [
+ {
+ "EXPR$0": 4
+ }
+ ]
+ },
+ {
+ "query": {
+ "query": "SELECT server_type FROM sys.servers WHERE tier IS NOT NULL AND server_type <> 'indexer'"
+ },
+ "expectedResults": [
+ {
+ "server_type":"historical"
+ },
+ {
+ "server_type":"broker"
+ }
+ ]
+ },
+ {
+ "query": {
+ "query": "SELECT status AS status FROM sys.tasks WHERE datasource LIKE 'wikipedia_index_test%' GROUP BY 1"
+ },
+ "expectedResults": [
+ {
+ "status": "SUCCESS"
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json
new file mode 100644
index 00000000000..8c4242c91e2
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json
@@ -0,0 +1,87 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "%%INPUT_SOURCE_TYPE%%",
+ "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%%
+ },
+ "inputFormat": {
+ "type": "%%INPUT_FORMAT_TYPE%%"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "partitionsSpec": %%PARTITIONS_SPEC%%,
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json
new file mode 100644
index 00000000000..742ab209f72
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json
@@ -0,0 +1,81 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "%%INPUT_SOURCE_TYPE%%",
+ "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%%
+ },
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig": {
+ "type": "index"
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json
new file mode 100644
index 00000000000..302d2fea284
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json
@@ -0,0 +1,141 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query": {
+ "queryType": "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults": [
+ {
+ "timestamp": "2013-08-31T01:02:33.000Z",
+ "result": {
+ "minTime": "2013-08-31T01:02:33.000Z",
+ "maxTime": "2013-09-01T18:22:39.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query": {
+ "queryType": "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity": "day",
+ "intervals": [
+ "2013-09-01T00:00/2013-09-02T00:00"
+ ],
+ "filter": null,
+ "aggregations": [
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type": "thetaSketch",
+ "name": "approxCountTheta",
+ "fieldName": "thetaSketch",
+ "size": 16384,
+ "shouldFinalize": true,
+ "isInputThetaSketch": false,
+ "errorBoundsStdDev": null
+ },
+ {
+ "type": "quantilesDoublesSketch",
+ "name": "quantilesSketch",
+ "fieldName": "quantilesDoublesSketch",
+ "k": 128
+ }
+ ]
+ },
+ "expectedResults": [
+ {
+ "timestamp": "2013-09-01T00:00:00.000Z",
+ "result": {
+ "quantilesSketch": 6,
+ "approxCountTheta": 6.0,
+ "approxCountHLL": 6
+ }
+ }
+ ]
+ },
+ {
+ "description": "having spec on post aggregation",
+ "query": {
+ "queryType": "groupBy",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity": "day",
+ "dimensions": [
+ "page"
+ ],
+ "filter": {
+ "type": "selector",
+ "dimension": "language",
+ "value": "zh"
+ },
+ "aggregations": [
+ {
+ "type": "count",
+ "name": "rows"
+ },
+ {
+ "type": "longSum",
+ "fieldName": "added",
+ "name": "added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type": "arithmetic",
+ "name": "added_count_times_ten",
+ "fn": "*",
+ "fields": [
+ {
+ "type": "fieldAccess",
+ "name": "added_count",
+ "fieldName": "added_count"
+ },
+ {
+ "type": "constant",
+ "name": "const",
+ "value": 10
+ }
+ ]
+ }
+ ],
+ "having": {
+ "type": "greaterThan",
+ "aggregation": "added_count_times_ten",
+ "value": 9000
+ },
+ "intervals": [
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults": [
+ {
+ "version": "v1",
+ "timestamp": "2013-08-31T00:00:00.000Z",
+ "event": {
+ "added_count_times_ten": 9050.0,
+ "page": "Crimson Typhoon",
+ "added_count": 905,
+ "rows": 1
+ }
+ },
+ {
+ "version": "v1",
+ "timestamp": "2013-08-31T00:00:00.000Z",
+ "event": {
+ "added_count_times_ten": 9770.0,
+ "page": "Gypsy Danger",
+ "added_count": 977,
+ "rows": 1
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json
new file mode 100644
index 00000000000..1e7deffc6b5
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json
@@ -0,0 +1,95 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "combining",
+ "delegates": [
+ {
+ "type": "local",
+ "baseDir": "/resources/indexer",
+ "filter": "wikipedia_combining_index_data.json"
+ },
+ {
+ "type": "ingestSegment",
+ "dataSource": "%%COMBINING_DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ ]
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json
new file mode 100644
index 00000000000..8e1d0941bf0
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json
@@ -0,0 +1,98 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "combining",
+ "delegates": [
+ {
+ "type": "local",
+ "baseDir": "/resources/indexer",
+ "filter": "wikipedia_combining_index_data.json"
+ },
+ {
+ "type": "druid",
+ "dataSource": "%%COMBINING_DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ ]
+ },
+ "appendToExisting": %%APPEND_TO_EXISTING%%,
+ "inputFormat": %%INPUT_FORMAT%%
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 4,
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ },
+ "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%,
+ "partitionsSpec": %%PARTITIONS_SPEC%%
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json
new file mode 100644
index 00000000000..fb620c11aa2
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json
@@ -0,0 +1,14 @@
+{
+ "type" : "compact",
+ "dataSource" : "%%DATASOURCE%%",
+ "ioConfig" : {
+ "type": "compact",
+ "inputSpec": {
+ "type": "interval",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ },
+ "context" : {
+ "storeCompactionState" : true
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json
new file mode 100644
index 00000000000..4fddeaa8aa3
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json
@@ -0,0 +1,15 @@
+{
+ "type" : "compact",
+ "dataSource" : "%%DATASOURCE%%",
+ "ioConfig" : {
+ "type": "compact",
+ "inputSpec": {
+ "type": "interval",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ },
+ "granularitySpec": %%GRANULARITY_SPEC%%,
+ "context" : {
+ "storeCompactionState" : true
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json
new file mode 100644
index 00000000000..7a2c016dc9d
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json
@@ -0,0 +1,15 @@
+{
+ "type" : "compact",
+ "dataSource" : "%%DATASOURCE%%",
+ "ioConfig" : {
+ "type": "compact",
+ "inputSpec": {
+ "type": "interval",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ },
+ "segmentGranularity": "%%SEGMENT_GRANULARITY%%",
+ "context" : {
+ "storeCompactionState" : true
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json
new file mode 100644
index 00000000000..586da63e3db
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json
@@ -0,0 +1,143 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-09-01T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type":"thetaSketch",
+ "name":"approxCountTheta",
+ "fieldName":"thetaSketch",
+ "size":16384,
+ "shouldFinalize":true,
+ "isInputThetaSketch":false,
+ "errorBoundsStdDev":null
+ },
+ {
+ "type":"quantilesDoublesSketch",
+ "name":"quantilesSketch",
+ "fieldName":"quantilesDoublesSketch",
+ "k":128
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "quantilesSketch":10,
+ "approxCountTheta":5.0,
+ "approxCountHLL":5
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 18100.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 1810,
+ "rows" : 2
+ }
+ } ]
+ },
+ {
+ "description": "timeseries, count aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "count",
+ "name": "rows"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "rows":10
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json
new file mode 100644
index 00000000000..eaa9592ca26
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json
@@ -0,0 +1,143 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-09-01T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type":"thetaSketch",
+ "name":"approxCountTheta",
+ "fieldName":"thetaSketch",
+ "size":16384,
+ "shouldFinalize":true,
+ "isInputThetaSketch":false,
+ "errorBoundsStdDev":null
+ },
+ {
+ "type":"quantilesDoublesSketch",
+ "name":"quantilesSketch",
+ "fieldName":"quantilesDoublesSketch",
+ "k":128
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "quantilesSketch":10,
+ "approxCountTheta":5.0,
+ "approxCountHLL":5
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 18100.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 1810,
+ "rows" : 1
+ }
+ } ]
+ },
+ {
+ "description": "timeseries, count aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "count",
+ "name": "rows"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "rows":5
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json
new file mode 100644
index 00000000000..f0cbb1cb859
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json
@@ -0,0 +1,47 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2016-06-27T00:00:11.000Z",
+ "result" : {
+ "minTime" : "2016-06-27T00:00:11.000Z",
+ "maxTime" : "2016-06-27T21:31:02.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "simple aggr",
+ "query":{
+ "queryType" : "topN",
+ "dataSource" : "%%DATASOURCE%%",
+ "intervals" : ["2016-06-27/2016-06-28"],
+ "granularity" : "all",
+ "dimension" : "page",
+ "metric" : "count",
+ "threshold" : 3,
+ "aggregations" : [
+ {
+ "type" : "count",
+ "name" : "count"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2016-06-27T00:00:11.000Z",
+ "result" :
+ [
+ {"count":58,"page":"Copa América Centenario"},
+ {"count":32,"page":"User:Cyde/List of candidates for speedy deletion/Subpage"},
+ {"count":32,"page":"Wikipedia:Administrators' noticeboard/Incidents"}
+ ]
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json
new file mode 100644
index 00000000000..0ac4e05af85
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json
@@ -0,0 +1,74 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2016-06/P1M" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "http",
+ "uris": ["https://druid.apache.org/data/wikipedia.json.gz", "https://druid.apache.org/data/wikipedia.json.gz"]
+ },
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "partitionsSpec": {
+ "type": "dynamic"
+ },
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json
new file mode 100644
index 00000000000..928effe65e9
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json
@@ -0,0 +1,150 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-09-01T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type":"thetaSketch",
+ "name":"approxCountTheta",
+ "fieldName":"thetaSketch",
+ "size":16384,
+ "shouldFinalize":true,
+ "isInputThetaSketch":false,
+ "errorBoundsStdDev":null
+ },
+ {
+ "type":"quantilesDoublesSketch",
+ "name":"quantilesSketch",
+ "fieldName":"quantilesDoublesSketch",
+ "k":128
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "quantilesSketch":5,
+ "approxCountTheta":5.0,
+ "approxCountHLL":5
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905,
+ "rows" : 1
+ }
+ } ]
+ },
+ {
+ "description": "timeseries, stringFirst/stringLast aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "user"
+ },
+ {
+ "type":"stringLast",
+ "name":"last_user",
+ "fieldName":"user"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "first_user":"nuclear",
+ "last_user":"stringer"
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json
new file mode 100644
index 00000000000..6ed715b89eb
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json
@@ -0,0 +1,150 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:00:00.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:00:00.000Z",
+ "maxTime" : "2013-09-01T12:00:00.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type":"thetaSketch",
+ "name":"approxCountTheta",
+ "fieldName":"thetaSketch",
+ "size":16384,
+ "shouldFinalize":true,
+ "isInputThetaSketch":false,
+ "errorBoundsStdDev":null
+ },
+ {
+ "type":"quantilesDoublesSketch",
+ "name":"quantilesSketch",
+ "fieldName":"quantilesDoublesSketch",
+ "k":128
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "quantilesSketch":5,
+ "approxCountTheta":5.0,
+ "approxCountHLL":5
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905,
+ "rows" : 1
+ }
+ } ]
+ },
+ {
+ "description": "timeseries, stringFirst/stringLast aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "user"
+ },
+ {
+ "type":"stringLast",
+ "name":"last_user",
+ "fieldName":"user"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "first_user":"nuclear",
+ "last_user":"stringer"
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json
new file mode 100644
index 00000000000..f0cfba67735
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json
@@ -0,0 +1,62 @@
+[
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page",
+ "city"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"language-zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"triple-added",
+ "name":"added_count"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"delta",
+ "name":"delta_sum"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 27150.0,
+ "page" : "Crimson Typhoon",
+ "city" : "Taiyuan",
+ "added_count" : 2715,
+ "delta_sum" : 900,
+ "rows" : 1
+ }
+ } ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json
new file mode 100644
index 00000000000..25f9203ac1b
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json
@@ -0,0 +1,150 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-01-01T00:00:00.000Z",
+ "result" : {
+ "minTime" : "2013-01-01T00:00:00.000Z",
+ "maxTime" : "2013-01-01T00:00:00.000Z"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries, datasketch aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-01-01T00:00/2014-01-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "HLLSketchMerge",
+ "name": "approxCountHLL",
+ "fieldName": "HLLSketchBuild",
+ "lgK": 12,
+ "tgtHllType": "HLL_4",
+ "round": true
+ },
+ {
+ "type":"thetaSketch",
+ "name":"approxCountTheta",
+ "fieldName":"thetaSketch",
+ "size":16384,
+ "shouldFinalize":true,
+ "isInputThetaSketch":false,
+ "errorBoundsStdDev":null
+ },
+ {
+ "type":"quantilesDoublesSketch",
+ "name":"quantilesSketch",
+ "fieldName":"quantilesDoublesSketch",
+ "k":128
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-01-01T00:00:00.000Z",
+ "result" : {
+ "quantilesSketch":10,
+ "approxCountTheta":5.0,
+ "approxCountHLL":5
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-01-01T00:00/2014-01-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-01-01T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 18100.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 1810,
+ "rows" : 1
+ }
+ } ]
+ },
+ {
+ "description": "timeseries, stringFirst/stringLast aggs, all",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-01-01T00:00/2014-01-01T00:00"
+ ],
+ "filter":null,
+ "aggregations":[
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "user"
+ },
+ {
+ "type":"stringLast",
+ "name":"last_user",
+ "fieldName":"user"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-01-01T00:00:00.000Z",
+ "result" : {
+ "first_user":"masterYi",
+ "last_user":"speed"
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json
new file mode 100644
index 00000000000..77854ea0acf
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json
@@ -0,0 +1,56 @@
+[
+ {
+ "description": "rows count",
+ "query":{
+ "queryType" : "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter": {
+ "type": "selector",
+ "dimension": "language",
+ "value": "en",
+ "extractionFn": null
+ },
+ "aggregations":[
+ {
+ "type": "count",
+ "name": "count"
+ }
+ ]
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "result" : {
+ "count":%%EXPECTED_COUNT_RESULT%%
+ }
+ }
+ ]
+ },
+ {
+ "description": "scan with filter",
+ "query":{
+ "queryType" : "scan",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter": {
+ "type": "selector",
+ "dimension": "language",
+ "value": "en",
+ "extractionFn": null
+ },
+ "columns": [
+ %%FIELD_TO_QUERY%%
+ ],
+ "resultFormat":"compactedList"
+ },
+ "expectedResults": %%EXPECTED_SCAN_RESULT%%,
+ "fieldsToTest": ["events"]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json
new file mode 100644
index 00000000000..00bf7721f2b
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json
@@ -0,0 +1,89 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "auto"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "local",
+ "files": [
+ "/resources/data/batch_index/json/wikipedia_index_data1.json",
+ "/resources/data/batch_index/json/wikipedia_index_data2.json",
+ "/resources/data/batch_index/json/wikipedia_index_data3.json"
+ ]
+ },
+ "inputFormat": {
+ "type": "json"
+ },
+ "appendToExisting": false
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3,
+ "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json
new file mode 100644
index 00000000000..1fa8b4eba32
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json
@@ -0,0 +1,86 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "DAY",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3,
+ "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json
new file mode 100644
index 00000000000..544d191d925
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json
@@ -0,0 +1,71 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": %%GRANULARITYSPEC%%,
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ {"type": "string", "name": "language", "createBitmapIndex": false}
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 10,
+ "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json
new file mode 100644
index 00000000000..e9bcbfd0aae
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json
@@ -0,0 +1,103 @@
+{
+ "type" : "index",
+ "spec" : {
+ "dataSchema" : {
+ "dataSource" : "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "triple-added",
+ "fieldName": "triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "transformSpec": {
+ "transforms": [
+ {
+ "type": "expression",
+ "name": "language",
+ "expression": "concat('language-', language)"
+ },
+ {
+ "type": "expression",
+ "name": "triple-added",
+ "expression": "added * 3"
+ }
+ ]
+ }
+ },
+ "ioConfig" : {
+ "type" : "index",
+ "inputSource" : {
+ "type" : "local",
+ "baseDir" : "/resources/data/batch_index/json",
+ "filter" : "wikipedia_index_data*"
+ },
+ "inputFormat" : {
+ "type" : "json"
+ },
+ "appendToExisting" : false
+ },
+ "tuningConfig" : {
+ "type" : "index",
+ "maxRowsPerSegment" : 5000000,
+ "maxRowsInMemory" : 25000
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json
new file mode 100644
index 00000000000..9f3128fb303
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json
@@ -0,0 +1,103 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "triple-added",
+ "fieldName": "triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ },
+ "transformSpec": {
+ "transforms": [
+ {
+ "type": "expression",
+ "name": "language",
+ "expression": "concat('language-', language)"
+ },
+ {
+ "type": "expression",
+ "name": "triple-added",
+ "expression": "added * 3"
+ }
+ ]
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json
new file mode 100644
index 00000000000..35b115c9f19
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json
@@ -0,0 +1,86 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3,
+ "maxColumnsToMerge" : 30
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json
new file mode 100644
index 00000000000..ee0fd73021a
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json
@@ -0,0 +1,89 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "local",
+ "filter" : "%%INPUT_SOURCE_FILTER%%",
+ "baseDir": "%%INPUT_SOURCE_BASE_DIR%%"
+ },
+ "appendToExisting": %%APPEND_TO_EXISTING%%,
+ "dropExisting": %%DROP_EXISTING%%,
+ "inputFormat": %%INPUT_FORMAT%%
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 4,
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ },
+ "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%,
+ "partitionsSpec": %%PARTITIONS_SPEC%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json
new file mode 100644
index 00000000000..ab4674999b5
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json
@@ -0,0 +1,42 @@
+[
+ {
+ "description": "groupby, stringFirst/stringLast rollup aggs, all",
+ "query":{
+ "queryType" : "groupBy",
+ "dataSource": "%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "continent"
+ ],
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"continent",
+ "value":"Asia"
+ },
+ "aggregations":[
+ {
+ "type": "stringFirst",
+ "name": "earliest_user",
+ "fieldName": "first_user"
+ },
+ {
+ "type":"stringLast",
+ "name":"latest_user",
+ "fieldName":"last_user"
+ }
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "continent":"Asia",
+ "earliest_user":"masterYi",
+ "latest_user":"stringer"
+ }
+ } ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json
new file mode 100644
index 00000000000..43264a8c675
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json
@@ -0,0 +1,70 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "user"
+ },
+ {
+ "type": "stringLast",
+ "name": "last_user",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "DAY",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "continent"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 5,
+ "maxRowsInMemory": 2
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json
new file mode 100644
index 00000000000..9daae62c8d4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json
@@ -0,0 +1,63 @@
+{
+ "type": "index",
+ "spec": {
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "druid",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "partitionsSpec": {
+ "type": "dynamic"
+ }
+ },
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "granularitySpec": {
+ "type": "uniform",
+ "queryGranularity": "DAY",
+ "segmentGranularity": "DAY"
+ },
+ "timestampSpec": {
+ "column": "__time",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "continent"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "first_user"
+ },
+ {
+ "type": "stringLast",
+ "name": "last_user",
+ "fieldName": "last_user"
+ }
+ ]
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json
new file mode 100644
index 00000000000..127461dd117
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json
@@ -0,0 +1,65 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "type": "stringFirst",
+ "name": "first_user",
+ "fieldName": "first_user"
+ },
+ {
+ "type": "stringLast",
+ "name": "last_user",
+ "fieldName": "last_user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "DAY",
+ "intervals" : [ "2013-08-31/2013-09-01" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "continent"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "ingestSegment",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index"
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json
new file mode 100644
index 00000000000..12b87977743
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json
@@ -0,0 +1,82 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "%%INPUT_SOURCE_TYPE%%",
+ "properties": %%INPUT_SOURCE_CONFIG%%,
+ "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%%
+ },
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig": {
+ "type": "index"
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json
new file mode 100644
index 00000000000..46af17af598
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json
@@ -0,0 +1,64 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "dimensionsSpec": {
+ "dimensionExclusions": [
+ "robot",
+ "continent"
+ ]
+ },
+ "timestampSpec": {
+ "column": "ignored-see-ignoreTimestampSpecForDruidInputSource",
+ "format": "iso"
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals": [
+ "2013-08-31/2013-09-02"
+ ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "druid",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-02"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
+ "partitionsSpec": %%PARTITIONS_SPEC%%,
+ "splitHintSpec": {
+ "type":"segments",
+ "maxInputSegmentBytesPerTask": 1
+ }
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json
new file mode 100644
index 00000000000..9618ba9e9b6
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json
@@ -0,0 +1,71 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-09-01T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905,
+ "rows" : 1
+ }
+ } ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json
new file mode 100644
index 00000000000..e83b1109da0
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json
@@ -0,0 +1,77 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxSplitSize": 1
+ },
+ "partitionsSpec": %%PARTITIONS_SPEC%%,
+ "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json
new file mode 100644
index 00000000000..b6b0d54c2aa
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json
@@ -0,0 +1,88 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "inputSource": {
+ "type": "sql",
+ "database": {
+ "type": "mysql",
+ "connectorConfig": {
+ "connectURI": "jdbc:mysql://druid-metadata-storage/sqlinputsource",
+ "user": "druid",
+ "password": "diurd"
+ }
+ },
+ "sqls": %%SQL_QUERY%%
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "partitionsSpec": %%PARTITIONS_SPEC%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json
new file mode 100644
index 00000000000..e995a0f299e
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json
@@ -0,0 +1,69 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals": [
+ "2013-08-31/2013-09-02"
+ ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format": "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensionExclusions": [
+ "robot",
+ "continent"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "firehose": {
+ "type": "ingestSegment",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-02",
+ "maxInputSegmentBytesPerTask": 1
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ },
+ "partitionsSpec": %%PARTITIONS_SPEC%%
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json
new file mode 100644
index 00000000000..bbbeca9b58a
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json
@@ -0,0 +1,18 @@
+[
+ {
+ "description": "timeseries, 1 agg, all should only show data2",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T11:58:39.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T11:58:39.000Z",
+ "maxTime" : "2013-09-01T01:02:33.000Z"
+ }
+ }
+ ]
+ }
+]
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json
new file mode 100644
index 00000000000..9ab153c6862
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json
@@ -0,0 +1,75 @@
+{
+ "type": "index_parallel",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second"
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index_parallel",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data2*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index_parallel",
+ "maxNumConcurrentSubTasks": 10,
+ "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
+ "splitHintSpec": {
+ "type": "maxSize",
+ "maxNumFiles": 1
+ },
+ "partitionsSpec": %%PARTITIONS_SPEC%%
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json
new file mode 100644
index 00000000000..46d5ec4395a
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json
@@ -0,0 +1,87 @@
+[
+ {
+ "description": "timeBoundary",
+ "query": {
+ "queryType":"timeBoundary",
+ "dataSource":"%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%",
+ "result": {
+ "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%",
+ "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries",
+ "query": {
+ "queryType": "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ],
+ "granularity": "all",
+ "aggregations": [
+ {"type": "longSum", "fieldName": "count", "name": "edit_count"},
+ {"type": "doubleSum", "fieldName": "added", "name": "chars_added"}
+ ]
+ },
+ "expectedResults": [
+ {
+ "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%",
+ "result" : {
+ "chars_added" : 1642.0,
+ "edit_count" : 22
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"minute",
+ "dimensions":[
+ "page"
+ ],
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905,
+ "rows" : 1
+ }
+ } ]
+ }
+]
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json
new file mode 100644
index 00000000000..9e773609cb7
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json
@@ -0,0 +1,94 @@
+{
+ "type": "index_realtime_appenderator",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "minute",
+ "queryGranularity": "second"
+ },
+ "parser": {
+ "type": "map",
+ "parseSpec": {
+ "format": "tsv",
+ "columns": [
+ "timestamp",
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city",
+ "added",
+ "deleted",
+ "delta"
+ ],
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "realtime",
+ "firehose": {
+ "type": "timed",
+ "shutoffTime": "#SHUTOFFTIME",
+ "delegate": {
+ "type": "receiver",
+ "serviceName": "eventReceiverServiceName",
+ "bufferSize": 100000
+ }
+ }
+ },
+ "tuningConfig": {
+ "type": "realtime_appenderator",
+ "maxRowsInMemory": 1,
+ "intermediatePersistPeriod": "PT1M"
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json
new file mode 100644
index 00000000000..e5fe33b6b15
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json
@@ -0,0 +1,147 @@
+[
+ {
+ "description": "timeBoundary",
+ "query": {
+ "queryType":"timeBoundary",
+ "dataSource":"%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%",
+ "result": {
+ "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%",
+ "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%"
+ }
+ }
+ ]
+ },
+ {
+ "description": "timeseries",
+ "query": {
+ "queryType": "timeseries",
+ "dataSource": "%%DATASOURCE%%",
+ "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ],
+ "granularity": "all",
+ "aggregations": [
+ {"type": "longSum", "fieldName": "count", "name": "edit_count"},
+ {"type": "doubleSum", "fieldName": "added", "name": "chars_added"}
+ ]
+ },
+ "expectedResults": [
+ {
+ "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%",
+ "result" : {
+ "chars_added" : 1595.0,
+ "edit_count" : 21
+ }
+ }
+ ]
+ },
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"minute",
+ "dimensions":[
+ "page"
+ ],
+ "aggregations":[
+ {
+ "type":"count",
+ "name":"rows"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905,
+ "rows" : 1
+ }
+ } ]
+ },
+ {
+ "description": "topN, 1 agg, join to inline",
+ "query": {
+ "queryType": "topN",
+ "dataSource": {
+ "type": "join",
+ "left": "%%DATASOURCE%%",
+ "right": {
+ "type": "inline",
+ "columnNames": ["language", "lookupLanguage"],
+ "columnTypes": ["string", "string"],
+ "rows": [
+ ["en", "inline join!"]
+ ]
+ },
+ "rightPrefix": "j.",
+ "condition": "language == \"j.language\"",
+ "joinType": "LEFT"
+ },
+ "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ],
+ "granularity": "all",
+ "virtualColumns": [
+ {
+ "type": "expression",
+ "name": "lookupLanguage",
+ "expression": "nvl(\"j.lookupLanguage\", \"language\")",
+ "outputType": "string"
+ }
+ ],
+ "aggregations": [
+ {
+ "type": "longSum",
+ "name": "count",
+ "fieldName": "count"
+ }
+ ],
+ "dimension": "lookupLanguage",
+ "metric": "count",
+ "threshold": 3
+ },
+ "expectedResults": [
+ {
+ "timestamp": "%%TIMESERIES_RESPONSE_TIMESTAMP%%",
+ "result": [
+ {
+ "lookupLanguage": "inline join!",
+ "count": 14
+ },
+ {
+ "lookupLanguage": "ja",
+ "count": 3
+ },
+ {
+ "lookupLanguage": "ru",
+ "count": 3
+ }
+ ]
+ }
+ ]
+ }
+]
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json
new file mode 100644
index 00000000000..5f48162c488
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json
@@ -0,0 +1,98 @@
+{
+ "type": "index_realtime",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "minute",
+ "queryGranularity": "second"
+ },
+ "parser": {
+ "type": "map",
+ "parseSpec": {
+ "format": "tsv",
+ "columns": [
+ "timestamp",
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city",
+ "added",
+ "deleted",
+ "delta"
+ ],
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "realtime",
+ "firehose": {
+ "type": "timed",
+ "shutoffTime": "#SHUTOFFTIME",
+ "delegate": {
+ "type": "receiver",
+ "serviceName": "eventReceiverServiceName",
+ "bufferSize": 100000
+ }
+ }
+ },
+ "tuningConfig": {
+ "type": "realtime",
+ "maxRowsInMemory": 1,
+ "intermediatePersistPeriod": "PT1M",
+ "windowPeriod": "PT1M",
+ "rejectionPolicy": {
+ "type": "serverTime"
+ }
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json
new file mode 100644
index 00000000000..cf2415c2b45
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json
@@ -0,0 +1,51 @@
+{
+ "type": "index",
+ "spec": {
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "druid",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "partitionsSpec": {
+ "type": "dynamic"
+ }
+ },
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "granularitySpec": {
+ "type": "uniform",
+ "queryGranularity": "SECOND",
+ "segmentGranularity": "DAY"
+ },
+ "timestampSpec": {
+ "column": "__time",
+ "format": "millis"
+ },
+ "dimensionsSpec": {
+ "dimensionExclusions" : ["robot", "continent"]
+ },
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ]
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json
new file mode 100644
index 00000000000..2c2b0372a56
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json
@@ -0,0 +1,106 @@
+{
+ "type": "index",
+ "spec": {
+ "ioConfig": {
+ "type": "index",
+ "inputSource": {
+ "type": "druid",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "partitionsSpec": {
+ "type": "dynamic"
+ }
+ },
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "granularitySpec": {
+ "type": "uniform",
+ "queryGranularity": "SECOND",
+ "segmentGranularity": "DAY"
+ },
+ "timestampSpec": {
+ "column": "__time",
+ "format": "millis"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "page",
+ "newPage",
+ "anonymous",
+ "namespace",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "transformSpec": {
+ "transforms": [
+ {
+ "type": "expression",
+ "name": "newPage",
+ "expression": "page"
+ },
+ {
+ "type": "expression",
+ "name": "city",
+ "expression": "concat('city-', city)"
+ },
+ {
+ "type": "expression",
+ "name": "one-plus-triple-added",
+ "expression": "\"triple-added\" + 1"
+ },
+ {
+ "type": "expression",
+ "name": "delta",
+ "expression": "\"delta\" / 2"
+ },
+ {
+ "type": "expression",
+ "name": "double-deleted",
+ "expression": "deleted * 2"
+ }
+ ]
+ },
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "triple-added",
+ "fieldName": "triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "one-plus-triple-added",
+ "fieldName": "one-plus-triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "double-deleted",
+ "fieldName": "double-deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ]
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json
new file mode 100644
index 00000000000..572e0f1b5db
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json
@@ -0,0 +1,66 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-08-31T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "page"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"zh"
+ },
+ "aggregations":[
+ {
+ "type":"longSum",
+ "fieldName":"added",
+ "name":"added_count"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 9050.0,
+ "page" : "Crimson Typhoon",
+ "added_count" : 905
+ }
+ } ]
+ }
+]
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json
new file mode 100644
index 00000000000..40503121b92
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json
@@ -0,0 +1,80 @@
+[
+ {
+ "description": "timeseries, 1 agg, all",
+ "query":{
+ "queryType" : "timeBoundary",
+ "dataSource": "%%DATASOURCE%%"
+ },
+ "expectedResults":[
+ {
+ "timestamp" : "2013-08-31T01:02:33.000Z",
+ "result" : {
+ "minTime" : "2013-08-31T01:02:33.000Z",
+ "maxTime" : "2013-08-31T12:41:27.000Z"
+ }
+ }
+ ]
+ },
+
+ {
+ "description":"having spec on post aggregation",
+ "query":{
+ "queryType":"groupBy",
+ "dataSource":"%%DATASOURCE%%",
+ "granularity":"day",
+ "dimensions":[
+ "newPage",
+ "city"
+ ],
+ "filter":{
+ "type":"selector",
+ "dimension":"language",
+ "value":"language-zh"
+ },
+ "aggregations":[
+ {
+ "type":"longSum",
+ "fieldName":"one-plus-triple-added",
+ "name":"added_count"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"double-deleted",
+ "name":"double_deleted_count"
+ },
+ {
+ "type":"longSum",
+ "fieldName":"delta",
+ "name":"delta_overshadowed"
+ }
+ ],
+ "postAggregations": [
+ {
+ "type":"arithmetic",
+ "name":"added_count_times_ten",
+ "fn":"*",
+ "fields":[
+ {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
+ {"type":"constant", "name":"const", "value":10}
+ ]
+ }
+ ],
+ "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
+ "intervals":[
+ "2013-08-31T00:00/2013-09-01T00:00"
+ ]
+ },
+ "expectedResults":[ {
+ "version" : "v1",
+ "timestamp" : "2013-08-31T00:00:00.000Z",
+ "event" : {
+ "added_count_times_ten" : 27160.0,
+ "newPage" : "Crimson Typhoon",
+ "city" : "city-Taiyuan",
+ "double_deleted_count" : 10,
+ "delta_overshadowed" : 450,
+ "added_count" : 2716
+ }
+ } ]
+ }
+]
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json
new file mode 100644
index 00000000000..e277a9127f4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json
@@ -0,0 +1,53 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-01" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensionExclusions" : ["robot", "continent"]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "ingestSegment",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index"
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json
new file mode 100644
index 00000000000..029b136d441
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json
@@ -0,0 +1,108 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%REINDEX_DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "triple-added",
+ "fieldName": "triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "one-plus-triple-added",
+ "fieldName": "one-plus-triple-added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "double-deleted",
+ "fieldName": "double-deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-01" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "page",
+ "newPage",
+ "anonymous",
+ "namespace",
+ "country",
+ "region",
+ "city"
+ ]
+ },
+ "transformSpec": {
+ "transforms": [
+ {
+ "type": "expression",
+ "name": "newPage",
+ "expression": "page"
+ },
+ {
+ "type": "expression",
+ "name": "city",
+ "expression": "concat('city-', city)"
+ },
+ {
+ "type": "expression",
+ "name": "one-plus-triple-added",
+ "expression": "\"triple-added\" + 1"
+ },
+ {
+ "type": "expression",
+ "name": "delta",
+ "expression": "\"delta\" / 2"
+ },
+ {
+ "type": "expression",
+ "name": "double-deleted",
+ "expression": "deleted * 2"
+ }
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "ingestSegment",
+ "dataSource": "%%DATASOURCE%%",
+ "interval": "2013-08-31/2013-09-01"
+ }
+ },
+ "tuningConfig": {
+ "type": "index"
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json
new file mode 100644
index 00000000000..75c1281fcd2
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json
@@ -0,0 +1,98 @@
+{
+ "type": "index_realtime",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second"
+ },
+ "parser": {
+ "type": "map",
+ "parseSpec": {
+ "format": "tsv",
+ "columns": [
+ "timestamp",
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city",
+ "added",
+ "deleted",
+ "delta"
+ ],
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "iso"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ "language",
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "realtime",
+ "firehose": {
+ "type": "timed",
+ "shutoffTime": "#SHUTOFFTIME",
+ "delegate": {
+ "type": "receiver",
+ "serviceName": "eventReceiverServiceName",
+ "bufferSize": 100000
+ }
+ }
+ },
+ "tuningConfig": {
+ "type": "realtime",
+ "maxRowsInMemory": 1,
+ "intermediatePersistPeriod": "PT1M",
+ "windowPeriod": "PT1M",
+ "rejectionPolicy": {
+ "type": "none"
+ }
+ }
+ }
+}
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json
new file mode 100644
index 00000000000..4f13b70cb72
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json
@@ -0,0 +1,86 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "metricsSpec": [
+ {
+ "type": "count",
+ "name": "count"
+ },
+ {
+ "type": "doubleSum",
+ "name": "added",
+ "fieldName": "added"
+ },
+ {
+ "type": "doubleSum",
+ "name": "deleted",
+ "fieldName": "deleted"
+ },
+ {
+ "type": "doubleSum",
+ "name": "delta",
+ "fieldName": "delta"
+ },
+ {
+ "name": "thetaSketch",
+ "type": "thetaSketch",
+ "fieldName": "user"
+ },
+ {
+ "name": "quantilesDoublesSketch",
+ "type": "quantilesDoublesSketch",
+ "fieldName": "delta"
+ },
+ {
+ "name": "HLLSketchBuild",
+ "type": "HLLSketchBuild",
+ "fieldName": "user"
+ }
+ ],
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": "second",
+ "intervals" : [ "2013-08-31/2013-09-02" ]
+ },
+ "parser": {
+ "parseSpec": {
+ "format" : "json",
+ "timestampSpec": {
+ "column": "timestamp"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "page",
+ {"type": "string", "name": "language", "createBitmapIndex": false},
+ "user",
+ "unpatrolled",
+ "newPage",
+ "robot",
+ "anonymous",
+ "namespace",
+ "continent",
+ "country",
+ "region",
+ "city",
+ "timestamp"
+ ]
+ }
+ }
+ }
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/resources/data/batch_index/json",
+ "filter": "wikipedia_index_data*"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "maxRowsPerSegment": 3
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json
new file mode 100644
index 00000000000..d450c7b9458
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json
@@ -0,0 +1,66 @@
+{
+ "type": "index",
+ "spec": {
+ "dataSchema": {
+ "dataSource": "%%DATASOURCE%%",
+ "granularitySpec": {
+ "type": "uniform",
+ "segmentGranularity": "day",
+ "queryGranularity": "none",
+ "intervals": [
+ "2015-09-12/2015-09-13"
+ ]
+ },
+ "parser": {
+ "type": "hadoopyString",
+ "parseSpec": {
+ "format": "json",
+ "dimensionsSpec": {
+ "dimensions": [
+ "channel",
+ "page",
+ "user"
+ ]
+ },
+ "timestampSpec": {
+ "format": "auto",
+ "column": "time"
+ }
+ }
+ },
+ "metricsSpec": [
+ {
+ "name": "count",
+ "type": "count"
+ },
+ {
+ "name": "added",
+ "type": "longSum",
+ "fieldName": "added"
+ },
+ {
+ "name": "deleted",
+ "type": "longSum",
+ "fieldName": "deleted"
+ },
+ {
+ "name": "delta",
+ "type": "longSum",
+ "fieldName": "delta"
+ }
+ ]
+ },
+ "ioConfig": {
+ "type": "index",
+ "firehose": {
+ "type": "local",
+ "baseDir": "/shared/wikiticker-it",
+ "filter": "wikiticker-2015-09-12-sampled.json.gz"
+ }
+ },
+ "tuningConfig": {
+ "type": "index",
+ "targetPartitionSize" : 10000
+ }
+ }
+}
\ No newline at end of file
diff --git a/integration-tests-ex/check-results.sh b/integration-tests-ex/check-results.sh
deleted file mode 100755
index 44aac0aa9f6..00000000000
--- a/integration-tests-ex/check-results.sh
+++ /dev/null
@@ -1,89 +0,0 @@
-#! /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.
-#--------------------------------------------------------------------
-
-# Run from Travis which has no good way to attach logs to a
-# build. Instead, we check if any IT failed. If so, we append
-# the last 100 lines of each server log to stdout. We have to
-# stay wihtin the 4MB limit which Travis applies, so we only
-# emit logs for the first failure, and only for servers that
-# don't report normal completion.
-#
-# The only good way to check for test failures is to parse
-# the Failsafe summary for each test located in
-# /target/failsafe-reports/failsafe-summary.xml
-#
-# This directory has many subdirectories, some of which are
-# tests. We rely on the fact that a test starts with "it-" AND
-# contains a failsafe report. (Some projects start with "it-"
-# but are not tests.)
-
-# Run in the docker-tests directory
-cd $(dirname $0)
-
-# Scan for candidate projects
-for PROJECT in it-*
-do
- # Check if a failsafe report exists. It will exist if the directory is
- # a test project and failsafe ran on that directory.
- REPORTS="$PROJECT/target/failsafe-reports/failsafe-summary.xml"
- if [ -f "$REPORTS" ]
- then
- # OK, so Bash isn't the world's best text processing language...
- ERRS=1
- FAILS=1
- while IFS= read -r line
- do
- if [ "$line" = " 0 " ]
- then
- ERRS=0
- fi
- if [ "$line" = " 0 " ]
- then
- FAILS=0
- fi
- done < "$REPORTS"
- if [ $ERRS -eq 1 -o $FAILS -eq 1 ]
- then
- FOUND_LOGS=0
- echo "======= $PROJECT Failed =========="
- # All logs except zookeeper
- for log in $(ls $PROJECT/target/shared/logs/[a-y]*.log)
- do
- # We assume that a successful exit includes a line with the
- # following:
- # Stopping lifecycle [module] stage [INIT]
- tail -5 "$log" | grep -Fq 'Stopping lifecycle [module] stage [INIT]'
- if [ $? -ne 0 ]
- then
- # Assume failure and report tail
- echo $(basename $log) "logtail ========================"
- tail -100 "$log"
- FOUND_LOGS=1
- fi
- done
-
- # Only emit the first failure to avoid output bloat
- if [ $FOUND_LOGS -eq 1 ]
- then
- exit 0
- else
- echo "All Druid services exited normally."
- fi
- fi
- fi
-done
diff --git a/integration-tests-ex/docs/compose.md b/integration-tests-ex/docs/compose.md
new file mode 100644
index 00000000000..01b41896adb
--- /dev/null
+++ b/integration-tests-ex/docs/compose.md
@@ -0,0 +1,238 @@
+
+
+# Docker Compose Configuration
+
+The integration tests use Docker Compose to launch Druid clusters. Each
+test defines its own cluster
+depending on what is to be tested. Since a large amount of the definition is
+common, we use inheritance to simplify cluster definition.
+
+Tests are split into categories so that they can run in parallel. Some of
+these categories use the same cluster configuration. To further reduce
+redundancy, test categories can share cluster configurations.
+
+See also:
+
+* [Druid configuration](druid-config.md) which is done via Compose.
+* [Test configuration](test-config.md) which tells tests about the
+ cluster configuration.
+* [Docker compose specification](https://github.com/compose-spec/compose-spec/blob/master/spec.md)
+
+## File Structure
+
+Docker Compose files live in the `druid-it-cases` module (`test-cases` folder)
+in the `cluster` directory. There is a separate subdirectory for each cluster type
+(subset of test categories), plus a `Common` folder for shared files.
+
+## Shared Directory
+
+Each test has a "shared" directory that is mounted into each container to hold things
+like logs, security files, etc. The directory is known as `/shared` within the container,
+and resides in `target/`. Even if two categories share a cluster configuration,
+they will have separate local versions of the shared directory. This is important to
+keep log files separate for each category.
+
+## Base Configurations
+
+Test clusters run some number of third-party "infrastructure" containers,
+and some number of Druid service containers. For the most part, each of
+these services (in Compose terms) is similar from test to test. Compose
+provides [an inheritance feature](
+https://github.com/compose-spec/compose-spec/blob/master/spec.md#extends)
+that we use to define base configurations.
+
+* `cluster/Common/dependencies.yaml` defines external dependencis (MySQL, Kafka, ZK
+ etc.)
+* `cluster/Common/druid.yaml` defines typical settings for each Druid service.
+
+Test-specific configurations extend and customize the above.
+
+### Druid Configuration
+
+Docker compose passes information to Docker in the form of environment variables.
+The test use a variation of the environment-variable-based configuration used in
+the [public Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html).
+That is, variables of the form `druid_my_config` are converted, by the image launch
+script, into properties of the form `my.config`. These properties are then written
+to a launch-specific `runtime.properties` file.
+
+Rather than have a test version of `runtime.properties`, instead we have a set of
+files that define properties as environment variables. All are located in
+`cases/cluster/Common/environment-configs`:
+
+* `common.env` - Properties common to all services. This is the test equivalent to
+ the `common.runtime.properties` file.
+* `.env` - Properties unique to one service. This is the test equivalent to
+ the `service/runtime.properties` files.
+
+### Special Environment Variables
+
+Druid properties can be a bit awkward and verbose in a test environment. A number of
+test-specific properties help:
+
+* `druid_standard_loadList` - Common extension load list for all tests, in the form
+ of a comma-delimited list of extensions (without the brackets.) Defined in
+ `common.env`.
+* `druid_test_loadList` - A list of additional extensions to load for a specific test.
+ Defined in the `docker-compose.yaml` file for that test category. Do not include
+ quotes.
+
+Example test-specific list:
+
+```text
+druid_test_loadList=druid-azure-extensions,my-extension
+```
+
+The launch script combines the two lists, and adds the required brackets and quotes.
+
+## Test-Specific Cluster
+
+Each test has a directory named `cluster/`. Docker Compose uses this name
+as the cluster name which appears in the Docker desktop UI. The folder contains
+the `docker-compose.yaml` file that defines the test cluster.
+
+In the simplest case, the file just lists the services to run as extensions
+of the base services:
+
+```text
+services:
+ zookeeper:
+ extends:
+ file: ../Common/dependencies.yaml
+ service: zookeeper
+
+ broker:
+ extends:
+ file: ../Common/compose/druid.yaml
+ service: broker
+...
+```
+
+## Cluster Configuration
+
+If a test wants to run two of some service (say Coordinator), then it
+can use the "standard" definition for only one of them and must fill in
+the details (especially distinct port numbers) for the second.
+(See `HighAvilability` for an example.)
+
+By default, the container and internal host name is the same as the service
+name. Thus, a `broker` service resides in a `broker` container known as
+host `broker` on the Docker overlay network.
+The service name is also usually the log file name. Thus `broker` logs
+to `/target//logs/broker.log`.
+
+An environment variable `DRUID_INSTANCE` adds a suffix to the service
+name and causes the log file to be `broker-one.log` if the instance
+is `one`. The service name should have the full name `broker-one`.
+
+Druid configuration comes from the common and service-specific environment
+files in `/compose/environment-config`. A test-specific service configuration
+can override any of these settings using the `environment` section.
+(See [Druid Configuration](druid-config.md) for details.)
+For special cases, the service can define its configuration in-line and
+not load the standard settings at all.
+
+Each service can override the Java options. However, in practice, the
+only options that actually change are those for memory. As a result,
+the memory settings reside in `DRUID_SERVICE_JAVA_OPTS`, which you can
+easily change on a service-by-service or test-by-test basis.
+
+Debugging is enabled on port 8000 in the container. Each service that
+wishes to expose debugging must map that container port to a distinct
+host port.
+
+The easiest way understand the above is to look at a few examples.
+
+## Service Names
+
+The Docker Compose file sets up an "overlay" network to connect the containers.
+Each is known via a host name taken from the service name. Thus "zookeeper" is
+the name of the ZK service and of the container that runs ZK. Use these names
+in configuration within each container.
+
+### Host Ports
+
+Outside of the application network, containers are accessible only via the
+host ports defined in the Docker Compose files. Thus, ZK is known as `localhost:2181`
+to tests and other code running outside of Docker.
+
+## Test-Specific Configuration
+
+In addition to the Druid configuration discussed above, the framework provides
+three ways to pass test-specific configuration to the tests. All of these methods
+override any configuration in the `docker-compose` or cluster `env` files.
+
+The values here are passed into the Druid server as configuration values. The
+values apply to all services. (This mechanism does not allow service-specific
+values.) In all three approaches, use the `druid_` environment variable form.
+
+Precendence is in the order below with the user file lowest priority and environment
+variables highest.
+
+### User-specific `~/druid-it/
+
+# Test Conversion
+
+Here is the process to convert an existing `integration-test`
+group to this new structure.
+
+The tests all go into the `druid-integration-test-cases` module
+(sub-directory `test-cases`). Move the tests into the existing
+`testsEx` name space so they do not collide with the existing
+integration test namespace.
+
+## Cluster Definition
+
+Define a category for your tests. See [tests](tests.md) for the
+details. The category is the name of the cluster definition by
+default.
+
+Determine if you can use an existing cluster definition, or if you
+need to define a new one. See [tests](tests.md) for how to share a
+cluster definition. If you share a definition, update `cluster.sh`
+to map from your category name to the shared cluster definition
+name.
+
+To create a new defnition,
+create a `druid-cluster/docker-compose.yaml` file by converting the
+previous `docker/docker-compose-.yml` file. Carefully review
+each service. Use existing files as a guide.
+
+In `integration-tests` all groups share a set of files with many
+conditionals to work out what is to be done. In this system, each
+test group stands alone: its Docker Compose file defines the cluster
+for that one test. There is some detangling of the existing conditionals
+to determine the setup used by each test group.
+
+Create the `yaml/docker.yaml` resource in `/src/test/resources` to
+define your cluster for the Java tests.
+
+Determine if the test group populates the metadata store using queries
+run in the Docker container. If so, copy those queries into the
+`docker.yaml` file in the `metadataInit` section. (In the new structure,
+these setup queries run in the test client, not in each Docker service.)
+See the former `druid.sh` script to see what SQL was used previously.
+
+###
+
+## Test Runner
+
+ITs require a large amount of setup. All that code is encapsulated in the
+`DruidTestRunner` class:
+
+```java
+@RunWith(DruidTestRunner.class)
+@Category(MyCategory.class)
+public class ITMyTest
+```
+
+It is helpful to know what the test runner does:
+
+* Loads the cluster configuration from the `docker.yaml` file, and
+ resolves any includes.
+* Builds up the set of Guice modules needed for the test.
+* Creates the Guice injector.
+* Uses the injector to inject dependencies into your test class.
+* Starts the Druid lifecycle.
+* Waits for each Druid service defined in `docker.yaml` to become
+ available.
+* Runs your test methods.
+* Ends the Druid lifecycle.
+
+You can customize the configuration for non-standard cases. See
+[tests](tests.md) for details.
+
+## Tests
+
+Convert the individual tests.
+
+### Basics
+
+Copy the existing tests for the target group into the
+`druid-it-cases`. For sanity, you may want to do one by one.
+
+When adding tests, leave the original tests in `integration-tests` for
+now. (Until we have the new system running in Travis.) Once Travis
+runs, you can move, rather than copy, the tests.
+
+While we are copying, copy to the `org.apache.druid.testsEx` package to
+prevent name conficts with `org.apache.druid.tests`.
+
+### Maven Dependencies
+
+You may need to add dependencies to `pom.xml`.
+
+The `docker-tests/pom.xml` file includes Maven dependencies for the most
+common Druid modules, which transitiviely include the third-party modules
+which the Druid modules reference. You test sub-project may need addition
+dependencies. To find them, review `integration-tests/pom.xml`. Careful,
+however, as that file is a bit of a "kitchen sink" that includes every
+possible dependency, even those already available transitively.
+
+If you feel the dependency is one used by multiple tests, go ahead and
+add it to `docker-tests/pom.xml`. If, however, it is somehwat unique to
+the test group, just add it to that sub-modules `pom.xml` file instead.
+
+Use the following to verify the `pom.xml`:
+
+```bash
+mvn dependency:analyze -DoutputXML=true -DignoreNonCompile=true \
+ -P skip-static-checks -Ddruid.console.skip=true -Dmaven.javadoc.skip=true \
+ -P skip-tests
+```
+
+Doing it now will save build cycles when submitting your PR.
+
+### Resources and Test Data
+
+The existing tests use the `/src/test/resources/` directory to hold both
+JSON specs used by the tests, as well as test data used by the cluster.
+To make the data available to tests, we mount the `/src/test/resources`
+folder into the Indexer at `/resources`.
+
+In the new version, we separate these two groups of files. Those used by
+tests continue to reside in `/src/test/resources` for the individual
+tests. Those shared by multiple tests can be in `base-test/src/test/resources`.
+Copy the resource files from `integration-tests` into one of these
+locations. Try to avoid doing a bulk copy: copy only the files used by
+the particular test group being converted.
+
+Then, copy the data into `/data`, keeping the same path. See
+`data/README.md` for more information.
+
+#### To Do
+
+It may turn out that data files are shared among tests. In that case, we'd
+want to put them in a common location, keeping test-specific data in the
+project for that test. But, we can't easily combine two directories into
+a single volume mount.
+
+Instead, we can use the `target/shared` folder: create a new `data`
+folder, copy in the required files, and mount that at `/resources`.
+Or, if we feel energetic, just change the specs to read their data
+from `/shared/data`, since `/shared` is already mounted.
+
+### Extensions
+
+You may see build or other code that passes a list of extensions to an old
+integration test. Such configuration represents a misunderstanding of how tests (as
+clients) actually work. Tests nave no visibility to a Druid installation directory.
+As a result, the "extension" concept does not apply. Instead, tests are run from
+Maven, and are subject to the usual Maven process for locating jar files. That
+means that any extensions which the test wants to use should be listed as dependencies
+in the `pom.xml` file, and will be available on the class path. There is no need for,
+or use of, the `druid_extensions_loadList` for tests (or, indeed, for any client.)
+
+### Starter Test (Optional)
+
+An optional step is to ease into the new system by doing a simple
+"starter test".
+Create a ad-hoc test file, say `StarterTest` to hold one of the
+tests to be converted. Copy any needed Guice injections. This will
+be a JUnit test.
+
+Define your test class like this:
+
+```java
+@RunWith(DruidTestRunner.class)
+public class StarterTest
+```
+
+The test runner handles the required startup, Guice configuration, cluster
+validation, and shutdown. Just add your own test cases.
+
+Determine if the test runs queries from `src/test/resources/queries`.
+If so, copy those to the new sub-project. Do the same with any other
+resources which the test requires.
+
+In this new structure, each group is its own sub-project, so resources
+are separted out per test group (sub-project), whereas in
+`integration-tests` the resources are all grouped together. If there
+are shared resources, put those in the `docker-tests/src/test/resources`
+folder so they can be shared. (This may require creating a test-jar.
+As an alternative, they can be put in `base-test` which is already
+available to all tests.)
+
+Run the one test. This will find bugs in the above. It will also likely
+point out that you need Druid modules not in the base set defined by
+`Initialization`. Add these modules via the `Builder.modules()` method.
+Resolve the other issues which will inevitably appear.
+
+This starter test will ensure that most of the dependency and configuration
+issues are resolved.
+
+### Revised Helper Classes
+
+The new test structure adopted shorter container and host names:
+`coordinator` instead of `druid-coordinator` etc. This is safe because the
+Docker application runs in isolation, we don't have to worry about a
+potential `coordinator` from application X.
+
+To handle these changes, there are new versions of several helper classes.
+Modify the tests to use the new versions:
+
+* `DruidClusterAdminClient` - interfaces with Docker using hard-coded
+ container names.
+
+The old versions are in `org.apache.druid.testing.utils` in
+`integration-tests`, the new versions in `org.apache.druid.testing2.utils`
+in this project.
+
+### Test Classes
+
+You can now convert the bulk of the tests.
+One-by-one, convert existing classes:
+
+* Remove the TestNG annotations and includes. Substitute JUnit includes.
+* Add the `@RunWith` annotation.
+* Run the test in the debugger to ensure it works.
+
+The test class definition should look like this:
+
+```java
+@RunWith(DruidTestRunner.class)
+public class ITIndexerTest ...
+{
+```
+
+Run the entire suite from Maven in the sub-module directory. It should
+start the cluster, run the tests, and shut down the cluster.
+
+## Improving Tests
+
+Once the tests work, an optional step is to improvement a bit beyond what
+was already done.
+
+### Retries
+
+The `Initializer` takes upon itself the task of ensuring that all services
+are up (at least enough that they each report that they are healthy.) So,
+it is not necessary for each test case to retry endlessly to handle the case
+that it is the first test run on a cluster still coming up. We can remove
+retries that don't represent valid server behavior. For example, if the goal
+is too ensure that the endpoint `/foo` returns `bar`, then there is no need
+to retry: if the server is up, then its `/foo` endpoint should be working, and
+so it should return `bar`, assuming that the server is deterministic.
+
+If `bar` represents something that takes time to compute (the result of a
+task, say), then retry is valid. If `bar` is deterministic, then retrying won't
+fix a bug that causes `bar` to be reported incorrectly.
+
+Use your judgement to determine when retries were added "just to be safe"
+(and can thus be removed), and when the represent actual race conditions in
+the operation under tests.
+
+### Cluster Client
+
+The tests obviously do a large number of API calls to the server. Some (most)
+seem to spell out the code inline, resulting in much copy/paste. An improvement
+is to use the cluster client instead: `ClusterClient`. Add methods for endpoints
+not yet covered by copying the code from the test in question. (Better, refactor
+that code to use the existing lower-level `get()` and similar methods. Then,
+use the cluster client method in place of the copy/paste wad of code.
+
+The result is a test that is smaller, easier to undestand, easier to maintain,
+and easier debug. Also, future tests are easier to write because they can reuse
+the method you added to the cluster client.
+
+You can inject the cluster client into your test:
+
+```java
+ @Inject
+ private ClusterClient clusterClient;
+```
+
+You may find that by using the cluster client, some of the dependencies which
+the test needed are now unused. Go ahead and remove them.
diff --git a/integration-tests-ex/docs/debugging.md b/integration-tests-ex/docs/debugging.md
new file mode 100644
index 00000000000..fb0da9aee37
--- /dev/null
+++ b/integration-tests-ex/docs/debugging.md
@@ -0,0 +1,159 @@
+
+
+# Debugging the Druid Image and Integration Tests
+
+The integration setup has as a primary goal the ability to quickly debug
+the Druid image and any individual tests. A first step is to move the
+image build into a separate project. A second step is to ensure each
+test can run in JUnit in an IDE against a cluster you start by hand.
+
+This section discusses how to use the various debugging features.
+
+See:
+
+* [Docker Configuration](docker.md) for information on debugging
+ docker builds.
+
+## General Debug Process
+
+Ease of debugging is a key goal of the revised structure.
+
+* Rebuild the Docker image only when the Druid code changes.
+ Do a normal distribution build, then build a new image.
+* Reuse the same image over and over if you only change tests
+ (such as when adding a new test.)
+* Reuse the same `shared` directory when the test does not
+ make permanent changes.
+* Change Druid configuration by changing the Docker compose
+ files, no need to rebuild the image.
+* Work primarily in the IDE when debugging tests.
+* To add more logging, change the `log4j2.xml` file in the shared
+ directory to increase the logging level.
+* Remote debug Druid services if needed.
+
+## Exploring the Test Cluster
+
+When run in Docker Compose, the endpoints known to Druid nodes differ from
+those needed by a client sitting outside the cluster. We could provide an
+explicit mapping. Better is to use the
+[Router](https://druid.apache.org/docs/latest/design/router.html#router-as-management-proxy)
+to proxy requests. Fortunately, the Druid Console already does this.
+
+## Docker Build Output
+
+Modern Docker seems to hide the output of commands, which is a hassle to debug
+a build. Oddly, the details appear for a failed build, but not for success.
+Use the followig to see at least some output:
+
+```bash
+export DOCKER_BUILDKIT=0
+```
+
+Once the base container is built, you can run it, log in and poke around. First
+identify the name. See the last line of the container build:
+
+```text
+Successfully tagged org.apache.druid/test:
+```
+
+Or ask Docker:
+
+```bash
+docker images
+```
+
+## Debug the Docker Image
+
+You can log into the Docker image and poke around to see what's what:
+
+
+```bash
+docker run --rm -it --entrypoint bash org.apache.druid/test:
+```
+
+Quite a few environment variables are provided by Docker and the setup scripts
+to see them, within the container, use:
+
+```bash
+env
+```
+
+## Debug an Integration Test
+
+To debug an integration test, you need a Docker image with the latest Druid.
+To get that, you need a full Druid build. So, we break the debugging process
+down into steps that depend on the state of your code. Assume `DRUID_DEV`
+points to your Druid development area.
+
+### On Each Druid Build
+
+If you need to rebuild Druid (because you fixed something), do:
+
+* Do a distribution build of Druid:
+* Build the test image.
+
+See [quickstart](quickstart.md) for the commands.
+
+### Start the Test Cluster
+
+* Pick a test "group" to use.
+* Start a test cluster configured for this test.
+* Run a test from the command line:
+
+Again, see [quickstart](quickstart.md) for the commands.
+
+### Debug the Test
+
+To run from your IDE, find the test to run and run it as a JUnit test (with the
+cluster up.)
+
+Depending on the test, you may be able to run the test over and over against the
+same cluster. (In fact, you should try to design your tests so that this is true:
+clean up after each run.)
+
+The tests are just plain old JUnit tests that happen to reach out to the
+test cluster and/or Docker to do their work. You can set breakpoints and debug
+in the usual way.
+
+Each test will first verify that the cluster is fully up before it starts, so
+you can launch your debug session immediately after starting the cluster: the tests
+will wait as needed.
+
+### Stop the Test Cluster
+
+When done, stop the cluster: [quickstart](quickstart.md) again for details.
+
+## Typical Issues
+
+For the most part, you can stop and restart the Druid services as often
+as you like and Druid will just work. There are a few combinations that
+can lead to trouble, however.
+
+* Services won't start: When doing a new build, stop the existing cluster
+ before doing the build. The build removes and rebuilds the shared
+ directory: services can't survive that.
+* Metastore failure: The metastore container will recreate the DB on
+ each restart. This will fail if your shared directory already contains
+ a DB. Do a `rm -r target//db` before restarting the DB container.
+* Coordinator fails with DB errors. The Coordinator will create the Druid
+ tables when it starts. This means the DB has to be created. If the DB
+ is removed after the Coordinator starts (to fix the above issue, say)
+ then you have to restart the Coordinator so it can create the needed
+ tables.
diff --git a/integration-tests-ex/docs/dependencies.md b/integration-tests-ex/docs/dependencies.md
new file mode 100644
index 00000000000..90047449f5c
--- /dev/null
+++ b/integration-tests-ex/docs/dependencies.md
@@ -0,0 +1,239 @@
+
+
+# Dependencies
+
+The Docker tests have a number of dependencies which are important to understand
+when making changes or debugging problems.
+
+## Third-Party Libraries
+
+As described in the [Docker](docker.md) section, the Docker image contains Druid
+plus three external dependencies:
+
+* The MySQL client library
+* The MariaDB client library
+* The Kafka protobuf provider
+
+These libraries are not shipped with Druid itself. Instead, we add them to the
+image as follows:
+
+* Dependencies are listed in the `test-image/pom.xml` file.
+* Maven fetches the dependencides from an upstream repo and places them
+ into the local Maven cache.
+* The `test-image/pom.xml` file uses the `maven-dependency-plugin`
+ to copy these dependencies from the local repo into the
+ `target/docker` directory.
+* The `Dockerfile` copies the dependencies into the `/usr/local/druid/lib`
+ directory after `build-image.sh` has unpacked the Druid distribution
+ into `/usr/local/druid`.
+
+The key benefit is that the dependencies are downloaded once and are
+served from the local repo afterwards.
+
+## Third-Party Servers
+
+As described in the [Docker](docker.md) section, we use third-party
+"official" images for three of our external server dependencies:
+
+* [MySQL](https://hub.docker.com/_/mysql). This image is configured
+ to create the Druid database and user upon startup.
+* [ZooKeeper](https://hub.docker.com/_/zookeeper).
+* [Kafka](https://hub.docker.com/r/bitnami/kafka/). There is no
+ "official" image so we use the one from Bitnami.
+
+See `compose/dependencies.yaml` for the Docker Compose configuration
+for each of these services.
+
+Other integration tests use additional servers such as Hadoop.
+We will want to track down official images for those as well.
+
+## Guice and Lifecycle
+
+Nothing will consume more of your time than fighting with Druid's
+Guice and Lifecycle mechanisms. These mechanisms are designed to do
+exactly one thing: configure the Druid server. They are a nightmare
+to use in other configurations such as unit or integration tests.
+
+### Guice Modules
+
+Druid has *many* Guice modules. There is no documentation to explain
+which components are available from which modules, or their dependencies.
+So, if one needs component X, one has to hunt through the source to
+find the module that provides X. (Or, one has to "just know.") There
+is no trick other than putting in the time to do the research, watching
+things fail, and trying harder.
+
+In addition, modules have implicit dependencies: to use module Y you
+must also include module Z. Again, there is no documentation, you have
+to know or figure it out.
+
+The modules are designed to work only in the server, so they assume
+the entire server is avaialble. Once we have a way that the modules
+work in the server, we don't mess with it. But, in tests, we want
+to use a subset because tests are clients, not a server. So, we end
+up fighting to reuse a system that was designed for exactly one use
+case: the server. The result is either a huge amount of time fiddling
+to get things right or (as in the original integration tests), we just
+include everything and pretend we are a server.
+
+There is no obvious solution, it is just a massive time sink at
+present.
+
+### Druid Modules
+
+Many of the modules we want to use in integration test are
+`DruidModule`s. These go beyond the usual Guice modules to provide
+extra functionality, some of which is vital in tests:
+
+* The modules have depenencies injected from the "startup injector."
+* The modules provide Jackson modules needed to serialized JSON.
+
+The `Initialization` class provides the mechanisms needed to work
+with `DruidModule`s, but only when creating a server: that same class
+has a strong opinion about which modules to include based on the
+assumption that the process is a server (or a Druid tool which acts
+like a server.)
+
+The code here refactored `Initialization` a bit to allow us to
+use the functionality without being forced to accept all the default
+server modules. The upside is that we don't end up having to fake the
+tests to look like servers. The downside is the issue above: we have to
+deal with the dependency nightmare.
+
+### Lifecycle Race Conditions
+
+Druid uses the `Lifecycle` class to manage object initialization. The
+Lifecycle expects instances to be registered before the lifecycle
+starts, after which it is impossible to register new instances.
+
+The lifecycle works out startup order based on Guice injection
+dependencies. Thus, if a constructor is `X(Y y, Z y)` Guice knows
+to create an `Y` and `Z` before creating `X`. `Lifecycle` leverages
+this knowledge to start `Y` and `Z` before starting `X`.
+
+This works only if, during module creation, something has a
+dependency on `X`. Else, if `X` is a `LazySingleton` it won't be
+created until it is first used. But, by then, the `Lifecycle` will have
+started and you'll get the dreaded "It doesn't work that way" error.
+
+### Guice and Lifecycle in Tests
+
+In the server, this works fine: there is exactly one usage of each
+singleton, and the various modules have appearently been tweaked to
+ensure every lifecycle-aware object is referenced (thus created,
+this registerd in the lifecycle) by some other module.
+
+In tests, however, this system breaks down. Maven runs a series of
+tests (via `failsafe`), each of which has any number of test methods.
+The test driver is free to create any number of test class instances.
+
+When using the `Lifecycle` mechanism in tests, we would prefer to
+set up the injector, and run the lifecycle, once per test class. This
+is easy to do with the JUnit `@BeforeClass` annotation. But, when we
+try this, the livecycle race condition issue slams us hard.
+
+Tests want to reference certain components, such as `DruidNodeDiscoveryProvider`
+which require `CuratorFramework` which is provided by a module that
+registers a component with the lifecycle. Because of the lazy singleton
+pattern, `DruidNodeDiscoveryProvider` (and hence its dependenencies)
+are created when first referenced, which occurs when JUnit instantiates
+the test class, which happens after the Guice/Lifecycle setup in
+`@BeforeClass`. And, we get our "It doesn't work that way" error.
+
+We can then try to move Guice/Lifecycle creation into the test class
+constuctor, but then we'll watch as JUnit creates multiple instances
+and we end up running initialization over and over. Further, it seems
+there are race conditions when we do that (haven't figure out the
+details), and we get strange errors. Further, we end up thrashing
+the very complex initializaiton logic (which is a great stress test,
+but we need to it only once, not on every test.)
+
+A hacky compromise is to add a caching layer: do the initialization in
+the constructor, so we can inject the member variables, which creates
+references, which causes the comonents to be created, which causes them
+to register with the `Lifecycle` at the proper time. In the second
+constructor call, we reuse the injector created in the first call.
+Since we simply reuse the same singletons, we should not run into
+Livecycle race conditions. The `@AfterClass` JUnit annotation is pressed
+into service to shut down the lifecycle after all tests run.
+
+## Testing Tools And the Custom Node Role
+
+The Druid extension `druid-testing-tools` (Maven project
+`extensions-core/testing-tools` provides an extension to be loaded
+into the Druid image along with the Druid distribution and third-party
+libraries.
+
+The `integration-tests` provides additional components (such as the
+custom node role) that must be placed in the image, but uses an
+entirely different mechanism.
+
+There is no documentation to explain why we do `testing-tools` one
+way, the custom node role a different way. Is there a reason other than
+the items were created by different people at different times who chose
+to use different approaches?
+
+In an ideal world, `testing-tools` would contain the custom node role:
+there would be a single way to provide test-only extensions. However,
+since we must maintain backward compatibility with `integration-tests`,
+and that module is a nightmare to modify, we must use a short-term
+compromise.
+
+For now, we punt: we make a copy of `druid-testing-tools`, add the
+`integraton-tools` custom node role, and call it `testing-tools-ex`.
+See [`testing-tools/README`](../testing-tools/README.md) for the
+details.
+
+## Integration Tests and `base-test`
+
+The `integration-tests` project contains the set of existing TestNG-based
+tests as well as a large number of utilities used by the tests.
+The revised framework adds its own utilities.
+
+The utilities speicfic to the new tests resides in the `base-test`
+sub-project. We include the `integration-test` project to reusse its
+utilities.
+
+This does create a potential conflict: as we convert tests, the tests
+here will have the same name as tests in the `integration-test`
+package, which causes duplicate class names on the class path: never
+a good thing.
+
+The ideal solution would be to move the test utilities to a new
+sub-project within `integration-tests` and have both the new and old test
+projects include the resulting jar.
+
+For now, we use a "shadow" approach, we use the `org.apache.druid.testsEx`
+package name for new tests so names do not conflict with the
+`org.apache.druid.tests` name used in `integration-tests`. Eventually,
+if all tests are renamed, we can rename the `testsEx` package back
+to `tests`.
+
+In a few cases, the utilitiy classes make asumptions about the test
+setup which does not match the new setup. In this case, we make a copy
+of the class and apply needed changes. At present, only one class has this
+issue:
+
+* `DruidClusterAdminClient` - interfaces with Docker using hard-coded
+ container names.
+
+The old versions are in `org.apache.druid.testing.utils` in
+`integration-tests`, the new versions in `org.apache.druid.testing2.utils`
+in this project.
diff --git a/integration-tests-ex/docs/docker.md b/integration-tests-ex/docs/docker.md
new file mode 100644
index 00000000000..afbc1064687
--- /dev/null
+++ b/integration-tests-ex/docs/docker.md
@@ -0,0 +1,302 @@
+
+
+# Docker Test Image for Druid
+
+Integration tests need a Druid cluster. While some tests support using
+Kubernetes for the Quickstart cluster, most need a cluster with some
+test-specific configuration. We use Docker Compose to create that cluster,
+based on a test-oriented Docker image built by the `it-image` Maven module
+(activated by the `test-image` profile.)
+The image contains the Druid distribution,
+unpacked, along with the MySQL and MariaDB client libaries and
+and the Kafka protobuf dependency. Docker Compose is
+used to pass configuration specific to each service.
+
+In addition to the Druid image, we use "official" images for dependencies such
+as ZooKeeper, MySQL and Kafka.
+
+The image here is distinct from the
+["retail" image](https://druid.apache.org/docs/latest/tutorials/docker.html)
+used for getting started. The test image:
+
+* Uses a shared directory to hold logs and some configuration.
+* Uses "official" images for dependencies.
+* Assumes the wrapper Docker compose scripts.
+* Has some additional test-specific extensions as defind in `it-tools`.
+
+## Build Process
+
+Assuming `DRUID_DEV` points to your Druid build directory,
+to build the image (only):
+
+```bash
+cd $DRUID_DEV/docker-tests/it-image
+mvn -P test-image install
+```
+
+Building of the image occurs in four steps:
+
+* The Maven `pom.xml` file gathers versions and other information from the build.
+ It also uses the normal Maven dependency mechanism to download the MySQL,
+ MariaDB and
+ Kafka client libraries, then copies them to the `target/docker` directory.
+ It then invokes the `build-image.sh` script.
+* `build-image.sh` adds the Druid build tarball from `distribution/target`,
+ copies the contents of `test-image/docker` to `target/docker` and
+ then invokes the `docker build` command.
+* `docker build` uses `target/docker` as the context, and thus
+ uses the `Dockerfile` to build the image. The `Dockerfile` copies artifacts into
+ the image, then defers to the `test-setup.sh` script.
+* The `test-setup.sh` script is copied into the image and run. This script does
+ the work of installing Druid.
+
+The resulting image is named `org.apache.druid/test:`.
+
+### Clean
+
+A normal `mvn clean` won't remove the Docker image because that is often not
+what you want. Instead, do:
+
+```bash
+mvn clean -P test-image
+```
+
+You can also remove the image using Docker or the Docker desktop.
+
+### `target/docker`
+
+Docker requires that all build resources be within the current directory. We don't want
+to change the source directory: in Maven, only the target directories should contain
+build artifacts. So, the `pom.xml` file builds up a `target/docker` directory. The
+`pom.xml` file then invokes the `build-image.sh` script to complete the setup. The
+resulting directory structure is:
+
+```text
+/target/docker
+|- Dockerfile (from docker/)
+|- scripts (from docker/)
+|- apache-druid--bin.tar.gz (from distribution, by build-image.sh)
+|- MySQL client (done by pom.xml)
+|- MariaDB client (done by pom.xml)
+|- Kafka protobuf client (done by pom.xml)
+```
+
+Then, we invoke `docker build` to build our test image. The `Dockerfile` copies
+files into the image. Actual setup is done by the `test-setup.sh` script copied
+into the image.
+
+Many Dockerfiles issue Linux commands inline. In some cases, this can speed up
+subsequent builds because Docker can reuse layers. However, such Dockerfiles are
+tedious to debug. It is far easier to do the detailed setup in a script within
+the image. With this approach, you can debug the script by loading it into
+the image, but don't run it in the Dockerfile. Instead, launch the image with
+a `bash` shell and run the script by hand to debug. Since our build process
+is quick, we don't lose much by reusing layers.
+
+### Manual Image Rebuilds
+
+You can quick rebuild the image if you've previously run a Maven image build.
+Assume `DRUID_DEV` points to your Druid development root. Start with a
+Maven build:
+
+```bash
+cd $DRUID_DEV/docker/test-image
+mvn -P test-image install
+```
+
+Maven is rather slow to do its part. Let it grind away once to populate
+`target/docker`. Then, as you debug the `Dockerfile`, or `test-setup.sh`,
+you can build faster:
+
+```bash
+cd $DRUID_DEV/docker/test-image
+./rebuild.sh
+```
+
+This works because the Maven build creates a file `target/env.sh` that
+contains the Maven-defined environment. `rebuild.sh` reads that
+environment, then proceeds as would the Maven build.
+Image build time shrinks from about a minute to just a few seconds.
+`rebuild.sh` will fail if `target/env.sh` is missing, which reminds
+you to do the full Maven build that first time.
+
+Remember to do a full Maven build if you change the actual Druid code.
+You'll need Maven to rebuild the affected jar file and to recreate the
+distribution image. You can do this the slow way by doing a full rebuild,
+or, if you are comfortable with maven, you can selectively run just the
+one module build followed by just the distribution build.
+
+## Image Contents
+
+The Druid test image adds the following to the base image:
+
+* A Debian base image with the target JDK installed.
+* Druid in `/usr/local/druid`
+* Script to run Druid: `/usr/local/launch.sh`
+* Extra libraries (Kafka, MySQL, MariaDB) placed in the Druid `lib` directory.
+
+The specific "bill of materials" follows. `DRUID_HOME` is the location of
+the Druid install and is set to `/usr/local/druid`.
+
+| Variable or Item | Source | Destination |
+| -------- | ------ | ----- |
+| Druid build | `distribution/target` | `$DRUID_HOME` |
+| MySQL Connector | Maven repo | `$DRUID_HOME/lib` |
+| Kafka Protobuf | Maven repo | `$DRUID_HOME/lib` |
+| Druid launch script | `docker/launch.sh` | `/usr/local/launch.sh` |
+| Env-var-to-config script | `docker/druid.sh` | `/usr/local/druid.sh` |
+
+Several environment variables are defined. `DRUID_HOME` is useful at
+runtime.
+
+| Name | Description |
+| ---- | ----------- |
+| `DRUID_HOME` | Location of the Druid install |
+| `DRUID_VERSION` | Druid version used to build the image |
+| `JAVA_HOME` | Java location |
+| `JAVA_VERSION` | Java version |
+| `MYSQL_VERSION` | MySQL version (DB, connector) (not actually used) |
+| `MYSQL_DRIVER_CLASSNAME` | Name of the MySQL driver (not actually used) |
+| `CONFLUENT_VERSION` | Kafka Protobuf library version (not actually used) |
+
+## Shared Directory
+
+The image assumes a "shared" directory passes in additional configuration
+information, and exports logs and other items for inspection.
+
+* Location in the container: `/shared`
+* Location on the host: `/target/shared`
+
+This means that each test group has a distinct shared directory,
+populated as needed for that test.
+
+Input items:
+
+| Item | Description |
+| ---- | ----------- |
+| `conf/` | `log4j.xml` config (optional) |
+| `hadoop-xml/` | Hadoop configuration (optional) |
+| `hadoop-dependencies/` | Hadoop dependencies (optional) |
+| `lib/` | Extra Druid class path items (optional) |
+
+Output items:
+
+| Item | Description |
+| ---- | ----------- |
+| `logs/` | Log files from each service |
+| `tasklogs/` | Indexer task logs |
+| `kafka/` | Kafka persistence |
+| `db/` | MySQL database |
+| `druid/` | Druid persistence, etc. |
+
+Note on the `db` directory: the MySQL container creates this directory
+when it starts. If you start, then restart the MySQL container, you *must*
+remove the `db` directory before restart or MySQL will fail due to existing
+files.
+
+### Third-Party Logs
+
+The three third-party containers are configured to log to the `/shared`
+directory rather than to Docker:
+
+* Kafka: `/shared/logs/kafka.log`
+* ZooKeeper: `/shared/logs/zookeeper.log`
+* MySQL: `/shared/logs/mysql.log`
+
+## Entry Point
+
+The container launches the `launch.sh` script which:
+
+* Converts environment variables to config files.
+* Assembles the Java command line arguments, including those
+ explained above, and the just-generated config files.
+* Launches Java as "pid 1" so it will receive signals.
+
+### Run Configuration
+
+The "raw" Java environment variables are a bit overly broad and result
+in copy/paste when a test wants to customize only part of the option, such
+as JVM arguments. To assist, the image breaks configuration down into
+smaller pieces, which it assembles prior to launch.
+
+| Enviornment Viable | Description |
+| ------------------ | ----------- |
+| `DRUID_SERVICE` | Name of the Druid service to run in the `server $DRUID_SERVICE` option |
+| `DRUID_INSTANCE` | Suffix added to the `DRUID_SERVICE` to create the log file name.
+Use when running more than one of the same service. |
+| `DRUID_COMMON_JAVA_OPTS` | Java options common to all services |
+| `DRUID_SERVICE_JAVA_OPTS` | Java options for this one service or instance |
+| `DEBUG_OPTS` | Optional debugging Java options |
+| `LOG4J_CONFIG` | Optional Log4J configuration used in `-Dlog4j.configurationFile=$LOG4J_CONFIG` |
+| `DRUID_CLASSPATH` | Optional extra Druid class path |
+
+In addition, three other shared directories are added to the class path if they exist:
+
+* `/shared/hadoop-xml` - included itself
+* `/shared/lib` - Included as `/shared/lib/*` to include extra jars
+* `/shared/resources` - included itself to hold extra class-path resources
+
+### `init` Process
+
+Middle Manager launches Peon processes which must be reaped.
+Add [the following option](https://docs.docker.com/compose/compose-file/compose-file-v2/#init)
+to the Docker Compose configuration for this service:
+
+```text
+ init: true
+```
+
+## Extensions
+
+The following extensions are installed in the image:
+
+```text
+druid-avro-extensions
+druid-aws-rds-extensions
+druid-azure-extensions
+druid-basic-security
+druid-bloom-filter
+druid-datasketches
+druid-ec2-extensions
+druid-google-extensions
+druid-hdfs-storage
+druid-histogram
+druid-kafka-extraction-namespace
+druid-kafka-indexing-service
+druid-kerberos
+druid-kinesis-indexing-service
+druid-kubernetes-extensions
+druid-lookups-cached-global
+druid-lookups-cached-single
+druid-orc-extensions
+druid-pac4j
+druid-parquet-extensions
+druid-protobuf-extensions
+druid-ranger-security
+druid-s3-extensions
+druid-stats
+it-tools
+mysql-metadata-storage
+postgresql-metadata-storage
+simple-client-sslcontext
+```
+
+If more are needed, they should be added during the image build.
+
diff --git a/integration-tests-ex/docs/druid-config.md b/integration-tests-ex/docs/druid-config.md
new file mode 100644
index 00000000000..5d715f43ca0
--- /dev/null
+++ b/integration-tests-ex/docs/druid-config.md
@@ -0,0 +1,138 @@
+
+
+# Druid Configuration
+
+In a normal install, Druid obtains configuration from properties files:
+
+* ` /_common/common.runtime.properties`
+* ` //runtime.properties`
+
+In the container, Druid uses the same mechanism, though the common properties
+file is empty. The container could simply mount the `runtime.properties` file.
+However, doing so runs into the normal issues with Druid configuration: Druid
+provides no form of inheritance: we'd have to copy/paste the same properties
+over and over, which would be a maintenance headache.
+
+Instead, the images use the same technique as the
+[production Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html):
+we pass in a large number of environment variables.
+
+The test configuration extends the production set to include extra
+variables. Thus there are two kinds:
+
+* General configuration (capitalized)
+* Druid configuration file settings (lower case)
+
+## Configuration Flow
+
+We use `docker-compose` to gather up the variables. From most specific
+(highest priority) to most general, configuration comes from:
+
+* An environment variable set by the script which launches Docker Compose.
+ (Use sparingly, only for different test "modes" such as choosing the
+ DB driver, when we will use a different mode across diffrerent test runs.)
+* As in-line settings in the `environment` section in the Docker Compose
+ definition for each service.
+* In the service-specific `compose/environment-configs/.env` file.
+* In the common `compose/environment-configs/common.env` file.
+
+Make test-specific changes in the test-specific Docker compose file. Make
+changes to the `*.env` files only if you are certain that the change should
+apply to all tests. An example is when we change something in our product
+configs.
+
+The set of defined environment variables starts with the
+`druid/conf/single-server/micro-quickstart` settings. It would be great to generate
+these files directly from the latest quickstart files. For now, it is a manual
+process to keep the definitions in sync.
+
+These are defined in a hierarchy:
+
+* `common.env` - roughly equivalent to the `_common` configuration area in Druid:
+ contains definitions common to all Druid services. Services can override any
+ of the definitions.
+* `.env` - base definitions for each service, assume it runs stand-alone.
+ Adjust if test cluster runs multiple instances. Rougly equivalent to the
+ service-specific `runtime.properties` file.
+* `docker-compose.yaml` - test-specific settings.
+
+The `launch.sh` script converts these variables to config files in
+`/tmp/conf/druid`. Those files are then added to the class path.
+
+## Druid Config Settings
+
+To set a Druid config variable, replace dots in the name with underscores.
+
+In the usual properties file:
+
+```text
+druid.auth.basic.common.maxSyncRetries=20
+```
+
+In an environment config file:
+
+```text
+druid_auth_basic_common_maxSyncRetries=20
+```
+
+```text
+ environment:
+ - druid_auth_basic_common_maxSyncRetries=20
+```
+
+For everyone's sanity, please include a comment to explain the reason for
+the setting if it differs from the Quickstart defaults.
+
+## Special Config Variables
+
+The test configuration goes beyond the production Druid image configuration
+to add several extensions specfically for tests. These are variables which
+handle some specific part of the configuration to avoid what would otherwise
+require redundant copy/paste. See the [Docker section](docker.md) for the
+details.
+
+## Shared Directory
+
+Druid configuration includes not just the config files, but also items
+on the Druid class path. These are provided via a `shared` directory mounted
+into the container at `/shared`.
+The shared directory is built in the `target/` folder for each test
+category.
+
+The `launch.sh` script fills in a number of implicit configuration items:
+
+| Item | Description |
+| ---- | ----------- |
+| Heap dump path | Set to `${SHARED}/logs/` |
+| Log4J config | Optional at `${SHARED}/conf/log4j.xml` |
+| Hadoop config | Optional at `${SHARED}/hadoop-xml` |
+| Extra libraries | Optional at `${SHARED}/lib` |
+| Extra resources | Optional at `${SHARED}/resources` |
+
+`${SHARED}/resources` is the place to put things like a custom `log4j2.xml`
+file.
+
+## Security Setup
+
+Tests can run with or without security enabled. (Security setup is a work in progress,
+the prior integration tests enabled security for all tests.)
+
+* `auth.env` - Additional definitions to create a secure cluster. Also requires that
+ the client certificates be created. Add this to tests which test security.
diff --git a/integration-tests-ex/docs/guide.md b/integration-tests-ex/docs/guide.md
new file mode 100644
index 00000000000..795a725e690
--- /dev/null
+++ b/integration-tests-ex/docs/guide.md
@@ -0,0 +1,241 @@
+
+
+# Test Creation Guide
+
+You've played with the existing tests and you are ready to create a new test.
+This section walks you through the process. If you are converting an existing
+test, then see the [conversion guide](conversion.md) instead. The details
+of each step are covered in other files, we'll link them from here.
+
+## Category
+
+The first quesetion is: should your new test go into an existing category,
+or should you create a new one?
+
+You should use an existing category if:
+
+* Your test is a new case within an obviously-existing category.
+* Your test needs the same setup as an existing category, and is quick
+ to run. Using the existing category avoids the need to fire up a
+ Docker cluster just for your test.
+
+You should create a new category if:
+
+* Your test uses a customized setup: set of services, service
+ configuration, set of external dependencies, instead.
+* Your test will run for an extended time, and is best run in
+ parallel with other tests in a build envrionment. Your test
+ can share a cluster configuration with an existing test, but
+ the new category allows the test to run by itself.
+
+When your test *can* reuse an existing cluser definition, then the question is
+about time. It takes significan time (minutes) to start a Docker cluster. We clearly
+don't want to pay that cost for a test that runs for seconds, if we could just add the
+test to another category. On the other hand, if you've gone crazy and added a huge
+suite of tests that take 20 minutes to run, then there is a huge win to be had by
+running the tests in parallel, even if they reuse an existing cluster configuration.
+Use your best judgment.
+
+The existing categories are listed in the
+`org.apache.druid.testsEx.categories` package. The classes there represent
+[JUnit categories](
+https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html).
+See [Test Category](tests.md#Test+Category) for details.
+
+If you create a new category, but want to reuse the configuration of
+an existing category, add the `@Cluster` annotation as described in the above
+link. Note: be sure to link to a "base" category, not to a category that, itself,
+has a `@Cluster` annotation.
+
+If you use the `@Cluster` annotation, you must also add a mapping in the
+`cluster.sh` file. See the top of the file for an example.
+
+## Cluster Configuration
+
+If you create a new category, you must define a new cluster. There are two parts:
+
+* Docker compose
+* Test configuration
+
+### Docker Compose
+
+Create a new folder: `custer/`, then create a `docker-compose.yaml` file
+in that folder. Define your cluster by borrowing heavily from existing files.
+See [compose](compose.md) for details.
+
+The only trick is if you want to include a new external dependency. The preferred
+approach is to use an "official" image. If you must, you can create a custom image
+in the `it-image` module. (We've not yet done that, so if you need a custom image,
+let us know and we'll figure it out.)
+
+### Test Configuration
+
+Tests need a variety of configuration information. This is, at present, more
+complex than we might like. You will at least need:
+
+* Describe the Docker Compose cluster
+* Provide test-specific properties
+
+You may also need:
+
+* Test-specific Guice modules
+* Environment variable bindings to various properties
+* MySQL statements to pre-populate the Druid metastore DB
+* And so on.
+
+### Test Config File
+
+The cluster and properties are defined in a config file. Create a folder
+`src/test/resources/cluster/`. Then add a file called `docker.yaml`.
+Crib the contents from the same category from which you borrowed the Docker
+Compose definitions. Strip out properties and metastore statements you don't
+need. Add those you do need. See [Test Configuration](test-config.md) for the
+gory details of this file.
+
+### Test Config Code
+
+You may also want to customize Guice, environment variable bindings, etc.
+This is done in the [test setup](tests.md#Initialization) method in your test.
+
+## Start Simple
+
+There are *many* things that can go wrong. It is best to start simple.
+
+### Verify the Cluster
+
+Start by ensuring your cluster works.
+
+* Define your cluster as described above. Or, pick one to reuse.
+* Verify the cluster using `it.sh up `.
+* Look at the Docker desktop UI to ensure the cluster says up. if not,
+ track down what went wrong. Look at both the Docker (stdout) and
+ Druid (`target//logs/.log`) files.
+
+### Starter Test
+
+Next, create your test file as described above and in [Tests](tests.md).
+
+* Create the test class.
+* Add the required annotations.
+* Create a simple test function that just prints "hello, world".
+* Create your `docker.yaml` file as decribed above.
+* Start your cluster, as described above, if not already started.
+* Run the test from your IDE.
+* Verify that the test "passes" (that is, it prints the message.)
+
+If so, then this means that your test connected to your custer and
+verified the health of all the services declared in your `docker.yaml` file.
+
+If something goes wrong, you'll know it is in the basics. Check your
+cluster status. Double-check the `docker.yaml` structure. Check ports.
+Etc.
+
+### Client
+
+Every test is a Druid client. Determine which service API you need. Find an
+existing test client. The `DruidClusterAdminClient` is the "modern" way to
+interact with the cluster, but thus far has a limited set of methods. There
+are older clients as well, but they tend to be quirky. Feel free to extend
+`DruidClusterAdminClient`, or use the older one: whatever works.
+
+Inject the client into your test. See existing tests for how this is done.
+
+Revise your "starter" test to do some trivial operation using the client.
+Retest to ensure things work.
+
+### Test Cases
+
+From here, you can start writing tests. Explore the existing mechanisms
+(including those in the original `druid-integration-tests` module which may
+not yet have been ported to the new framework yet.) For example, there are
+ways to store specs as files and parameterize them in tests. There is a
+syntax for running queries and specifying expected results.
+
+You may have to create a new tool to help with your test. If you do,
+try to use the new mechanisms, such as `ResolvedClusterConfig` rather than
+using the old, cumbersome ones. Post questions in Slack so we can help.
+
+### Extensions
+
+Your test may need a "non-default" extension. See [Special Environment Variables](
+compose.md#Special+Environment+Variables) for how to specify test-specific
+extensions. (Hint: don't copy/paste the full load list!)
+
+Extensions have two aspects in ITs. They act like extensions in the Druid servers
+running in Docker. So, the extension must be avaialble in the Docker image. All
+standard Druid extensions which are available in the Druid distribution, are also
+available in the image. The may not be enabled, however. Hence the need to define
+the custom load list.
+
+Your test may use code from the extension. To the *tests*, however, the extension
+is just another jar: it must be listed in the `pom.xml` file. There is no such
+thing as a "Druid extensions" to the tests themselves.
+
+If you test an extension that is *not* part of the Druid distributeion, then it
+has to get into the image. Reach out on the slack mailing list so we can discuss
+solutions (such as mounting a directory that contains the extension).
+
+### Retries
+
+The old IT framework was very liberal in its use of retries. Retires were
+used to handle:
+
+* the time lag in starting a cluster,
+* the latency inherent in events propagaing through a distributed system
+ (such as when segments get published),
+* random network failures,
+* flaky tests.
+
+The new framework takes a stricter view. The framework itself will ensure
+service are ready (using the Druid API for that purpose.) If a server reports
+itself ready, but still fails on one of your API calls, then we've got a bug
+to fix. Don't use retries to work around this issue because users won't know
+to do this.
+
+In the new framwork, tests should not be flaky. Flaky tests are a drag on
+development; they waste time. If your test is flaky, please fix it. Don't count
+on the amount of times things take: a busy build system will run much slower than
+your dedicated laptop. And so on.
+
+Ideally, Druid would provide a way to positively confirm that an action has
+occurred. Perhaps this might be a test-only API. Otherwise, a retry is fine, but
+should be coded into your test. (Or, better, implemented in a client.) Do this only
+if we document that, for that API, users should poll. Otherwise, again, users of
+the API under test won't know to retry, and so the test shouldn't do so either.
+
+This leaves random failures. The right place to handle those is in the client,
+since they are independent of the usage of the API.
+
+The result of the above is that you should not need (or use) the `ITRetryUtil`
+mechanism. No reason for your test to retry 240 times if something is really wrong
+or your test is flaky.
+
+This is an area under development. If you see a reason to retry, lets discuss it
+and put it in the proper place.
+
+### Travis
+
+Run your tests in the IDE. Try them using `it.sh test `. If that passes
+add the test to Travis. The details on how to do so are still being worked out.
+Likely, you will just copy/paste an existing test "stanza" to define your new
+test. Your test will run in parallel with all other IT categories, which is why
+we offered the advice above: the test has to have a good reason to fire up yet
+another build task.
+
diff --git a/integration-tests-ex/docs/history.md b/integration-tests-ex/docs/history.md
new file mode 100644
index 00000000000..1a35be30323
--- /dev/null
+++ b/integration-tests-ex/docs/history.md
@@ -0,0 +1,96 @@
+
+
+
+## History
+
+This revision of the integration test Docker scripts is based on a prior
+integration test version, which is, in turn, based on
+the build used for the public Docker image used in the Druid tutorial. If you are familiar
+with the prior structure, here are some of the notable changes.
+
+* Use of "official" images for third-party dependencies, rather than adding them
+ to the Druid image. (Results in *far* faster image builds.)
+* This project splits the prior `druid-integration-tests` project into several parts. This
+ project holds the Druid Docker image, while sibling projects hold the cluster definition
+ and test for each test group.
+ This allows the projects to better utilize the standard Maven build phases, and allows
+ better partial build support.
+* The prior approach built the Docker image in the `pre-integration-test` phase. Here, since
+ the project is separate, we can use the Maven `install` phase.
+* The prior structure ran *before* the Druid `distribution` module, hence the Druid artifacts
+ were not available, and the scripts did its own build, which could end up polluting the
+ Maven build cache. This version runs after `distribution` so it can reuse the actual build
+ artifacts.
+* The `pom.xml` file in this project does some of the work that that `build_run_cluster.sh`
+ previously did, such as passing Maven versions into Docker.
+* The work from the prior Dockerfile and `base-setup.sh` are combined into the revised
+ `base-setup.sh` here so that the work is done in the target container.
+* Since the prior approach was "all-in-one", it would pass test configuration options into
+ the container build process so that the container is specific to the test options. This
+ project attempts to create a generic container and instead handle test-specific options
+ at container run time.
+* The detailed launch commands formerly in the Dockerfile now reside in
+ `$DRUID_HOME/launch.sh`.
+* The prior version used a much-extended version of the public launch script. Those
+ extensions moved into `launch.sh` with the eventual goal of using the same launch
+ scripts in both cases.
+* The various `generate_*_cert.sh` scripts wrote into the source directory. The revised
+ scripts write into `target/shared/tls`.
+* The shared directory previously was in `~/shared`, but that places the directory outside
+ of the Maven build tree. The new location is `$DRUID_DEV/docker/base-docker/target/shared`.
+ As a result, the directory is removed and rebuild on each Maven build. The old location was
+ removed via scripts, but the new one is very clearly a Maven artifact, and thus to be
+ removed on a Maven `clean` operation.
+* The prior approach had security enabled for all tests, which makes debugging hard.
+ This version makes security optional, it should be enabled for just a security test.
+* The orginal design was based on TestNG. Revised tests are based on JUnit.
+* The original tests had "test groups" within the single directory. This version splits
+ the former groups into projects, so each can have its own tailored cluster definition.
+* Prior images would set up MySQL inline in the container by starting the MySQL engine.
+ This led to some redundancy (all images would do the same thing) and also some lost
+ work (since the DBs in each container are not those used when running.) Here, MySQL
+ is in its own image. Clients can update MySQL as needed using JDBC.
+* Prior code used Supervisor to launch tasks. This version uses Docker directly and
+ runs one process per container (except for Middle Manager, which runs Peons.)
+
+## History
+
+The current work builds on the prior integration tests, with changes to
+simplify and speed up the process.
+
+* The prior tests required a separate Docker build for each test "group"
+ Here, the former groups are sub-projects. All use the same Docker image.
+* The prior code used the long-obsolte TestNG. Tests here use JUnit.
+* The prior test used a TestNG suite to create test intances and inject
+ various items using Guice. This version uses an `Initializer` class to
+ do roughly the same job.
+* The prior tests required test configuration be passed in on the command
+ line, which is tedious when debugging. This version uses a cluster
+ configuation file instead.
+* The prior version perfomed MySQL initialization in the Docker container.
+ But, since each test would launch multiple containers, that work was
+ done multiple times. Here the work is done by the test itself.
+* The prior version had a single "shared" directory for all tests in
+ `~/shared`. This version creates a separate shared folder for each
+ test module, in `/target/shared`. This ensures that Maven will
+ delete everything between test runs.
+* This version removes many of the `druid-` prefixes on the container
+ names. We assume that the cluster runs as the only Docker app locally,
+ so the extra naming just clutters things.
\ No newline at end of file
diff --git a/integration-tests-ex/docs/maven.md b/integration-tests-ex/docs/maven.md
new file mode 100644
index 00000000000..6d17aebe452
--- /dev/null
+++ b/integration-tests-ex/docs/maven.md
@@ -0,0 +1,255 @@
+
+
+# Maven Structure
+
+The integration tests are built and run as part of Druid's Maven script.
+Maven itself is used by hand, and as part of the [Travis](travis.md) build
+proces. Running integration tests in maven is a multi-part process.
+
+* Build the product `distribution`.
+* Build the test image. The tests run against the Maven-created Druid build,
+ and so appear in the root `pom.xml` file *after* the `distribution`
+ project which builds the Druid tarball.
+* Run one or more ITs. Each Maven run includes a single test category and its
+ required Druid cluster.
+
+Travis orchestrates the above process to run the ITs in parallel. When you
+run tests locally, you do the above steps one by one. You can, of course, reuse
+the same disribution for multiple image builds, and the same image for multiple
+test runs.
+
+## Build the Distribution and Image
+
+Use the following command to run the ITs, assuming `DRUID_DEV` points
+to your Druid development directory:
+
+```bash
+cd $DRUID_DEV
+mvn clean package -P dist,test-image,skip-static-checks \
+ -Dmaven.javadoc.skip=true -DskipUTs=true
+```
+
+The various pieces are:
+
+* `clean`: Remove any existing artifacts, and any existing Docker image.
+* `install`: Build the Druid code and write it to the local Maven repo.
+* `-P dist`: Create the Druid distribution tarball by pulling jars from
+ the local Maven repo.
+* `-P test-image`: Build the Docker images by grabbing the Druid tarball
+ and pulling additional dependencies into the local repo, then stage them
+ for Docker.
+* Everything else: ignore parts of the build not needed for the ITs, such
+ as static checks, unit tests, Javadoc, etc.
+
+Once you've done the above once, you can do just the specific part you want
+to repeat during development. See below for details.
+
+See [quickstart](quickstart.md) for how to run the two steps separately.
+
+## Run Each Integration Test Category
+
+Each pass through Maven runs a single test category. Running a test category
+has three parts, spelled out in Maven:
+
+* Launch the required cluster.
+* Run the test category.
+* Shut down the cluster.
+
+Again, see [quickstart](quickstart.md) for how to run the three steps separately,
+and how to run the tests in an IDE.
+
+To do the task via Maven:
+
+```bash
+cd $DRUID_DEV
+mvn verify -P docker-tests,skip-static-checks,IT- \
+ -Dmaven.javadoc.skip=true -DskipUTs=true
+```
+
+The various pieces are:
+
+* `verify`: Run the steps up to the one that checks the output of the ITs. Because of
+ the extra cluster step in an IT, the build does not fail if an IT failse. Instead,
+ it continues on to clean up the cluster, and only after that does it check test
+ sucess in the `verify` step.
+* `` selects the category to run.
+
+The profiles allow you to build the test image once during debugging,
+and reuse it across multiple test runs. (See [Debugging](debugging.md).)
+
+## Dependencies
+
+The Docker image inclues three third-party dependencies not included in the
+Druid build:
+
+* MySQL connector
+* MariaDB connector
+* Kafka Protobuf provider
+
+We use dependency rules in the `test-image/pom.xml` file to cause Maven to download
+these dependencies into the Maven cache, then we use the
+`maven-dependency-plugin` to copy those dependencies into a Docker directory,
+and we use Docker to copy the files into the image. This approach avoids the need
+to pull the dependency from a remote repository into the image directly, and thus
+both speeds up the build, and is kinder to the upstream repositories.
+
+If you add additional dependencies, please follow the above process. See the
+`pom.xml` files for examples.
+
+## Environment Variables
+
+The build environment users environment variables to pass information to Maven.
+Maven communicates with Docker and Docker Compose via environment variables
+set in the `exec-maven-plugin` of various `pom.xml` files. The environment
+variables then flow into either the Docker build script (`Dockerfile`) or the
+various Docker Compose scripts (`docker-compose.yaml`). It can be tedious to follow
+this flow. A quick outline:
+
+* The build environment (such as Travis) sets environment variables, or passes values
+ to maven via the `-d=
+
+# Future Work
+
+The present version establishes the new IT framework. Work completed to
+date includes:
+
+* Restructure the Docker images to use the Druid produced from the
+ Maven build. Use "official" images for dependencies.
+* Restructure the Docker compose files.
+* Create the cluster configuration mechanisms.
+* Convert one "test group" to a sub-module: "high-availability".
+* Create the `pom.xml`, scripts and other knick-knacks needed to tie
+ everything together.
+* Create the initial test without using security settings to aid
+ debugging.
+
+However, *much* work remains:
+
+* Convert remaining tests.
+* Decide when we need full security. Convert the many certificate
+ setup scripts.
+* Support cluster types other than Docker.
+
+## Open Tasks
+
+The following detail items are open:
+
+* Disable a test if the `disabled` type is set in the test configuration
+ file. Apply it to disable the HA tests for all but Docker.
+* Handle missing config files: generate a "dummy" that is disabled.
+* When launching a container build or test run from Maven, write
+ environment variables to a `target/env.sh` file so that the user
+ doesn't have to find them manually to run the helper scripts.
+* There is some redundancy in each test group project. Figure out
+ solutions:
+ * The `cluster.sh` script
+ * Boilerplate in the `pom.xml` file.
+* Move test data from `/resources` to `/shared/data`. Build up the
+ data directory from multiple sources during cluster launch.
+* Sort out which data and spec files are actually used. Remove those
+ which are not used. Sort the files by test-specific and shared
+ across tests by moving them into different directories.
+
+## Later Tasks
+
+The "public" and "integration test" versions of the Docker images have diverged significantly,
+which makes it harder to "test what we ship." Differences include:
+
+* Different base image
+* Different ways to set up dependencies.
+* Different paths within the container.
+* Different launch scripts.
+* The test images place Druid in `/usr/local`, the public images in `/opt`.
+
+The tests do want to do things beyond what the "public" image does. However, this should
+not require a fork of the builds. To address this issue:
+
+* Extend this project to create a base common to the "public" and integration test images.
+* Extend the integration test image to build on top of the public image.
diff --git a/integration-tests-ex/docs/quickstart.md b/integration-tests-ex/docs/quickstart.md
new file mode 100644
index 00000000000..1a3fcb22f7b
--- /dev/null
+++ b/integration-tests-ex/docs/quickstart.md
@@ -0,0 +1,201 @@
+
+
+# Quickstart
+
+If you just need to know how to build, run and use the tests, this
+is the place. You can refer to the detailed material later as you
+add new tests or work to improve the tests.
+
+## Example Test
+
+When first learning the framework, you can try thing out using the
+`HighAvailability` test. Of the tests converted thus far, it is the
+one that runs for the shortest time (on the order of a minute or two.)
+
+## Working with Individual Tests
+
+To work with tests for development and debugging, you can break the
+above all-in-one step into a number of sub-steps.
+
+* [Build Druid](#Build Druid).
+* [Build the Docker image](#Build the Docker Image).
+* [Start a cluster](#Start a Cluster).
+* [Run a test from the command line](#Run a Test from the Command Line).
+* [Run a test from an IDE](#Run a Test from an IDE).
+* [Stop the cluster](#Stop the Cluster).
+* [Clean up](#Clean Up).
+
+## Build Druid
+
+The integration tests start with a Druid distribution in `distribution/target`,
+which you can build using your preferred Maven command line. Simplest:
+
+```
+cd $DRUID_DEV
+./it.sh dist
+```
+
+Or, in detail:
+
+For example:
+To make the text a bit simpler, define a variable for the standard settings:
+
+```bash
+export MAVEN_IGNORE=-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true
+```
+
+```bash
+mvn clean package -P dist $MAVEN_IGNORE -T1.0C
+```
+
+Hint: put this into a script somewhere, such as a `~/bin` directory and
+add that to your `PATH`. A good name is `bdru` (Build DRUid).
+Here is the full script:
+
+```bash
+#! /bin/bash
+
+mvn clean package -P dist,skip-static-checks,skip-tests \
+ -Dmaven.javadoc.skip=true -T1.0C $*
+```
+
+Now you can run the any Druid build with the above script. To resume a
+build:
+
+```bash
+> bdru -rf :foo
+```
+
+## Build the Docker Image
+
+You must rebuild the Docker image whenever you rebuild the Druid distribution,
+since the image includes the distribution. You also will want to rebuild the
+image if you change the `it-image` project which contains the build scripts.
+
+```bash
+./it.sh image
+```
+
+In detail, and assuming `DRUID_DEV` points to your Druid build directory,
+to build the image (only):
+
+```bash
+cd $DRUID_DEV/integration-tests-ex/image
+mvn install -P test-image $MAVEN_IGNORE
+```
+
+The above has you `cd` into the project to avoid the need to disable all the
+unwanted bits of the Maven build.
+
+See [this page](docker.md) for more information.
+
+## Run an IT from the Command Line
+
+```bash
+./it.sh test
+```
+
+Or, in detail:
+
+```bash
+mvn verify -P docker-tests,IT- -pl :druid-it-cases \
+ -P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true -DskipUTs=true
+```
+
+Where `` is one of the test categories.
+
+Or
+
+```bash
+cd $DRUID_DEV/integration-tests-ex/cases
+mvn verify -P skip-static-checks,docker-tests,IT- \
+ -Dmaven.javadoc.skip=true -DskipUTs=true \
+ -pl :druid-it-cases
+```
+
+If the test fails, find the Druid logs in `target/shared/logs` within the
+test group project.
+
+## Start a Cluster
+
+The previous generation of tests were organized into TestNG groups. This
+revision moves those groups into Maven modules. Each group has a distinct
+cluster configuration. (In fact, it is the cluster configuration which defines
+the group: we combine all tests with the same configuration into the same module.)
+So, to start a cluster, you have to pick a group to run. See
+[this list](maven.md#Modules) for the list of groups.
+
+```bash
+./it.sh up
+```
+
+Or, in detail:
+
+```bash
+cd $DRUID_DEV/integration-tests-ex/cases
+./cluster.sh up
+```
+
+You can use Docker Desktop to monitor the cluster. Give things about 30 seconds
+or a minute: if something is going to fail, it will happen during starup and you'll
+see that one or more containers exited unexpectedly.
+
+Remember to first shut down any Druid cluster you may already be running on
+your machine.
+
+See [this page](docker.md) for more information.
+
+## Run a Test from an IDE
+
+To run an IT in your IDE:
+
+* Find the IT to run.
+* Run it as a JUnit test.
+
+The tests are specifically designed to require no command-line setup: you can
+just run them directly.
+
+## Stop the Cluster
+
+Once you are done with your cluster, you can stop it as follows:
+
+
+```bash
+./it.sh down
+```
+
+Or, in detail:
+
+```bash
+cd $DRUID_DEV/integration-tests-ex/
+./cluster.sh down
+```
+
+## Clean Up
+
+You can remove the Docker image when you no longer need it:
+
+```bash
+cd $DRUID_DEV
+mvn clean -P test-image
+```
+
+It is usually fine to skip this step: the next image build will replace
+the current one anyway.
diff --git a/integration-tests-ex/docs/runtime.md b/integration-tests-ex/docs/runtime.md
new file mode 100644
index 00000000000..c276eee207e
--- /dev/null
+++ b/integration-tests-ex/docs/runtime.md
@@ -0,0 +1,105 @@
+
+
+## Test Runtime Behavior
+
+This section explains how the various configuration pieces come together
+to run tests.
+
+See also:
+
+* [Docker setup](docker.md)
+* [Docker Compose setup](compose.md)
+* [Test setup](test-config.md)
+
+## Shared Resources
+
+This module has a number of folders that are used by all tests:
+
+* `compose`: A collection of Docker Compose scripts that define the basics
+ of the cluster. Each test "inherits" the bits that it needs.
+* `compose/environment-configs`: Files which define, as environment variables,
+ the runtime properties for each service in the cluster. (See below
+ for details.)
+* `assets`: The `log4j2.xml` file used by images for logging.
+
+## Container Runtime Structure
+
+The container itself is a bit of a hybrid. The Druid distribution, along
+with some test-specific extensions, is reused. The container also contains
+libraries for Kafka, MySQL and MariaDB.
+
+Druid configuration is passed into the container as environment variables,
+and then converted to a `runtime.properties` file by the container launch
+script. Though a bit of a [Rube Goldberg](https://en.wikipedia.org/wiki/Rube_Goldberg)
+mechanism, it does have one important advantage over the usual Druid configs:
+we can support inheritance and overrides. The various `.env` files
+provide the standard configurations. Test-specific Docker Compose files can
+modify any setting.
+
+The container mounts a shared volume, defined in the `target/shared` directory
+of each test module. This volume can provide extra libraries and class path
+items. The one made available by default is `log4j2.xml`, but tests can add
+more as needed.
+
+Container "output" also goes into the shared folder: logs, "cold storage"
+and so on.
+
+Each container exposes the Java debugger on port 8000, mapped to a different
+host port for each container.
+
+Each container exposes the usual Druid ports so you can work with the
+container as you would a local cluster. Two handy tools are the Druid
+console and the scriptable [Python client](https://github.com/paul-rogers/druid-client).
+
+## Test Execution
+
+Tests run using the Maven [failsafe](https://maven.apache.org/surefire/maven-failsafe-plugin/)
+plugin which is designed for integration tests. The Maven phases are:
+
+* `pre-integration-test`: Starts the test cluster with `cluster.sh up` using Docker Compose.
+* `integration-test`: Runs tests that start or end with `IT`.
+* `post-integration-test`: Stops the test cluster using `cluster.sh down`
+* `verify`: Checks the result of the integration tests.
+
+See [this example](https://maven.apache.org/surefire/maven-failsafe-plugin/examples/junit.html)
+for JUnit setup with failsafe.
+
+The basic process for running a test group (sub-module) is:
+
+* Cluser startup builds a `target/shared` directory with items to be mounted
+ into the containers, such as the `log4j2.xml` file, sample data, etc.
+ The shared directory also holds log files, Druid persistent storage,
+ the metastore (MySQL) DB, etc. See `test-image/README.md` for details.
+* The test is configured via a `druid-cluster/compose.yaml` file.
+ This file defines the services to run and their configuration.
+* The `cluster.sh up` script builds the shared directory, loads the env vars
+ defined when the image was created and starts the cluster.
+* Tests run on the local host within JUnit.
+* The `Initialization` class loads the cluster configuration (see below),
+ optionally populates the Druid metadata storage, and is used to
+ inject instances into the test.
+* The individual tests run.
+* The `cluster.sh down` script shuts down the cluster.
+
+`cluster.sh` uses the generated `test-image/target/env.sh` for versions and
+and other environment variables. This ensures that tests run with the same
+versions used to build the image. It also simplifies the Maven boilerplate to
+be copy/pasted into each test sub-project.
+
diff --git a/integration-tests-ex/docs/scripts.md b/integration-tests-ex/docs/scripts.md
new file mode 100644
index 00000000000..2c9ceb2a874
--- /dev/null
+++ b/integration-tests-ex/docs/scripts.md
@@ -0,0 +1,74 @@
+
+
+# Scripts
+
+The IT framework uses a number of scripts and it can be a bit of a mystery
+what each one does. This guide lists each of them.
+
+## `integration-tests-ex`
+
+* `it.sh` - Utility to perform many IT-related actions such as building Druid,
+ running ITs, starting a cluster, etc. Use `it.sh help` to see the list of commands.
+
+### `it-image`
+
+* `build-image.sh` - Internal script to (you guessed it), build the image.
+ Creates the `target/env.sh` file above, then invokes `docker-build.sh`.
+* `rebuild.sh` - Rebuilds the image after the above script has created the
+ `env.sh` file. Used to debug changes to the image build itself: use `rebuild.sh`
+ rather than waiting for Maven to do its thing.
+* `docker-build.sh` - Internal script to set up the needed environment
+ variables and invoke Docker to build the image. Modify this if you ned to change
+ the information passed into Docker via the command line.
+* `docker/Dockerfile` - Docker script to build the image.
+* `docker/test-setup.sh` - Script copied into the image at build time and run to
+ set up the image. Keeps the `Dockerfile` simpler (and, it is somewhat easier to
+ debug this script than the `Dockerfile` itself.)
+* `docker/launch.sh` - Container entrypoint which runs inside the container.
+ Sets up configuration and calls `druid.sh` to launch Druid itself.
+* `druid.sh` - Creates a Druid configuration file from environment variables,
+ then runs Druid within the container.
+* `target/env.sh` - Created when the image is built. Provides environment
+ variables for things like the image name, versions and so on. Used to
+ quickly rebuild the image (see [Maven configuration](docs/maven.md)) and
+ to launch tests.
+
+### `test-cases`
+
+* `cluster//*.yaml` - Base Docker Compose scripts that define the "standard"
+ Druid cluster. Tests use these files to avoid redundant copy/past of the
+ standard items.
+* `cluster.sh` - Launches or tears down a cluster for a test. Called from Maven
+ and `it.sh`. Can be used manually. See below.
+
+The options for `cluster.sh` are:
+
+```bash
+cluster.sh [-h|help|up|down|status|compose-cmd] category
+```
+
+* `up` - starts the cluster.
+* `down` - shuts down the cluster.
+* `status` - displays cluster status for debugging. Expecially useful for debugging
+ issues in Travis where we cannot directly inspect the Docker cluster itself.
+* `help` - repeats the usage line.
+* Others - passes the command on to Docker Compose.
+* `category` - the test category to launch. Performs mapping from the category name
+ to cluster name when categories share definitions.
diff --git a/integration-tests-ex/docs/test-config.md b/integration-tests-ex/docs/test-config.md
new file mode 100644
index 00000000000..4a5c7a583f7
--- /dev/null
+++ b/integration-tests-ex/docs/test-config.md
@@ -0,0 +1,588 @@
+
+
+## Test Configuration
+
+Tests typically need to understand how the cluster is structured.
+To create a test, you must supply at least three key components:
+
+* A `cluster//docker-compose.yaml` file that launches the desired cluster.
+ (The folder name `` becomes the application name in Docker.)
+* A `src/test/resources/cluster//docker.yaml` file that describes the cluster
+ for tests. This file can also include Metastore SQL statements needed to
+ populate the metastore.
+* The test itself, as a JUnit test that uses the `Initializer` class to
+ configure the tests to match the cluster.
+
+This section explains the test configuration file which defines the test
+cluster.
+
+Note that you can create multiple versions of the `docker.yaml` file. For example,
+you might want to create one that lists hosts and credentials unique to your
+debugging environment. You then use your custom version in place of the standard
+one.
+
+## Cluster Types
+
+The integration tests can run in a variety of cluster types, depending
+on the details of the test:
+
+* Docker Compose: the normal configuration that all tests support.
+* Micro Quickstart: allows for a manual cluster setup, if, say, you
+ want to run services in your IDE. Supported by a subset of tests.
+* Kubernetes: (Details needed.)
+
+Each cluster type has its own quirks. The job of the tests's cluster configuration
+file is to communicate those quirks to the test.
+
+Docker and Kubernetes use proxies to communicate to the cluster. Thus, the host
+known to the tests is different than the hosts known within the cluster. Ports
+may also are mapped differently "outside" than "inside."
+
+Clusters outside of Docker don't provide a good way to start and stop
+services, so tests that want to do that (to, say, test high availability)
+can't run except in a Docker cluster.
+
+### Specify the Cluster Type
+
+To reflect this, tests provide named configuration files. The configuration
+itself is passed in via the environment:
+
+```bash
+export TEST_CONFIG=quickstart
+```
+
+```bash
+java ... -DtestConfig=quickstart
+```
+
+The system property taskes precedence over the environment variable.
+If neither are set, `docker` is the default. The configuration file
+itself is assumed to be a resource named `/yaml/.yaml`.
+
+As a debug aide, a test can specify and ad-hoc file in the file system
+to load for one-off special cases. See `Initialization.Builder` for
+details.
+
+## Cluster Configuration Files
+
+Cluster configuration is specified in a file for ease of debugging. Since
+configuration is in a file (resource), and not in environment variables or
+system properties, you
+should need no special launch setup in your IDE to run a test that uses
+the standard Docker Compose cluster for that test.
+
+The configuration file has the same name as the cluster type and resides on
+the class path at `/yaml/.yaml` and in the source tree at
+`/src/test/resources/yaml/.yaml`. The standard names are:
+
+* `docker.yaml`: the default and required for all tests. Describes a Docker
+ Compose based test.
+* `k8s.yaml`: a test cluster running in Kubernetes. (Details needed.)
+* `local.yaml`: a local cluser such as Micro Quickstart cluster. (Details needed.)
+* `.yaml`: custom cluster configuration.
+
+Configuration files support include files. Most of the boiler-plate configuration
+should appear in commmon files. As a result, you should only need to specify
+test-specific differences in your `docker.yaml` file, with all else obtained
+from the included files.
+
+## Configuration File Syntax
+
+The configuration is a [YAML](https://yaml.org/spec/1.2.2/) file that
+has a few top-level properties and an entry for each service in your
+cluster.
+
+### `type`
+
+```yaml
+type: docker|k8s|local|disabled
+```
+
+The type explains the infrastructure that runs the cluster:
+
+* `docker`: a cluster launched in Docker, typically via Docker Compose.
+ A proxy host is needed. (See below.)
+* `k8s`: a cluster run in Kubernets. (Details needed). A proxy host
+ is needed.
+* `local`: a cluster running as processes on a network directly reachable
+ by the tests. Example: a micro-quickstart cluster running locally.
+* `disabled`: the configuration is not supported by the test.
+
+The `disabled` type is handy for tests that require Docker: you can say that
+the test is not available when the cluster is local.
+
+If the test tries to load a cluster name that does not exist, a "dummy"
+configuration is loaded instead with the type set to `disabled`.
+
+The type is separate from the cluster name (as explained earlier): there
+may be multiple names for the same type. For example, you might have two
+or three local cluster setups you wish to test.
+
+### `include`
+
+```yaml:
+include:
+ -
+```
+
+Allows including any number of other files. Similar to inheritance for
+Docker Compose. The inheritance rules are:
+
+* Properties set later in the list take precedence over properties set in
+ files earlier in the list.
+* Properties set in the file take precedence over properties set in
+ included files.
+* Includes can nest to any level.
+
+Merging occurs as follows:
+
+* Top level scalars: newer values replace older values.
+* Services: newer values replace all older settings for that service.
+* Metastore init: newer values add more queries to any list defined
+ by an earlier file.
+* Properties: newer values replace values defined by earlier files.
+
+The files are assumed to be resources (on the class path) and require
+the full path name. Example: `/cluster/Commmon/base.yaml`
+
+### `proxyHost`
+
+```yaml
+proxyHost:
+```
+
+When tests run in either Docker or Kubernetes, the test communicate with
+a proxy, which forwards requests to the cluster hosts and ports. In
+Docker, the proxy host is the machine that runs Docker. In Kubernetes,
+the proxy host is the host running the Kubernetes proxy service.
+
+There is no proxy host for clusters running directly on a machine.
+
+If the proxy host is omitted for Docker, `localhost` is assumed.
+
+### `datasourceSuffix`
+
+```yaml
+datasourceSuffix:
+```
+
+Suffix to append to data source names in indexer tests. The default
+is the empty string.
+
+### `zk`
+
+```yaml
+zk:
+
+```
+
+Specifies the ZooKeeper instances.
+
+#### `startTimeoutSecs`
+
+```yaml
+startTimeoutSecs:
+```
+
+Specifies the amount of time to wait for ZK to become available when using the
+test client. Optional.
+
+### `metastore`
+
+```yaml
+metastore:
+
+```
+
+Describes the Druid "metadata storage" (metastore) typically
+hosted in the offical MySql container. See `MetastoreConfig` for
+configuration options.
+
+#### `driver`
+
+```yaml
+driver:
+```
+
+The Driver to use to work with the metastore. The driver must be
+available on the tests's class path.
+
+#### `connectURI`
+
+```yaml:
+connectURI:
+```
+
+The JDBC connetion URL. Example:
+
+```text
+jdbc:mysql://:/druid
+```
+
+The config system supports two special fields: `` and ``.
+A string of form `` will be replaced by the resolved host name
+(proxy host for Docker) and `` with the resolved port number.
+
+#### `user`
+
+```yaml
+user:
+```
+
+The MySQL user name.
+
+
+#### `password`
+
+```yaml
+user:
+```
+
+The MySQL password.
+
+#### `properties`
+
+```yaml
+properties:
+ :
+```
+
+Optional map of additional key/value pairs to pass to the JDBC driver.
+
+### `kafka`
+
+```yaml
+zk:
+
+```
+
+Describes the optional Kafka service.
+
+### `druid`
+
+```yaml
+druid:
+ :
+
+```
+
+Describes the set of Druid services using the `ServiceConfig` object.
+Each service is keyed by the standard service name: the same name used
+by the Druid `server` option.
+
+When using inheritance, overrides replace entire services: it is not possible
+to override individual instances of the service. That is, an include file might
+define `coordinator`, but a test-specific file might override this with a
+definition of two Coordinators.
+
+### `properties`
+
+```yaml
+properties:
+ :
+```
+
+Optional set of properties to use to configuration the Druid components loaded
+by tests. This is the test-specific form of the standard Druid `common.runtime.properties`
+and `runtime.properties` files. Because the test runs as a client, the server
+files are not available, and might not even make sense. (The client is not
+a "service", for example.) Technically, the properties listed here are added to
+Guice as the one and only `Properties` object.
+
+Typically most components work using the default values. Tests are free to change
+any of these values for a given test scenario. The properties are
+the same for all tests within a category. However, they can be changed via environment
+variables via the environment variable "binding" mechanism described in
+[tests](tests.md).
+
+The "JSON configuration" mechanism wants all properties to be strings. YAML
+will deserialize number-like properties as numbers. To avoid confusion, all
+properties are converted to strings before being passed to Druid.
+
+When using inheritance, later properties override earlier properties. Environment
+variables, if bound, override the defaults specified in this section. Command-line
+settings, if provided, have the highest priority.
+
+A number of test-specific properties are avilable:
+
+* `druid.test.config.cloudBucket`
+* `druid.test.config.cloudPath`
+
+### `settings`
+
+The settings section is much like the properties section, and, indeed, are converted
+to properties internally. Settings are a fixed set of values that map to the config
+files used in the prior tests. Keys include:
+
+| Setting | Property | Environment Variable |
+| `druid_storage_type` | - | - |
+| `druid_storage_bucket` | `druid.test.config.cloudBucket` | `DRUID_STORAGE_BUCKET` |
+| `druid_storage_baseKey` | `druid.test.config.cloudPath` | `DRUID_STORAGE_BASEKEY` |
+| `druid_s3_accessKey` | - | `AWS_ACCESS_KEY_ID` |
+| `druid_s3_secretKey` | - | AWS_SECRET_ACCESS_KEY` |
+
+The above replaces the config file mechanism from the older tests. In general, when a
+setting is fixed for a test category, list it in the `docker.yaml` configuration file.
+When it varies, pass it in as an environment variable. As a result, the prior configuration
+file is not needed. As a result, the prior `override.config.path` property is not supported.
+
+### `metastoreInit`
+
+```yaml
+metastoreInit:
+ - sql: |
+
+```
+
+A set of MySQL statements to be run against the
+metadata storage before the test starts. Queries run in the
+order specified. Ensure each is idempotent to
+allow running tests multiple times against the same database.
+
+To be kind to readers, please format the statements across multiple lines.
+The code will compress out extra spaces before submitting the query so
+that JSON payloads are as compact as possible.
+
+The `sql` keyword is the only one supported at present. The idea is that
+there may need to be context for some queries in some tests. (To be
+enhanced as query conversion proceeds.)
+
+When using inheritance, the set of queries is the union of all queries
+in all configuration files. Base statements appear first, then included
+statements.
+
+### `metastoreInitDelaySec`
+
+```yaml
+metastoreInitDelaySec:
+```
+
+The default value is 6 seconds.
+
+The metastore init section issues queries to the MySQL DB read by the
+Coordinator. For performance, the Coordinator *does not* directly query
+the database: instead, it queries an in-memory cache. This leads to the
+following behavior:
+
+* The Coordinator starts, checks the DB, and records the poll time.
+* The test starts and updates the DB.
+* The test runs and issues a query that needs the DB contents.
+* The Coordinator checks that its poll timeout has not yet occurred
+ and returns the (empty) contents of the cache.
+* The test checks the empty contents against the expected contents,
+ notices the results differ, and fails the test.
+
+To work around this, we must change _two_ settings. First, change
+the following Druid configuration for the Coordinator:
+
+```yaml
+ - druid_manager_segments_pollDuration=PT5S
+```
+
+Second, change the `metastoreInitDelaySec` to be a bit longer:
+
+```yaml
+metastoreInitDelaySec: 6
+```
+
+The result is that the test will sit idle for 6 seconds, but that is better
+than random failures.
+
+**Note:** a better fix would be for the Coordinator to have an API that causes
+it to flush its cache. Since some tests run two coordinators, the message must be
+sent to both. An even better fix would be fore the Coordinator to detect such
+changes itself somehow.
+
+### Service Object
+
+Generic object to describe Docker Compose services.
+
+#### `service`
+
+```yaml
+service:
+```
+
+Name of the service as known to Docker Compose. Defaults to be
+the same as the service name used in this configuration file.
+
+#### `instances`
+
+```yaml
+instances:
+ -
+```
+
+Describes the instances of the service as `ServiceInstance` objects.
+Each service requires at least one instance. If more than one, then
+each instance must define a `tag` that is a suffix that distinguishes
+the instances.
+
+### Service Instance Object
+
+The service sections all allow multiple instances of each service. Service
+instances define each instance of a service and provide a number of properties:
+
+#### `tag`
+
+When a service has more than one instance, the instances must have unique
+names. The name is made up of the a base name (see below) with the tag
+appended. Thus, if the service is `cooordinator` and the tag is `one`,
+then the instance name is `coordinator-one`.
+
+The tag is required when there is more than one instance of a service,
+and is optional if there is only one instance. The tag corresponds to the
+`DRUID_INSTANCE` environment variable passed into the container.
+
+#### `container`
+
+```yaml
+container:
+```
+
+Name of the Docker container. If omitted, defaults to:
+
+* `-` if a `tag` is provided (see below.)
+* The name of the service (if there is only one instance).
+
+#### `host`
+
+```yaml
+host:
+```
+
+The host name or IP address on which the instance runs. This is
+the host name known to the _cluster_: the name inside a Docker overlay network.
+Has the same defaults as `container`.
+
+#### `port`
+
+```yaml
+port:
+```
+
+The port number of the service on the container as seen by other
+services running within Docker. Required.
+
+(TODO: If TLS is enabled, this is the TLS port.)
+
+#### `proxyPort`
+
+```yaml
+proxyPort:
+```
+
+The port number for the service as exposed on the proxy host.
+Defaults to the same as `port`. You must specify a value if
+you run multiple instances of the same service.
+
+## Conversion Guide
+
+In prior tests, a config file, and the `ConfigFileConfigProvider` class,
+provided test configuration. In this version, the file described here
+provides configuration. This section presents a mapping from the old to
+the new form.
+
+The `IntegrationTestingConfig` class, which the above class used to provide,
+is reimplemented to provide the same information
+to tests as before; only the source of the information has changed.
+
+The new framework assumes that each Druid node is configured either for
+plain text or for TLS. (If this assumption is wrong, we'll change the config
+file to match.)
+
+Many of the properties are derived from information in the configuration file.
+For example, host names (within Docker) are those given in the `druid` section,
+and ports (within the cluster and for the client) are given in `druid..intances.port`,
+from which the code computes the URL.
+
+The old system hard-codes the idea that there are two coordinators or overlords. The
+new system allows any number of instances.
+
+| Method | Old Property | New Format |
+| ------ | ------------ | ---------- |
+| Router | | |
+| `getRouterHost()` | `router_host` | `'router'` |
+| `getRouterUrl()` | `router_url` | `'router'` & `instances.port` |
+| `getRouterTLSUrl()` | `router_tls_url` | " |
+| `getPermissiveRouterUrl()` | `router_permissive_url` | " |
+| `getPermissiveRouterTLSUrl()` | `router_permissive_tls_url` | " |
+| `getNoClientAuthRouterUrl()` | `router_no_client_auth_url` | " |
+| `getNoClientAuthRouterTLSUrl()` | `router_no_client_auth_tls_url` | " |
+| `getCustomCertCheckRouterUrl()` | | " |
+| `getCustomCertCheckRouterTLSUrl()` | | " |
+| Broker | | |
+| `getBrokerHost()` | `broker_host` | `'broker'` |
+| `getBrokerUrl()` | `broker_url` | `'broker'` & `instances.port` |
+| `getBrokerTLSUrl()` | `broker_tls_url` | " |
+| Coordinator | | |
+| `getCoordinatorHost()` | `coordinator_host` | `'coordinator'` + `tag` |
+| `getCoordinatorTwoHost()` | `coordinator_two_host` | " |
+| `getCoordinatorUrl()` | `coordinator_url` | host & `instances.port` |
+| `getCoordinatorTLSUrl()` | `coordinator_tls_url` | " |
+| `getCoordinatorTwoUrl()` | `coordinator_two_url` | " |
+| `getCoordinatorTwoTLSUrl()` | `coordinator_two_tls_url` | " |
+| Overlord | | |
+| `getOverlordUrl()` | ? | `'overlord'` + `tag` |
+| `getOverlordTwoHost()` | `overlord_two_host` | " |
+| `getOverlordTwoUrl()` | `overlord_two_url` | host & `instances.port` |
+| `getOverlordTLSUrl()` | ? | " |
+| `getOverlordTwoTLSUrl()` | `overlord_two_tls_url` | " |
+| Overlord | | |
+| `getHistoricalHost()` | `historical_host` | `historical'` |
+| `getHistoricalUrl()` | `historical_url` | `'historical'` & `instances.port` |
+| `getHistoricalTLSUrl()` | `historical_tls_url` | " |
+| Overlord | | |
+| `getMiddleManagerHost()` | `middlemanager_host` | `'middlemanager'` |
+| Dependencies | | |
+| `getZookeeperHosts()` | `zookeeper_hosts` | `'zk'` |
+| `getKafkaHost()` | `kafka_host` | '`kafka`' |
+| `getSchemaRegistryHost()` | `schema_registry_host` | ? |
+| `getProperty()` | From config file | From `settings` |
+| `getProperties()` | " | " |
+| `getUsername()` | `username` | Setting |
+| `getPassword()` | `password` | Setting |
+| `getCloudBucket()` | `cloud_bucket` | Setting |
+| `getCloudPath()` | `cloud_path` | Setting |
+| `getCloudRegion()` | `cloud_region` | Setting |
+| `getS3AssumeRoleWithExternalId()` | `s3_assume_role_with_external_id` | Setting |
+| `getS3AssumeRoleExternalId()` | `s3_assume_role_external_id` | Setting |
+| `getS3AssumeRoleWithoutExternalId()` | `s3_assume_role_without_external_id` | Setting |
+| `getAzureKey()` | `azureKey` | Setting |
+| `getHadoopGcsCredentialsPath()` | `hadoopGcsCredentialsPath` | Setting |
+| `getStreamEndpoint()` | `stream_endpoint` | Setting |
+| `manageKafkaTopic()` | ? | ? |
+| `getExtraDatasourceNameSuffix()` | ? | ? |
+
+Pre-defined environment bindings:
+
+| Setting | Env. Var. |
+| `cloudBucket` | `DRUID_CLOUD_BUCKET` |
+| `cloudPath` | `DRUID_CLOUD_PATH` |
+| `s3AccessKey` | `AWS_ACCESS_KEY_ID` |
+| `s3SecretKey` | `AWS_SECRET_ACCESS_KEY` |
+| `azureContainer` | `AZURE_CONTAINER` |
+| `azureAccount` | `AZURE_ACCOUNT` |
+| `azureKey` | `AZURE_KEY` |
+| `googleBucket` | `GOOGLE_BUCKET` |
+| `googlePrefix` | `GOOGLE_PREFIX` |
+
+Others can be added in `Initializer.Builder`.
+
diff --git a/integration-tests-ex/docs/tests.md b/integration-tests-ex/docs/tests.md
new file mode 100644
index 00000000000..1ef74a77338
--- /dev/null
+++ b/integration-tests-ex/docs/tests.md
@@ -0,0 +1,411 @@
+
+
+# Test Structure
+
+The structure of these integration tests is heavily influenced by the existing
+integration test structure. In that previous structure:
+
+* Each test group ran as separate Maven build.
+* Each would build an image, start a cluster, run the test, and shut down the cluster.
+* Tests were created using [TestNG](https://testng.org/doc/), a long-obsolete
+ test framework.
+* A `IntegrationTestingConfig` is created from system properties (passed in from
+ Maven via `-D=` options).
+* A TestNG test runner uses a part of the Druid Guice configuration to inject
+ test objects into the tests.
+* The test then runs.
+
+To minimize test changes, we try to keep much of the "interface" while changing
+the "implementation". Basically:
+
+* The same Docker image is used for all tests.
+* Each test defines its own test cluster using Docker Compose.
+* Tests are grouped into categories, represented by [JUnit categories](
+ https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html).
+* Maven runs one selected category, starting and stopping the test-specific cluster
+ for each.
+* A cluster-specific directory contains the `docker-compose.yaml` file that defines
+ that cluster. Each of these files imports from common definitions.
+* Each test is annotated with the `DruidTestRunner` to handle initialization, and
+ JUnit `Category` to group the test into a category.
+* Categories can share cluster configuration to reduce redundant definitions.
+* A `docker.yaml` file defines the test configuration and creates the
+ `IntegrationTestingConfig` object.
+* Tests run as JUnit tests.
+
+The remainder of this section describes the test internals.
+
+## Test Name
+
+Due to the way the [Failsafe](
+https://maven.apache.org/surefire/maven-failsafe-plugin/integration-test-mojo.html)
+Maven plugin works, it will look for ITs with
+names of the form "IT*.java". This is the preferred form for Druid ITs. That is,
+name your test "ITSomething", not "SomethingTest" or "IntegTestSomething", etc.
+Many tests are called "ITSomethingTest", but this is a bit repetitious and redundant
+since "IT" stands for "Integration Test".
+
+## Cluster Configuration
+
+A test must have a [cluster configuration](compose.md) to define the cluster.
+There is a many-to-one relationship between test categories and test clusters.
+
+## Test Configuration
+
+See [Test Configuration](test-config.md) for details on the `docker.yaml` file
+that you create for each test module to tell the tests about the cluster you
+have defined.
+
+Test configuration allows inheritance so, as in Docker Compose, we define
+standard bits in one place, just providing test-specific information in each
+tests `docker.yaml` file.
+
+The test code assumes that the test configuration file is in
+`src/test/resources/cluster//docker.yaml`, where `` is
+the test category. The test runner loads the configuration file into
+(or, specifically that it is on the class path at `/yaml/docker.yaml`)
+a `ClusterConfig` instance.
+
+The `ClusterConfig` instance provides the backward-compatible
+`IntegrationTestingConfig` instance tha that most existing test cases use.
+New tests may want to work with `ClusterConfig` directly as the older interface
+is a bit of a muddle in several areas.
+
+## Test Category
+
+Each test is associated with a cluster definition. Maven starts the required
+cluster, runs a group of tests, and shuts down the cluster. We use the JUnit
+`Category` to identify the category for each test:
+
+```java
+@RunWith(DruidTestRunner.class)
+@Category(BatchIndex.class)
+public class ITIndexerTest extends AbstractITBatchIndexTest
+{
+ ...
+```
+
+The category is a trivial class that exists just to provide the category name.
+It can also hold annotations, which will use in a moment. When adding tests, use
+and existing category, or define a new one if you want your tests to run in
+parallel with other categories.
+
+The `test-cases` module contains all integration tests. However,
+Maven can run only one category per Maven run. You specify the category using a
+profile of the same name, but with "IT-" prefixed. Thus the Maven profile for the
+above `BatchIndex` category is `IT-BatchIndex`.
+
+Test categories may share the same cluster definition. We mark this by adding an
+annotation to the category (_not_ test) class. The test class itself:
+
+```java
+@RunWith(DruidTestRunner.class)
+@Category(InputFormat.class)
+public class ITLocalInputSourceAllInputFormatTest extends AbstractLocalInputSourceParallelIndexTest
+{
+ ...
+```
+
+The test category class:
+
+```java
+@Cluster(BatchIndex.class)
+public class InputFormat
+{
+}
+```
+
+This says that the test above is in the `InputFormat` category, and tests in that
+category use the same cluster definition as the `BatchIndex` category. Specifically,
+to look for the cluster definition in the `BatchIndex` folders.
+
+### Defined Categories
+
+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 |
+
+The new names correspond to class names. The Test NG names were strings.
+
+## Test Runner
+
+The ITs are JUnit test, but use a special test runner to handle configuration.
+Test configuration is complex. The easiest way to configure, once the configuration
+files are set, is to use the `DruidTestRunner` class:
+
+```java
+@RunWith(DruidTestRunner.class)
+@Category(MyCategory.class)
+public class MyTest
+{
+ @Inject
+ private SomeObject myObject;
+ ...
+
+ @Test
+ public void myTest()
+ {
+ ...
+```
+
+The test runner loads the configuration files, configures Guice, starts the
+Druid lifecycle, and injects the requested values into the class each time
+a test method runs. For simple tests, this is all you need.
+
+The test runner validates that the test has a category, and handles the
+above mapping from category to cluster definition.
+
+### Parameterization
+
+The `DruidTestRunner` extends `JUnitParamsRunner` to allow parameterized tests.
+This class stays discretely out of the way if you don't care about parameters.
+To use parameters, see the `CalciteJoinQueryTest` class for an example.
+
+## Initialization
+
+The JUnit-based integration tests are designed to be as simple as possible
+to debug. Each test class uses annotations and configuration files to provide
+all the information needed to run a test. Once the customer is started
+(using `cluster.sh` as described [here](quickstart.md)), each test can
+be run from the command line or IDE with no additional command-line parameters.
+To do that, we use a `docker.yaml` configuration file that defines all needed
+parameters, etc.
+
+A test needs both configuration and a Guice setup. The `DruidTestRunner` ,
+along with a number ofm support classes, mostly hide the details from the tests.
+However, you should know what's being done so you can debug.
+
+* JUnit uses the annotation to notice that we've provided a custom
+ test runner. (When converting tests, remember to add the required
+ annotation.)
+* JUnit calls the test class constructor one or more times per test class.
+* On the first creation of the test class, `DruidTestRunner` creates an
+ instance of the `Initializer` class, via its `Builder` to
+ load test configuration, create the Guice injector,
+ inject dependencies into the class instanance, and
+ start the Druid lifecycle.
+* JUnit calls one of the test methods in the class.
+* On the second creation of the test class in the same JVM, `DruidTestRunner`
+ reuses the existing injector to inject dependencies into the test,
+ which avoids the large setup overhead.
+* During the first configuration, `DruidTestRunner` causes initialization
+ to check the health of each service prior to starting the tests.
+* The test is now configured just as it would be from TestNG, and is ready to run.
+* `DruidTestRunner` ends the lifecycle after the last test within this class runs.
+
+See [this explanation](dependencies.md) for the gory details.
+
+`DruidTestRunner` loads the basic set of Druid modules to run the basic client
+code. Tests may wish to load additional modules specific to that test.
+
+## Custom Configuration
+
+There are times when a test needs additional Guice modules beyond what the
+`Initializer` provides. In such cases, you can add a method to customize
+configuration.
+
+### Guice Modules
+
+If your test requires additional Guice modules, add them as follows:
+
+```java
+@Configure
+public static void configure(Initializer.Builder builder)
+{
+ builder.modules(
+ new MyExtraModule(),
+ new AnotherModule()
+ );
+}
+```
+
+### Properties
+
+Druid makes heavy use of properties to configure objects via the 'JsonConfigProvider`
+mechanism. Integration tests don't read the usual `runtime.properties` files: there
+is no such file to read. Instead, properties are set in the test configuration
+file. There are times, however, when it makes more sense to hard-code a property
+value. This is done in the `@Configure` method:
+
+```java
+ builder.property(key, value);
+```
+
+You can also bind a property to an environment variable. This value is used when
+the environment variable is set. You should also bind a default value:
+
+```java
+ builder.property("druid.my.property", 42);
+ builder.propertyEnvVarBinding("druid.my.property", "ULTIMATE_ANSWER");
+```
+
+A property can also be passed in as either a system property or an environment
+variable of the "Docker property environment variable form":
+
+```bash
+druid_property_a=foo
+./it.sh Category test
+```
+
+Or, directly on the command line:
+
+```text
+-Ddruid_property_b=bar
+```
+
+Property precedence is:
+
+* Properties set in code, as above.
+* Properties from the configuration file.
+* Properties bound to environment variables, and the environment variable is set.
+* Properties from the command line.
+
+The test properties can also be seen as default values for properties provided
+in config files or via the command line.
+
+## Resolving Lifecycle Issues
+
+If your test get the dreaded "it doesn't work that way" message, it means that
+an injected property in your test is asking Guice to instantiate a lifecycle-managed
+class after the lifecycle itself was started. This typically happens if the class
+in question is bound via the polymorphic `PolyBind` mechanism which doesn't support
+"eager singletons". (If the class in question is not created via `PolyBind`, change
+its Guice binding to include `.asEagerSingleton()` rather than `.as(LazySingleton.class)`.
+See [this reference](https://github.com/google/guice/wiki/Scopes#eager-singletons).
+
+A quick workaround is to tell the initializer to create an instance before the
+lifecycle starts. The easy way to do that is simply to inject the object into a
+field in your class. Otherwise, give the builder a hint:
+
+```java
+ builder.eagerInstance(ThePeskyComponent.class);
+```
+
+## Test Operation
+
+When working with tests, it is helpful to know a bit more about the "magic"
+behind `DruidTestRunner`.
+
+Druid's code is designed to run in a server, not a client. Yet, the tests are
+clients. This means that tests want to run code in a way that it was not
+intended to be run. The existing ITs have mostly figured out how to make that
+happen, but result is not very clean. This is an opportunity for improvement.
+
+Druid introduced a set of "injector builders" to organize Guice initialization
+a bit. The builders normally build the full server Guice setup. For the ITs,
+the builders also allow us to pick and choose which modules to use to define
+a client. The `Initializer` class in `it-base` uses the injector builders to
+define the "client" modules needed to run tests.
+
+Druid uses the `Lifecycle` class to start and stop services. For this to work,
+the managed instance must be created *before* the lifecycle starts. There are
+a few items that are lazy singletons. When run in the server, they work fine.
+But, when run in tests, we run into a race condition: we want to start the
+lifecycle once before the tests start, the inject dependencies into each test
+class instance as tests run. But, those injections create the insteance we want
+the lifecycle to manage, resulting in a muddle. This is why the `DruidTestRunner`
+has that odd "first test. vs. subsequent test" logic.
+
+The prior ITs would start running tests immediately. But, it can take up to a
+minute or more for a Druid cluster to stabilize as all the services start
+running simultaneously. The previous ITs would use a generic retry up to 240
+times to work around the fact that any given test could fail due to the cluster
+not being ready. This version does that startup check as part if `DruidTestRunner`.
+By the time the tests run, the cluster is up and has reported itself healthy.
+That is, your tests can assume a healthy cluster. If a test fails: it indicates
+an actual error or race condition.
+
+Specifically, if tests still randomly fail, those tests are telling you something: something
+in Druid itself is non-deterministic (such as the delay to see changes to the DB, etc.),
+or the tests are making invalid assumptions such as assuming an ordering when there
+is none, using a time delay to try to synchronize actions when there should be
+some specific synchronization, etc. This means that, in general, you should avoid
+the use of the generic retry facility: if you have to retry to get your tests to
+work, then the Druid user has to also retry. Unless we document the need to retry
+in the API documentation, then having to retry should be considered a bug to be fixed
+(perhaps by documenting the need to retry, perhaps by fixing a bug, perhaps by adding
+a synchronization API.)
+
+Another benefit of the startup check is that the startup and health-check costs are
+paid once per test class. This allows you to structure your
+tests as a large number of small tests rather than a few big tests.
+
+## `ClusterConfig` and `ResolvedClusterConfig`
+
+The `ClusterConfig` class is the Java representation of the
+[test configuration](test-config.md). The instance is available from the
+`Initializer` and by Guice injection.
+
+It is a Jackson-serialized class that handles the "raw" form of
+configuration.
+
+The `ClusterConfig.resolve()` method expands includes, applies defaults,
+validates values, and returns a `ResolvedClusterConfig` instance used
+by tests. `ResolvedClusterConfig` is available via Guice injection.
+In most cases, however, you'll use it indirecty via the various clients
+described below. Each of those uses `IntegrationTestingConfig` class, an
+instance of which is created to read from `ResolvedClusterConfig`.
+
+Remember that each host has two names and two ports:
+
+* The external (or "proxy") host and port, as seen by the machine running
+ the tests.
+* The internal host and port, as seen by the service itself running
+ in the Docker cluster.
+
+The various [config files](test-config.md) provide configurations for
+the Docker, K8s and local cluster cases. This means that `resolveProxyHost()`
+will resolve to the proxy for Docker, but the actual host for a local cluster.
+
+The original test setup was designed before Druid introduced the router.
+A good future improvement is to modify the code to use the router to do the
+routing rather than doing it "by hand" in the tests. This means that each
+test would use the router port and router API for things like the Overlord
+and Coordinator. Then, configuration need only specify the router, not the
+other services.
+
+It is also possible to use Router APIs to obtain the server list dynamically
+rather than hard-coding the services and ports. If we find cases where tests
+must use the APIs directly, then we could either extend the Router API or
+implement client-side service lookup.
+
+## `ClusterClient`
+
+The integration tests make many REST calls to the Druid cluster. The tests
+contain much copy/paste code to make these calls. The `ClusterClient` class
+is intended to gather up these calls so we have a single implementation
+rather than many copies. Add methods as needed for additional APIs.
+
+The cluster client is "test aware": it uses the information in
+`ClusterConfig` to know how to send the requested API. The methods handle
+JSON deserialization, so tests can focus simply on making a call and
+checking the results.
+
+## `org.apache.druid.testing.clients`
+
+This package in `integration-tests` has clients for most other parts of
+Druid. For example, `CoordinatorResourceTestClient` is a
+client for Coordinator calls. These clients are also aware of the test
+configuration, by way of the `IntegrationTestingConfig` class, an
+instance of which is created to read from `ResolvedClusterConfig`.
diff --git a/integration-tests-ex/docs/travis.md b/integration-tests-ex/docs/travis.md
new file mode 100644
index 00000000000..f4b454b6df2
--- /dev/null
+++ b/integration-tests-ex/docs/travis.md
@@ -0,0 +1,168 @@
+
+
+# Travis Integration
+
+Apache Druid uses Travis to manage builds, including running the integration
+tests. You can find the Travis build file at `$DRUID_DEV/.travis.yml`, where
+`DRUID_DEV` is the root of your Druid development directory. Information
+about Travis can be found at:
+
+* [Documentation](https://docs.travis-ci.com/)
+* [Job lifecycle](https://docs.travis-ci.com/user/job-lifecycle/)
+* [Environment variables](https://docs.travis-ci.com/user/environment-variables/)
+* [Travis file reference](https://config.travis-ci.com/)
+* [Travis YAML](https://docs.travis-ci.com/user/build-config-yaml)
+
+## Running ITs In Travis
+
+Travis integration is still experimental. The latest iteration is:
+
+```yaml
+ - name: "experimental docker tests"
+ stage: Tests - phase 1
+ script: ${MVN} install -P test-image,docker-tests -rf :it-tools ${MAVEN_SKIP} -DskipUTs=true
+ after_failure:
+ - docker-tests/check-results.sh
+```
+
+The above is a Travis job definition. The job "inherits" an `install` task defined
+earlier in the file. That install task builds all of Druid and creates the distribution
+tarball. Since the tests are isolated in specialized Maven profiles, the `install`
+task does not build any of the IT-related artifacts.
+
+We've placed the test run in "Phase 1" for debugging convenience. Later, the tests
+will run in "Phase 2" along with the other ITs. Once conversion is complete, the
+"previous generation" IT tests will be replaced by the newer revisions.
+
+The `script` runs the ITs. The components of the command line are:
+
+* `install` - Run Maven though the install [lifecycle phase](
+ https://maven.apache.org/guides/introduction/introduction-to-the-lifecycle.html)
+ for each module. This allows us to build and install the "testing tools"
+ (see the [Maven notes](maven.md)). The test image is also built during the
+ `install` phase. The tests themselves only need the `verify` phase, which occurs
+ before `install`. `install` does nothing for ITs.
+* `-P test-image,docker-tests` - activates the image to build the image
+ (`test-image`) and then runs the ITs (`docker-tests`).
+* `-rf :it-tools` - The `it-tools` module is the first of the IT modules: it contains
+ the "testing tools" added into the image. Using `-rf` skips all the other projects
+ which we already built in the Travis `install` step. Doing so saves the time
+ otherwise required for Maven to figure out it has nothing to do for those modules.
+* `${MAVEN_SKIP}` - Omits the static checks: they are not needed for ITs.
+* `-DskipUTs=true` - The ITs use the [Maven Failsafe plugin](
+ https://maven.apache.org/surefire/maven-failsafe-plugin/index.html)
+ which shares code with the [Maven Surefire plugin](
+ https://maven.apache.org/surefire/maven-surefire-plugin/index.html). We don't want
+ to run unit tests. If we did the usual `-DskipTests`, then we'd also disable the
+ ITs. The `-DskipUTs=true` uses a bit of [Maven trickery](
+ https://stackoverflow.com/questions/6612344/prevent-unit-tests-but-allow-integration-tests-in-maven)
+ to skip only the Surefire, but not Faisafe tests.
+
+## Travis Diagnostics
+
+A common failure when running ITs is that they uncover a bug in a Druid service;
+typically in the code you added that you want to test. Or, if you are changing the
+Docker or Docker Compose infratructure, then the tests will often fail because the
+Druid services are mis-configured. (Bad configuration tends to result in services
+that don't start, or start and immediately exit.)
+
+The standard way to diagnose such failures is to look at the Druid logs. However,
+Travis provides no support for attaching files to a build. The best alternative
+seems to be to upload the files somewhere else. As a compromise, the Travis build
+will append to the build log a subset of the Druid logs.
+
+Travis has a limit of 4MB per build log, so we can't append the entire log for
+every Druid service for every IT. We have to be selective. In most cases, we only
+care about the logs for ITs that fail.
+
+Now, it turns out to be *very hard* indeed to capture failues! Eventually, we want
+Maven to run many ITs for each test run: we need to know which failed. Each IT
+creates its own "shared" directory, so to find the logs, we need to know which IT
+failed. Travis does not have this information: Travis only knows that Maven itself
+exited with a non-zero status. Maven doesn't know: it only knows that Failsafe
+failed the build. Failsafe is designed to run all ITs, then check the results in
+the `verify` state, so Maven doesn't even know about the failures.
+
+### Failsafe Error Reports
+
+To work around all this, we mimic Failsafe: we look at the Failsafe error report
+in `$DRUID_DEV/docker-tests//target/failsafe-reports/failsafe-summary.xml`
+which looks like this:
+
+```xml
+
+ 3
+ 1
+ 0
+ 0
+
+
+```
+
+The above shows one error and no failures. A successful run will show 0 for the
+`errors` tag. This example tells us "something didn't work". The corresponding
+Druid service logs are candidates for review.
+
+### Druid Service Failures
+
+The Druid logs are in `$DRUID_DEV/docker-tests//target/shared/logs`.
+We could append all of them, but recall the 4MB limit. We generally are
+interested only in those services that failed. So, we look at the logs and
+see that a successful run is indicated by a normal Lifecycle shutdown:
+
+```text
+2022-04-16T20:54:37,997 INFO [Thread-56] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [INIT]
+```
+
+The key bit of text is:
+
+```text
+Stopping lifecycle [module] stage [INIT]
+```
+
+This says that 1) we're shutting down the lifecycle (which means no exception was thrown),
+and 2) that we got all the way to the end (`[INIT]`). Since Druid emits no final
+"exited normally" message, we take the above as the next-best thing.
+
+So, we only care about logs that *don't* have the above line. For those, we want to
+append the log to the build output. Or, because of the size limit, we append the
+last 100 lines.
+
+All of this is encapsulated in the `docker-tests/check-results.sh` script which
+is run if the build fails (in the `after_failure`) tag.
+
+### Druid Log Output
+
+For a failed test, the build log will end with something like this:
+
+```text
+======= it-high-availability Failed ==========
+broker.log logtail ========================
+022-04-16T03:53:10,492 INFO [CoordinatorRuleManager-Exec--0] org.apache.druid.discovery.DruidLeaderClient - Request[http://coordinator-one:8081/druid/coordinator/v1/rules] received redirect response to location [http://coordinator-two:8081/druid/coordinator/v1/rules].
+...
+```
+
+To keep below the limit, on the first failed test is reported.
+
+The above won't catch all cases: maybe the service exited normally, but might still have
+log lines of interest. Since all tests run, those lines could be anywhere in the file
+and the scripts can't know which might be of interest. To handle that, we either
+have to upload all logs somewhere, or you can use the convenience of the new
+IT framework to rerun the tests on your development machine.
diff --git a/integration-tests-ex/it-image/build-image.sh b/integration-tests-ex/image/build-image.sh
similarity index 100%
rename from integration-tests-ex/it-image/build-image.sh
rename to integration-tests-ex/image/build-image.sh
diff --git a/integration-tests-ex/it-image/docker-build.sh b/integration-tests-ex/image/docker-build.sh
similarity index 100%
rename from integration-tests-ex/it-image/docker-build.sh
rename to integration-tests-ex/image/docker-build.sh
diff --git a/integration-tests-ex/it-image/docker/Dockerfile b/integration-tests-ex/image/docker/Dockerfile
similarity index 100%
rename from integration-tests-ex/it-image/docker/Dockerfile
rename to integration-tests-ex/image/docker/Dockerfile
diff --git a/integration-tests-ex/it-image/docker/druid.sh b/integration-tests-ex/image/docker/druid.sh
similarity index 100%
rename from integration-tests-ex/it-image/docker/druid.sh
rename to integration-tests-ex/image/docker/druid.sh
diff --git a/integration-tests-ex/it-image/docker/launch.sh b/integration-tests-ex/image/docker/launch.sh
similarity index 76%
rename from integration-tests-ex/it-image/docker/launch.sh
rename to integration-tests-ex/image/docker/launch.sh
index 4b8d0293c8c..1f64b4e14df 100644
--- a/integration-tests-ex/it-image/docker/launch.sh
+++ b/integration-tests-ex/image/docker/launch.sh
@@ -33,14 +33,38 @@ cd /
# TODO: enable only for security-related tests?
#/tls/generate-server-certs-and-keystores.sh
-. /druid.sh
# The image contains both the MySQL and MariaDB JDBC drivers.
# The MySQL driver is selected by the Docker Compose file.
# Set druid.metadata.mysql.driver.driverClassName to the preferred
# driver.
+# Test-specific way to define extensions. Compose defines two test-specific
+# variables. We combine these to create the final form converted to a property.
+if [ -n "$druid_extensions_loadList" ]; then
+ echo "Using the provided druid_extensions_loadList=$druid_extensions_loadList"
+else
+ mkdir -p /tmp/conf
+ EXTNS_FILE=/tmp/conf/extns
+ echo $druid_standard_loadList | tr "," "\n" > $EXTNS_FILE
+ if [ -n "$druid_test_loadList" ]; then
+ echo $druid_test_loadList | tr "," "\n" >> $EXTNS_FILE
+ fi
+ druid_extensions_loadList="["
+ delim=""
+ while read -r line; do
+ druid_extensions_loadList="$druid_extensions_loadList$delim\"$line\""
+ delim=","
+ done < $EXTNS_FILE
+ export druid_extensions_loadList="${druid_extensions_loadList}]"
+ unset druid_standard_loadList
+ unset druid_test_loadList
+ rm $EXTNS_FILE
+ echo "Effective druid_extensions_loadList=$druid_extensions_loadList"
+fi
+
# Create druid service config files with all the config variables
+. /druid.sh
setupConfig
# Export the service config file path to use in supervisord conf file
@@ -92,7 +116,9 @@ fi
LOG_FILE=$LOG_DIR/${INSTANCE_NAME}.log
echo "" >> $LOG_FILE
-echo "--- Service runtime.properties ---" >> $LOG_FILE
+echo "--- env ---" >> $LOG_FILE
+env >> $LOG_FILE
+echo "--- runtime.properties ---" >> $LOG_FILE
cat $DRUID_SERVICE_CONF_DIR/*.properties >> $LOG_FILE
echo "---" >> $LOG_FILE
echo "" >> $LOG_FILE
diff --git a/integration-tests-ex/it-image/docker/test-setup.sh b/integration-tests-ex/image/docker/test-setup.sh
similarity index 100%
rename from integration-tests-ex/it-image/docker/test-setup.sh
rename to integration-tests-ex/image/docker/test-setup.sh
diff --git a/integration-tests-ex/it-image/pom.xml b/integration-tests-ex/image/pom.xml
similarity index 98%
rename from integration-tests-ex/it-image/pom.xml
rename to integration-tests-ex/image/pom.xml
index cd0d4e4e272..e352aa41793 100644
--- a/integration-tests-ex/it-image/pom.xml
+++ b/integration-tests-ex/image/pom.xml
@@ -38,8 +38,9 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven
pom
- it-image
- it-image
+ org.apache.druid.integration-tests
+ druid-it-image
+ druid-it-image
Build the Docker image for integration tests.
@@ -199,15 +200,15 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven
- ${project.version}
${mysql.version}
- ${mysql.image.version}
${mariadb.version}
- ${apache.kafka.version}
+ com.mysql.jdbc.Driver
+ ${mysql.image.version}
${confluent-version}
+ ${apache.kafka.version}
${zookeeper.version}
${hadoop.compile.version}
- com.mysql.jdbc.Driver
+ ${project.version}
${druid.it.image-name}
${project.build.directory}
runtime
diff --git a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java
index f2ac334b913..de8970b20ee 100644
--- a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java
+++ b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java
@@ -23,6 +23,7 @@ import com.github.rvesse.airline.builder.CliBuilder;
public class CustomNodeRoleCommandCreator implements CliCommandCreator
{
+ @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
public void addCommands(CliBuilder builder)
{
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java
index e39c630cf28..8e3c5a83e8d 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java
@@ -221,7 +221,7 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide
overlordTwoTLSUrl = StringUtils.format("https://%s:%s", overlordTwoHost, props.get("overlord_two_tls_port"));
}
}
-
+
middleManagerHost = props.get("middlemanager_host");
zookeeperHosts = props.get("zookeeper_hosts");
@@ -259,7 +259,6 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide
{
return new IntegrationTestingConfig()
{
-
@Override
public String getCoordinatorUrl()
{
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java
index 3ce1dd21f68..a815391fb22 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java
@@ -30,4 +30,5 @@ import com.google.inject.Provider;
})
public interface IntegrationTestingConfigProvider extends Provider
{
+ String PROPERTY_BASE = "druid.test.config";
}
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
index 3df3d8a49d5..e3f2a98dc63 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
@@ -323,6 +323,8 @@ public class OverlordResourceTestClient
{
TaskState status = getTaskStatus(taskID).getStatusCode();
if (status == TaskState.FAILED) {
+ LOG.error("Task failed: %s", taskID);
+ LOG.error("Message: %s", getTaskErrorMessage(taskID));
throw new ISE("Indexer task FAILED");
}
return status == TaskState.SUCCESS;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java
index 1e239ae57db..b0bef034083 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java
@@ -51,7 +51,7 @@ public class DruidTestModule implements Module
binder.bind(IntegrationTestingConfig.class)
.toProvider(IntegrationTestingConfigProvider.class)
.in(ManageLifecycle.class);
- JsonConfigProvider.bind(binder, "druid.test.config", IntegrationTestingConfigProvider.class);
+ JsonConfigProvider.bind(binder, IntegrationTestingConfigProvider.PROPERTY_BASE, IntegrationTestingConfigProvider.class);
binder.bind(CuratorConfig.class).to(IntegrationTestingCuratorConfig.class);
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java
index 0c88619bc12..1bb6e8e12c8 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java
@@ -24,6 +24,7 @@ import com.google.inject.Inject;
import org.apache.commons.io.IOUtils;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.guice.annotations.Smile;
+import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
@@ -165,6 +166,9 @@ public abstract class AbstractIndexerTest
public static String getResourceAsString(String file) throws IOException
{
try (final InputStream inputStream = getResourceAsStream(file)) {
+ if (inputStream == null) {
+ throw new ISE("Failed to load resource: [%s]", file);
+ }
return IOUtils.toString(inputStream, StandardCharsets.UTF_8);
}
}
diff --git a/it.sh b/it.sh
new file mode 100755
index 00000000000..6cf0b54a631
--- /dev/null
+++ b/it.sh
@@ -0,0 +1,130 @@
+#! /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.
+#--------------------------------------------------------------------
+
+# Utility script for running the new integration tests, since the Maven
+# commands are unwieldy.
+
+export DRUID_DEV=$(cd $(dirname $0) && pwd)
+
+function usage
+{
+ cat <
+ start the cluster for category
+ down
+ stop the cluster for category
+ test
+ start the cluster, run the test for category, and stop the cluster
+ tail
+ show the last 20 lines of each container log
+ travis
+ run one IT in Travis (build dist, image, run test, tail logs)
+ prune
+ prune Docker volumes
+EOF
+}
+
+function tail_logs
+{
+ category=$1
+ cd integration-tests-ex/cases/target/$category/logs
+ ls *.log | while read log;
+ do
+ echo "----- $category/$log -----"
+ tail -20 $log
+ done
+}
+
+CMD=$1
+shift
+MAVEN_IGNORE="-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true"
+
+case $CMD in
+ "help" )
+ usage
+ ;;
+ "build" )
+ mvn clean package -P dist $MAVEN_IGNORE -T1.0C
+ ;;
+ "dist" )
+ mvn package -P dist $MAVEN_IGNORE -pl :distribution
+ ;;
+ "image" )
+ cd $DRUID_DEV/integration-tests-ex/image
+ mvn install -P test-image $MAVEN_IGNORE
+ ;;
+ "up" )
+ if [ -z "$1" ]; then
+ usage
+ exit 1
+ fi
+ cd $DRUID_DEV/integration-tests-ex/cases
+ ./cluster.sh up $1
+ ;;
+ "down" )
+ if [ -z "$1" ]; then
+ usage
+ exit 1
+ fi
+ cd $DRUID_DEV/integration-tests-ex/cases
+ ./cluster.sh down $1
+ ;;
+ "test" )
+ if [ -z "$1" ]; then
+ usage
+ exit 1
+ fi
+ cd $DRUID_DEV/integration-tests-ex/cases
+ mvn verify -P skip-static-checks,docker-tests,IT-$1 \
+ -Dmaven.javadoc.skip=true -DskipUTs=true \
+ -pl :druid-it-cases
+ ;;
+ "tail" )
+ if [ -z "$1" ]; then
+ usage
+ exit 1
+ fi
+ tail_logs $1
+ ;;
+ "travis" )
+ if [ -z "$1" ]; then
+ usage
+ exit 1
+ fi
+ $0 dist
+ $0 image
+ $0 test $1
+ $0 tail $1
+ ;;
+ "prune" )
+ # Caution: this removes all volumes, which is generally what you
+ # want when testing.
+ docker system prune --volumes
+ ;;
+ * )
+ usage
+ exit -1
+ ;;
+esac
diff --git a/pom.xml b/pom.xml
index 1eb1979a31b..7dede645166 100644
--- a/pom.xml
+++ b/pom.xml
@@ -214,8 +214,9 @@