mirror of https://github.com/apache/druid.git
Add the new integration test framework (#12368)
This commit is a first draft of the revised integration test framework which provides: - A new directory, integration-tests-ex that holds the new integration test structure. (For now, the existing integration-tests is left unchanged.) - Maven module druid-it-tools to hold code placed into the Docker image. - Maven module druid-it-image to build the Druid-only test image from the tarball produced in distribution. (Dependencies live in their "official" image.) - Maven module druid-it-cases that holds the revised tests and the framework itself. The framework includes file-based test configuration, test-specific clients, test initialization and updated versions of some of the common test support classes. The integration test setup is primarily a huge mass of details. This approach refactors many of those details: from how the image is built and configured to how the Docker Compose scripts are structured to test configuration. An extensive set of "readme" files explains those details. Rather than repeat that material here, please consult those files for explanations.
This commit is contained in:
parent
0bc9f9f303
commit
cfed036091
|
@ -25,5 +25,5 @@ README
|
|||
.pmdruleset.xml
|
||||
.java-version
|
||||
integration-tests/gen-scripts/
|
||||
/bin/
|
||||
bin/
|
||||
*.hprof
|
||||
|
|
50
.travis.yml
50
.travis.yml
|
@ -46,7 +46,7 @@ addons:
|
|||
# Add various options to make 'mvn install' fast and skip javascript compile (-Ddruid.console.skip=true) since it is not
|
||||
# needed. Depending on network speeds, "mvn -q install" may take longer than the default 10 minute timeout to print any
|
||||
# output. To compensate, use travis_wait to extend the timeout.
|
||||
install: ./check_test_suite.py && travis_terminate 0 || echo 'Running Maven install...' && MAVEN_OPTS='-Xmx3000m' travis_wait 15 ${MVN} clean install -q -ff -pl '!distribution,!:it-tools,!:it-image' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C && ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS}
|
||||
install: ./check_test_suite.py && travis_terminate 0 || echo 'Running Maven install...' && MAVEN_OPTS='-Xmx3000m' travis_wait 15 ${MVN} clean install -q -ff -pl '!distribution,!:druid-it-tools,!:druid-it-image,!:druid-it-cases' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C && ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS}
|
||||
|
||||
# There are 3 stages of tests
|
||||
# 1. Tests - phase 1
|
||||
|
@ -347,7 +347,7 @@ jobs:
|
|||
<<: *test_processing_module
|
||||
name: "(openjdk8) other modules test"
|
||||
env:
|
||||
- MAVEN_PROJECTS='!processing,!indexing-hadoop,!indexing-service,!extensions-core/kafka-indexing-service,!extensions-core/kinesis-indexing-service,!server,!web-console,!integration-tests,!:it-image,!:it-tools'
|
||||
- MAVEN_PROJECTS='!processing,!indexing-hadoop,!indexing-service,!extensions-core/kafka-indexing-service,!extensions-core/kinesis-indexing-service,!server,!web-console,!integration-tests,!:druid-it-tools,!:druid-it-image,!:druid-it-cases'
|
||||
|
||||
- <<: *test_other_modules
|
||||
name: "(openjdk11) other modules test"
|
||||
|
@ -457,9 +457,9 @@ jobs:
|
|||
docker exec -it druid-$v sh -c 'dmesg | tail -3' ;
|
||||
done
|
||||
|
||||
- <<: *integration_batch_index
|
||||
name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer"
|
||||
env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer'
|
||||
#- <<: *integration_batch_index
|
||||
# name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer"
|
||||
# env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer'
|
||||
|
||||
- &integration_input_format
|
||||
name: "(Compile=openjdk8, Run=openjdk8) input format integration test"
|
||||
|
@ -666,16 +666,33 @@ jobs:
|
|||
name: "(Compile=openjdk8, Run=openjdk8) other integration tests with Indexer"
|
||||
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer'
|
||||
|
||||
- <<: *integration_tests
|
||||
name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests"
|
||||
jdk: openjdk8
|
||||
env: TESTNG_GROUPS='-Dgroups=high-availability' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data'
|
||||
#- <<: *integration_tests
|
||||
# name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests"
|
||||
# jdk: openjdk8
|
||||
# env: TESTNG_GROUPS='-Dgroups=high-availability' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data'
|
||||
|
||||
- <<: *integration_query
|
||||
name: "(Compile=openjdk8, Run=openjdk8) query integration test (mariaDB)"
|
||||
jdk: openjdk8
|
||||
env: TESTNG_GROUPS='-Dgroups=query' USE_INDEXER='middleManager' MYSQL_DRIVER_CLASSNAME='org.mariadb.jdbc.Driver' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data'
|
||||
|
||||
# Revised ITs.
|
||||
- &integration_tests_ex
|
||||
name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests (new)"
|
||||
stage: Tests - phase 2
|
||||
jdk: openjdk8
|
||||
services: *integration_test_services
|
||||
env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager'
|
||||
# Uses the install defined above. Then, builds the test tools and docker image,
|
||||
# and runs one IT. If tests fail, echos log lines of any of
|
||||
# the Druid services that did not exit normally.
|
||||
script: ./it.sh travis HighAvailability
|
||||
|
||||
- <<: *integration_tests_ex
|
||||
name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer (new)"
|
||||
env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer'
|
||||
script: ./it.sh travis BatchIndex
|
||||
|
||||
# END - Integration tests for Compile with Java 8 and Run with Java 8
|
||||
|
||||
# START - Integration tests for Compile with Java 8 and Run with Java 11
|
||||
|
@ -756,21 +773,6 @@ jobs:
|
|||
|
||||
# END - Integration tests for Compile with Java 8 and Run with Java 11
|
||||
|
||||
# BEGIN - Revised integration tests
|
||||
|
||||
# Experimental build of the revised integration test Docker image.
|
||||
# Actual tests will come later.
|
||||
- name: "experimental docker tests"
|
||||
stage: Tests - phase 2
|
||||
# Uses the install defined above. Then, builds the test tools and docker image,
|
||||
# and run the various IT tests. If tests fail, echos log lines of any of
|
||||
# the Druid services that did not exit normally.
|
||||
# Run though install to ensure the test tools are installed, and the docker
|
||||
# image is built. The tests only need verify.
|
||||
script: ${MVN} install -P dist,test-image -rf :distribution ${MAVEN_SKIP} -DskipUTs=true
|
||||
|
||||
# END - Revised integration tests
|
||||
|
||||
- &integration_batch_index_k8s
|
||||
name: "(Compile=openjdk8, Run=openjdk8, Cluster Build On K8s) ITNestedQueryPushDownTest integration test"
|
||||
stage: Tests - phase 2
|
||||
|
|
|
@ -40,12 +40,13 @@ import java.util.Map;
|
|||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Provides the ability to create "polymorphic" bindings. Where the polymorphism is actually just making a decision
|
||||
* based on a value in a Properties.
|
||||
*
|
||||
* The workflow is that you first create a choice by calling createChoice(). Then you create options using the binder
|
||||
* returned by the optionBinder() method. Multiple different modules can call optionBinder and all options will be
|
||||
* reflected at injection time as long as equivalent interface Key objects are passed into the various methods.
|
||||
* Provides the ability to create "polymorphic" bindings where the polymorphism is actually just making a decision
|
||||
* based on a value in Properties.
|
||||
* <p>
|
||||
* The workflow is that you first create a choice by calling {@code createChoice()}. Then you create options using
|
||||
* the binder returned by the {@code optionBinder()} method. Multiple different modules can call
|
||||
* {@code optionBinder()} and all options will be reflected at injection time as long as equivalent interface
|
||||
* {@code Key} objects are passed into the various methods.
|
||||
*/
|
||||
@PublicApi
|
||||
public class PolyBind
|
||||
|
@ -110,10 +111,10 @@ public class PolyBind
|
|||
}
|
||||
|
||||
/**
|
||||
* Binds an option for a specific choice. The choice must already be registered on the injector for this to work.
|
||||
* Binds an option for a specific choice. The choice must already be registered on the injector for this to work.
|
||||
*
|
||||
* @param binder the binder for the injector that is being configured
|
||||
* @param interfaceKey the interface that will have an option added to it. This must equal the
|
||||
* @param interfaceKey the interface that will have an option added to it. This must equal the
|
||||
* Key provided to createChoice
|
||||
* @param <T> interface type
|
||||
* @return A MapBinder that can be used to create the actual option bindings.
|
||||
|
@ -195,7 +196,7 @@ public class PolyBind
|
|||
|
||||
if (provider == null) {
|
||||
throw new ProvisionException(
|
||||
StringUtils.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet())
|
||||
StringUtils.format("Unknown provider [%s] of %s, known options [%s]", implName, key, implsMap.keySet())
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.Properties;
|
|||
*/
|
||||
public class MetadataStorageConnectorConfig
|
||||
{
|
||||
public static final String PROPERTY_BASE = "druid.metadata.storage.connector";
|
||||
|
||||
@JsonProperty
|
||||
private boolean createTables = true;
|
||||
|
||||
|
|
|
@ -30,6 +30,8 @@ import java.util.Map;
|
|||
*/
|
||||
public class MetadataStorageTablesConfig
|
||||
{
|
||||
public static final String PROPERTY_BASE = "druid.metadata.storage.tables";
|
||||
|
||||
public static MetadataStorageTablesConfig fromBase(String base)
|
||||
{
|
||||
return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null);
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.hamcrest.CoreMatchers;
|
|||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
@ -249,6 +250,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
|||
}
|
||||
|
||||
@Test
|
||||
@Ignore("See issue #12638")
|
||||
public void testWithLargeCacheAndSmallFetch() throws IOException
|
||||
{
|
||||
final TestPrefetchableTextFilesFirehoseFactory factory =
|
||||
|
@ -336,6 +338,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
|||
}
|
||||
|
||||
@Test
|
||||
@Ignore("See issue #12638")
|
||||
public void testReconnectWithCacheAndPrefetch() throws IOException
|
||||
{
|
||||
final TestPrefetchableTextFilesFirehoseFactory factory =
|
||||
|
|
|
@ -112,7 +112,7 @@ public class PolyBindTest
|
|||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof ProvisionException);
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo"));
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo"));
|
||||
}
|
||||
try {
|
||||
Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
|
||||
|
@ -120,7 +120,7 @@ public class PolyBindTest
|
|||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof ProvisionException);
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo"));
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo"));
|
||||
}
|
||||
|
||||
// test default property value
|
||||
|
@ -136,7 +136,7 @@ public class PolyBindTest
|
|||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof ProvisionException);
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$GogoSally"));
|
||||
Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$GogoSally"));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -61,7 +61,7 @@ public class MySQLConnector extends SQLMetadataConnector
|
|||
)
|
||||
{
|
||||
super(config, dbTables);
|
||||
log.info("Loading \"MySQL\" metadata connector driver %s", driverConfig.getDriverClassName());
|
||||
log.info("Loading MySQL metadata connector driver %s", driverConfig.getDriverClassName());
|
||||
tryLoadDriverClass(driverConfig.getDriverClassName(), true);
|
||||
|
||||
if (driverConfig.getDriverClassName().contains("mysql")) {
|
||||
|
|
|
@ -25,8 +25,11 @@ import java.util.Objects;
|
|||
|
||||
public class MySQLConnectorDriverConfig
|
||||
{
|
||||
public static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver";
|
||||
public static final String MARIA_DB_DRIVER = "org.mariadb.jdbc.Driver";
|
||||
|
||||
@JsonProperty
|
||||
private String driverClassName = "com.mysql.jdbc.Driver";
|
||||
private String driverClassName = MYSQL_DRIVER;
|
||||
|
||||
@JsonProperty
|
||||
public String getDriverClassName()
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
/bin/
|
|
@ -19,17 +19,116 @@
|
|||
|
||||
# Revised Integration Tests
|
||||
|
||||
This directory builds a Docker image for Druid. Later revisions
|
||||
use the image to run revised integration tests.
|
||||
This directory builds a Docker image for Druid, then uses that image, along
|
||||
with test configuration to run tests. This version greatly evolves the
|
||||
integration tests from the earlier form. See the [History](docs/history.md)
|
||||
section for details.
|
||||
|
||||
The `it-base` project is built as part of the normal build,
|
||||
though it is used only for the Docker image.
|
||||
## Shortcuts
|
||||
|
||||
To build the image:
|
||||
List of the most common commands once you're familiar with the framework.
|
||||
If you are new to the framework, see [Quickstart](docs/quickstart.md) for
|
||||
an explanation.
|
||||
|
||||
### Build Druid
|
||||
|
||||
To make the text a bit simpler, define a variable for the standard settings:
|
||||
|
||||
```bash
|
||||
mvn $USUAL_CAVEATS -P test-image
|
||||
export MAVEN_IGNORE=-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true
|
||||
|
||||
```bash
|
||||
mvn clean package -P dist $MAVEN_IGNORE -T1.0C
|
||||
```
|
||||
|
||||
Where `$USUAL_CAVEATS` are your favorite options to turn
|
||||
off static checks, UTs, etc.
|
||||
### Build the Test Image
|
||||
|
||||
```bash
|
||||
cd $DRUID_DEV/integration-tests-ex/image
|
||||
mvn install -P test-image $MAVEN_IGNORE
|
||||
```
|
||||
|
||||
### Run an IT from the Command Line
|
||||
|
||||
```bash
|
||||
mvn verify -P IT-<category> -pl :druid-it-cases $MAVEN_IGNORE
|
||||
```
|
||||
|
||||
Where `<category>` is one of the test categories.
|
||||
|
||||
Or
|
||||
|
||||
```bash
|
||||
cd $DRUID_DEV/integration-tests-ex/cases
|
||||
mvn verify -P skip-static-checks,docker-tests,IT-<category> \
|
||||
-Dmaven.javadoc.skip=true -DskipUTs=true \
|
||||
-pl :druid-it-cases
|
||||
```
|
||||
|
||||
### Run an IT from the IDE
|
||||
|
||||
Start the cluster:
|
||||
|
||||
```bash
|
||||
cd $DRUID_DEV/integration-tests-ex/cases
|
||||
./cluster.sh <category> up
|
||||
```
|
||||
|
||||
Where `<category>` is one of the test categories. Then launch the
|
||||
test as a JUnit test.
|
||||
|
||||
## Contents
|
||||
|
||||
* [Goals](#Goals)
|
||||
* [Quickstart](docs/quickstart.md)
|
||||
* [Create a new test](docs/guide.md)
|
||||
* [Maven configuration](docs/maven.md)
|
||||
* [Travis integration](docs/travis.md)
|
||||
* [Docker image](docs/docker.md)
|
||||
* [Druid configuration](docs/druid-config.md)
|
||||
* [Docker Compose configuration](docs/compose.md)
|
||||
* [Test configuration](docs/test-config.md)
|
||||
* [Test structure](docs/tests.md)
|
||||
* [Test runtime semantics](docs/runtime.md)
|
||||
* [Scripts](docs/scripts.md)
|
||||
* [Dependencies](docs/dependencies.md)
|
||||
* [Debugging](docs/debugging.md)
|
||||
|
||||
Background information
|
||||
|
||||
* [Next steps](docs/next-steps.md)
|
||||
* [Test conversion](docs/conversion.md) - How to convert existing tests.
|
||||
* [History](docs/history.md) - Comparison with prior integration tests.
|
||||
|
||||
## Goals
|
||||
|
||||
The goal of the present version is to simplify development.
|
||||
|
||||
* Speed up the Druid test image build by avoiding download of
|
||||
dependencies. (Instead, any such dependencies are managed by
|
||||
Maven and reside in the local build cache.)
|
||||
* Use official images for dependencies to avoid the need to
|
||||
download, install, and manage those dependencies.
|
||||
* Make it is easy to manually build the image, launch
|
||||
a cluster, and run a test against the cluster.
|
||||
* Convert tests to JUnit so that they will easily run in your
|
||||
favorite IDE, just like other Druid tests.
|
||||
* Use the actual Druid build from `distribution` so we know
|
||||
what is tested.
|
||||
* Leverage, don't fight, Maven.
|
||||
* Run the integration tests easily on a typical development machine.
|
||||
|
||||
By meeting these goals, you can quickly:
|
||||
|
||||
* Build the Druid distribution.
|
||||
* Build the Druid image. (< 1 minute)
|
||||
* Launch the cluster for the particular test. (a few seconds)
|
||||
* Run the test any number of times in your debugger.
|
||||
* Clean up the test artifacts.
|
||||
|
||||
The result is that the fastest path to develop a Druid patch or
|
||||
feature is:
|
||||
|
||||
* Create a normal unit test and run it to verify your code.
|
||||
* Create an integration test that double-checks the code in
|
||||
a live cluster.
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
/bin/
|
|
@ -0,0 +1,201 @@
|
|||
#! /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.
|
||||
#--------------------------------------------------------------------
|
||||
|
||||
# Starts the test-specific test cluster using Docker compose using
|
||||
# versions and other settings gathered when the test image was built.
|
||||
# Maps category names to cluster names. The mapping here must match
|
||||
# that in the test category classes when @Cluster is used.
|
||||
|
||||
# Enable for debugging
|
||||
#set -x
|
||||
|
||||
export MODULE_DIR=$(cd $(dirname $0) && pwd)
|
||||
|
||||
function usage {
|
||||
cat <<EOF
|
||||
Usage: $0 cmd [category]
|
||||
-h, help
|
||||
Display this message
|
||||
up category
|
||||
Start the cluster
|
||||
down category
|
||||
Stop the cluster
|
||||
status category
|
||||
Status of the cluster (for debugging within build scripts)
|
||||
compose-cmd category
|
||||
Pass the command to Docker compose.
|
||||
EOF
|
||||
}
|
||||
|
||||
# Command name is required
|
||||
if [ $# -eq 0 ]; then
|
||||
usage 1>&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
CMD=$1
|
||||
shift
|
||||
|
||||
function category {
|
||||
if [ $# -eq 0 ]; then
|
||||
usage 1>&2
|
||||
exit 1
|
||||
fi
|
||||
export CATEGORY=$1
|
||||
|
||||
# All commands need env vars
|
||||
ENV_FILE=$MODULE_DIR/../image/target/env.sh
|
||||
if [ ! -f $ENV_FILE ]; then
|
||||
echo "Please build the Docker test image before testing" 1>&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
source $ENV_FILE
|
||||
# The untranslated category is used for the local name of the
|
||||
# shared folder.
|
||||
|
||||
# DRUID_INTEGRATION_TEST_GROUP is used in
|
||||
# docker-compose files and here. Despite the name, it is the
|
||||
# name of the cluster configuration we want to run, not the
|
||||
# test category. Multiple categories an map to the same cluster
|
||||
# definition.
|
||||
|
||||
# Map from category name to shared cluster definition name.
|
||||
# Add an entry here if you create a new category that shares
|
||||
# a definition.
|
||||
case $CATEGORY in
|
||||
"InputFormat")
|
||||
export DRUID_INTEGRATION_TEST_GROUP=BatchIndex
|
||||
;;
|
||||
*)
|
||||
export DRUID_INTEGRATION_TEST_GROUP=$CATEGORY
|
||||
;;
|
||||
esac
|
||||
|
||||
export CLUSTER_DIR=$MODULE_DIR/cluster/$DRUID_INTEGRATION_TEST_GROUP
|
||||
if [ ! -d $CLUSTER_DIR ]; then
|
||||
echo "Cluster directory $CLUSTER_DIR does not exist." 1>&2
|
||||
echo "$USAGE" 1>&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
export TARGET_DIR=$MODULE_DIR/target
|
||||
export SHARED_DIR=$TARGET_DIR/$CATEGORY
|
||||
export ENV_FILE="$TARGET_DIR/${CATEGORY}.env"
|
||||
}
|
||||
|
||||
function build_override {
|
||||
|
||||
mkdir -p target
|
||||
rm -f "$ENV_FILE"
|
||||
touch "$ENV_FILE"
|
||||
|
||||
# User-local settings?
|
||||
LOCAL_ENV="$HOME/druid-it/${CATEGORY}.env"
|
||||
if [ -f "$LOCAL_ENV" ]; then
|
||||
cat "$LOCAL_ENV" >> "$ENV_FILE"
|
||||
fi
|
||||
|
||||
# Provided override file
|
||||
if [ -n "$OVERRIDE_ENV" ]; then
|
||||
if [ ! -f "$OVERRIDE_ENV" ]; then
|
||||
echo "Environment override file (OVERRIDE_ENV) not found: $OVERRIDE_ENV" 1>&2
|
||||
exit 1
|
||||
fi
|
||||
cat "$OVERRIDE_ENV" >> "$ENV_FILE"
|
||||
fi
|
||||
|
||||
# Add all environment variables of the form druid_*
|
||||
env | grep "^druid_" >> "$ENV_FILE"
|
||||
|
||||
# Reuse the OVERRIDE_ENV variable to pass the full list to Docker compose
|
||||
export OVERRIDE_ENV="$ENV_FILE"
|
||||
}
|
||||
|
||||
# Dump lots of information to debug Docker failures when run inside
|
||||
# of a build environment where we can't inspect Docker directly.
|
||||
function show_status {
|
||||
echo "===================================="
|
||||
ls -l target/shared
|
||||
echo "docker ps -a"
|
||||
docker ps -a
|
||||
# Was: --filter status=exited
|
||||
for id in $(docker ps -a --format "{{.ID}}"); do
|
||||
echo "===================================="
|
||||
echo "Logs for Container ID $id"
|
||||
docker logs $id | tail -n 20
|
||||
done
|
||||
echo "===================================="
|
||||
}
|
||||
|
||||
function build_shared_dir {
|
||||
mkdir -p $SHARED_DIR
|
||||
# Must start with an empty DB to keep MySQL happy
|
||||
rm -rf $SHARED_DIR/db
|
||||
mkdir -p $SHARED_DIR/logs
|
||||
mkdir -p $SHARED_DIR/tasklogs
|
||||
mkdir -p $SHARED_DIR/db
|
||||
mkdir -p $SHARED_DIR/kafka
|
||||
mkdir -p $SHARED_DIR/resources
|
||||
cp $MODULE_DIR/assets/log4j2.xml $SHARED_DIR/resources
|
||||
# Permissions in some build setups are screwed up. See above. The user
|
||||
# which runs Docker does not have permission to write into the /shared
|
||||
# directory. Force ownership to allow writing.
|
||||
chmod -R a+rwx $SHARED_DIR
|
||||
}
|
||||
|
||||
# Print environment for debugging
|
||||
#env
|
||||
|
||||
case $CMD in
|
||||
"-h" )
|
||||
usage
|
||||
;;
|
||||
"help" )
|
||||
usage
|
||||
docker-compose help
|
||||
;;
|
||||
"up" )
|
||||
category $*
|
||||
echo "Starting cluster $DRUID_INTEGRATION_TEST_GROUP"
|
||||
build_override
|
||||
build_shared_dir
|
||||
cd $CLUSTER_DIR
|
||||
docker-compose up -d
|
||||
# Enable the following for debugging
|
||||
show_status
|
||||
;;
|
||||
"status" )
|
||||
category $*
|
||||
cd $CLUSTER_DIR
|
||||
show_status
|
||||
;;
|
||||
"down" )
|
||||
category $*
|
||||
# Enable the following for debugging
|
||||
show_status
|
||||
cd $CLUSTER_DIR
|
||||
echo OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD
|
||||
OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD
|
||||
;;
|
||||
"*" )
|
||||
category $*
|
||||
cd $CLUSTER_DIR
|
||||
OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD
|
||||
;;
|
||||
esac
|
|
@ -0,0 +1,98 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
networks:
|
||||
druid-it-net:
|
||||
name: druid-it-net
|
||||
ipam:
|
||||
config:
|
||||
- subnet: 172.172.172.0/24
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: zookeeper
|
||||
|
||||
metadata:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: metadata
|
||||
|
||||
coordinator:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: coordinator
|
||||
container_name: coordinator
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
# The frequency with which the coordinator polls the database
|
||||
# for changes. The DB population code has to wait at least this
|
||||
# long for the coordinator to notice changes.
|
||||
- druid_manager_segments_pollDuration=PT5S
|
||||
- druid_coordinator_period=PT10S
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
overlord:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: overlord
|
||||
container_name: overlord
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
broker:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: broker
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
router:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: router
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
historical:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: historical
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
indexer:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: indexer
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
volumes:
|
||||
# Test data
|
||||
- ../../resources:/resources
|
||||
depends_on:
|
||||
- zookeeper
|
|
@ -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.
|
||||
|
||||
# Service definitions for Druid's dependencies: ZooKeeper, MySQL, and Kafka.
|
||||
# All tests need ZK and MySQL, ingestion tasks may need Kafka.
|
||||
#
|
||||
# These services use "official" images from the project or other sources.
|
||||
# Some amount of fiddling is needed to map them into configuration which
|
||||
# Druid requires.
|
||||
|
||||
services:
|
||||
|
||||
# Uses the official Zookeeper image
|
||||
# See https://hub.docker.com/_/zookeeper
|
||||
zookeeper:
|
||||
image: zookeeper:${ZK_VERSION}
|
||||
container_name: zookeeper
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.4
|
||||
ports:
|
||||
- 2181:2181
|
||||
volumes:
|
||||
- ${SHARED_DIR}/logs:/logs
|
||||
environment:
|
||||
ZOO_LOG4J_PROP: INFO,ROLLINGFILE
|
||||
|
||||
# Uses the Bitnami Kafka image
|
||||
# See https://hub.docker.com/r/bitnami/kafka/
|
||||
kafka:
|
||||
image: bitnami/kafka:${KAFKA_VERSION}
|
||||
container_name: kafka
|
||||
ports:
|
||||
- 9092:9092
|
||||
- 9093:9093
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.2
|
||||
volumes:
|
||||
- ${SHARED_DIR}/kafka:/bitnami/kafka
|
||||
environment:
|
||||
# This is the default: making it explicit
|
||||
KAFKA_CFG_ZOOKEEPER_CONNECT: zookeeper:2181
|
||||
# Plaintext is disabled by default
|
||||
ALLOW_PLAINTEXT_LISTENER: "yes"
|
||||
# Adapted from base-setup.sh and Bitnami docs
|
||||
KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: "INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT"
|
||||
KAFKA_CFG_LISTENERS: "INTERNAL://:9092,EXTERNAL://:9093"
|
||||
KAFKA_CFG_ADVERTISED_LISTENERS: "INTERNAL://kafka:9092,EXTERNAL://localhost:9093"
|
||||
KAFKA_CFG_INTER_BROKER_LISTENER_NAME: INTERNAL
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
# Uses the official MySQL image
|
||||
# See https://hub.docker.com/_/mysql
|
||||
# The image will intialize the user and DB upon first start.
|
||||
metadata:
|
||||
# platform: linux/x86_64 - Add when running on M1 Macs
|
||||
image: mysql:$MYSQL_IMAGE_VERSION
|
||||
container_name: metadata
|
||||
command:
|
||||
- --character-set-server=utf8mb4
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.3
|
||||
ports:
|
||||
- 3306:3306
|
||||
volumes:
|
||||
- ${SHARED_DIR}/db:/var/lib/mysql
|
||||
environment:
|
||||
MYSQL_ROOT_PASSWORD: driud
|
||||
MYSQL_DATABASE: druid
|
||||
MYSQL_USER: druid
|
||||
MYSQL_PASSWORD: diurd
|
||||
|
||||
## TODO: Not yet retested
|
||||
### Optional supporting infra
|
||||
|
||||
openldap:
|
||||
image: osixia/openldap:1.4.0
|
||||
container_name: openldap
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.102
|
||||
ports:
|
||||
- 8389:389
|
||||
- 8636:636
|
||||
privileged: true
|
||||
volumes:
|
||||
- ./ldap-configs:/container/service/slapd/assets/config/bootstrap/ldif/custom
|
||||
command: --copy-service
|
||||
|
||||
schema-registry:
|
||||
image: confluentinc/cp-schema-registry:5.5.1
|
||||
container_name: schema-registry
|
||||
ports:
|
||||
- 8085:8085
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.103
|
||||
volumes:
|
||||
- ./schema-registry/jaas_config.file:/usr/lib/druid/conf/jaas_config.file
|
||||
- ./schema-registry/password-file:/usr/lib/druid/conf/password-file
|
||||
privileged: true
|
||||
environment:
|
||||
SCHEMA_REGISTRY_HOST_NAME: schema-registry
|
||||
SCHEMA_REGISTRY_LISTENERS: "http://0.0.0.0:8085"
|
||||
SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: kafka:9092
|
||||
SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC
|
||||
SCHEMA_REGISTRY_AUTHENTICATION_REALM: druid
|
||||
SCHEMA_REGISTRY_AUTHENTICATION_ROLES: users
|
||||
SCHEMA_REGISTRY_OPTS: -Djava.security.auth.login.config=/usr/lib/druid/conf/jaas_config.file -Xmx32m
|
||||
|
||||
druid-it-hadoop:
|
||||
## Giving fake version
|
||||
image: druid-it/hadoop:9.9.9
|
||||
container_name: druid-it-hadoop
|
||||
ports:
|
||||
- 2049:2049
|
||||
- 2122:2122
|
||||
- 8020:8020
|
||||
- 8021:8021
|
||||
- 8030:8030
|
||||
- 8031:8031
|
||||
- 8032:8032
|
||||
- 8033:8033
|
||||
- 8040:8040
|
||||
- 8042:8042
|
||||
- 8088:8088
|
||||
- 8443:8443
|
||||
- 9000:9000
|
||||
- 10020:10020
|
||||
- 19888:19888
|
||||
- 34455:34455
|
||||
- 50010:50010
|
||||
- 50020:50020
|
||||
- 50030:50030
|
||||
- 50060:50060
|
||||
- 50070:50070
|
||||
- 50075:50075
|
||||
- 50090:50090
|
||||
- 51111:51111
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.101
|
||||
privileged: true
|
||||
volumes:
|
||||
- ${HOME}/shared:/shared
|
||||
- ./../src/test/resources:/resources
|
||||
hostname: "druid-it-hadoop"
|
||||
command: "bash -c 'echo Start druid-it-hadoop container... && \
|
||||
/etc/bootstrap.sh && \
|
||||
tail -f /dev/null'"
|
||||
|
|
@ -0,0 +1,177 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
version: "2.2"
|
||||
|
||||
# IP address ranges:
|
||||
# 172.172.172.2: zookeeper + kafka (dockerfile depends on this)
|
||||
# 172.172.172.3: metadata store
|
||||
# 172.172.172.10-19: overlord
|
||||
# 172.172.172.20-29: coordinator
|
||||
# 172.172.172.30-39: historicals
|
||||
# 172.172.172.40-49: middle managers
|
||||
# 172.172.172.50-59: indexers
|
||||
# 172.172.172.60-69: brokers
|
||||
# 172.172.172.70-79: routers
|
||||
# 172.172.172.101+: hadoop, other supporting infra
|
||||
#
|
||||
# See https://docs.docker.com/compose/networking/
|
||||
# Key things to know:
|
||||
# * The containers communicate using the Compose network.
|
||||
# * Containers use service name as the host, container port as port
|
||||
# * Ports are also exposed to the host: "host:container", use the
|
||||
# host port to connect from the local machine (outside of a container).
|
||||
#
|
||||
# TODO: The static IPs should not be required. Remove?
|
||||
|
||||
networks:
|
||||
druid-it-net:
|
||||
name: druid-it-net
|
||||
ipam:
|
||||
config:
|
||||
- subnet: 172.172.172.0/24
|
||||
|
||||
services:
|
||||
|
||||
overlord:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: overlord
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.10
|
||||
ports:
|
||||
- 8090:8090
|
||||
- 8290:8290
|
||||
- 5009:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/overlord.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
coordinator:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: coordinator
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.20
|
||||
ports:
|
||||
- 8081:8081
|
||||
- 8281:8281
|
||||
- 5006:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/coordinator.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
historical:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: historical
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.30
|
||||
ports:
|
||||
- 8083:8083
|
||||
- 8283:8283
|
||||
- 5007:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/historical.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
middlemanager:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: middlemanager
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.40
|
||||
ports:
|
||||
- 5008:8000
|
||||
- 8091:8091
|
||||
- 8291:8291
|
||||
- 8100:8100
|
||||
- 8101:8101
|
||||
- 8102:8102
|
||||
- 8103:8103
|
||||
- 8104:8104
|
||||
- 8105:8105
|
||||
- 8300:8300
|
||||
- 8301:8301
|
||||
- 8302:8302
|
||||
- 8303:8303
|
||||
- 8304:8304
|
||||
- 8305:8305
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/middlemanager.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
indexer:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: indexer
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.50
|
||||
ports:
|
||||
- 5008:8000
|
||||
- 8091:8091
|
||||
- 8291:8291
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/indexer.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
broker:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: broker
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.60
|
||||
ports:
|
||||
- 5005:8000
|
||||
- 8082:8082
|
||||
- 8282:8282
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/broker.env
|
||||
- ${OVERRIDE_ENV}
|
||||
|
||||
router:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: router
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.70
|
||||
ports:
|
||||
- 5004:8000
|
||||
- 8888:8888
|
||||
- 9088:9088
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- environment-configs/common.env
|
||||
- environment-configs/router.env
|
||||
- ${OVERRIDE_ENV}
|
|
@ -0,0 +1,53 @@
|
|||
# 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.
|
||||
|
||||
# Settings for a secure cluster.
|
||||
# Disabled by default for each of testing.
|
||||
# Use only for security tests.
|
||||
|
||||
druid_auth_authenticator_basic_authorizerName=basic
|
||||
druid_auth_authenticator_basic_initialAdminPassword=priest
|
||||
druid_auth_authenticator_basic_initialInternalClientPassword=warlock
|
||||
druid_auth_authenticator_basic_type=basic
|
||||
druid_auth_authenticatorChain=["basic"]
|
||||
druid_auth_authorizer_basic_type=basic
|
||||
druid_auth_authorizers=["basic"]
|
||||
druid_client_https_certAlias=druid
|
||||
druid_client_https_keyManagerPassword=druid123
|
||||
druid_client_https_keyStorePassword=druid123
|
||||
druid_client_https_keyStorePath=/tls/server.p12
|
||||
druid_client_https_protocol=TLSv1.2
|
||||
druid_client_https_trustStoreAlgorithm=PKIX
|
||||
druid_client_https_trustStorePassword=druid123
|
||||
druid_client_https_trustStorePath=/tls/truststore.jks
|
||||
druid_enableTlsPort=true
|
||||
druid_server_http_allowedHttpMethods=["OPTIONS"]
|
||||
druid_server_https_certAlias=druid
|
||||
druid_server_https_keyManagerPassword=druid123
|
||||
druid_server_https_keyStorePassword=druid123
|
||||
druid_server_https_keyStorePath=/tls/server.p12
|
||||
druid_server_https_keyStoreType=PKCS12
|
||||
druid_server_https_requireClientCertificate=true
|
||||
druid_server_https_trustStoreAlgorithm=PKIX
|
||||
druid_server_https_trustStorePassword=druid123
|
||||
druid_server_https_trustStorePath=/tls/truststore.jks
|
||||
druid_server_https_validateHostnames=true
|
||||
druid_escalator_authorizerName=basic
|
||||
druid_escalator_internalClientPassword=warlock
|
||||
druid_escalator_internalClientUsername=druid_system
|
||||
druid_escalator_type=basic
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
|
@ -0,0 +1,53 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
DRUID_SERVICE=broker
|
||||
|
||||
# JAVA OPTS
|
||||
SERVICE_DRUID_JAVA_OPTS=-Xms192m -Xmx256m
|
||||
|
||||
# Druid configs
|
||||
druid_host=broker
|
||||
|
||||
# HTTP server settings
|
||||
druid_server_http_numThreads=40
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
||||
|
||||
# HTTP client settings
|
||||
druid_broker_http_numConnections=20
|
||||
druid_broker_http_readTimeout=PT5M
|
||||
|
||||
# Processing threads and buffers
|
||||
druid_processing_buffer_sizeBytes=25000000
|
||||
druid_processing_numMergeBuffers=2
|
||||
druid_processing_numThreads=1
|
||||
druid_processing_tmpDir=/tmp/druid/processing
|
||||
|
||||
# Query cache
|
||||
druid_broker_cache_useCache=true
|
||||
druid_broker_cache_populateCache=true
|
||||
druid_cache_sizeInBytes=40000000
|
||||
|
||||
druid_query_groupBy_maxOnDiskStorage=300000000
|
||||
druid_lookup_namespace_cache_type=onHeap
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/broker
|
||||
druid_sql_avatica_enable=true
|
||||
druid_query_scheduler_laning_strategy=manual
|
||||
druid_query_scheduler_laning_lanes_one=1
|
||||
druid_segmentCache_locations=[{"path":"/shared/druid/brokerIndexCache","maxSize":1000000000}]
|
||||
druid_server_maxSize=1000000000
|
||||
druid_sql_planner_metadataRefreshPeriod=PT15S
|
|
@ -0,0 +1,117 @@
|
|||
# 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.
|
||||
|
||||
LANG=C.UTF-8
|
||||
LANGUAGE=C.UTF-8
|
||||
LC_ALL=C.UTF-8
|
||||
|
||||
# JAVA OPTS
|
||||
# -XX:HeapDumpPath set in container based on the service and intance
|
||||
DRUID_COMMON_JAVA_OPTS=-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+UseG1GC -XX:+ExitOnOutOfMemoryError -XX:+HeapDumpOnOutOfMemoryError
|
||||
DRUID_SERVICE_JAVA_OPTS=
|
||||
|
||||
# Debugging. Enabled by default. Always uses port 8000 in the container.
|
||||
# May to unique ports onto the host in Docker Compose.
|
||||
DEBUG_PORT=
|
||||
DEBUG_OPTS=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
|
||||
|
||||
# Extra classpath. Standard classpath includes Druid libs, Hadoop config, Druid config.
|
||||
DRUID_CLASSPATH=
|
||||
|
||||
# Number of an instance when there are two or more of the same type.
|
||||
DRUID_INSTANCE=
|
||||
|
||||
# Druid configs
|
||||
# Mostly adapted from the micro-quickstart/_common file.
|
||||
|
||||
# Hostname
|
||||
# druid.host is set on each host by the launch script
|
||||
|
||||
# Extensions specified in the load list will be loaded by Druid at runtime.
|
||||
# The extension jars must be installed as part of Druid, or via the image
|
||||
# build script.
|
||||
#
|
||||
# The launch script creates druid_extensions_loadList by combining two test-specific
|
||||
# variables: druid_standard_loadList defined here, and druid_test_loadList, defined
|
||||
# in a docker-compose.yaml file, for any test-specific extensions.
|
||||
# See compose.md for more details.
|
||||
druid_standard_loadList=mysql-metadata-storage,it-tools,druid-lookups-cached-global,druid-histogram,druid-datasketches,druid-parquet-extensions,druid-avro-extensions,druid-protobuf-extensions,druid-orc-extensions,druid-kafka-indexing-service,druid-s3-extensions
|
||||
|
||||
# Location of Hadoop dependencies provided at runtime in the shared directory.
|
||||
druid_extensions_hadoopDependenciesDir=/shared/hadoop-dependencies
|
||||
|
||||
# Logging
|
||||
druid_startup_logging_logProperties=true
|
||||
|
||||
# Zookeeper
|
||||
# Name of the ZK container, mapped to a host name
|
||||
druid_zk_service_host=zookeeper
|
||||
druid_zk_paths_base=/druid
|
||||
|
||||
# Metadata storage
|
||||
# For MySQL (MySQL JDBC installed in $DRUID_HOME/lib)
|
||||
druid_metadata_storage_type=mysql
|
||||
druid_metadata_storage_connector_connectURI=jdbc:mysql://metadata/druid
|
||||
druid_metadata_storage_connector_user=druid
|
||||
druid_metadata_storage_connector_password=diurd
|
||||
|
||||
# Deep storage
|
||||
druid_storage_type=local
|
||||
druid_storage_storageDirectory=/shared/druid/storage
|
||||
|
||||
# Indexing service logs
|
||||
druid_indexer_logs_type=file
|
||||
druid_indexer_logs_directory=/shared/tasklogs
|
||||
|
||||
# Service discovery
|
||||
druid_selectors_indexing_serviceName=druid/overlord
|
||||
druid_selectors_coordinator_serviceName=druid/coordinator
|
||||
|
||||
# Monitoring
|
||||
druid_monitoring_monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
|
||||
druid_emitter=noop
|
||||
druid_emitter_logging_logLevel=info
|
||||
|
||||
# Storage type of double columns
|
||||
druid_indexing_doubleStorage=double
|
||||
|
||||
# SQL
|
||||
druid_sql_enable=true
|
||||
druid_sql_planner_authorizeSystemTablesDirectly=true
|
||||
|
||||
# Lookups
|
||||
druid_lookup_numLookupLoadingThreads=1
|
||||
|
||||
# Test-specific
|
||||
druid_server_http_numThreads=20
|
||||
# Allow OPTIONS method for ITBasicAuthConfigurationTest.testSystemSchemaAccess
|
||||
druid_auth_basic_common_maxSyncRetries=20
|
||||
druid_request_logging_type=slf4j
|
||||
|
||||
# TODO: Can these 4 move to the Coordinator config?
|
||||
druid_coordinator_kill_supervisor_on=true
|
||||
druid_coordinator_kill_supervisor_period=PT10S
|
||||
druid_coordinator_kill_supervisor_durationToRetain=PT0M
|
||||
druid_coordinator_period_metadataStoreManagementPeriod=PT10S
|
||||
|
||||
# TODO: Can the following be moved to Overlord? Or removed?
|
||||
# Testing the legacy config from https://github.com/apache/druid/pull/10267
|
||||
# Can remove this when the flag is no longer needed
|
||||
druid_indexer_task_ignoreTimestampSpecForDruidInputSource=true
|
||||
|
||||
|
||||
# TODO: Pass this from the test (AzureDeepStorage)
|
|
@ -0,0 +1,39 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
DRUID_SERVICE=coordinator
|
||||
|
||||
# JAVA OPTS
|
||||
DRUID_SERVICE_JAVA_OPTS=-Xmx64m -Xms64m
|
||||
|
||||
# Druid configs
|
||||
druid_host=coordinator
|
||||
druid_coordinator_startDelay=PT5S
|
||||
druid_manager_lookups_hostUpdateTimeout=PT30S
|
||||
druid_manager_lookups_period=10000
|
||||
druid_manager_lookups_threadPoolSize=2
|
||||
druid_manager_config_pollDuration=PT10S
|
||||
druid_manager_rules_pollDuration=PT10S
|
||||
druid_manager_segments_pollDuration=PT2S
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/coordinator
|
||||
druid_auth_unsecuredPaths=["/druid/coordinator/v1/loadqueue"]
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
||||
druid_coordinator_period_indexingPeriod=PT180000S
|
||||
|
||||
# 2x indexing period so that kill period is valid
|
||||
druid_coordinator_kill_period=PT360000S
|
||||
druid_coordinator_period=PT1S
|
|
@ -0,0 +1,30 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
DRUID_SERVICE=historical
|
||||
|
||||
# JAVA OPTS
|
||||
SERVICE_DRUID_JAVA_OPTS=-Xmx512m -Xms512m
|
||||
|
||||
# Druid configs
|
||||
druid_host=historical
|
||||
druid_processing_buffer_sizeBytes=25000000
|
||||
druid_processing_numThreads=2
|
||||
druid_query_groupBy_maxOnDiskStorage=300000000
|
||||
druid_segmentCache_locations=[{"path":"/shared/druid/indexCache","maxSize":5000000000}]
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/historical
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
|
@ -0,0 +1,35 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
DRUID_SERVICE=indexer
|
||||
|
||||
# JAVA OPTS
|
||||
SERVICE_DRUID_JAVA_OPTS=-Xmx1g -Xms512m
|
||||
|
||||
# Druid configs
|
||||
druid_host=indexer
|
||||
druid_server_http_numThreads=4
|
||||
druid_storage_storageDirectory=/shared/storage
|
||||
|
||||
druid_processing_buffer_sizeBytes=25000000
|
||||
druid_processing_numThreads=1
|
||||
druid_selectors_indexing_serviceName=druid/overlord
|
||||
druid_indexer_task_chathandler_type=announce
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/indexer
|
||||
druid_startup_logging_logProperties=true
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
||||
druid_worker_capacity=10
|
|
@ -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.
|
||||
|
||||
DRUID_SERVICE=middleManager
|
||||
|
||||
# JAVA OPTS
|
||||
SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m
|
||||
|
||||
# Druid configs
|
||||
druid_host=middlemanager
|
||||
druid_server_http_numThreads=100
|
||||
druid_storage_storageDirectory=/shared/storage
|
||||
druid_indexer_runner_javaOptsArray=["-server", "-Xmx256m", "-Xms256m", "-XX:NewSize=128m", "-XX:MaxNewSize=128m", "-XX:+UseG1GC", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-Dlog4j.configurationFile=/shared/resources/log4j2.xml"]
|
||||
|
||||
druid_indexer_fork_property_druid_processing_buffer_sizeBytes=25000000
|
||||
druid_indexer_fork_property_druid_processing_numThreads=1
|
||||
druid_indexer_fork_server_http_numThreads=20
|
||||
druid_selectors_indexing_serviceName=druid/overlord
|
||||
druid_indexer_task_chathandler_type=announce
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/middleManager
|
||||
druid_startup_logging_logProperties=true
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
||||
druid_worker_capacity=10
|
|
@ -0,0 +1,30 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
DRUID_SERVICE=overlord
|
||||
|
||||
# JAVA OPTS
|
||||
SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m
|
||||
|
||||
# Druid configs
|
||||
druid_host=overlord
|
||||
|
||||
druid_indexer_queue_startDelay=PT5S
|
||||
druid_indexer_storage_type=metadata
|
||||
druid_indexer_runner_type=remote
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/overlord
|
||||
druid_server_https_crlPath=/tls/revocations.crl
|
|
@ -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.
|
||||
#
|
||||
|
||||
DRUID_SERVICE=router
|
||||
|
||||
# JAVA OPTS
|
||||
DRUID_SERVICE_JAVA_OPTS=-Xmx64m -Xms64m
|
||||
|
||||
# Druid configs
|
||||
druid_host=router
|
||||
druid_auth_basic_common_cacheDirectory=/tmp/authCache/router
|
||||
druid_sql_avatica_enable=true
|
||||
|
||||
# HTTP proxy
|
||||
druid_router_http_numConnections=50
|
||||
druid_router_http_readTimeout=PT5M
|
||||
druid_router_http_numMaxThreads=100
|
||||
druid_server_http_numThreads=100
|
||||
|
||||
# Management proxy to coordinator / overlord: required for unified web console.
|
||||
druid_router_managementProxy_enabled=true
|
|
@ -0,0 +1,157 @@
|
|||
# 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.
|
||||
|
||||
networks:
|
||||
druid-it-net:
|
||||
name: druid-it-net
|
||||
ipam:
|
||||
config:
|
||||
- subnet: 172.172.172.0/24
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: zookeeper
|
||||
|
||||
metadata:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: metadata
|
||||
|
||||
coordinator-one:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: coordinator
|
||||
container_name: coordinator-one
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
- DRUID_INSTANCE=one
|
||||
# The frequency with which the coordinator polls the database
|
||||
# for changes. The DB population code has to wait at least this
|
||||
# long for the coordinator to notice changes.
|
||||
- druid_manager_segments_pollDuration=PT5S
|
||||
- druid_coordinator_period=PT10S
|
||||
- druid_host=coordinator-one
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
# The second Coordinator (and Overlord) cannot extend
|
||||
# The base service: they need distinct ports.
|
||||
coordinator-two:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: coordinator-two
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.120
|
||||
ports:
|
||||
- 18081:8081
|
||||
- 18281:8281
|
||||
- 15006:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- ../Common/environment-configs/common.env
|
||||
- ../Common/environment-configs/coordinator.env
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
- DRUID_INSTANCE=two
|
||||
- druid_manager_segments_pollDuration=PT5S
|
||||
- druid_coordinator_period=PT10S
|
||||
- druid_host=coordinator-two
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
overlord-one:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: overlord
|
||||
container_name: overlord-one
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
- DRUID_INSTANCE=one
|
||||
- druid_host=overlord-one
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
overlord-two:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: overlord-two
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.110
|
||||
ports:
|
||||
- 18090:8090
|
||||
- 18290:8290
|
||||
- 15009:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- ../Common/environment-configs/common.env
|
||||
- ../Common/environment-configs/overlord.env
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
- DRUID_INSTANCE=two
|
||||
- druid_host=overlord-two
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
broker:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: broker
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
router:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: router
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
# The custom node role has no base definition. Also, there is
|
||||
# no environment file: the needed environment settings are
|
||||
# given here.
|
||||
custom-node-role:
|
||||
image: ${DRUID_IT_IMAGE_NAME}
|
||||
container_name: custom-node-role
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.90
|
||||
ports:
|
||||
- 50011:50011
|
||||
- 9301:9301
|
||||
- 9501:9501
|
||||
- 5010:8000
|
||||
volumes:
|
||||
- ${SHARED_DIR}:/shared
|
||||
env_file:
|
||||
- ../Common/environment-configs/common.env
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
- DRUID_SERVICE=custom-node-role
|
||||
- SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m
|
||||
- druid_host=custom-node-role
|
||||
- druid_auth_basic_common_cacheDirectory=/tmp/authCache/custom_node_role
|
||||
- druid_server_https_crlPath=/tls/revocations.crl
|
|
@ -0,0 +1,374 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>org.apache.druid.integration-tests</groupId>
|
||||
<artifactId>druid-it-cases</artifactId>
|
||||
<name>druid-it-cases</name>
|
||||
<description>New Integration Tests</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.24.0-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<properties>
|
||||
<it.category>MustSetACategory</it.category>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-integration-tests</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<!-- See https://maven.apache.org/plugins/maven-jar-plugin/examples/create-test-jar.html -->
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-integration-tests</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-services</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-multibindings</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
<artifactId>curator-framework</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
<artifactId>curator-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.dataformat</groupId>
|
||||
<artifactId>jackson-dataformat-yaml</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-core</artifactId>
|
||||
<version>3.2.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-api</artifactId>
|
||||
<version>3.2.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-transport-netty</artifactId>
|
||||
<version>3.2.11</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java</artifactId>
|
||||
<version>3.2.0</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-transport-jersey</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-transport-native-kqueue</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>mysql</groupId>
|
||||
<artifactId>mysql-connector-java</artifactId>
|
||||
<version>${mysql.version}</version>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.jdbi</groupId>
|
||||
<artifactId>jdbi</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>mysql-metadata-storage</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-azure-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-hdfs-storage</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-bundle</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-clients</artifactId>
|
||||
<version>${apache.kafka.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>pl.pragmatists</groupId>
|
||||
<artifactId>JUnitParams</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<!-- Exclude ITs from surefire. Required because they end with "Test". -->
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>**/IT*</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<configuration>
|
||||
<!--
|
||||
~ The analysis gets confused between javax.xml.bind:jaxb-api and jakarta.xml.bind:jakarta.xml.bind-api.
|
||||
~ The latter is a transitive dependency of jackson 2.10+.
|
||||
-->
|
||||
<ignoredUsedUndeclaredDependencies>
|
||||
<ignoredUsedUndeclaredDependency>org.glassfish.hk2.external:jakarta.inject</ignoredUsedUndeclaredDependency>
|
||||
</ignoredUsedUndeclaredDependencies>
|
||||
<!-- Dynamically loaded. -->
|
||||
<ignoredUnusedDeclaredDependencies>
|
||||
<ignoredUnusedDeclaredDependency>mysql:mysql-connector-java:jar</ignoredUnusedDeclaredDependency>
|
||||
</ignoredUnusedDeclaredDependencies>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>IT-HighAvailability</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<it.category>HighAvailability</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>IT-BatchIndex</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<it.category>BatchIndex</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>IT-InputFormat</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<it.category>InputFormat</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>IT-AzureDeepStorage</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<it.category>AzureDeepStorage</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>docker-tests</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-failsafe-plugin</artifactId>
|
||||
<version>3.0.0-M7</version>
|
||||
<dependencies>
|
||||
<!-- Required to force Failsafe to use JUnit instead of TestNG.
|
||||
junit47 is required to use test categories. -->
|
||||
<dependency>
|
||||
<groupId>org.apache.maven.surefire</groupId>
|
||||
<artifactId>surefire-junit47</artifactId>
|
||||
<version>3.0.0-M7</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>functional-test</id>
|
||||
<phase>integration-test</phase>
|
||||
<goals>
|
||||
<goal>integration-test</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<!-- our tests are very verbose, let's keep the volume down -->
|
||||
<redirectTestOutputToFile>true</redirectTestOutputToFile>
|
||||
<!-- Can run only one test category per Maven run. -->
|
||||
<groups>org.apache.druid.testsEx.categories.${it.category}</groups>
|
||||
</configuration>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>verify</id>
|
||||
<phase>verify</phase>
|
||||
<goals>
|
||||
<goal>verify</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>exec-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>start-cluster</id>
|
||||
<phase>pre-integration-test</phase>
|
||||
<goals>
|
||||
<goal>exec</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<workingDirectory>${project.basedir}</workingDirectory>
|
||||
<executable>bash</executable>
|
||||
<arguments>
|
||||
<argument>cluster.sh</argument>
|
||||
<argument>up</argument>
|
||||
<argument>${it.category}</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</execution>
|
||||
|
||||
<execution>
|
||||
<id>stop-cluster</id>
|
||||
<phase>post-integration-test</phase>
|
||||
<goals>
|
||||
<goal>exec</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<workingDirectory>${project.basedir}</workingDirectory>
|
||||
<executable>bash</executable>
|
||||
<arguments>
|
||||
<argument>cluster.sh</argument>
|
||||
<argument>down</argument>
|
||||
<argument>${it.category}</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</profile>
|
||||
</profiles>
|
||||
</project>
|
|
@ -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.
|
||||
-->
|
||||
|
||||
# Test Data
|
||||
|
||||
This directory contains resources, such as test data, mounted into
|
||||
Druid indexer container at `/resources`.
|
||||
That name is chosen for backward compatibility with the previous IT
|
||||
version that mounted `/src/test/resources` to that mount point.
|
||||
See the `indexer` node in `docker-compose.yaml` for `BatchIndex`
|
||||
for the actual volume mount.
|
||||
|
||||
The many indexer specs expect data to be at
|
||||
`/resources/data/batch_index/json/spec.json`, where `spec` is the
|
||||
name of some ingest spec. The structure here was chosen to allow
|
||||
the existing specs to be reused without change.
|
||||
|
||||
Put data for the Indexer in this folder. Put files to be used by
|
||||
tests in `/src/test/resources`. That way, we only mount into the container
|
||||
the data which the server needs. Resources which tests need, such as
|
||||
queries, specs and other resources, are on the class path of the test,
|
||||
but are not visible to the server.
|
||||
|
||||
Paths within this folder are the same as the former
|
||||
`/src/test/resources` folder so that the many indexer specs don't
|
||||
have to change.
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
|
@ -0,0 +1,4 @@
|
|||
timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta
|
||||
2013-08-31T01:02:33Z,Gypsy Danger,en,nuclear,TRUE,TRUE,FALSE,FALSE,article,North America,United States,Bay Area,San Francisco,57,200,-143
|
||||
2013-08-31T03:32:45Z,Striker Eureka,en,speed,FALSE,TRUE,TRUE,FALSE,wikipedia,Australia,Australia,Cantebury,Syndey,459,129,330
|
||||
2013-08-31T07:11:21Z,Cherno Alpha,ru,masterYi,FALSE,TRUE,TRUE,FALSE,article,Asia,Russia,Oblast,Moscow,123,12,111
|
|
|
@ -0,0 +1,4 @@
|
|||
timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta
|
||||
2013-08-31T11:58:39Z,Crimson Typhoon,zh,triplets,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,China,Shanxi,Taiyuan,905,5,900
|
||||
2013-08-31T12:41:27Z,Coyote Tango,ja,stringer,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,Japan,Kanto,Tokyo,1,10,-9
|
||||
2013-09-01T01:02:33Z,Gypsy Danger,en,nuclear,TRUE,TRUE,FALSE,FALSE,article,North America,United States,Bay Area,San Francisco,57,200,-143
|
|
|
@ -0,0 +1,5 @@
|
|||
timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta
|
||||
2013-09-01T03:32:45Z,Striker Eureka,en,speed,FALSE,TRUE,TRUE,FALSE,wikipedia,Australia,Australia,Cantebury,Syndey,459,129,330
|
||||
2013-09-01T07:11:21Z,Cherno Alpha,ru,masterYi,FALSE,TRUE,TRUE,FALSE,article,Asia,Russia,Oblast,Moscow,123,12,111
|
||||
2013-09-01T11:58:39Z,Crimson Typhoon,zh,triplets,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,China,Shanxi,Taiyuan,905,5,900
|
||||
2013-09-01T12:41:27Z,Coyote Tango,ja,stringer,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,Japan,Kanto,Tokyo,1,10,-9
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,3 @@
|
|||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
|
@ -0,0 +1,3 @@
|
|||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
|
@ -0,0 +1,4 @@
|
|||
{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
|
@ -0,0 +1,4 @@
|
|||
timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta
|
||||
2013-08-31T01:02:33Z Gypsy Danger en nuclear true true false false article North America United States Bay Area San Francisco 57 200 -143
|
||||
2013-08-31T03:32:45Z Striker Eureka en speed false true true false wikipedia Australia Australia Cantebury Syndey 459 129 330
|
||||
2013-08-31T07:11:21Z Cherno Alpha ru masterYi false true true false article Asia Russia Oblast Moscow 123 12 111
|
|
|
@ -0,0 +1,4 @@
|
|||
timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta
|
||||
2013-08-31T11:58:39Z Crimson Typhoon zh triplets true false true false wikipedia Asia China Shanxi Taiyuan 905 5 900
|
||||
2013-08-31T12:41:27Z Coyote Tango ja stringer true false true false wikipedia Asia Japan Kanto Tokyo 1 10 -9
|
||||
2013-09-01T01:02:33Z Gypsy Danger en nuclear true true false false article North America United States Bay Area San Francisco 57 200 -143
|
|
|
@ -0,0 +1,5 @@
|
|||
timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta
|
||||
2013-09-01T03:32:45Z Striker Eureka en speed false true true false wikipedia Australia Australia Cantebury Syndey 459 129 330
|
||||
2013-09-01T07:11:21Z Cherno Alpha ru masterYi false true true false article Asia Russia Oblast Moscow 123 12 111
|
||||
2013-09-01T11:58:39Z Crimson Typhoon zh triplets true false true false wikipedia Asia China Shanxi Taiyuan 905 5 900
|
||||
2013-09-01T12:41:27Z Coyote Tango ja stringer true false true false wikipedia Asia Japan Kanto Tokyo 1 10 -9
|
|
|
@ -0,0 +1,3 @@
|
|||
{"timestamp": "2013-08-31T05:18:22Z", "page": "Gypsy Danger", "language" : "zh", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 977, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T17:57:01Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-09-01T18:22:39Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYo", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
|
@ -0,0 +1,198 @@
|
|||
[
|
||||
{
|
||||
"description": "timeseries, 1 agg, all",
|
||||
"query":{
|
||||
"queryType" : "timeBoundary",
|
||||
"dataSource": "%%DATASOURCE%%"
|
||||
},
|
||||
"expectedResults":[
|
||||
{
|
||||
"timestamp" : "2013-09-01T03:32:45.000Z",
|
||||
"result" : {
|
||||
"minTime" : "2013-09-01T03:32:45.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":[]
|
||||
},
|
||||
{
|
||||
"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":4,
|
||||
"approxCountTheta":4.0,
|
||||
"approxCountHLL":4
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"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":[]
|
||||
},
|
||||
{
|
||||
"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-09-01T00:00/2013-09-02T00:00"
|
||||
]
|
||||
},
|
||||
"expectedResults":[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2013-09-01T00:00:00.000Z",
|
||||
"event" : {
|
||||
"added_count_times_ten" : 9050.0,
|
||||
"page" : "Crimson Typhoon",
|
||||
"added_count" : 905,
|
||||
"rows" : 1
|
||||
}
|
||||
} ]
|
||||
}
|
||||
]
|
|
@ -0,0 +1,22 @@
|
|||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 1, "deleted": 1, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 3, "deleted": 3, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 5, "deleted": 5, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 7, "deleted": 3, "delta": 4}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 11, "deleted": 11, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 13, "deleted": 13, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 17, "deleted": 17, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 19, "deleted": 19, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 23, "deleted": 23, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 29, "deleted": 31, "delta": -1}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 31, "deleted": 31, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 37, "deleted": 37, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 41, "deleted": 41, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 43, "deleted": 43, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 47, "deleted": 47, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 59, "deleted": 59, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 61, "deleted": 61, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 67, "deleted": 67, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 71, "deleted": 71, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 73, "deleted": 73, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 79, "deleted": 79, "delta": 0}
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.testsEx.categories;
|
||||
|
||||
public class BatchIndex
|
||||
{
|
||||
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.testsEx.categories;
|
||||
|
||||
public class HighAvailability
|
||||
{
|
||||
|
||||
}
|
|
@ -0,0 +1,340 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||
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.DruidNode;
|
||||
import org.apache.druid.testing.guice.TestClient;
|
||||
import org.apache.druid.testsEx.config.ResolvedConfig;
|
||||
import org.apache.druid.testsEx.config.ResolvedDruidService;
|
||||
import org.apache.druid.testsEx.config.ResolvedService.ResolvedInstance;
|
||||
import org.jboss.netty.handler.codec.http.HttpMethod;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||
|
||||
import javax.inject.Inject;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Client to the Druid cluster described by the test cluster
|
||||
* configuration. Various clients exist for specific services or tasks:
|
||||
* this client is about the cluster as a whole, with operations used
|
||||
* by tests.
|
||||
*/
|
||||
public class DruidClusterClient
|
||||
{
|
||||
private static final Logger log = new Logger(DruidClusterClient.class);
|
||||
|
||||
private final ResolvedConfig config;
|
||||
private final HttpClient httpClient;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
@Inject
|
||||
public DruidClusterClient(
|
||||
ResolvedConfig config,
|
||||
@TestClient HttpClient httpClient,
|
||||
ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.httpClient = httpClient;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
public ResolvedConfig config()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the configuration object for the leader for
|
||||
* the given service.
|
||||
*/
|
||||
public ResolvedInstance leader(ResolvedDruidService service)
|
||||
{
|
||||
if (service.requireInstances().size() == 1) {
|
||||
return service.instance();
|
||||
}
|
||||
String leader = getLeader(service.service());
|
||||
return service.findHost(leader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the leader URL for the given service.
|
||||
*/
|
||||
public String getLeader(String service)
|
||||
{
|
||||
String url = StringUtils.format(
|
||||
"%s/druid/%s/v1/leader",
|
||||
config.routerUrl(),
|
||||
service
|
||||
);
|
||||
return get(url).getContent();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if a node is healthy, given the service and instance.
|
||||
*
|
||||
* @return `true` if the message returns `true`, `false` if the
|
||||
* message fails (indicating the node is not healthy.)
|
||||
*/
|
||||
public boolean isHealthy(ResolvedDruidService service, ResolvedInstance instance)
|
||||
{
|
||||
return isHealthy(service.resolveUrl(instance));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if a node is healthy given the URL for that node.
|
||||
*
|
||||
* @return `true` if the message returns `true`, `false` if the
|
||||
* message fails (indicating the node is not healthy.)
|
||||
*/
|
||||
public boolean isHealthy(String serviceUrl)
|
||||
{
|
||||
try {
|
||||
String url = StringUtils.format(
|
||||
"%s/status/health",
|
||||
serviceUrl
|
||||
);
|
||||
return getAs(url, Boolean.class);
|
||||
}
|
||||
catch (Exception e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the URL for the lead coordinator.
|
||||
*/
|
||||
public String leadCoordinatorUrl()
|
||||
{
|
||||
ResolvedDruidService coord = config.requireCoordinator();
|
||||
ResolvedInstance leader = leader(coord);
|
||||
return coord.resolveUrl(leader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls the `/v1/cluster` endpoint on the lead coordinator.
|
||||
*/
|
||||
public Map<String, Object> coordinatorCluster()
|
||||
{
|
||||
String url = StringUtils.format(
|
||||
"%s/druid/coordinator/v1/cluster",
|
||||
leadCoordinatorUrl()
|
||||
);
|
||||
return getAs(url, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls the `/v1/cluster` endpoint on the router.
|
||||
*/
|
||||
public Map<String, Object> routerCluster()
|
||||
{
|
||||
String url = StringUtils.format(
|
||||
"%s/druid/router/v1/cluster",
|
||||
config.routerUrl()
|
||||
);
|
||||
return getAs(url, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Low-level HTTP get for the given URL.
|
||||
*/
|
||||
public StatusResponseHolder get(String url)
|
||||
{
|
||||
try {
|
||||
StatusResponseHolder response = httpClient.go(
|
||||
new Request(HttpMethod.GET, new URL(url)),
|
||||
StatusResponseHandler.getInstance()
|
||||
).get();
|
||||
|
||||
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
|
||||
throw new ISE(
|
||||
"Error from GET [%s] status [%s] content [%s]",
|
||||
url,
|
||||
response.getStatus(),
|
||||
response.getContent()
|
||||
);
|
||||
}
|
||||
return response;
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Issue a GET command and deserialize the JSON result to the given class.
|
||||
*/
|
||||
public <T> T getAs(String url, Class<T> clazz)
|
||||
{
|
||||
StatusResponseHolder response = get(url);
|
||||
try {
|
||||
return jsonMapper.readValue(response.getContent(), clazz);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Issue a GET command and deserialize the JSON result to the given type reference.
|
||||
*/
|
||||
public <T> T getAs(String url, TypeReference<T> typeRef)
|
||||
{
|
||||
StatusResponseHolder response = get(url);
|
||||
try {
|
||||
return jsonMapper.readValue(response.getContent(), typeRef);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Call the `/status/selfDiscovered` given a node URL.
|
||||
*/
|
||||
public boolean selfDiscovered(String nodeUrl)
|
||||
{
|
||||
String url = StringUtils.format(
|
||||
"%s/status/selfDiscovered",
|
||||
nodeUrl
|
||||
);
|
||||
try {
|
||||
get(url);
|
||||
}
|
||||
catch (Exception e) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates the cluster by waiting for each service declared in the
|
||||
* test configuration to report that it is healthy. By doing this at the
|
||||
* start of the test, individual tests don't have to retry to handle the
|
||||
* race condition that otherwise occurs between cluster and test startup.
|
||||
*/
|
||||
public void validate()
|
||||
{
|
||||
log.info("Starting cluster validation");
|
||||
for (ResolvedDruidService service : config.requireDruid().values()) {
|
||||
for (ResolvedInstance instance : service.requireInstances()) {
|
||||
validateInstance(service, instance);
|
||||
}
|
||||
}
|
||||
log.info("Cluster validated.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate an instance by waiting for it to report that it is healthy.
|
||||
*/
|
||||
private void validateInstance(ResolvedDruidService service, ResolvedInstance instance)
|
||||
{
|
||||
int timeoutMs = config.readyTimeoutSec() * 1000;
|
||||
int pollMs = config.readyPollMs();
|
||||
long startTime = System.currentTimeMillis();
|
||||
long updateTime = startTime + 5000;
|
||||
while (System.currentTimeMillis() - startTime < timeoutMs) {
|
||||
if (isHealthy(service, instance)) {
|
||||
log.info(
|
||||
"Service %s, host %s is ready",
|
||||
service.service(),
|
||||
instance.clientHost());
|
||||
return;
|
||||
}
|
||||
long currentTime = System.currentTimeMillis();
|
||||
if (currentTime > updateTime) {
|
||||
log.info(
|
||||
"Service %s, host %s not ready, retrying",
|
||||
service.service(),
|
||||
instance.clientHost());
|
||||
updateTime = currentTime + 5000;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(pollMs);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("Interrupted during cluster validation");
|
||||
}
|
||||
}
|
||||
throw new RE(
|
||||
StringUtils.format("Service %s, instance %s not ready after %d ms.",
|
||||
service.service(),
|
||||
instance.tag() == null ? "<default>" : instance.tag(),
|
||||
timeoutMs));
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for an instance to become ready given the URL and a description of
|
||||
* the service.
|
||||
*/
|
||||
public void waitForNodeReady(String label, String url)
|
||||
{
|
||||
int timeoutMs = config.readyTimeoutSec() * 1000;
|
||||
int pollMs = config.readyPollMs();
|
||||
long startTime = System.currentTimeMillis();
|
||||
while (System.currentTimeMillis() - startTime < timeoutMs) {
|
||||
if (isHealthy(url)) {
|
||||
log.info(
|
||||
"Service %s, url %s is ready",
|
||||
label,
|
||||
url);
|
||||
return;
|
||||
}
|
||||
log.info(
|
||||
"Service %s, url %s not ready, retrying",
|
||||
label,
|
||||
url);
|
||||
try {
|
||||
Thread.sleep(pollMs);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("Interrupted while waiting for note to be ready");
|
||||
}
|
||||
}
|
||||
throw new RE(
|
||||
StringUtils.format("Service %s, url %s not ready after %d ms.",
|
||||
label,
|
||||
url,
|
||||
timeoutMs));
|
||||
}
|
||||
|
||||
public String nodeUrl(DruidNode node)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"http://%s:%s",
|
||||
config.hasProxy() ? config.proxyHost() : node.getHost(),
|
||||
node.getPlaintextPort()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.testsEx.config.ResolvedService.ResolvedKafka;
|
||||
import org.apache.kafka.clients.admin.Admin;
|
||||
import org.apache.kafka.clients.admin.DescribeClusterResult;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
/**
|
||||
* Simple Kafka client for a test cluster.
|
||||
* <p>
|
||||
* At present, this client contains just enough functionality to validate
|
||||
* that Kafka is up. It can be expanded to gather Kafka code currently
|
||||
* copy/pasted in tests.
|
||||
*/
|
||||
public class KafkaClient
|
||||
{
|
||||
private final ResolvedKafka config;
|
||||
private Admin admin;
|
||||
|
||||
public KafkaClient(ResolvedKafka config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public void open()
|
||||
{
|
||||
validate();
|
||||
}
|
||||
|
||||
public Admin adminClient()
|
||||
{
|
||||
if (admin == null) {
|
||||
final Map<String, Object> props = new HashMap<>();
|
||||
props.put("bootstrap.servers", config.bootstrap());
|
||||
admin = Admin.create(props);
|
||||
}
|
||||
return admin;
|
||||
}
|
||||
|
||||
public void validate()
|
||||
{
|
||||
DescribeClusterResult result = adminClient().describeCluster();
|
||||
try {
|
||||
if (result.nodes().get().isEmpty()) {
|
||||
throw new ISE("No nodes found in Kafka cluster");
|
||||
}
|
||||
}
|
||||
catch (InterruptedException | ExecutionException e) {
|
||||
throw new ISE(e, "Could not connect to Kafka");
|
||||
}
|
||||
}
|
||||
|
||||
public void close()
|
||||
{
|
||||
if (admin != null) {
|
||||
admin.close();
|
||||
admin = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.metadata.MetadataStorageConnector;
|
||||
import org.apache.druid.metadata.SQLMetadataConnector;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
|
||||
import javax.inject.Inject;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.SQLException;
|
||||
|
||||
/**
|
||||
* Simple test-time client to the MySQL metastore.
|
||||
* <p>
|
||||
* Used to verify that the DB is up and available. The JDBC
|
||||
* connection can be used to query the metadata DB for tests.
|
||||
* <p>
|
||||
* Also handles running metastore setup queries on test
|
||||
* startup. Write such queries to be idempotent: REPLACE
|
||||
* rather than INSERT, for example.
|
||||
*/
|
||||
public class MetastoreClient
|
||||
{
|
||||
// See SQLMetadataConnector.getValidationQuery()
|
||||
// That instance isn't available here, so we punt.
|
||||
public static String VALIDATION_QUERY = "SELECT 1";
|
||||
|
||||
private DBI dbi;
|
||||
private Handle handle;
|
||||
|
||||
@Inject
|
||||
public MetastoreClient(MetadataStorageConnector connector)
|
||||
{
|
||||
SQLMetadataConnector sqlConnector = (SQLMetadataConnector) connector;
|
||||
this.dbi = sqlConnector.getDBI();
|
||||
this.handle = dbi.open();
|
||||
}
|
||||
|
||||
public void validate()
|
||||
{
|
||||
boolean ok = execute(VALIDATION_QUERY);
|
||||
if (!ok) {
|
||||
throw new ISE("Metadata store validation failed");
|
||||
}
|
||||
}
|
||||
|
||||
public Connection connection()
|
||||
{
|
||||
return handle.getConnection();
|
||||
}
|
||||
|
||||
public boolean execute(String sql)
|
||||
{
|
||||
try {
|
||||
return connection().prepareStatement(sql).execute();
|
||||
}
|
||||
catch (SQLException e) {
|
||||
throw new ISE(e, "Metadata query failed");
|
||||
}
|
||||
}
|
||||
|
||||
public void close()
|
||||
{
|
||||
handle.close();
|
||||
handle = null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.testsEx.cluster;
|
||||
|
||||
import org.apache.curator.ensemble.EnsembleProvider;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.druid.curator.CuratorConfig;
|
||||
import org.apache.druid.curator.CuratorModule;
|
||||
import org.apache.druid.curator.ExhibitorConfig;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.testsEx.config.ResolvedConfig;
|
||||
import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Test oriented ZooKeeper client.
|
||||
* <p>
|
||||
* Currently contains just enough functionality to verify that
|
||||
* ZK is ready.
|
||||
*/
|
||||
public class ZooKeeperClient
|
||||
{
|
||||
private final ResolvedConfig clusterConfig;
|
||||
private final ResolvedZk config;
|
||||
private CuratorFramework curatorFramework;
|
||||
|
||||
public ZooKeeperClient(ResolvedConfig config)
|
||||
{
|
||||
this.clusterConfig = config;
|
||||
this.config = config.zk();
|
||||
if (this.config == null) {
|
||||
throw new ISE("ZooKeeper not configured");
|
||||
}
|
||||
prepare();
|
||||
awaitReady();
|
||||
}
|
||||
|
||||
private void prepare()
|
||||
{
|
||||
CuratorConfig curatorConfig = clusterConfig.toCuratorConfig();
|
||||
ExhibitorConfig exhibitorConfig = clusterConfig.toExhibitorConfig();
|
||||
EnsembleProvider ensembleProvider = CuratorModule.createEnsembleProvider(curatorConfig, exhibitorConfig);
|
||||
curatorFramework = CuratorModule.createCurator(curatorConfig, ensembleProvider);
|
||||
}
|
||||
|
||||
private void awaitReady()
|
||||
{
|
||||
int timeoutSec = config.startTimeoutSecs();
|
||||
if (timeoutSec == 0) {
|
||||
timeoutSec = 5;
|
||||
}
|
||||
try {
|
||||
curatorFramework.start();
|
||||
curatorFramework.blockUntilConnected(timeoutSec, TimeUnit.SECONDS);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new ISE("ZooKeeper timed out waiting for connect");
|
||||
}
|
||||
}
|
||||
|
||||
public CuratorFramework curator()
|
||||
{
|
||||
return curatorFramework;
|
||||
}
|
||||
|
||||
public void close()
|
||||
{
|
||||
curatorFramework.close();
|
||||
curatorFramework = null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.Inherited;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* Indicates the cluster configuration to use for a test. If omitted, the
|
||||
* cluster configuration is the same as the category. Use this annotation
|
||||
* on a category class to indicate that category shares configuration with
|
||||
* another category. Example:<pre>{@code
|
||||
* @Cluster(BatchIndex.class)
|
||||
* public class InputFormat
|
||||
* {
|
||||
* }</pre>
|
||||
* Note that a category uses a single cluster configuration. It is not
|
||||
* possible to have <i>tests</i> choose the cluster type because Maven must
|
||||
* start the cluster before running tests, thus clusters are a category-level
|
||||
* concept, not a test-level concept.
|
||||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Inherited
|
||||
@Target(ElementType.TYPE)
|
||||
public @interface Cluster
|
||||
{
|
||||
Class<?> value();
|
||||
}
|
|
@ -0,0 +1,342 @@
|
|||
/*
|
||||
* 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.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig;
|
||||
import org.apache.druid.testsEx.config.ServiceConfig.ZKConfig;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Java representation of the test configuration YAML.
|
||||
* <p>
|
||||
* This object is primarily de-serialized as the files are written by hand
|
||||
* to define a test. It is serialized only for debugging.
|
||||
*/
|
||||
public class ClusterConfig
|
||||
{
|
||||
public enum ClusterType
|
||||
{
|
||||
docker,
|
||||
k8s,
|
||||
local,
|
||||
disabled
|
||||
}
|
||||
|
||||
private boolean isResource;
|
||||
private String basePath;
|
||||
|
||||
@JsonProperty("type")
|
||||
private ClusterType type;
|
||||
@JsonProperty("proxyHost")
|
||||
private String proxyHost;
|
||||
@JsonProperty("include")
|
||||
private List<String> include;
|
||||
@JsonProperty("readyTimeoutSec")
|
||||
private int readyTimeoutSec;
|
||||
@JsonProperty("readyPollMs")
|
||||
private int readyPollMs;
|
||||
@JsonProperty("zk")
|
||||
private ZKConfig zk;
|
||||
@JsonProperty("metastore")
|
||||
private MetastoreConfig metastore;
|
||||
@JsonProperty("kafka")
|
||||
private KafkaConfig kafka;
|
||||
@JsonProperty("druid")
|
||||
private Map<String, DruidConfig> druidServices;
|
||||
@JsonProperty("settings")
|
||||
private Map<String, Object> settings;
|
||||
@JsonProperty("properties")
|
||||
private Map<String, Object> properties;
|
||||
@JsonProperty("metastoreInit")
|
||||
private List<MetastoreStmt> metastoreInit;
|
||||
@JsonProperty("datasourceSuffix")
|
||||
private String datasourceSuffix;
|
||||
|
||||
/**
|
||||
* Delay after initializing the DB to wait for the coordinator to notice
|
||||
* the changes. This is a pure hack to work around an optimization in
|
||||
* the coordinator that would otherwise ignore the changes.
|
||||
* Set to a bit longer than the coordinator druid.manager.segments.pollDuration
|
||||
* property.
|
||||
*/
|
||||
@JsonProperty("metastoreInitDelaySec")
|
||||
private int metastoreInitDelaySec;
|
||||
|
||||
public ClusterConfig()
|
||||
{
|
||||
}
|
||||
|
||||
public ClusterConfig(ClusterConfig from)
|
||||
{
|
||||
this.type = from.type;
|
||||
this.proxyHost = from.proxyHost;
|
||||
this.include = null; // Tell IntelliJ inspections we don't want to copy this.
|
||||
this.readyTimeoutSec = from.readyTimeoutSec;
|
||||
this.readyPollMs = from.readyPollMs;
|
||||
this.isResource = from.isResource;
|
||||
this.basePath = from.basePath;
|
||||
this.metastoreInitDelaySec = from.metastoreInitDelaySec;
|
||||
this.datasourceSuffix = from.datasourceSuffix;
|
||||
if (from.include != null) {
|
||||
this.include = new ArrayList<>(from.include);
|
||||
}
|
||||
this.zk = from.zk;
|
||||
this.metastore = from.metastore;
|
||||
this.kafka = from.kafka;
|
||||
if (from.druidServices != null) {
|
||||
this.druidServices = new HashMap<>(from.druidServices);
|
||||
}
|
||||
if (from.properties != null) {
|
||||
this.properties = new HashMap<>(from.properties);
|
||||
}
|
||||
if (from.settings != null) {
|
||||
this.settings = new HashMap<>(from.settings);
|
||||
}
|
||||
if (from.metastoreInit != null) {
|
||||
this.metastoreInit = new ArrayList<>(from.metastoreInit);
|
||||
}
|
||||
}
|
||||
|
||||
public static ClusterConfig loadFromFile(String filePath)
|
||||
{
|
||||
return loadFromFile(new File(filePath));
|
||||
}
|
||||
|
||||
public static ClusterConfig loadFromFile(File configFile)
|
||||
{
|
||||
ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
|
||||
try {
|
||||
ClusterConfig config = mapper.readValue(configFile, ClusterConfig.class);
|
||||
config.isResource = false;
|
||||
config.basePath = configFile.getParent();
|
||||
return config;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Failed to load config file: " + configFile);
|
||||
}
|
||||
}
|
||||
|
||||
public static ClusterConfig loadFromResource(String resource)
|
||||
{
|
||||
ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
|
||||
try (InputStream is = TestConfigs.class.getResourceAsStream(resource)) {
|
||||
if (is == null) {
|
||||
throw new ISE("Config resource not found: " + resource);
|
||||
}
|
||||
ClusterConfig config = mapper.readValue(is, ClusterConfig.class);
|
||||
config.isResource = true;
|
||||
return config;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Failed to load config resource: " + resource);
|
||||
}
|
||||
}
|
||||
|
||||
public ResolvedConfig resolve(String clusterName)
|
||||
{
|
||||
return new ResolvedConfig(clusterName, resolveIncludes());
|
||||
}
|
||||
|
||||
public ClusterConfig resolveIncludes()
|
||||
{
|
||||
if (include == null || include.isEmpty()) {
|
||||
return this;
|
||||
}
|
||||
ClusterConfig included = null;
|
||||
for (String entry : include) {
|
||||
ClusterConfig child = loadInclude(entry);
|
||||
if (included == null) {
|
||||
included = child;
|
||||
} else {
|
||||
included = included.merge(child);
|
||||
}
|
||||
}
|
||||
return included.merge(this);
|
||||
}
|
||||
|
||||
private ClusterConfig loadInclude(String includeName)
|
||||
{
|
||||
if (isResource) {
|
||||
return loadFromResource(includeName);
|
||||
} else {
|
||||
File file = new File(new File(basePath), includeName);
|
||||
return loadFromFile(file);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("type")
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public ClusterType type()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
@JsonProperty("readyTimeoutSec")
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public int readyTimeoutSec()
|
||||
{
|
||||
return readyTimeoutSec;
|
||||
}
|
||||
|
||||
@JsonProperty("readyPollMs")
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public int readyPollMs()
|
||||
{
|
||||
return readyPollMs;
|
||||
}
|
||||
|
||||
@JsonProperty("proxyHost")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public String proxyHost()
|
||||
{
|
||||
return proxyHost;
|
||||
}
|
||||
|
||||
@JsonProperty("include")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public List<String> include()
|
||||
{
|
||||
return include;
|
||||
}
|
||||
|
||||
@JsonProperty("zk")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public ZKConfig zk()
|
||||
{
|
||||
return zk;
|
||||
}
|
||||
|
||||
@JsonProperty("metastore")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public MetastoreConfig metastore()
|
||||
{
|
||||
return metastore;
|
||||
}
|
||||
|
||||
@JsonProperty("kafka")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public KafkaConfig kafka()
|
||||
{
|
||||
return kafka;
|
||||
}
|
||||
|
||||
@JsonProperty("druid")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public Map<String, DruidConfig> druid()
|
||||
{
|
||||
return druidServices;
|
||||
}
|
||||
|
||||
@JsonProperty("settings")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public Map<String, Object> settings()
|
||||
{
|
||||
return settings;
|
||||
}
|
||||
|
||||
@JsonProperty("properties")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public Map<String, Object> properties()
|
||||
{
|
||||
return properties;
|
||||
}
|
||||
|
||||
@JsonProperty("metastoreInit")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public List<MetastoreStmt> metastoreInit()
|
||||
{
|
||||
return metastoreInit;
|
||||
}
|
||||
|
||||
@JsonProperty("metastoreInitDelaySec")
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public int metastoreInitDelaySec()
|
||||
{
|
||||
return metastoreInitDelaySec;
|
||||
}
|
||||
|
||||
@JsonProperty("datasourceSuffix")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public String datasourceSuffix()
|
||||
{
|
||||
return datasourceSuffix;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return TestConfigs.toYaml(this);
|
||||
}
|
||||
|
||||
public ClusterConfig merge(ClusterConfig overrides)
|
||||
{
|
||||
ClusterConfig merged = new ClusterConfig(this);
|
||||
if (overrides.readyTimeoutSec != 0) {
|
||||
merged.readyTimeoutSec = overrides.readyTimeoutSec;
|
||||
}
|
||||
if (overrides.proxyHost != null) {
|
||||
merged.proxyHost = overrides.proxyHost;
|
||||
}
|
||||
// Includes are already considered.
|
||||
if (overrides.zk != null) {
|
||||
merged.zk = overrides.zk;
|
||||
}
|
||||
if (overrides.metastore != null) {
|
||||
merged.metastore = overrides.metastore;
|
||||
}
|
||||
if (overrides.kafka != null) {
|
||||
merged.kafka = overrides.kafka;
|
||||
}
|
||||
if (merged.druidServices == null) {
|
||||
merged.druidServices = overrides.druidServices;
|
||||
} else if (overrides.druidServices != null) {
|
||||
merged.druidServices.putAll(overrides.druidServices);
|
||||
}
|
||||
if (merged.settings == null) {
|
||||
merged.settings = overrides.settings;
|
||||
} else if (overrides.settings != null) {
|
||||
merged.settings.putAll(overrides.settings);
|
||||
}
|
||||
if (merged.properties == null) {
|
||||
merged.properties = overrides.properties;
|
||||
} else if (overrides.properties != null) {
|
||||
merged.properties.putAll(overrides.properties);
|
||||
}
|
||||
if (merged.metastoreInit == null) {
|
||||
merged.metastoreInit = overrides.metastoreInit;
|
||||
} else if (overrides.metastoreInit != null) {
|
||||
merged.metastoreInit.addAll(overrides.metastoreInit);
|
||||
}
|
||||
return merged;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,123 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.testsEx.config;
|
||||
|
||||
import org.apache.druid.testing.IntegrationTestingConfig;
|
||||
import org.apache.druid.testsEx.config.ClusterConfig.ClusterType;
|
||||
import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.PrintWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
/**
|
||||
* Sanity check of an example YAML config file using the Java
|
||||
* deserialization classes.
|
||||
*/
|
||||
public class ClusterConfigTest
|
||||
{
|
||||
@Test
|
||||
public void testYaml() throws FileNotFoundException
|
||||
{
|
||||
ClusterConfig config = ClusterConfig.loadFromResource("/config-test/test.yaml");
|
||||
// Uncomment this line to see the full config with includes resolved.
|
||||
//System.out.println(config.resolveIncludes());
|
||||
|
||||
ResolvedConfig resolved = config.resolve("Test");
|
||||
assertEquals(ClusterType.docker, resolved.type());
|
||||
assertEquals(ResolvedConfig.DEFAULT_READY_TIMEOUT_SEC, resolved.readyTimeoutSec());
|
||||
assertEquals(ResolvedConfig.DEFAULT_READY_POLL_MS, resolved.readyPollMs());
|
||||
assertEquals(3, resolved.properties().size());
|
||||
|
||||
ResolvedZk zk = resolved.zk();
|
||||
assertNotNull(zk);
|
||||
assertEquals("zookeeper", zk.service());
|
||||
assertEquals(1, zk.requireInstances().size());
|
||||
assertEquals(2181, zk.instance().port());
|
||||
assertEquals(2181, zk.instance().clientPort());
|
||||
assertEquals("zookeeper", zk.instance().host());
|
||||
assertEquals("localhost", zk.instance().clientHost());
|
||||
assertEquals("zookeeper:2181", zk.clusterHosts());
|
||||
assertEquals("localhost:2181", zk.clientHosts());
|
||||
|
||||
ResolvedMetastore ms = resolved.metastore();
|
||||
assertNotNull(ms);
|
||||
assertEquals("metastore", ms.service());
|
||||
assertEquals(1, ms.requireInstances().size());
|
||||
assertEquals("jdbc:mysql://localhost:3306/druid", ms.connectURI());
|
||||
assertEquals("druid", ms.user());
|
||||
assertEquals("diurd", ms.password());
|
||||
|
||||
ResolvedDruidService service = resolved.requireBroker();
|
||||
assertNotNull(service);
|
||||
assertEquals("broker", service.service());
|
||||
assertEquals("http://localhost:8082", service.clientUrl());
|
||||
|
||||
service = resolved.requireRouter();
|
||||
assertNotNull(service);
|
||||
assertEquals("router", service.service());
|
||||
assertEquals("http://localhost:8888", service.clientUrl());
|
||||
assertEquals("http://localhost:8888", resolved.routerUrl());
|
||||
|
||||
File userEnv = new File(
|
||||
new File(
|
||||
System.getProperty("user.home"),
|
||||
"druid-it"),
|
||||
"Test.env");
|
||||
try (PrintWriter out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(userEnv), StandardCharsets.UTF_8))) {
|
||||
out.println("druid_user_var=user");
|
||||
}
|
||||
|
||||
System.setProperty("druid_sys_prop", "sys");
|
||||
Map<String, Object> 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());
|
||||
}
|
||||
}
|
|
@ -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:
|
||||
* <pre><code>
|
||||
* @Configure
|
||||
* public static void configure(Initializer.Builder builder)
|
||||
* {
|
||||
* builder.something(arg);
|
||||
* }
|
||||
* </code</pre>
|
||||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Target(ElementType.METHOD)
|
||||
public @interface Configure
|
||||
{
|
||||
}
|
|
@ -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.
|
||||
* <p>
|
||||
* 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 <i>before</i> 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.
|
||||
* <p>
|
||||
* 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));
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
* <p>
|
||||
* 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.)
|
||||
* <p>
|
||||
* 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<Module> modules = new ArrayList<>();
|
||||
private boolean validateCluster;
|
||||
private List<Class<?>> eagerCreation = new ArrayList<>();
|
||||
private Map<String, String> 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.
|
||||
* <p>
|
||||
* 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<Module> 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:
|
||||
* <ul>
|
||||
* <li>System properties (highest)</li>
|
||||
* <li>Environment variable bindings</li>
|
||||
* <li>Configuration file</li>
|
||||
* <li>Hard-coded values (lowest></li>
|
||||
* </ul>
|
||||
*/
|
||||
private static Properties properties(
|
||||
Builder builder,
|
||||
ResolvedConfig clusterConfig
|
||||
)
|
||||
{
|
||||
Properties finalProperties = new Properties();
|
||||
finalProperties.putAll(builder.testProperties);
|
||||
finalProperties.putAll(clusterConfig.toProperties());
|
||||
for (Entry<String, String> 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.
|
||||
* <p>
|
||||
* <b>WARNING</b>: 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<MetastoreStmt> 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();
|
||||
}
|
||||
}
|
|
@ -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<String, String> properties;
|
||||
|
||||
@Inject
|
||||
public IntegrationTestingConfigEx(
|
||||
final ResolvedConfig config,
|
||||
final Properties properties)
|
||||
{
|
||||
this.config = config;
|
||||
ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
|
||||
for (Map.Entry<Object, Object> 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<String, String> 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();
|
||||
}
|
||||
}
|
|
@ -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<ServiceInstance> instances
|
||||
)
|
||||
{
|
||||
super(service, instances);
|
||||
}
|
||||
}
|
|
@ -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<String, Object> 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<String, Object> properties,
|
||||
@JsonProperty("instances") List<ServiceInstance> 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<String, Object> properties()
|
||||
{
|
||||
return properties;
|
||||
}
|
||||
|
||||
public boolean validate(List<String> errs)
|
||||
{
|
||||
if (Strings.isNullOrEmpty(connectURI)) {
|
||||
errs.add("Metastore connect URI is required");
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -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.)
|
||||
* <p>
|
||||
* 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;
|
||||
}
|
||||
}
|
|
@ -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<String, Object> properties;
|
||||
private Map<String, Object> settings;
|
||||
|
||||
private final ResolvedZk zk;
|
||||
private final ResolvedKafka kafka;
|
||||
private final ResolvedMetastore metastore;
|
||||
private final Map<String, ResolvedDruidService> 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<String, DruidConfig> 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<String, Object> settings()
|
||||
{
|
||||
return settings;
|
||||
}
|
||||
|
||||
public Map<String, Object> properties()
|
||||
{
|
||||
return properties;
|
||||
}
|
||||
|
||||
public Map<String, ResolvedDruidService> 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<String, String> SETTINGS_MAP =
|
||||
ImmutableMap.<String, String>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<String, Object> 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<String, Object> 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<String, Object> toProperties()
|
||||
{
|
||||
Map<String, Object> 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<String, Object> 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<String, String> 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<Object, Object> entry : System.getProperties().entrySet()) {
|
||||
String key = (String) entry.getKey();
|
||||
if (!key.startsWith("druid_")) {
|
||||
continue;
|
||||
}
|
||||
setDruidProperyVar(properties, key, entry.getValue());
|
||||
}
|
||||
return properties;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<String, Object> properties;
|
||||
private final int initDelaySec;
|
||||
private List<MetastoreStmt> 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(),
|
||||
"<port>",
|
||||
Integer.toString(instance.clientPort())),
|
||||
"<host>",
|
||||
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<String, Object> properties()
|
||||
{
|
||||
return properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the properties Guice needs to create the connector config.
|
||||
*
|
||||
* @see <a href="https://druid.apache.org/docs/0.23.0/development/extensions-core/mysql.html#setting-up-mysql">
|
||||
* Setting up MySQL</a>
|
||||
*/
|
||||
public Map<String, Object> toProperties()
|
||||
{
|
||||
final String base = MetadataStorageConnectorConfig.PROPERTY_BASE;
|
||||
Map<String, Object> 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<MetastoreStmt> initStmts()
|
||||
{
|
||||
return initStmts;
|
||||
}
|
||||
|
||||
public int initDelaySec()
|
||||
{
|
||||
return initDelaySec;
|
||||
}
|
||||
}
|
|
@ -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<ResolvedInstance> 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<ResolvedInstance> 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<String> hosts = new ArrayList<>();
|
||||
for (ResolvedInstance instance : instances) {
|
||||
hosts.add(formatHost(instance.clientHost(), instance.clientPort()));
|
||||
}
|
||||
return String.join(",", hosts);
|
||||
}
|
||||
|
||||
public String clusterHosts()
|
||||
{
|
||||
List<String> 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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<ServiceInstance> instances;
|
||||
|
||||
public ServiceConfig(
|
||||
String service,
|
||||
List<ServiceInstance> 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<ServiceInstance> 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<ServiceInstance> 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: <code><pre>
|
||||
* druid:
|
||||
* broker: # <-- key (service name)
|
||||
* instances:
|
||||
* ...
|
||||
* </pre></code>
|
||||
*/
|
||||
public static class DruidConfig extends ServiceConfig
|
||||
{
|
||||
@JsonCreator
|
||||
public DruidConfig(
|
||||
// Note: service is not actually used.
|
||||
@JsonProperty("service") String service,
|
||||
@JsonProperty("instances") List<ServiceInstance> instances
|
||||
)
|
||||
{
|
||||
super(service, instances);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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 <service>} or @{code <service>-<tag>}
|
||||
* 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<service>-<tag>} 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);
|
||||
}
|
||||
}
|
|
@ -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 "<conversion failed>";
|
||||
}
|
||||
}
|
||||
|
||||
public static void putProperty(Map<String, Object> properties, String key, Object value)
|
||||
{
|
||||
if (value == null) {
|
||||
return;
|
||||
}
|
||||
properties.put(key, value);
|
||||
}
|
||||
|
||||
public static void putProperty(Map<String, Object> properties, String base, String key, Object value)
|
||||
{
|
||||
if (value == null) {
|
||||
return;
|
||||
}
|
||||
properties.put(base + "." + key, value);
|
||||
}
|
||||
}
|
|
@ -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<Boolean, Boolean> segmentAvailabilityConfirmationPair
|
||||
) throws IOException
|
||||
{
|
||||
doIndexTest(
|
||||
dataSource,
|
||||
indexTaskFilePath,
|
||||
Function.identity(),
|
||||
queryFilePath,
|
||||
waitForNewVersion,
|
||||
runTestQueries,
|
||||
waitForSegmentsToLoad,
|
||||
segmentAvailabilityConfirmationPair
|
||||
);
|
||||
}
|
||||
|
||||
protected void doIndexTest(
|
||||
String dataSource,
|
||||
String indexTaskFilePath,
|
||||
Function<String, String> taskSpecTransform,
|
||||
String queryFilePath,
|
||||
boolean waitForNewVersion,
|
||||
boolean runTestQueries,
|
||||
boolean waitForSegmentsToLoad,
|
||||
Pair<Boolean, Boolean> 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<Boolean, Boolean> segmentAvailabilityConfirmationPair
|
||||
) throws IOException
|
||||
{
|
||||
doReindexTest(
|
||||
baseDataSource,
|
||||
reindexDataSource,
|
||||
Function.identity(),
|
||||
reindexTaskFilePath,
|
||||
queryFilePath,
|
||||
segmentAvailabilityConfirmationPair
|
||||
);
|
||||
}
|
||||
|
||||
void doReindexTest(
|
||||
String baseDataSource,
|
||||
String reindexDataSource,
|
||||
Function<String, String> taskSpecTransform,
|
||||
String reindexTaskFilePath,
|
||||
String queryFilePath,
|
||||
Pair<Boolean, Boolean> 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<String> 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<String, String> taskSpecTransform
|
||||
) throws IOException
|
||||
{
|
||||
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
|
||||
final String taskSpec = taskSpecTransform.apply(
|
||||
StringUtils.replace(
|
||||
getResourceAsString(indexTaskFilePath),
|
||||
"%%DATASOURCE%%",
|
||||
fullDatasourceName
|
||||
)
|
||||
);
|
||||
|
||||
Pair<Boolean, Boolean> 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<Boolean, Boolean> segmentAvailabilityConfirmationPair
|
||||
)
|
||||
{
|
||||
final List<DataSegment> 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<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(
|
||||
coordinator.getAvailableSegments(dataSourceName)
|
||||
);
|
||||
|
||||
final List<TimelineObjectHolder<String, DataSegment>> 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<DataSegment> 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<DataSegment> 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<String> 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<String> actualIntervals = coordinator.getSegmentIntervals(
|
||||
dataSource + config.getExtraDatasourceNameSuffix()
|
||||
);
|
||||
actualIntervals.sort(null);
|
||||
return actualIntervals.equals(intervalsBeforeCompaction);
|
||||
},
|
||||
"Compaction interval check"
|
||||
);
|
||||
}
|
||||
|
||||
void verifySegmentsCompacted(String dataSource, int expectedCompactedSegmentCount)
|
||||
{
|
||||
List<DataSegment> segments = coordinator.getFullSegmentsMetadata(
|
||||
dataSource + config.getExtraDatasourceNameSuffix()
|
||||
);
|
||||
List<DataSegment> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<TaskResponseObject> 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<String> 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<Boolean>()
|
||||
{
|
||||
@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<String> listResources(String dir) throws IOException
|
||||
{
|
||||
List<String> 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;
|
||||
}
|
||||
}
|
|
@ -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<Boolean, Boolean> segmentAvailabilityConfirmationPair
|
||||
) throws Exception
|
||||
{
|
||||
doIndexTest(inputFormatDetails, ImmutableMap.of(), segmentAvailabilityConfirmationPair);
|
||||
}
|
||||
|
||||
public void doIndexTest(
|
||||
InputFormatDetails inputFormatDetails,
|
||||
@Nonnull Map<String, Object> extraInputFormatMap,
|
||||
Pair<Boolean, Boolean> segmentAvailabilityConfirmationPair
|
||||
) throws Exception
|
||||
{
|
||||
final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
|
||||
Map<String, Object> inputFormatMap = new ImmutableMap.Builder<String, Object>().putAll(extraInputFormatMap)
|
||||
.put("type", inputFormatDetails.getInputFormatType())
|
||||
.build();
|
||||
try (
|
||||
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
|
||||
) {
|
||||
final Function<String, String> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, String> 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<String, String> 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<String, String> 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"
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, String> combiningFirehoseSpecTransform = spec -> {
|
||||
try {
|
||||
return StringUtils.replace(
|
||||
spec,
|
||||
"%%COMBINING_DATASOURCE%%",
|
||||
INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
};
|
||||
final Function<String, String> 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)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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<String, Object> inputFormatMap = new ImmutableMap
|
||||
.Builder<String, Object>()
|
||||
.put("type", "json")
|
||||
.build();
|
||||
try (
|
||||
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
|
||||
final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
|
||||
) {
|
||||
final Function<String, String> 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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, String> 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<Boolean, Boolean> 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<String, String> 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<String, String> 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<String, Integer> minTaskPriority = Collections.singletonMap(datasourceName, 0);
|
||||
final Map<String, List<Interval>> 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");
|
||||
}
|
||||
}
|
|
@ -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<String, Object> inputFormatMap = new ImmutableMap.Builder<String, Object>()
|
||||
.put("type", INPUT_FORMAT_DETAILS.getInputFormatType())
|
||||
.build();
|
||||
final Function<String, String> 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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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<String, String> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, Object> results = clusterClient.coordinatorCluster();
|
||||
assertEquals(5, results.size());
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> coordNodes = (List<Object>) results.get(NodeRole.COORDINATOR.getJsonName());
|
||||
assertEquals(2, coordNodes.size());
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> histNodes = (List<Object>) results.get(NodeRole.HISTORICAL.getJsonName());
|
||||
assertTrue(histNodes.isEmpty());
|
||||
}
|
||||
}
|
|
@ -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<String, String> runCommandInCoordinatorContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInCoordinatorTwoContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInHistoricalContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInOverlordContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInOverlordTwoContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInBrokerContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(BROKER_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInRouterContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(ROUTER_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> runCommandInMiddleManagerContainer(String... cmd) throws Exception
|
||||
{
|
||||
return runCommandInDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME, cmd);
|
||||
}
|
||||
|
||||
public Pair<String, String> 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<Container> containers = dockerClient.listContainersCmd().exec();
|
||||
Optional<String> 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"
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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"
|
|
@ -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
|
|
@ -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
|
|
@ -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://<host>:<port>/druid
|
||||
user: druid
|
||||
password: diurd
|
||||
instances:
|
||||
- port: 3306
|
|
@ -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"
|
||||
}'
|
||||
)
|
|
@ -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": []
|
||||
}
|
||||
]
|
|
@ -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"
|
||||
}'
|
||||
)
|
|
@ -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"]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
|
|
@ -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()"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
[
|
||||
{
|
||||
"query": {
|
||||
"queryType": "segmentMetadata",
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"analysisTypes": [
|
||||
"%%ANALYSIS_TYPE%%"
|
||||
],
|
||||
"intervals": [
|
||||
"%%INTERVALS%%"
|
||||
]
|
||||
},
|
||||
"expectedResults": %%EXPECTED_QUERY_GRANULARITY%%,
|
||||
"fieldsToTest": ["queryGranularity"]
|
||||
}
|
||||
]
|
|
@ -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%%
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -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"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue