From cfed036091038dc05cdb246e2849fdbed39aec0d Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Wed, 24 Aug 2022 04:33:23 -0700 Subject: [PATCH] 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. --- .gitignore | 2 +- .travis.yml | 52 +- .../java/org/apache/druid/guice/PolyBind.java | 19 +- .../MetadataStorageConnectorConfig.java | 2 + .../metadata/MetadataStorageTablesConfig.java | 2 + ...fetchableTextFilesFirehoseFactoryTest.java | 3 + .../org/apache/druid/guice/PolyBindTest.java | 8 +- .../storage/mysql/MySQLConnector.java | 2 +- .../mysql/MySQLConnectorDriverConfig.java | 5 +- extensions-core/testing-tools/.gitignore | 1 + integration-tests-ex/README.md | 115 +- integration-tests-ex/cases/.gitignore | 1 + .../{ => cases}/assets/log4j2.xml | 0 integration-tests-ex/cases/cluster.sh | 201 + .../cluster/BatchIndex/docker-compose.yaml | 98 + .../cases/cluster/Common/dependencies.yaml | 166 + .../cases/cluster/Common/druid.yaml | 177 + .../Common/environment-configs/auth.env | 53 + .../Common/environment-configs/broker.env | 53 + .../Common/environment-configs/common.env | 117 + .../environment-configs/coordinator.env | 39 + .../Common/environment-configs/historical.env | 30 + .../Common/environment-configs/indexer.env | 35 + .../environment-configs/middlemanager.env | 37 + .../Common/environment-configs/overlord.env | 30 + .../Common/environment-configs/router.env | 37 + .../HighAvailability/docker-compose.yaml | 157 + integration-tests-ex/cases/pom.xml | 374 ++ .../cases/resources/README.md | 42 + .../avro/wikipedia_index_data1.avro | Bin 0 -> 1066 bytes .../avro/wikipedia_index_data2.avro | Bin 0 -> 1055 bytes .../avro/wikipedia_index_data3.avro | Bin 0 -> 1151 bytes .../batch_index/csv/wikipedia_index_data1.csv | 4 + .../batch_index/csv/wikipedia_index_data2.csv | 4 + .../batch_index/csv/wikipedia_index_data3.csv | 5 + .../batch_index/hadoop_tsv/batch_hadoop.data | 3360 +++++++++++++++++ .../json/wikipedia_index_data1.json | 3 + .../json/wikipedia_index_data2.json | 3 + .../json/wikipedia_index_data3.json | 4 + .../batch_index/orc/wikipedia_index_data1.orc | Bin 0 -> 2035 bytes .../batch_index/orc/wikipedia_index_data2.orc | Bin 0 -> 2026 bytes .../batch_index/orc/wikipedia_index_data3.orc | Bin 0 -> 2008 bytes .../parquet/wikipedia_index_data1.parquet | Bin 0 -> 4119 bytes .../parquet/wikipedia_index_data2.parquet | Bin 0 -> 4116 bytes .../parquet/wikipedia_index_data3.parquet | Bin 0 -> 4160 bytes .../batch_index/tsv/wikipedia_index_data1.tsv | 4 + .../batch_index/tsv/wikipedia_index_data2.tsv | 4 + .../batch_index/tsv/wikipedia_index_data3.tsv | 5 + .../wikipedia_combining_index_data.json | 3 + .../wikipedia_index_queries_only_data3.json | 198 + .../wikipedia_realtime_index_data.json | 22 + .../druid/testsEx/categories/BatchIndex.java | 25 + .../testsEx/categories/HighAvailability.java | 25 + .../testsEx/cluster/DruidClusterClient.java | 340 ++ .../druid/testsEx/cluster/KafkaClient.java | 83 + .../testsEx/cluster/MetastoreClient.java | 88 + .../testsEx/cluster/ZooKeeperClient.java | 89 + .../apache/druid/testsEx/config/Cluster.java | 48 + .../druid/testsEx/config/ClusterConfig.java | 342 ++ .../testsEx/config/ClusterConfigTest.java | 123 + .../druid/testsEx/config/Configure.java | 42 + .../druid/testsEx/config/DruidTestRunner.java | 208 + .../druid/testsEx/config/Initializer.java | 590 +++ .../config/IntegrationTestingConfigEx.java | 421 +++ .../druid/testsEx/config/KafkaConfig.java | 37 + .../druid/testsEx/config/MetastoreConfig.java | 121 + .../druid/testsEx/config/MetastoreStmt.java | 78 + .../druid/testsEx/config/ResolvedConfig.java | 406 ++ .../testsEx/config/ResolvedDruidService.java | 145 + .../testsEx/config/ResolvedMetastore.java | 128 + .../druid/testsEx/config/ResolvedService.java | 219 ++ .../druid/testsEx/config/ServiceConfig.java | 116 + .../druid/testsEx/config/ServiceInstance.java | 124 + .../druid/testsEx/config/TestConfigs.java | 66 + .../indexer/AbstractITBatchIndexTest.java | 502 +++ .../testsEx/indexer/AbstractIndexerTest.java | 188 + ...ractLocalInputSourceParallelIndexTest.java | 115 + .../ITBestEffortRollupParallelIndexTest.java | 250 ++ .../ITCombiningFirehoseFactoryIndexTest.java | 101 + ...CombiningInputSourceParallelIndexTest.java | 130 + .../druid/testsEx/indexer/ITIndexerTest.java | 386 ++ .../indexer/ITOverwriteBatchIndexTest.java | 166 + .../ITSystemTableBatchIndexTaskTest.java | 72 + .../testsEx/indexer/ITTransformTest.java | 125 + .../leadership/ITHighAvailabilityTest.java | 246 ++ .../utils/DruidClusterAdminClient.java | 328 ++ .../cluster/AzureDeepStorage/docker.yaml | 58 + .../resources/cluster/BatchIndex/docker.yaml | 40 + .../test/resources/cluster/Common/kafka.yaml | 22 + .../cluster/Common/zk-metastore.yaml | 28 + .../cluster/HighAvailability/docker.yaml | 204 + .../cluster/HighAvailability/queries/sys.json | 39 + .../src/test/resources/config-test/test.yaml | 85 + .../indexer/broadcast_join_index_task.json | 82 + .../indexer/json_path_index_queries.json | 49 + .../indexer/json_path_index_task.json | 80 + .../indexer/segment_metadata_query.json | 16 + .../indexer/sparse_column_index_queries.json | 80 + .../indexer/sparse_column_index_task.json | 57 + ...parse_column_with_dim_compaction_task.json | 19 + ...se_column_without_dim_compaction_task.json | 16 + .../sys_segment_batch_index_queries.json | 35 + .../indexer/wikipedia_cloud_index_task.json | 87 + .../wikipedia_cloud_simple_index_task.json | 81 + ...edia_combining_firehose_index_queries.json | 141 + ...kipedia_combining_firehose_index_task.json | 95 + ...ning_input_source_index_parallel_task.json | 98 + .../indexer/wikipedia_compaction_task.json | 14 + ...compaction_task_with_granularity_spec.json | 15 + ...paction_task_with_segment_granularity.json | 15 + ...tion_non_perfect_rollup_index_queries.json | 143 + ...ngestion_perfect_rollup_index_queries.json | 143 + .../wikipedia_http_inputsource_queries.json | 47 + .../wikipedia_http_inputsource_task.json | 74 + .../indexer/wikipedia_index_queries.json | 150 + ..._index_queries_hour_query_granularity.json | 150 + ...ikipedia_index_queries_with_transform.json | 62 + ..._index_queries_year_query_granularity.json | 150 + .../wikipedia_index_rollup_queries.json | 56 + .../indexer/wikipedia_index_task.json | 89 + ...ipedia_index_task_with_dimension_spec.json | 86 + ...edia_index_task_with_granularity_spec.json | 71 + ...index_task_with_inputsource_transform.json | 103 + .../wikipedia_index_task_with_transform.json | 103 + ...ia_index_with_merge_column_limit_task.json | 86 + ...kipedia_local_input_source_index_task.json | 89 + .../wikipedia_merge_index_queries.json | 42 + .../indexer/wikipedia_merge_index_task.json | 70 + ...merge_reindex_druid_input_source_task.json | 63 + .../indexer/wikipedia_merge_reindex_task.json | 65 + ...pedia_override_credentials_index_task.json | 82 + ...arallel_druid_input_source_index_task.json | 64 + .../wikipedia_parallel_index_queries.json | 71 + .../wikipedia_parallel_index_task.json | 77 + ...allel_index_using_sqlinputsource_task.json | 88 + ...ia_parallel_ingest_segment_index_task.json | 69 + .../wikipedia_parallel_reindex_queries.json | 18 + .../wikipedia_parallel_reindex_task.json | 75 + ...a_realtime_appenderator_index_queries.json | 87 + ...edia_realtime_appenderator_index_task.json | 94 + .../wikipedia_realtime_index_queries.json | 147 + .../wikipedia_realtime_index_task.json | 98 + ...pedia_reindex_druid_input_source_task.json | 51 + ...uid_input_source_task_with_transforms.json | 106 + .../indexer/wikipedia_reindex_queries.json | 66 + ...pedia_reindex_queries_with_transforms.json | 80 + .../indexer/wikipedia_reindex_task.json | 53 + ...ikipedia_reindex_task_with_transforms.json | 108 + .../indexer/wikipedia_union_index_task.json | 98 + .../wikipedia_with_timestamp_index_task.json | 86 + .../indexer/wikiticker_index_task.json | 66 + integration-tests-ex/check-results.sh | 89 - integration-tests-ex/docs/compose.md | 238 ++ integration-tests-ex/docs/conversion.md | 292 ++ integration-tests-ex/docs/debugging.md | 159 + integration-tests-ex/docs/dependencies.md | 239 ++ integration-tests-ex/docs/docker.md | 302 ++ integration-tests-ex/docs/druid-config.md | 138 + integration-tests-ex/docs/guide.md | 241 ++ integration-tests-ex/docs/history.md | 96 + integration-tests-ex/docs/maven.md | 255 ++ integration-tests-ex/docs/next-steps.md | 77 + integration-tests-ex/docs/quickstart.md | 201 + integration-tests-ex/docs/runtime.md | 105 + integration-tests-ex/docs/scripts.md | 74 + integration-tests-ex/docs/test-config.md | 588 +++ integration-tests-ex/docs/tests.md | 411 ++ integration-tests-ex/docs/travis.md | 168 + .../{it-image => image}/build-image.sh | 0 .../{it-image => image}/docker-build.sh | 0 .../{it-image => image}/docker/Dockerfile | 0 .../{it-image => image}/docker/druid.sh | 0 .../{it-image => image}/docker/launch.sh | 30 +- .../{it-image => image}/docker/test-setup.sh | 0 .../{it-image => image}/pom.xml | 13 +- .../{it-image => image}/rebuild.sh | 0 .../it-image/scripts/build-shared.sh | 40 - .../{it-tools => tools}/README.md | 0 .../{it-tools => tools}/pom.xml | 7 +- .../testing/tools/CliCustomNodeRole.java | 0 .../tools/CliHistoricalForQueryErrorTest.java | 0 .../tools/CustomNodeRoleClientModule.java | 0 .../tools/CustomNodeRoleCommandCreator.java | 0 .../tools/QueryRetryTestCommandCreator.java | 0 .../tools/ServerManagerForQueryErrorTest.java | 0 .../druid/testing/tools/SleepExprMacro.java | 0 .../druid/testing/tools/SleepModule.java | 0 .../tools/SleepOperatorConversion.java | 0 .../apache/druid/testing/tools/package.java | 0 .../org.apache.druid.cli.CliCommandCreator | 0 ...rg.apache.druid.initialization.DruidModule | 0 integration-tests/README.md | 20 +- integration-tests/pom.xml | 1 + .../cli/CustomNodeRoleCommandCreator.java | 1 + .../testing/ConfigFileConfigProvider.java | 3 +- .../IntegrationTestingConfigProvider.java | 1 + .../clients/OverlordResourceTestClient.java | 2 + .../druid/testing/guice/DruidTestModule.java | 2 +- .../tests/indexer/AbstractIndexerTest.java | 4 + it.sh | 130 + pom.xml | 24 +- .../druid/guice/DruidInjectorBuilder.java | 40 +- .../druid/guice/MetadataConfigModule.java | 4 +- .../ExtensionInjectorBuilder.java | 2 +- .../org/apache/druid/cli/GuiceRunnable.java | 4 +- 205 files changed, 20703 insertions(+), 230 deletions(-) create mode 100644 extensions-core/testing-tools/.gitignore create mode 100644 integration-tests-ex/cases/.gitignore rename integration-tests-ex/{ => cases}/assets/log4j2.xml (100%) create mode 100755 integration-tests-ex/cases/cluster.sh create mode 100644 integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml create mode 100644 integration-tests-ex/cases/cluster/Common/dependencies.yaml create mode 100644 integration-tests-ex/cases/cluster/Common/druid.yaml create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/auth.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/broker.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/common.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/historical.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env create mode 100644 integration-tests-ex/cases/cluster/Common/environment-configs/router.env create mode 100644 integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml create mode 100644 integration-tests-ex/cases/pom.xml create mode 100644 integration-tests-ex/cases/resources/README.md create mode 100644 integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro create mode 100644 integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro create mode 100644 integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro create mode 100644 integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data1.csv create mode 100644 integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data2.csv create mode 100644 integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data3.csv create mode 100644 integration-tests-ex/cases/resources/data/batch_index/hadoop_tsv/batch_hadoop.data create mode 100644 integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data1.json create mode 100644 integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data2.json create mode 100644 integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data3.json create mode 100644 integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data1.orc create mode 100644 integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc create mode 100644 integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc create mode 100644 integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet create mode 100644 integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet create mode 100644 integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet create mode 100644 integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data1.tsv create mode 100644 integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data2.tsv create mode 100644 integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data3.tsv create mode 100644 integration-tests-ex/cases/resources/indexer/wikipedia_combining_index_data.json create mode 100644 integration-tests-ex/cases/resources/indexer/wikipedia_index_queries_only_data3.json create mode 100644 integration-tests-ex/cases/resources/indexer/wikipedia_realtime_index_data.json create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BatchIndex.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json create mode 100644 integration-tests-ex/cases/src/test/resources/config-test/test.yaml create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json create mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json delete mode 100755 integration-tests-ex/check-results.sh create mode 100644 integration-tests-ex/docs/compose.md create mode 100644 integration-tests-ex/docs/conversion.md create mode 100644 integration-tests-ex/docs/debugging.md create mode 100644 integration-tests-ex/docs/dependencies.md create mode 100644 integration-tests-ex/docs/docker.md create mode 100644 integration-tests-ex/docs/druid-config.md create mode 100644 integration-tests-ex/docs/guide.md create mode 100644 integration-tests-ex/docs/history.md create mode 100644 integration-tests-ex/docs/maven.md create mode 100644 integration-tests-ex/docs/next-steps.md create mode 100644 integration-tests-ex/docs/quickstart.md create mode 100644 integration-tests-ex/docs/runtime.md create mode 100644 integration-tests-ex/docs/scripts.md create mode 100644 integration-tests-ex/docs/test-config.md create mode 100644 integration-tests-ex/docs/tests.md create mode 100644 integration-tests-ex/docs/travis.md rename integration-tests-ex/{it-image => image}/build-image.sh (100%) rename integration-tests-ex/{it-image => image}/docker-build.sh (100%) rename integration-tests-ex/{it-image => image}/docker/Dockerfile (100%) rename integration-tests-ex/{it-image => image}/docker/druid.sh (100%) rename integration-tests-ex/{it-image => image}/docker/launch.sh (76%) rename integration-tests-ex/{it-image => image}/docker/test-setup.sh (100%) rename integration-tests-ex/{it-image => image}/pom.xml (98%) rename integration-tests-ex/{it-image => image}/rebuild.sh (100%) delete mode 100644 integration-tests-ex/it-image/scripts/build-shared.sh rename integration-tests-ex/{it-tools => tools}/README.md (100%) rename integration-tests-ex/{it-tools => tools}/pom.xml (96%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/CliCustomNodeRole.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/CliHistoricalForQueryErrorTest.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/CustomNodeRoleClientModule.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/CustomNodeRoleCommandCreator.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/QueryRetryTestCommandCreator.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/SleepExprMacro.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/SleepModule.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/SleepOperatorConversion.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/java/org/apache/druid/testing/tools/package.java (100%) rename integration-tests-ex/{it-tools => tools}/src/main/resources/META-INF/services/org.apache.druid.cli.CliCommandCreator (100%) rename integration-tests-ex/{it-tools => tools}/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule (100%) create mode 100755 it.sh diff --git a/.gitignore b/.gitignore index f906e242679..89eabfcbdd0 100644 --- a/.gitignore +++ b/.gitignore @@ -25,5 +25,5 @@ README .pmdruleset.xml .java-version integration-tests/gen-scripts/ -/bin/ +bin/ *.hprof diff --git a/.travis.yml b/.travis.yml index 9ca6fee1ad7..14d16e5e363 100644 --- a/.travis.yml +++ b/.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 @@ -72,7 +72,7 @@ jobs: - name: "animal sniffer checks" stage: Tests - phase 1 script: ${MVN} animal-sniffer:check --fail-at-end - + - name: "checkstyle" script: ${MVN} checkstyle:checkstyle --fail-at-end @@ -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 diff --git a/core/src/main/java/org/apache/druid/guice/PolyBind.java b/core/src/main/java/org/apache/druid/guice/PolyBind.java index e7e53b6c984..19931a6b63e 100644 --- a/core/src/main/java/org/apache/druid/guice/PolyBind.java +++ b/core/src/main/java/org/apache/druid/guice/PolyBind.java @@ -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. + *

+ * 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 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()) ); } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java index 512cd57d892..2e6767f0b77 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java @@ -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; diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index 766efabb526..267f4fe6b69 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -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); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index dd9c384e91a..38fb843a34c 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -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 = diff --git a/core/src/test/java/org/apache/druid/guice/PolyBindTest.java b/core/src/test/java/org/apache/druid/guice/PolyBindTest.java index d6a50d51396..372d428458d 100644 --- a/core/src/test/java/org/apache/druid/guice/PolyBindTest.java +++ b/core/src/test/java/org/apache/druid/guice/PolyBindTest.java @@ -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,9 +120,9 @@ 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 Assert.assertEquals("B", injector.getInstance(GogoSally.class).go()); props.setProperty("sally", "a"); @@ -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")); } } diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index bbe238ffb0f..4098e30a544 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -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")) { diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java index 0f6fa03245d..4aad21651c7 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java @@ -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() diff --git a/extensions-core/testing-tools/.gitignore b/extensions-core/testing-tools/.gitignore new file mode 100644 index 00000000000..ae3c1726048 --- /dev/null +++ b/extensions-core/testing-tools/.gitignore @@ -0,0 +1 @@ +/bin/ diff --git a/integration-tests-ex/README.md b/integration-tests-ex/README.md index 9619897baeb..9c29ec101fe 100644 --- a/integration-tests-ex/README.md +++ b/integration-tests-ex/README.md @@ -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. \ No newline at end of file +### 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- -pl :druid-it-cases $MAVEN_IGNORE +``` + +Where `` is one of the test categories. + +Or + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +mvn verify -P skip-static-checks,docker-tests,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases +``` + +### Run an IT from the IDE + +Start the cluster: + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +./cluster.sh up +``` + +Where `` 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. diff --git a/integration-tests-ex/cases/.gitignore b/integration-tests-ex/cases/.gitignore new file mode 100644 index 00000000000..ae3c1726048 --- /dev/null +++ b/integration-tests-ex/cases/.gitignore @@ -0,0 +1 @@ +/bin/ diff --git a/integration-tests-ex/assets/log4j2.xml b/integration-tests-ex/cases/assets/log4j2.xml similarity index 100% rename from integration-tests-ex/assets/log4j2.xml rename to integration-tests-ex/cases/assets/log4j2.xml diff --git a/integration-tests-ex/cases/cluster.sh b/integration-tests-ex/cases/cluster.sh new file mode 100755 index 00000000000..0b19b478fcb --- /dev/null +++ b/integration-tests-ex/cases/cluster.sh @@ -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 <&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 diff --git a/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml b/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml new file mode 100644 index 00000000000..f8235db6255 --- /dev/null +++ b/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml new file mode 100644 index 00000000000..ccfce630a2a --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -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'" + diff --git a/integration-tests-ex/cases/cluster/Common/druid.yaml b/integration-tests-ex/cases/cluster/Common/druid.yaml new file mode 100644 index 00000000000..bd5caad2232 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/druid.yaml @@ -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} diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env new file mode 100644 index 00000000000..fec58cc0a00 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/broker.env b/integration-tests-ex/cases/cluster/Common/environment-configs/broker.env new file mode 100644 index 00000000000..652bbaf69df --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/broker.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. + +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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env new file mode 100644 index 00000000000..350a1f5b792 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env @@ -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) diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env b/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env new file mode 100644 index 00000000000..fa20cf6ccaf --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env b/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env new file mode 100644 index 00000000000..cf969f62894 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env b/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env new file mode 100644 index 00000000000..1c6afffb182 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env b/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env new file mode 100644 index 00000000000..8ea21dac6e2 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env new file mode 100644 index 00000000000..1e73199ee58 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/router.env b/integration-tests-ex/cases/cluster/Common/environment-configs/router.env new file mode 100644 index 00000000000..2e8c5bc6f43 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/router.env @@ -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 diff --git a/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml b/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml new file mode 100644 index 00000000000..bcecf4d9eee --- /dev/null +++ b/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml @@ -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 diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml new file mode 100644 index 00000000000..5c34af881d6 --- /dev/null +++ b/integration-tests-ex/cases/pom.xml @@ -0,0 +1,374 @@ + + + + + 4.0.0 + + org.apache.druid.integration-tests + druid-it-cases + druid-it-cases + New Integration Tests + + + org.apache.druid + druid + 0.24.0-SNAPSHOT + ../../pom.xml + + + + MustSetACategory + + + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + tests + test-jar + test + + + org.apache.druid + druid-core + ${project.parent.version} + + + org.apache.druid + druid-processing + ${project.parent.version} + + + org.apache.druid + druid-server + ${project.parent.version} + + + org.apache.druid + druid-services + ${project.parent.version} + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-client + + + com.google.guava + guava + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + joda-time + joda-time + + + commons-io + commons-io + + + com.github.docker-java + docker-java-core + 3.2.0 + + + com.github.docker-java + docker-java-api + 3.2.0 + + + com.github.docker-java + docker-java-transport-netty + 3.2.11 + + + com.github.docker-java + docker-java + 3.2.0 + + + com.github.docker-java + docker-java-transport-jersey + + + io.netty + netty-transport-native-kqueue + + + + + mysql + mysql-connector-java + ${mysql.version} + runtime + + + org.jdbi + jdbi + + + org.apache.druid.extensions + mysql-metadata-storage + ${project.parent.version} + + + org.apache.druid.extensions + druid-azure-extensions + ${project.parent.version} + provided + + + org.apache.druid.extensions + druid-hdfs-storage + ${project.parent.version} + provided + + + com.amazonaws + aws-java-sdk-bundle + + + + + org.apache.commons + commons-lang3 + + + io.netty + netty + + + org.apache.kafka + kafka-clients + ${apache.kafka.version} + test + + + com.google.code.findbugs + jsr305 + + + junit + junit + test + + + pl.pragmatists + JUnitParams + test + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/IT* + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.glassfish.hk2.external:jakarta.inject + + + + mysql:mysql-connector-java:jar + + + + + + + + + IT-HighAvailability + + false + + + HighAvailability + + + + IT-BatchIndex + + false + + + BatchIndex + + + + IT-InputFormat + + false + + + InputFormat + + + + IT-AzureDeepStorage + + false + + + AzureDeepStorage + + + + docker-tests + + false + + + + + maven-failsafe-plugin + 3.0.0-M7 + + + + org.apache.maven.surefire + surefire-junit47 + 3.0.0-M7 + + + + + functional-test + integration-test + + integration-test + + + + true + + org.apache.druid.testsEx.categories.${it.category} + + + + verify + verify + + verify + + + + + + + org.codehaus.mojo + exec-maven-plugin + + + start-cluster + pre-integration-test + + exec + + + ${project.basedir} + bash + + cluster.sh + up + ${it.category} + + + + + + stop-cluster + post-integration-test + + exec + + + ${project.basedir} + bash + + cluster.sh + down + ${it.category} + + + + + + + + + + diff --git a/integration-tests-ex/cases/resources/README.md b/integration-tests-ex/cases/resources/README.md new file mode 100644 index 00000000000..7ae95d8a55f --- /dev/null +++ b/integration-tests-ex/cases/resources/README.md @@ -0,0 +1,42 @@ + + +# 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. diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro new file mode 100644 index 0000000000000000000000000000000000000000..5ed0d60703560ba020d2e803dfc0bd682af94db1 GIT binary patch literal 1066 zcmah|PiqrF6gM?$ETNPl5`u?i5%kc8Y_?L}TdaRhS}_V5QFxhoO~%a5EN^BS*3yeN zZ#{@Nzkr{`&me*qKY<5N&L(L|i``!4-+S|Zf9B23;C-cf9rj4K3>#qzf3NO`HqS6j zLPDmAhwGt&wAfF?SY*fr%#Tb4tT@*+Y=K{ou*Jy<-r``PB~<3D&)gL#xe1du`$u)G zNYPkGGasEp8B0P9!-MtXqzr~w7MGxg?wlNFkjiQ}m4PKKB^D~rq@r;0>>MQ016_zc zvj!-w@>DxhmQ?sWGqt1=)5;2ksuYfCr!3`VV1z@VRapiTwzL9pzG$5AlolS4qeNRC zw?!w-Et}qcdvI;qicE84HRYtd@af*q5Aye|Z(E1y%~$XCeqH{4>2{|b#T)Ir8*%ia z9VP8f636|<<2*BY@W8hPQWe#Ta!evn&630Eu)GV>;A~t0vckukPc_+5&`ps9gXT*m zJmsKg!J=8c4|&ifgk}#^@Q6S$Vf>Fcy!dtL1cf+>JIUsq{*9jR;xSV2&=HQolUOzx zBA-E?CbD+?oz}X}_diGh3#)BV76*>uK>fKh#k4b!{!;yk_6jF|DvckN)5iY;mk(!F literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro new file mode 100644 index 0000000000000000000000000000000000000000..aa25803d5d221f9c851b63dab653ecb504c04150 GIT binary patch literal 1055 zcmaizL2J}d5XZHyvIbeJ6e)X8tazBdOq16pGvU1#Yb-q$ zJc@TeL=pT5#qZ#U@b1Z%?pk7@>2306GQXMs`@h%QyH`8+;Dh3!7AfZN=jMTDssg>x zQz%%)-iAkz*WdecPZk(USRUyDM19UG8G?;d9LB1Y@h}DhLrFzxY~_x`EY@E5?%-b^ zBXiUS^1^19C~cIa#5>$LD=HwtwtNPZl$X?$8dY15k_s@2vkdDkFC2w%|IH;N#oJsj zdtnbCrYh%Bw-wboFGOwWB$$jOq*k2Zl8ma_URP_mD$Pf5dd&-~n-XbH|Vl?1nDDbPpR zi$GQ-(AS>xx%VG$6g~{X(c}60lq)vqM%IVi*@e}Gg;6yKjqp6|g?%vOyaY1rM7&q= z%G#YJXKUaGkspNqC>)RH>(8q~SMF1bLDfO(Q7JNn1*QR5tk(l&q|NtVu`;PU$x%rG m{kJ6TRk$+)2KDL_sN9J{=+A)MX9`G2ZD0DAYu{GRtNj;6QDa{K literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro new file mode 100644 index 0000000000000000000000000000000000000000..2e0f9f2e1d08ffc5b8295d8847dae7dc5253356a GIT binary patch literal 1151 zcma)+yKWOf6ozqRz^p9GL=mJw$ZQ2tV)+s)WIM~E0ihu51|y2&ofGfGvu9Rw!C4|V zCL-GP`m?gm8|P9!We$vI`&PP zpda`OL#dd*^lY6EfNW|t^!6sNI2+d2zMV2pY50%&Znh_M<~UJgqM7$t?kOy#9pA)LOrf~4d~ zW@0bR0VFAnrPX;sna>MTb25dJ3@4;qTqrFWmFB?|j=7|~h>)Ay48Yi`aolZb|w?7hYQTe&lp`4x;Hh&gYQI6Aj9JKd__j`)RT2gJ0CXtj>d>2)n=*0P~=K@!d z7iG?QUN7PV<$eTo#47{fsRjC2@jg_}@tmJd+OCr(Q)#d^a92$h523oarKtW7_SC%z U;$(V}_x|n9wLRfS0dLvGcIYD z$hpLGDE=pHDmfcRt-!N@EgDFFJ&-mUh^SG7CFkO)ay1sPLra_2OJoh4iKZqN=2NhU zh`NCV_MtV?^_n4C)7Mmo5Fsxz9;OhF^5S?&a1;-sGFZT2tt->(U76s7C?>g9R@Q!c z*c=K4Z3<^Cg;JHm9!-H~7%R`qrHv_gSz+3FCcSxFBJG`#o8TW0OXV|6VhNhWd#y$5 z!L86OItTL z*d#X+;OIJ}=+B)qwKnJV7U*3~nt0ODSBJ$?|IZPIEa0`KqEOj?gQ`G`tkHIbL@w?0 zr9V=n$>qAl3#bV?JP}xc;PSvC9vvH?4$zG(EdW$PXpe&+KFte(_>nw8WMYt*9|ep3 z64B*RB1&Q(aqI!`Peg>VG`9p8;YC7jB7qQr;(3spfCs6VC$4({*w5z);V4>=5E8%RT1=sdbZVwxKQ zLmm)c2y;b9GksqPS^Hex{dU?2a!4Oxv-n`tET&p7R|cOk$tO`oyn zHN|OBkIXhV3-)h3mEUC?M(R=;BoVjnr9V;Cx1g*uPX49~33C>WDb0~k=g=r*+QHmB1oy~i%Yd2D!)llCsHl1A1 zbBt3GIAHF(AQAuc4u4?jae{KL>F(3z`mpKhw)uqAkGZ!h>taYSqC$zV2A!{it*biD`ubewWBxPjJ4 z8si3sTO%4+UI_15y*I%SDD;eN_S!_>mu*Q|{<8kZhZ`G?{$m*X;7wYB!u8PWm*R+p z-!gw*EDKEw`qiS&Gy3~q0{?#bU{mJ>qt4iLuO&Uh>{-cdl8d!gT<}UxL-(x*%*D#u zgRK+KO=*pS@sVo{Zgm62))P(Y+Zd;_eRfxKwp*8_J67KE8pHo%U6%dpa8Or3$ zt_Oznj85{`WP3X$P8l*QVWP;~j(q8d5n)f-;itdoPj|=agbAR+j3W1IIj}itYxp-Y zet{hBpqwK!p1&z=>3*mE{6J&L+)4~){Ea*zHca3C=CgZ= zrpiu(SI)ac(IJ&z#feBeTh4ax4CW7V*gLvblpX){bhNrx;rr&D<;6hbp<~1H{%-ES z#pQY*d>sYDZhhNU#cBbJ6CbSFPh=eT9TRsIWS`O|n^E1%)vt)}OgNdDn#sPDJSwjQ z`qgsw{_GuHb61noU#1fJ9a^g#mEHl_Ww(}&VL}Ro>|m`ff?doAC4Qx}=4S4y!HU{S zmZjM2sgUbdJA3Y&`=eRwWyiy9??28+AI#prj%$9BlG0LKR!~yXQhEKO!$tCQX37N# v#qM4+`?Ja&sf8u}+Qt#Z9|;;`I5LKsno2gof(@3fSQ|px0uC3z_AvMlJ1iCB literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc new file mode 100644 index 0000000000000000000000000000000000000000..3f0927403c0e263d9382f986cf5d2c0bf4ac8f1f GIT binary patch literal 2026 zcmaKqeKgzm9>>2)1PLNZOI=S*JXEV)O6LwOim+;0#ACI)Gry1!B_WBD7IVACL#w$? zG<#@iH}z1R>I2F&ZqXBkX@Aba9iju^_gQtpLru8_;7dR0@|CkjDpPy%NP%ky0X;C>WSF zxaOa@vG@{{MLpb+E9_>z>1GOdgfT2f;tO#W!$x%Arfq3T$BclPDU@o37rFYTG3F{< z@d%t}znNzZQ<`H9AfA97pTZIrxm zObcr|I03sno7TCQ!7iQ@+gVMIMiMD1k;|5{$+pWqXH_)xHj-0QP{-ap1PIbjaz>}$ z1?veET#cND?ygV-knbyt<+AffF-;Tw9FsZ@Hp~%5c_Vu^Ri=>Jk;RgV z-m_&2$>!+c>xo4g5CIbK(D2;=0|4F!bRz(Cb9fhkNQf=WfH;xdlR|C;#38-UgE;YQ zemoZgE0GxkaDp$Aq)kHv+XJ9IhCiGFyjyA#n*-6HG>8+$VGEcEqzC~dVDfbZq6TyX z01LJvN;Y8(0RdrS0>J5{WQfVpQXzFF?K%@78Gv7R`P(Lc#}K*YS9I+HLt$?oJC)Dn zkRpX?Nn9?6$`x`2Oj0DoN#q_9rtyWO061jwbS#>7CjcUnAWjzh2*eR^{UIUAkH>^` zpUk}9LHxnR27psYG&+%tMn@3PsO4L_LH=)=zHq*l2A+g~$4*OO3iy0D(Ash`;FE+l z-3!y&5g_UxeIehv;LlkdNVh(~K6+u`h6Y2?*eG33&+m?umN-W(DV7tzD8^dXaWhXP zW|sRKGxN)GW1ztOFG_8>2^Y8B3$s-WsV*0{I3DmS#h4~^8H}iltSkFp2Z>wkjdG+5 z{gu>sL08;TSJ0R`zxB{l8EdR)JnL}VWX``len$7Db{2|%e(WtB^X_};R`adowB%}~ z(~{!d*xJOt#q>_uF`vio+Se_*>l$+A&>BB-V&}($Yv_fJ&M&;1Z`_J4x9^@?yrS4> znwX!DmG+%#-gvP%KGc0%Jt^?WL7hf|)DpiCr0drFKI0%^hYK561w6y6J48lY7lvE9 zUr;;BjhbP3$FZDfW9s`iOZOcPv5&I!w(M1g^ecPpSM-?EuckNFH`+Asc&(J!=W@A8kCJjzMB*q*tIOLz4`U4@ei@zP`cr7q zFQ@D8QYHth8lBBkxJ_3V9cb4-u9lWk$1^*gTw1uD?3*>)P;>2snIiDVIZ-oWb=ViL zLV(cZS7Q!!=PHXTacHRge7D5>wjwBaie40W&9YVcFmtM`oc26G_DKu6O8lg_G@G;} z`l_llMZ(e;iW=)K)aqB?^u+~s+3!Ywxaj2n(eC@J8-Bk^}G_EqL=?rJ8VXF5ivOwobC$VV74?A}aK? z8mm7VVRZ0Vy;&HiG2%qBQ^yOp??C&<_J$T%EQak1iTQ!pbE5h)tL^0N4+~rqUX)TeKKlL|tj3n?4LY`~JqpbFL+H2IUx5!Rni~iXq6e={F`j`I#UjY=p literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc new file mode 100644 index 0000000000000000000000000000000000000000..5c5f54486a076853949cdfad0c378ec7014498c1 GIT binary patch literal 2008 zcmaizX;_n27RPTAkR;duVGC%LP-R#&kcl85C=js9W)fq83Y0exAuLHu2qA!MqJkoo zU~w#{fQYqVQKl?|KJ0>8S};_`xtbet_lY({1H% zm`TH%BGv;BILsJeu>pd36lb*6o#gOxSvj;UAU-*_JozvMa~!M#Ua+{?2XPa$hPggg z!(GUM@ECWIFkS$~av*{=4S25!y;2KtU?Domze}c+mUR?&dlKtPWJ+bXC*~?F_6aFg zHxo%{K>iO8db-&-!k7UK>{#m5q-RbQT>DegqaAmpedV$;Z(`j*ZpYoz zzL)|y&tc8L2unSLS7CJJAj`_i+RqSKfXBi!*u54VWQ7A1Q4eQiyE_LHX}nr&SaJ)R zt2Oy23Q73??7pxBV_3~@0<)7zd5R9Dw6E;GFEJ0s&MfYevM`k}i6H=?C{!VMtAz(h zO#l!;3xK_VrVW6)9b^CwO1Kehi62iG&J!~sjzk1;nYq)H;@k0_YnIx z0I;?Uf2{%qKL5EE43Q8HQKL9q$Q$BAT=-@i3Zv=>AV zE#UIZ+++Ds5LLj56Y{ubObI`V$Kz6Y5?(ypj0tfgKRBFRwg-R;apT!6kw9{Q6$=UD z4@N=UM2-*qllRet?tgpd1~+D{D%^?}sC&vh%Y|+P^(!p;&kQ$|-KU7V01)!o?1(s6 zoh=CBz_|QacDxXNQ5j6n8Z1z+9~kLGbQDgWm!H907-G!i z9u~AvoE{nn=i4aq3udi1T{_!iTTXQT^_kOg$|J4f$6E6#_46aeb6+!;vbC3fX!Jjp zOrB{T$xX2h;<5ObA4M*1yF!-fB$jUex$FLFk??ZB&uR1!`is+lc#XP<8LRsx1xo>n z-hlGjYbQ1zuHVE<<#zT@QTm&2r$sxw`S-E<)u&UJL%+S!)jrN&LNbs*l()yH-Hp)c z<2&)jipk07s>o@Z zhDBEGrQ*ENcXyIsXB3(6FK#&05>ATjo|$q*@x4>38~e@<4Wi^Z;_RIzbyteq zcMtvE?qV|hbnESXEQF%hvULLDR2kjCH@HXVFtsbr$#LG)cYNV2c_9GT3KD+dHcd7D4m^L9G@QE2a%kilS9m5zhV&)OrX@JEB3 z9Z%wg{SHXO=VEiKs+s4OgI>W;pCrn28kpbB+I`;E>XTu^vA9`mnNM&E@Y}Avm37r< z(S-WunW>ZgqbJ$D&%!6Xl=kbBoT~37#YB%^x@pb#jJv9^Ye6uVztOxfeQ0at9Kw{C zJ6lUXt=H~Y!egvJI!htBC(mc~MEq`9r&!oM?K*IHOHx!(8!2$1<2UDDble)SZHAeh zXA8HP)SngC$fPfkx@Q}YQ)-c_NPP3>jsL_DSvk?wvLQ-_(^pmVmx1* zR1uueEQ+>RoI6u>N|q>6H0F7Id5{wtm8#Vdr+aW#K7U12SFXcIoqhj!N-RlOTI}k7 zw>a!n_-ZEDA@pC@zlXAE?Z%u1%Wv)^&yMM!yHUCbLVCKcITEb5tU}tY%bZ7{4lt-D F{{iiL5gY&j literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..aaed7672c5d15a32e29db8f159217c31f2019eda GIT binary patch literal 4119 zcmcJT&r2L<7{}*bXUE+{MHAo2Yr9RSM&Ejf;%aOYDJ3R04YEx{?V%x;*WI@|xX!He zqq4Mz)`R3AAqNj7c*wbgl0zt^l#+my(B=?&2%(goij*Gm2NY@gyfZsHGwWE1+C_KV z=Y8Ji{k+e<&-Zr?+BVF`C)a#}dSuaYB~ zz=LGY)qGT`FTo0fM|x85uAo#?k3|1KIV^XL$db}U(yUbCC7eTIEe#8O%*r+gU>FF2-;(;$-T?6*;Fr~WETl;EmYl;aHL5H#OryGJy0%BQ ziqxBk-fyBSO}T_9Krn#|LHc^fTOsl^Gqg(jy#5xYit&ggs&+07S&m*Tb8%9a^##4cOI|%SYEia6NSbR?Vmv8& zUk!P(23zEIg%y3Edr18j(Z`r>cM727SOC|KI|F6R{qW&;FRw7@&8gk2(0QzsFuB7O_SLod?0lim5*v2J15p3hs zjSQ@EfmO9Ji+j3xkr@u3buCZWug;d4?Ib9t;*XF_Ftw3&^<>DZSbR=54e!{2xIaep z&TI6$oS;pL1BBg}1_!X?eEs&@^UL0O2{tM8wic44EpyiN#d{s88;Cx5o&Iw&{I2T( zfZ-Oxvq73JU+;ts4h`mq`v=eV59enF^P_`9qr=13-X#(l9vvPUJu`BFNV&I1^ZC)C z{I!IXbw;NLek5iMf;J}e&Z%$15aDmY2Wpk-CZKk8jc~DzFlQ;8fRk$B0A} zyExAm8P+5$_oixHo`hTA@36P(l?pjn*t2>aP1F#!4^eAyKL<7uJh%XBOL@16AG9vw zfxEYUT}h+XK-!`#Y#o{gj+T=4O)I+WFX%^ePwKN`tsEo&=Szsm->+&!VNMB{}T8Y`O_Qr literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6f26f1f07e39e802bc0798691e3c24337649ea16 GIT binary patch literal 4116 zcmcIoL2MI86y0&wYdbh4gv>H-O<6%Q^&vraLMTKCHGpVDBN{dC1##%qS@0HT*X*tf z=2TS((F;OV#i8QRLsj+A0|$gSpk6=~s!)YE^nfakJyaFqP^mpcZU4-Ayz9-T5^=gp zcJ04^-n@B#{+}-|&b?Q`DlX&l0@koPq9`Q$*?^KphmaCn2Rl!!;89E_xEK6uMd2sj zk*xI9l@cEvv%=TGu@u}jlo}XQ=&z`V)xwObXa$mH%gg36PU)Kd6zR_ptDLgwGe9p76cbQ@!3n$|^&6ka0CJUC?H04sXv?B=uI&tgX^Gj!ahKUO%d0tYH92gR zdy=6ZKN#1e-7_s@sn#sNEeJkF`p<~INYkrnL2w-;AUcKjFTjHby6}Mt=yRUMIJ=XM z>(`CB71On9EJ3=V9tWcGaWJf_;DAbRyh#IEg{uR&1u=b z8z}h*=}!>-dysxPD2T2D229I%uVhNpw30)I24NO{Jq#X7oE*u@Hzdn*ZIzK?Fzt8TqG-f z_!Mic@fG~EEnGnwR^T(qA9DWQFnzHV_^*Xo7aaJH;gSr%$T6PBm+Jif5#x8$j&t9y zlv0=KBxTxhZdXRb&5&&{`)rGyH3hwENWX#T?d|l%2;U9icp=|DnMvey4lplR=d7l= zY%!ky%=N8WBdG@Ex_%_n@dE#79(|*Sd~!BhF})_M1%z%P{Vt;4WBPoQ6H3Mca=S5= zofP{21ws~Pus`CZXZzB5J4*Km5P9_EcEY9FFGD6b} zTVe8}uC?Mhwoz?2>yBgR(e{$lc6`&Qg6a+$o_M?6^xDQT_-48Z0nA6=ll;-hU_kFl znOm=gtktHZ{1?(!p_0E!Zx=Y_|HgwA9L7iF3U*!8V2-6avoBg)(RY~bJ6ZT}gtd)1 z*W^XB=YRx!EpR7U=@VzUx-(>{x#n5RvBk!_hx7-C-q=a+7CFx(oOqOFaJ$h4KJoJ} z7NY2;yKaO`(ysHN6BOqnKVDCc@pXh7Qlk0Oqp-lKa%Fn5d|-0AQms^GXAaCx zA3SxKC}e7OUuAac&7EZ5!O3!^TCU8Nr)H<8PbKI!8esJLk&d+&Ij-ned%|_(d<&At zXDdx1>VuU2b&6iuO;-)RmY16ZGy4JQkDv^`K_64jED;8$3ai^@6ry>xy1BxzpjPoQKb0 zS2eCI6k%#%6S2P*C3xT707py>Z~9Q0Aj>Bft!{*dUJQrg{Q?)jGKPu7}E<73nHplvWJ hduR4mCYMgmEiTQ!TZL)@751h6l>5*+EW+;r{{l>qDDeOQ literal 0 HcmV?d00001 diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet new file mode 100644 index 0000000000000000000000000000000000000000..fd318da28b89a963f88f073f18b0e968311175fe GIT binary patch literal 4160 zcmcIoO^6&t6z-bs>FM5#n@ybR^yp~DBwJ2`8+P_5Ka0q&F(7d@!|os&a!6;U+-Y~V zyX_z2#ETIL(SziWQwS0y1P^HRduqC8wnNCy z+6>*buU~)ly;t?UdTeEOzKlgYgI}J*G8Q)rLQ(z+327vJoR(e_gwF?r;6mX;zyi); z?Iq@mB`g&*nR{q5_fdsXf)xM;cckDhqtw9f0{Igau$Z3`1u3tk^?IGwaY~id2MB)T z7?Gb08t81s>Ksl5F(}0}O!S`KbSV&q7ZKq-LMa`*(834!DaDW$k_GaWSispfD@8Xg zv$bsbZoup`Qolsx`T#jQP)I-l1{FNdvsl_80rEo2tkV`6tiNfwHS0tMOjX@z`MSyY zRjs9#)tgX;9K6t^@FQ^S%G*6DVkp)OuN9EHiPT$&d@7N1>401>1hCtMchA9ttud}B z06Xiuo})JnU1Wc_hMtfsrQ@WD%mq9<=^7X!59NrAQnO6YFsbPUj4mN{1(EYI`8mTF zB>(}LdHgod*v^YxoLKtieHW%=sXEXw%wTXxw{q4mCTI5*s*Icd?X0uRQru4b6*t%vf9c( z;sUuk6bU3#r%mcb61k7a*H4jKSrEx;phv<7QN)CPC=j|82>r9R(Et87yn1u2$=pMe zxeuFYwY4{n8=Imy(h!HI$Ao@mnB3V2QLKh}BsfGdhR69@XgMZcY9eXU@!9ZXgWca} zo4Kn}?GZ|;lMO9J&0d11bJ5O_o4UE^>x(qtcpj-25xM#_xxblhjp+M;!z7oJc9d=1 z6ULHZ)4HKEnd^>c)S5}vN9p)!nYI`BN1{rR%WA|&G4*BY+IlS{^es}qL*z$H{>TNQ zO2h(k+c4qeHq9`3dyY@p_Ip%K6q!RpNgT+M8(KssW6~29o}F_@y#T`5LcZC;$n-z} zi!Ts9gZM%nJTnCIFL;izM4jS6-=Rx-7AyxE)G@8%Y}0P&8`1EjV=TLtS**6~hGm)Q z?Io*ad9+xCbxA6gL*0m4HjW4IU6Q)etnMIFEG-HK)`eu=lG#07x+dQsl-q zvho6pC=U1dNg(1b&2_Z)*qpC!LqKyYB%-;&H*MW>tY))a|3lr~9wBE(qSSX_fXNYj zl&5~Q!kD-=rF9tw%dj!2^R9uib09<;&J>rKp${<3Md~EvTIPP5%s=kU>FJ#jS 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 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 getAs(String url, Class 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 getAs(String url, TypeReference 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 ? "" : 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() + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java new file mode 100644 index 00000000000..91da889c685 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java @@ -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. + *

+ * 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 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; + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java new file mode 100644 index 00000000000..86e0ece8d52 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java @@ -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. + *

+ * Used to verify that the DB is up and available. The JDBC + * connection can be used to query the metadata DB for tests. + *

+ * 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; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java new file mode 100644 index 00000000000..cad00d1ae4a --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.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. + *

+ * 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; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java new file mode 100644 index 00000000000..d45ec864d2e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java @@ -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:

{@code
+ * @Cluster(BatchIndex.class)
+ * public class InputFormat
+ * {
+ * }
+ * Note that a category uses a single cluster configuration. It is not + * possible to have tests 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(); +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java new file mode 100644 index 00000000000..363f7648b07 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java @@ -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. + *

+ * 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 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 druidServices; + @JsonProperty("settings") + private Map settings; + @JsonProperty("properties") + private Map properties; + @JsonProperty("metastoreInit") + private List 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 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 druid() + { + return druidServices; + } + + @JsonProperty("settings") + @JsonInclude(Include.NON_NULL) + public Map settings() + { + return settings; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_NULL) + public Map properties() + { + return properties; + } + + @JsonProperty("metastoreInit") + @JsonInclude(Include.NON_NULL) + public List 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; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java new file mode 100644 index 00000000000..1531edff0fd --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java @@ -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 props = resolved.toProperties(); + // Added from ZK section + assertEquals("localhost:2181", props.get("druid.zk.service.zkHosts")); + // Generic property + assertEquals("howdy", props.get("my.test.property")); + // Mapped from settings + assertEquals("myBucket", props.get("druid.test.config.cloudBucket")); + assertEquals("myPath", props.get("druid.test.config.cloudPath")); + assertEquals("secret", props.get("druid.test.config.s3AccessKey")); + // From settings, overridden in properties + assertEquals("myRegion", props.get("druid.test.config.cloudRegion")); + // System property + assertEquals("sys", props.get("druid.test.config.sys_prop")); + // From user override + assertEquals("user", props.get("druid.test.config.user_var")); + + // Test plumbing through the test config + Properties properties = new Properties(); + properties.putAll(props); + IntegrationTestingConfig testingConfig = new IntegrationTestingConfigEx(resolved, properties); + assertEquals("myBucket", testingConfig.getCloudBucket()); + assertEquals("myPath", testingConfig.getCloudPath()); + // From settings, overridden in properties + assertEquals("myRegion", testingConfig.getCloudRegion()); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java new file mode 100644 index 00000000000..99e8b7e3cbb --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Indicates a method to use to configure the {@link Initializer.Builder}. Must be of the + * form: + *


+ * @Configure
+ * public static void configure(Initializer.Builder builder)
+ * {
+ *    builder.something(arg);
+ * }
+ * 
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface Configure
+{
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
new file mode 100644
index 00000000000..5e65ec7f9d4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.testsEx.config;
+
+import junitparams.JUnitParamsRunner;
+import org.apache.druid.java.util.common.UOE;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.Statement;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+
+/**
+ * Test runner for Druid integration tests. Loads test configuration. Initializes
+ * Guice. Injects dependencies into the test. Runs the tests.
+ * Shuts down afterwards. Since cluster configuration and health checks are a bit
+ * expensive, does that work once per test class rather than once per test method.
+ * 

+ * Note that our Guice usage is a bit awkward for tests. Druid modules define + * objects that must be lifecycle managed, but as lazy singletons, which means + * that they might be created after the lifecycle starts, which causes the dreaded + * "it doesn't work that way" message. The awkward workaround is to ask to inject + * test members before starting the lifecycle, so that the injection creates + * a reference, which creates the object, which registers it in the lifecycle. We + * should fix this issue. Until then, the awkwardness is hidden in this test runner. + *

+ * Extends the parameterize test runner, so your Druid ITs can also use parameters. + */ +public class DruidTestRunner extends JUnitParamsRunner +{ + private class CloseInitializer extends Statement + { + private final Statement next; + + public CloseInitializer(Statement next) + { + this.next = next; + } + + @Override + public void evaluate() throws Throwable + { + next.evaluate(); + if (initializer != null) { + initializer.close(); + initializer = null; + } + } + } + + private Initializer initializer; + + public DruidTestRunner(Class testClass) throws InitializationError + { + super(testClass); + } + + @Override + protected Object createTest() throws Exception + { + Object test = super.createTest(); + if (initializer == null) { + initializer = buildInitializer(test); + } else { + initializer.injector().injectMembers(test); + } + return test; + } + + private Initializer buildInitializer(Object test) + { + Class testClass = test.getClass(); + Category[] annotations = testClass.getAnnotationsByType(Category.class); + if (annotations.length == 0) { + throw new UOE( + "Class % must have a @Category annotation", + testClass.getSimpleName() + ); + } + if (annotations.length != 1) { + throw new UOE( + "Class % must have exactly one @Category annotation", + testClass.getSimpleName() + ); + } + Class[] categories = annotations[0].value(); + if (categories.length == 0) { + throw new UOE( + "Class % must have a @Category value", + testClass.getSimpleName() + ); + } + if (categories.length != 1) { + throw new UOE( + "Class % must have exactly one @Category value", + testClass.getSimpleName() + ); + } + Class category = category(testClass); + String clusterName = inferCluster(category); + Initializer.Builder builder = Initializer.builder(clusterName) + .test(test) + .validateCluster(); + for (Method method : testClass.getMethods()) { + if (method.getAnnotation(Configure.class) == null) { + continue; + } + final int requiredMods = Modifier.STATIC | Modifier.PUBLIC; + if ((method.getModifiers() & requiredMods) != requiredMods) { + throw new UOE( + "Method %s annotated with @Configure must be public static", + method.getName() + ); + } + try { + method.invoke(null, builder); + } + catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) { + throw new UOE( + "Call to Method %s annotated with @Configure failed: %s", + method.getName(), + e.getMessage() + ); + } + } + return builder.build(); + } + + /** + * Resolve the {@code @Category} annotation for the test class. + */ + private Class category(Class testClass) + { + Category[] annotations = testClass.getAnnotationsByType(Category.class); + if (annotations.length == 0) { + throw new UOE( + "Class % must have a @Category annotation", + testClass.getSimpleName() + ); + } + if (annotations.length != 1) { + throw new UOE( + "Class % must have exactly one @Category annotation", + testClass.getSimpleName() + ); + } + Class[] categories = annotations[0].value(); + if (categories.length == 0) { + throw new UOE( + "Class % must have a @Category value", + testClass.getSimpleName() + ); + } + if (categories.length != 1) { + throw new UOE( + "Class % must have exactly one @Category value", + testClass.getSimpleName() + ); + } + return categories[0]; + } + + /** + * Resolve the optional {@code @Cluster} annotation on the test category. + * If omitted, the category itself is the cluster name. + */ + private String inferCluster(Class category) + { + String categoryName = category.getSimpleName(); + Cluster[] annotations = category.getAnnotationsByType(Cluster.class); + if (annotations.length == 0) { + return categoryName; + } + if (annotations.length != 1) { + throw new UOE( + "Category % must have no more than one @Cluster annotation", + category.getSimpleName() + ); + } + Class clusterClass = annotations[0].value(); + return clusterClass.getSimpleName(); + } + + @Override + protected Statement withAfterClasses(Statement statement) + { + return new CloseInitializer(super.withAfterClasses(statement)); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java new file mode 100644 index 00000000000..a2899a08448 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java @@ -0,0 +1,590 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.apache.druid.cli.GuiceRunnable; +import org.apache.druid.curator.CuratorModule; +import org.apache.druid.curator.discovery.DiscoveryModule; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.guice.AnnouncerModule; +import org.apache.druid.guice.DruidProcessingConfigModule; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.PolyBind; +import org.apache.druid.guice.SQLMetadataStorageDruidModule; +import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.guice.StorageNodeModule; +import org.apache.druid.guice.annotations.Client; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.http.HttpClientModule; +import org.apache.druid.guice.security.EscalatorModule; +import org.apache.druid.initialization.CoreInjectorBuilder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.jackson.DruidServiceSerializerModifier; +import org.apache.druid.jackson.StringObjectPairList; +import org.apache.druid.jackson.ToStringObjectPairListDeserializer; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.LoggingEmitter; +import org.apache.druid.java.util.emitter.core.LoggingEmitterConfig; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageProvider; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.NoopMetadataStorageProvider; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig; +import org.apache.druid.metadata.storage.mysql.MySQLConnectorSslConfig; +import org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule; +import org.apache.druid.server.DruidNode; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.IntegrationTestingConfigProvider; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.cluster.MetastoreClient; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +/** + * The magic needed to piece together enough of Druid to allow clients to + * run without server dependencies being pulled in. Used to set up the + * Guice injector used to inject members into integration tests, while + * reading configuration from the docker.yaml or similar test + * configuration file. + *

+ * Much of the work here deals the tedious task of assembling Druid + * modules, sometimes using copy/past to grab the part that a client + * wants (such as object deserialization) without the parts that the + * server needs (and which would introduce the need for unused configuration + * just to make dependencies work.) + *

+ * See the documentation for these test for the "user view" of this + * class and its configuration. + */ +public class Initializer +{ + public static final String TEST_CONFIG_PROPERTY = "testConfig"; + public static final String TEST_CONFIG_VAR = "TEST_CONFIG"; + public static final String CLUSTER_RESOURCES = "/cluster/"; + public static final String CLUSTER_CONFIG_RESOURCE = CLUSTER_RESOURCES + "%s/%s.yaml"; + public static final String CLUSTER_CONFIG_DEFAULT = "docker"; + public static final String METASTORE_CONFIG_PROPERTY = "sqlConfig"; + public static final String METASTORE_CONFIG_RESOURCE = "/metastore/%s.sql"; + public static final String METASTORE_CONFIG_DEFAULT = "init"; + + private static final Logger log = new Logger(Initializer.class); + + public static String queryFile(Class category, String fileName) + { + return CLUSTER_RESOURCES + category.getSimpleName() + "/queries/" + fileName; + } + + private static class TestModule implements DruidModule + { + ResolvedConfig config; + + public TestModule(ResolvedConfig config) + { + this.config = config; + } + + @Override + public void configure(Binder binder) + { + binder + .bind(ResolvedConfig.class) + .toInstance(config); + binder + .bind(IntegrationTestingConfig.class) + .to(IntegrationTestingConfigEx.class) + .in(LazySingleton.class); + binder + .bind(MetastoreClient.class) + .in(LazySingleton.class); + + // Dummy DruidNode instance to make Guice happy. This instance is unused. + binder + .bind(DruidNode.class) + .annotatedWith(Self.class) + .toInstance( + new DruidNode("integration-tests", "localhost", false, 9191, null, null, true, false)); + + // Reduced form of SQLMetadataStorageDruidModule + String prop = SQLMetadataStorageDruidModule.PROPERTY; + String defaultValue = MySQLMetadataStorageModule.TYPE; + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageConnector.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageProvider.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(SQLMetadataConnector.class), defaultValue); + + // Reduced form of MetadataConfigModule + // Not actually used here (tests don't create tables), but needed by MySQLConnector constructor + JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class); + + // Build from properties provided in the config + JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class); + } + + @Provides + @TestClient + public HttpClient getHttpClient( + IntegrationTestingConfig config, + Lifecycle lifecycle, + @Client HttpClient delegate + ) + { + return delegate; + } + + @Provides + @ManageLifecycle + public ServiceEmitter getServiceEmitter(ObjectMapper jsonMapper) + { + return new ServiceEmitter("", "", new LoggingEmitter(new LoggingEmitterConfig(), jsonMapper)); + } + + // From ServerModule to allow deserialization of DiscoveryDruidNode objects from ZK. + // We don't want the other dependencies of that module. + @Override + public List 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 getJacksonModules() + { + return new MySQLMetadataStorageModule().getJacksonModules(); + } + } + + /** + * Class used by test to identify test-specific options, load configuration + * and "inject themselves" with dependencies. + */ + public static class Builder + { + private final String clusterName; + private String configFile; + private Object test; + private List modules = new ArrayList<>(); + private boolean validateCluster; + private List> eagerCreation = new ArrayList<>(); + private Map envVarBindings = new HashMap<>(); + private Properties testProperties = new Properties(); + + public Builder(String clusterName) + { + this.clusterName = clusterName; + + // Node discovery is lifecycle managed. If we're using it, we have to + // create the instance within Guice during setup so it can be lifecycle + // managed. Using LazySingleon works in a server, but not in test clients, + // because test clients declare their need of node discovery after the + // the lifecycle starts. + eagerInstance(DruidNodeDiscoveryProvider.class); + + // Set properties from environment variables, or hard-coded values + // previously set in Maven. + propertyEnvVarBinding("druid.test.config.dockerIp", "DOCKER_IP"); + propertyEnvVarBinding("druid.zk.service.host", "DOCKER_IP"); + propertyEnvVarBinding("druid.test.config.hadoopDir", "HADOOP_DIR"); + property("druid.client.https.trustStorePath", "client_tls/truststore.jks"); + property("druid.client.https.trustStorePassword", "druid123"); + property("druid.client.https.keyStorePath", "client_tls/client.jks"); + property("druid.client.https.certAlias", "druid"); + property("druid.client.https.keyManagerPassword", "druid123"); + property("druid.client.https.keyStorePassword", "druid123"); + + // More env var bindings for properties formerly passed in via + // a generated config file. + final String base = IntegrationTestingConfigProvider.PROPERTY_BASE + "."; + propertyEnvVarBinding(base + "cloudBucket", "DRUID_CLOUD_BUCKET"); + propertyEnvVarBinding(base + "cloudPath", "DRUID_CLOUD_PATH"); + propertyEnvVarBinding(base + "s3AccessKey", "AWS_ACCESS_KEY_ID"); + propertyEnvVarBinding(base + "s3SecretKey", "AWS_SECRET_ACCESS_KEY"); + propertyEnvVarBinding(base + "azureContainer", "AZURE_CONTAINER"); + propertyEnvVarBinding(base + "azureAccount", "AZURE_ACCOUNT"); + propertyEnvVarBinding(base + "azureKey", "AZURE_KEY"); + propertyEnvVarBinding(base + "googleBucket", "GOOGLE_BUCKET"); + propertyEnvVarBinding(base + "googlePrefix", "GOOGLE_PREFIX"); + + // Other defaults + // druid.global.http.numMaxThreads avoids creating 40+ Netty threads. + // We only ever use 1. + property("druid.global.http.numMaxThreads", 3); + property("druid.broker.http.numMaxThreads", 3); + } + + /** + * Load a configuration from the named file. Primarily for + * debugging to use a one-off, custom configuration file. + */ + public Builder configFile(String configFile) + { + this.configFile = configFile; + return this; + } + + /** + * The test class with members to be injected. + */ + public Builder test(Object test) + { + this.test = test; + return this; + } + + /** + * Druid provides the {@code PolyBind} abstraction and the {@code Lifecycle} + * abstraction. When used together, we can encounter initialization issues. We won't create + * and instance of a polymorphic binding until it is first needed, and only then does + * the instance add itself to the lifecycle. However, if it is a test that acks for + * the instance, that is too late: the lifecycle has started. A test should call this + * method to "register" polymorphic lifecycle classes that will be injected later. + *

+ * The builder registers {@code DruidNodeDiscoveryProvider} by default: add any + * test-specific instances as needed. + */ + public Builder eagerInstance(Class theClass) + { + this.eagerCreation.add(theClass); + return this; + } + + /** + * Optional test-specific modules to load. + */ + public Builder modules(List modules) + { + this.modules.addAll(modules); + return this; + } + + public Builder modules(Module...modules) + { + return modules(Arrays.asList(modules)); + } + + /** + * Validates the cluster before running tests. Ensures that each + * Druid service reports itself as healthy. Since Druid services + * depend on ZK and the metadata DB, this indirectly checks their + * health as well. + */ + public Builder validateCluster() + { + this.validateCluster = true; + return this; + } + + /** + * Set a property value in code. Such values go into the {@link Properties} + * object in Guice, and act as defaults to properties defined in the config + * file or via system properties. These properties can also "hard code" items + * that would normally be user-settable in a server. The value can be of any + * type: it is converted to a String internally. + */ + public Builder property(String key, Object value) + { + if (value == null) { + testProperties.remove(key); + } else { + testProperties.put(key, value.toString()); + } + return this; + } + + /** + * Bind a property value to an environment variable. Useful if the property + * is set in the environment via the build system, Maven or other means. + * Avoids the need to add command-line arguments of the form + * {@code -Dproperty.name=$ENV_VAR}. Environment variable bindings take + * precedence over values set via {@link #property(String, Object)}, or + * the config file, but are lower priority than system properties. The + * environment variable is used only if set, else it is ignored. + */ + public Builder propertyEnvVarBinding(String property, String envVar) + { + this.envVarBindings.put(property, envVar); + return this; + } + + public synchronized Initializer build() + { + return new Initializer(this); + } + } + + private final ResolvedConfig clusterConfig; + private final Injector injector; + private final Lifecycle lifecycle; + private MetastoreClient metastoreClient; + private DruidClusterClient clusterClient; + + private Initializer(Builder builder) + { + if (builder.configFile != null) { + this.clusterConfig = loadConfigFile(builder.clusterName, builder.configFile); + } else { + this.clusterConfig = loadConfig(builder.clusterName, builder.configFile); + } + this.injector = makeInjector(builder, clusterConfig); + + // Do the injection of test members early, for force lazy singleton + // instance creation to avoid problems when lifecycle-managed objects + // are combined with PolyBind. + if (builder.test != null) { + this.injector.injectMembers(builder.test); + } + + // Backup: instantiate any additional instances that might be referenced + // later outside of injection. + for (Class eagerClass : builder.eagerCreation) { + this.injector.getInstance(eagerClass); + } + + // Now that we've created lifecycle-managed instances, start the lifecycle. + log.info("Starting lifecycle"); + this.lifecycle = GuiceRunnable.initLifecycle(injector, log); + + // Verify the cluster to ensure it is ready. + log.info("Creating cluster client"); + this.clusterClient = this.injector.getInstance(DruidClusterClient.class); + if (builder.validateCluster) { + clusterClient.validate(); + } + + // Now that the cluster is ready (which implies that the metastore is ready), + // load any "starter data" into the metastore. Warning: there is a time-lag between + // when the DB is updated and when Coordinator or Overlord learns about the updates. + // At present, there is no API to force a cache flush. Caveat emptor. + prepareDB(); + } + + public static Builder builder(String clusterName) + { + return new Builder(clusterName); + } + + private static ResolvedConfig loadConfig(String category, String configName) + { + if (configName == null) { + configName = System.getProperty(TEST_CONFIG_PROPERTY); + } + if (configName == null) { + configName = System.getenv(TEST_CONFIG_VAR); + } + if (configName == null) { + configName = CLUSTER_CONFIG_DEFAULT; + } + String loadName = StringUtils.format(CLUSTER_CONFIG_RESOURCE, category, configName); + ClusterConfig config = ClusterConfig.loadFromResource(loadName); + return config.resolve(category); + } + + private static ResolvedConfig loadConfigFile(String category, String path) + { + ClusterConfig config = ClusterConfig.loadFromFile(path); + return config.resolve(category); + } + + private static Injector makeInjector( + Builder builder, + ResolvedConfig clusterConfig + ) + { + Injector startupInjector = new StartupInjectorBuilder() + .withProperties(properties(builder, clusterConfig)) + .build(); + return new CoreInjectorBuilder(startupInjector) + .withLifecycle() + .add( + // Required by clients + new EscalatorModule(), + HttpClientModule.global(), + HttpClientModule.escalatedGlobal(), + new HttpClientModule("druid.broker.http", Client.class), + new HttpClientModule("druid.broker.http", EscalatedClient.class), + // For ZK discovery + new CuratorModule(), + new AnnouncerModule(), + new DiscoveryModule(), + // Dependencies from other modules + new DruidProcessingConfigModule(), + // Dependencies from other modules + new StorageNodeModule(), + + // Test-specific items, including bits copy/pasted + // from modules that don't play well in a client setting. + new TestModule(clusterConfig), + new TestMySqlModule() + ) + .addAll(builder.modules) + .build(); + } + + /** + * Define test properties similar to how the server does. Property precedence + * is: + *

    + *
  • System properties (highest)
  • + *
  • Environment variable bindings
  • + *
  • Configuration file
  • + *
  • Hard-coded values (lowest>
  • + *
+ */ + private static Properties properties( + Builder builder, + ResolvedConfig clusterConfig + ) + { + Properties finalProperties = new Properties(); + finalProperties.putAll(builder.testProperties); + finalProperties.putAll(clusterConfig.toProperties()); + for (Entry entry : builder.envVarBindings.entrySet()) { + String value = System.getenv(entry.getValue()); + if (value != null) { + finalProperties.put(entry.getKey(), value); + } + } + finalProperties.putAll(System.getProperties()); + log.info("Properties:"); + log.info(finalProperties.toString()); + return finalProperties; + } + + /** + * Some tests need a known set of metadata in the metadata DB. To avoid the + * complexity of do the actual actions (such as creating segments), the tests + * "seed" the database directly. The result is not entirely valid and consistent, + * but is good enough for the test at hand. + *

+ * WARNING: At present, there is no way to force the Coordinator or + * Overlord to flush its cache to learn about these new entries. Instead, we have + * to sleep for the cache timeout period. This solution is unsatisfying, and error-prone. + */ + private void prepareDB() + { + ResolvedMetastore metastoreConfig = clusterConfig.metastore(); + if (metastoreConfig == null) { + return; + } + List stmts = metastoreConfig.initStmts(); + if (stmts == null || stmts.isEmpty()) { + return; + } + log.info("Preparing database"); + MetastoreClient client = injector.getInstance(MetastoreClient.class); + for (MetastoreStmt stmt : stmts) { + client.execute(stmt.toSQL()); + } + try { + Thread.sleep(metastoreConfig.initDelaySec() * 1000); + } + catch (InterruptedException e) { + throw new RuntimeException("Interrupted while waiting for coordinator to notice DB changes"); + } + log.info("Database prepared"); + } + + public Injector injector() + { + return injector; + } + + public ResolvedConfig clusterConfig() + { + return clusterConfig; + } + + public MetastoreClient metastoreClient() + { + if (clusterConfig.metastore() == null) { + throw new IAE("Please provide a metastore section in docker.yaml"); + } + return injector.getInstance(MetastoreClient.class); + } + + public DruidClusterClient clusterClient() + { + return clusterClient; + } + + public void close() + { + lifecycle.stop(); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java new file mode 100644 index 00000000000..c14ea745aa8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.IntegrationTestingConfigProvider; + +import javax.inject.Inject; + +import java.util.Map; +import java.util.Properties; + +/** + * Adapter to the "legacy" cluster configuration used by tests. + */ +class IntegrationTestingConfigEx implements IntegrationTestingConfig +{ + private final ResolvedConfig config; + private final Map properties; + + @Inject + public IntegrationTestingConfigEx( + final ResolvedConfig config, + final Properties properties) + { + this.config = config; + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : properties.entrySet()) { + String key = (String) entry.getKey(); + if (key.startsWith(IntegrationTestingConfigProvider.PROPERTY_BASE)) { + key = key.substring(IntegrationTestingConfigProvider.PROPERTY_BASE.length() + 1); + builder.put(key, (String) entry.getValue()); + } + } + this.properties = builder.build(); + } + + @Override + public String getZookeeperHosts() + { + return config.requireZk().clientHosts(); + } + + @Override + public String getKafkaHost() + { + return config.requireKafka().instance().clientHost(); + } + + @Override + public String getKafkaInternalHost() + { + return config.requireKafka().instance().host(); + } + + @Override + public String getBrokerHost() + { + return config.requireBroker().instance().clientHost(); + } + + @Override + public String getBrokerInternalHost() + { + return config.requireBroker().instance().host(); + } + + @Override + public String getRouterHost() + { + return config.requireRouter().instance().clientHost(); + } + + @Override + public String getRouterInternalHost() + { + return config.requireRouter().instance().host(); + } + + @Override + public String getCoordinatorHost() + { + return config.requireCoordinator().tagOrDefault("one").clientHost(); + } + + @Override + public String getCoordinatorInternalHost() + { + return config.requireCoordinator().tagOrDefault("one").host(); + } + + @Override + public String getCoordinatorTwoInternalHost() + { + return config.requireCoordinator().requireInstance("two").host(); + } + + @Override + public String getCoordinatorTwoHost() + { + return config.requireCoordinator().tagOrDefault("one").clientHost(); + } + + @Override + public String getOverlordHost() + { + return config.requireOverlord().tagOrDefault("one").clientHost(); + } + + @Override + public String getOverlordTwoHost() + { + return config.requireOverlord().tagOrDefault("two").clientHost(); + } + + @Override + public String getOverlordInternalHost() + { + return config.requireOverlord().tagOrDefault("one").host(); + } + + @Override + public String getOverlordTwoInternalHost() + { + return config.requireOverlord().requireInstance("two").host(); + } + + @Override + public String getMiddleManagerHost() + { + return config.requireMiddleManager().instance().clientHost(); + } + + @Override + public String getMiddleManagerInternalHost() + { + return config.requireMiddleManager().instance().host(); + } + + @Override + public String getHistoricalHost() + { + return config.requireHistorical().instance().clientHost(); + } + + @Override + public String getHistoricalInternalHost() + { + return config.requireHistorical().instance().host(); + } + + @Override + public String getCoordinatorUrl() + { + ResolvedDruidService serviceConfig = config.requireCoordinator(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one")); + } + + @Override + public String getCoordinatorTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCoordinatorTwoUrl() + { + ResolvedDruidService serviceConfig = config.requireCoordinator(); + return serviceConfig.resolveUrl(serviceConfig.requireInstance("two")); + } + + @Override + public String getCoordinatorTwoTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getOverlordUrl() + { + ResolvedDruidService serviceConfig = config.requireOverlord(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one")); + } + + @Override + public String getOverlordTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getOverlordTwoUrl() + { + ResolvedDruidService serviceConfig = config.requireOverlord(); + return serviceConfig.resolveUrl(serviceConfig.requireInstance("two")); + } + + @Override + public String getOverlordTwoTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getIndexerUrl() + { + ResolvedDruidService indexer = config.druidService(ResolvedConfig.INDEXER); + if (indexer == null) { + indexer = config.requireMiddleManager(); + } + return indexer.resolveUrl(indexer.instance()); + } + + @Override + public String getIndexerTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getRouterUrl() + { + return config.routerUrl(); + } + + @Override + public String getRouterTLSUrl() + { + ResolvedDruidService serviceConfig = config.requireRouter(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls")); + } + + @Override + public String getPermissiveRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getPermissiveRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getNoClientAuthRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getNoClientAuthRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCustomCertCheckRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCustomCertCheckRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getBrokerUrl() + { + ResolvedDruidService serviceConfig = config.requireBroker(); + return serviceConfig.resolveUrl(serviceConfig.instance()); + } + + @Override + public String getBrokerTLSUrl() + { + ResolvedDruidService serviceConfig = config.requireBroker(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls")); + } + + @Override + public String getHistoricalUrl() + { + return config.requireHistorical().resolveUrl(); + } + + @Override + public String getHistoricalTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getProperty(String prop) + { + return properties.get(prop); + } + + @Override + public String getUsername() + { + return getProperty("username"); + } + + @Override + public String getPassword() + { + return getProperty("password"); + } + + @Override + public Map getProperties() + { + return properties; + } + + @Override + public boolean manageKafkaTopic() + { + throw new ISE("Not implemented"); + } + + @Override + public String getExtraDatasourceNameSuffix() + { + return config.datasourceNameSuffix; + } + + @Override + public String getCloudBucket() + { + return getProperty("cloudBucket"); + } + + @Override + public String getCloudPath() + { + return getProperty("cloudPath"); + } + + @Override + public String getCloudRegion() + { + return getProperty("cloudRegion"); + } + + @Override + public String getS3AssumeRoleWithExternalId() + { + return getProperty("s3AssumeRoleWithExternalId"); + } + + @Override + public String getS3AssumeRoleExternalId() + { + return getProperty("s3AssumeRoleExternalId"); + } + + @Override + public String getS3AssumeRoleWithoutExternalId() + { + return getProperty("s3AssumeRoleWithoutExternalId"); + } + + @Override + public String getAzureKey() + { + return getProperty("azureKey"); + } + + @Override + public String getHadoopGcsCredentialsPath() + { + return getProperty("hadoopGcsCredentialsPath"); + } + + @Override + public String getStreamEndpoint() + { + return getProperty("streamEndpoint"); + } + + @Override + public String getSchemaRegistryHost() + { + return getProperty("schemaRegistryHost"); + } + + @Override + public boolean isDocker() + { + return config.isDocker(); + } + + @Override + public String getDockerHost() + { + return config.proxyHost(); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java new file mode 100644 index 00000000000..00e785d940d --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class KafkaConfig extends ServiceConfig +{ + @JsonCreator + public KafkaConfig( + @JsonProperty("service") String service, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java new file mode 100644 index 00000000000..cad2fd5293e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Strings; + +import java.util.List; +import java.util.Map; + +public class MetastoreConfig extends ServiceConfig +{ + /** + * Driver. Defaults to the MySQL Driver. + * @see {@link org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig} + */ + private final String driver; + + /** + * JDBC connection URI. Required. + */ + private final String connectURI; + + /** + * User for the metastore DB. + */ + private final String user; + + /** + * Password for the metastore DB. + */ + private final String password; + + /** + * Optional connection properties. + */ + private final Map properties; + + @JsonCreator + public MetastoreConfig( + @JsonProperty("service") String service, + @JsonProperty("driver") String driver, + @JsonProperty("connectURI") String connectURI, + @JsonProperty("user") String user, + @JsonProperty("password") String password, + @JsonProperty("properties") Map properties, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + this.driver = driver; + this.connectURI = connectURI; + this.user = user; + this.password = password; + this.properties = properties; + } + + @JsonProperty("driver") + @JsonInclude(Include.NON_NULL) + public String driver() + { + return driver; + } + + @JsonProperty("connectURI") + @JsonInclude(Include.NON_NULL) + public String connectURI() + { + return connectURI; + } + + @JsonProperty("user") + @JsonInclude(Include.NON_NULL) + public String user() + { + return user; + } + + @JsonProperty("password") + @JsonInclude(Include.NON_NULL) + public String password() + { + return password; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_NULL) + public Map properties() + { + return properties; + } + + public boolean validate(List errs) + { + if (Strings.isNullOrEmpty(connectURI)) { + errs.add("Metastore connect URI is required"); + return false; + } + return true; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java new file mode 100644 index 00000000000..5213678f562 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.commons.lang3.RegExUtils; + +/** + * Represents a statement (query) to send to the Druid metadata + * storage database (metastore) before running tests. At present, + * each query is only a SQL statement. The statements are represented + * as objects to allow for other options (such as ignoring failures, + * etc.) + *

+ * Metastore queries often include a JSON payload. The metastore wants + * to store the payload in compact form without spaces. However, such + * JSON is hard for humans to understand. So, the configuration file + * should format the SQL statement and JSON for readability. This class + * will "compactify" the statement prior to execution. + */ +public class MetastoreStmt +{ + private final String sql; + + @JsonCreator + public MetastoreStmt( + @JsonProperty("sql") String sql + ) + { + this.sql = sql; + } + + @JsonProperty("sql") + public String sql() + { + return sql; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } + + /** + * Convert the human-readable form of the statement in YAML + * into the compact JSON form preferred in the DB. Also + * compacts the SQL, but that's OK. + */ + public String toSQL() + { + String stmt = RegExUtils.replaceAll(sql, "\n", " "); + stmt = RegExUtils.replaceAll(stmt, " +", " "); + stmt = RegExUtils.replaceAll(stmt, ": ", ":"); + stmt = RegExUtils.replaceAll(stmt, ", ", ","); + stmt = RegExUtils.replaceAll(stmt, " }", "}"); + stmt = RegExUtils.replaceAll(stmt, "\\{ ", "{"); + return stmt; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java new file mode 100644 index 00000000000..6bdfe96b2f9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.curator.CuratorConfig; +import org.apache.druid.curator.ExhibitorConfig; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testing.IntegrationTestingConfigProvider; +import org.apache.druid.testsEx.config.ClusterConfig.ClusterType; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedKafka; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk; +import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +public class ResolvedConfig +{ + public static final String COORDINATOR = "coordinator"; + public static final String HISTORICAL = "historical"; + public static final String OVERLORD = "overlord"; + public static final String BROKER = "broker"; + public static final String ROUTER = "router"; + public static final String MIDDLEMANAGER = "middlemanager"; + public static final String INDEXER = "indexer"; + + public static final int DEFAULT_READY_TIMEOUT_SEC = 120; + public static final int DEFAULT_READY_POLL_MS = 2000; + + private final String category; + private final ClusterType type; + private final String proxyHost; + private final int readyTimeoutSec; + private final int readyPollMs; + final String datasourceNameSuffix; + private Map properties; + private Map settings; + + private final ResolvedZk zk; + private final ResolvedKafka kafka; + private final ResolvedMetastore metastore; + private final Map druidServices = new HashMap<>(); + + public ResolvedConfig(String category, ClusterConfig config) + { + this.category = category; + type = config.type() == null ? ClusterType.docker : config.type(); + if (!hasProxy()) { + proxyHost = null; + } else if (Strings.isNullOrEmpty(config.proxyHost())) { + proxyHost = "localhost"; + } else { + proxyHost = config.proxyHost(); + } + readyTimeoutSec = config.readyTimeoutSec() > 0 ? + config.readyTimeoutSec() : DEFAULT_READY_TIMEOUT_SEC; + readyPollMs = config.readyPollMs() > 0 ? config.readyPollMs() : DEFAULT_READY_POLL_MS; + if (config.properties() == null) { + this.properties = ImmutableMap.of(); + } else { + this.properties = config.properties(); + } + if (config.settings() == null) { + this.settings = ImmutableMap.of(); + } else { + this.settings = config.settings(); + } + if (config.datasourceSuffix() == null) { + this.datasourceNameSuffix = ""; + } else { + this.datasourceNameSuffix = config.datasourceSuffix(); + } + + if (config.zk() == null) { + this.zk = null; + } else { + this.zk = new ResolvedZk(this, config.zk()); + } + if (config.kafka() == null) { + this.kafka = null; + } else { + this.kafka = new ResolvedKafka(this, config.kafka()); + } + if (config.metastore() == null) { + this.metastore = null; + } else { + this.metastore = new ResolvedMetastore(this, config.metastore(), config); + } + + if (config.druid() != null) { + for (Entry entry : config.druid().entrySet()) { + druidServices.put(entry.getKey(), + new ResolvedDruidService(this, entry.getValue(), entry.getKey())); + } + } + } + + public ClusterType type() + { + return type; + } + + public String proxyHost() + { + return proxyHost; + } + + public int readyTimeoutSec() + { + return readyTimeoutSec; + } + + public int readyPollMs() + { + return readyPollMs; + } + + public boolean isDocker() + { + return type == ClusterType.docker; + } + + public boolean hasProxy() + { + switch (type) { + case docker: + case k8s: + return true; + default: + return false; + } + } + + public ResolvedZk zk() + { + return zk; + } + + public ResolvedMetastore metastore() + { + return metastore; + } + + public ResolvedKafka kafka() + { + return kafka; + } + + public Map settings() + { + return settings; + } + + public Map properties() + { + return properties; + } + + public Map requireDruid() + { + if (druidServices == null) { + throw new ISE("Please configure Druid services"); + } + return druidServices; + } + + public ResolvedZk requireZk() + { + if (zk == null) { + throw new ISE("Please specify the ZooKeeper configuration"); + } + return zk; + } + + public ResolvedMetastore requireMetastore() + { + if (metastore == null) { + throw new ISE("Please specify the Metastore configuration"); + } + return metastore; + } + + public ResolvedKafka requireKafka() + { + if (kafka == null) { + throw new ISE("Please specify the Kafka configuration"); + } + return kafka; + } + + public ResolvedDruidService druidService(String serviceKey) + { + return requireDruid().get(serviceKey); + } + + public ResolvedDruidService requireService(String serviceKey) + { + ResolvedDruidService service = druidService(serviceKey); + if (service == null) { + throw new ISE("Please configure Druid service " + serviceKey); + } + return service; + } + + public ResolvedDruidService requireCoordinator() + { + return requireService(COORDINATOR); + } + + public ResolvedDruidService requireOverlord() + { + return requireService(OVERLORD); + } + + public ResolvedDruidService requireBroker() + { + return requireService(BROKER); + } + + public ResolvedDruidService requireRouter() + { + return requireService(ROUTER); + } + + public ResolvedDruidService requireMiddleManager() + { + return requireService(MIDDLEMANAGER); + } + + public ResolvedDruidService requireHistorical() + { + return requireService(HISTORICAL); + } + + public String routerUrl() + { + return requireRouter().clientUrl(); + } + + public CuratorConfig toCuratorConfig() + { + if (zk == null) { + throw new ISE("ZooKeeper not configured"); + } + // TODO: Add a builder for other properties + return CuratorConfig.create(zk.clientHosts()); + } + + public ExhibitorConfig toExhibitorConfig() + { + // Does not yet support exhibitors + return ExhibitorConfig.create(Collections.emptyList()); + } + + /** + * Map from old-style config file (and settings) name to the + * corresponding property. + */ + private static final Map SETTINGS_MAP = + ImmutableMap.builder() + .put("cloud_bucket", "cloudBucket") + .put("cloud_path", "cloudPath") + .put("cloud_region", "cloudRegion") + .put("s3_assume_role_with_external_id", "s3AssumeRoleWithExternalId") + .put("s3_assume_role_external_id", "s3AssumeRoleExternalId") + .put("s3_assume_role_without_external_id", "s3AssumeRoleWithoutExternalId") + .put("stream_endpoint", "streamEndpoint") + .put("s3_accessKey", "s3AccessKey") + .put("s3_secretKey", "s3SecretKey") + .put("azure_account", "azureAccount") + .put("azure_key", "azureKey") + .put("azure_container", "azureContainer") + .put("google_bucket", "googleBucket") + .put("google_prefix", "googlePrefix") + .build(); + + private static void setDruidProperyVar(Map properties, String key, Object value) + { + if (value == null) { + return; + } + if (key.startsWith("druid_")) { + key = key.substring("druid_".length()); + } + String mapped = SETTINGS_MAP.get(key); + key = mapped == null ? key : mapped; + TestConfigs.putProperty(properties, IntegrationTestingConfigProvider.PROPERTY_BASE, key, value.toString()); + } + + private void loadPropertyFile(Map properties, File file) + { + try (BufferedReader in = new BufferedReader( + new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))) { + String line; + while ((line = in.readLine()) != null) { + if (Strings.isNullOrEmpty(line) || line.startsWith("#")) { + continue; + } + String[] parts = line.split("="); + if (parts.length != 2) { + continue; + } + setDruidProperyVar(properties, parts[0], parts[1]); + } + } + catch (IOException e) { + throw new IAE(e, "Cannot read file %s", file.getAbsolutePath()); + } + } + + /** + * Convert the config in this structure the the properties + * used to configure Guice. + */ + public Map toProperties() + { + Map properties = new HashMap<>(); + // druid.test.config.dockerIp is used by some older test code. Remove + // it when that code is updated. + TestConfigs.putProperty(properties, "druid.test.config.dockerIp", proxyHost); + + // Start with implicit properties from various sections. + if (zk != null) { + properties.putAll(zk.toProperties()); + } + if (metastore != null) { + properties.putAll(metastore.toProperties()); + } + + // Add settings, converted to properties. Map both old and + // "property-style" settings to the full property path. + // Settings are converted to properties so they can be overridden + // by environment variables and -D command-line settings. + for (Map.Entry entry : settings.entrySet()) { + setDruidProperyVar(properties, entry.getKey(), entry.getValue()); + } + + // Add explicit properties + if (this.properties != null) { + properties.putAll(this.properties); + } + + // Override with a user-specific config file. + File userEnv = new File( + new File( + System.getProperty("user.home"), + "druid-it"), + category + ".env"); + if (userEnv.exists()) { + loadPropertyFile(properties, userEnv); + } + + // Override with a user-specific config file. + String overrideEnv = System.getenv("OVERRIDE_ENV"); + if (overrideEnv != null) { + loadPropertyFile(properties, new File(overrideEnv)); + } + + // Override with any environment variables of the form "druid_" + for (Map.Entry entry : System.getenv().entrySet()) { + String key = entry.getKey(); + if (!key.startsWith("druid_")) { + continue; + } + setDruidProperyVar(properties, key, entry.getValue()); + } + + // Override with any system properties of the form "druid_" + for (Map.Entry entry : System.getProperties().entrySet()) { + String key = (String) entry.getKey(); + if (!key.startsWith("druid_")) { + continue; + } + setDruidProperyVar(properties, key, entry.getValue()); + } + return properties; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java new file mode 100644 index 00000000000..58cbdcdc33c --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class ResolvedDruidService extends ResolvedService +{ + public ResolvedDruidService(ResolvedConfig root, DruidConfig config, String serviceKey) + { + super(root, config, serviceKey); + } + + /** + * Get the URL (visible to the test) of the service. + */ + public String clientUrl() + { + return resolveUrl(instance()); + } + + /** + * Find an instance given the instance name (tag). + */ + public ResolvedInstance findInstance(String instanceName) + { + for (ResolvedInstance instance : requireInstances()) { + if (instance.tag() != null && instance.tag().equals(instanceName)) { + return instance; + } + } + return null; + } + + /** + * Find an instance given the instance name (tag). Raises + * an error (which fails the test) if the tag is not defined. + */ + public ResolvedInstance requireInstance(String instanceName) + { + ResolvedInstance instance = findInstance(instanceName); + if (instance != null) { + return instance; + } + throw new ISE( + StringUtils.format( + "No Druid instance of service %s with name %s is defined", + service, + instanceName)); + } + + public String resolveUrl() + { + return resolveUrl(instance()); + } + + /** + * Return the URL for the given instance name (tag) of this service + * as visible to the test. + */ + public String resolveUrl(String instanceName) + { + return resolveUrl(requireInstance(instanceName)); + } + + /** + * Return the URL, known to the test, of the given service instance. + */ + public String resolveUrl(ResolvedInstance instance) + { + return StringUtils.format( + "http://%s:%d", + instance.clientHost(), + instance.clientPort()); + } + + /** + * Return the named service instance. If not found, return the + * "default" instance. This is used by the somewhat awkward test + * config object so that if a test asks for "Coordinator one" in + * a cluster with a single Coordinator, it will get that Coordinator. + * Same for Overlord. + */ + public ResolvedInstance tagOrDefault(String tag) + { + ResolvedInstance taggedInstance = findInstance(tag); + return taggedInstance == null ? instance() : taggedInstance; + } + + /** + * Returns the "default" host for this service as known to the + * cluster. The host is that of the only instance and is undefined + * if there are multiple instances. + */ + public String resolveHost() + { + ResolvedInstance instance = instance(); + if (instances.size() > 1) { + throw new ISE( + StringUtils.format("Service %s has %d hosts, default is ambiguous", + service, + instances.size())); + } + return instance.host(); + } + + public ResolvedInstance findHost(String host) + { + Pattern p = Pattern.compile("https?://(.*):(\\d+)"); + Matcher m = p.matcher(host); + if (!m.matches()) { + return null; + } + String hostName = m.group(1); + int port = Integer.parseInt(m.group(2)); + for (ResolvedInstance instance : instances) { + if (instance.host().equals(hostName) && instance.port() == port) { + return instance; + } + } + return null; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java new file mode 100644 index 00000000000..f65790eb1c9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.RegExUtils; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ResolvedMetastore extends ResolvedService +{ + // Set to be 1 sec. longer than the setting in the + // docker-compose.yaml file: + // druid_manager_segments_pollDuration=PT5S + public static final int DEFAULT_METASTORE_INIT_DELAY_SEC = 6; + + private final String driver; + private final String connectURI; + private final String user; + private final String password; + private final Map properties; + private final int initDelaySec; + private List initStmts; + + public ResolvedMetastore(ResolvedConfig root, MetastoreConfig config, ClusterConfig clusterConfig) + { + super(root, config, "metastore"); + this.driver = config.driver(); + if (config.connectURI() != null) { + ResolvedInstance instance = instance(); + this.connectURI = RegExUtils.replaceAll( + RegExUtils.replaceAll( + config.connectURI(), + "", + Integer.toString(instance.clientPort())), + "", + instance.clientHost()); + } else { + this.connectURI = null; + } + this.user = config.user(); + this.password = config.password(); + if (config.properties() == null) { + this.properties = ImmutableMap.of(); + } else { + this.properties = config.properties(); + } + + this.initDelaySec = clusterConfig.metastoreInitDelaySec() > 0 + ? clusterConfig.metastoreInitDelaySec() + : DEFAULT_METASTORE_INIT_DELAY_SEC; + this.initStmts = clusterConfig.metastoreInit(); + } + + public String driver() + { + return driver; + } + + public String connectURI() + { + return connectURI; + } + + public String user() + { + return user; + } + + public String password() + { + return password; + } + + public Map properties() + { + return properties; + } + + /** + * Create the properties Guice needs to create the connector config. + * + * @see + * Setting up MySQL + */ + public Map toProperties() + { + final String base = MetadataStorageConnectorConfig.PROPERTY_BASE; + Map properties = new HashMap<>(); + TestConfigs.putProperty(properties, "druid.metadata.mysql.driver.driverClassName", driver); + TestConfigs.putProperty(properties, "druid.metadata.storage.type", "mysql"); + TestConfigs.putProperty(properties, base, "connectURI", connectURI); + TestConfigs.putProperty(properties, base, "user", user); + TestConfigs.putProperty(properties, base, "password", password); + TestConfigs.putProperty(properties, base, "dbcp", this.properties); + return properties; + } + + public List initStmts() + { + return initStmts; + } + + public int initDelaySec() + { + return initDelaySec; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java new file mode 100644 index 00000000000..5bd377f7079 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.curator.CuratorConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.config.ServiceConfig.ZKConfig; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ResolvedService +{ + protected final String service; + protected final List instances = new ArrayList<>(); + + public ResolvedService(ResolvedConfig root, ServiceConfig config, String name) + { + this.service = config.service() == null ? name : config.service(); + for (ServiceInstance instanceConfig : config.instances()) { + this.instances.add(new ResolvedInstance(root, instanceConfig, this)); + } + } + + public String service() + { + return service; + } + + public List requireInstances() + { + if (instances.isEmpty()) { + throw new ISE("Please specify a " + service + " instance"); + } + return instances; + } + + public ResolvedInstance instance() + { + return requireInstances().get(0); + } + + public class ResolvedInstance + { + private final String container; + private final String host; + private final String clientHost; + private final String tag; + private final int port; + private final int clientPort; + + public ResolvedInstance(ResolvedConfig root, ServiceInstance config, ResolvedService service) + { + this.tag = config.tag(); + + // The actual (cluster) host is... + if (config.host() != null) { + // The specified host, if provided + this.host = config.host(); + } else { + String baseHost; + if (root.hasProxy()) { + // The same as the service, if there is a proxy + baseHost = service.service; + // with the tag appended + if (tag != null) { + baseHost += "-" + config.tag(); + } + this.host = baseHost; + } else { + // The local host otherwise + this.host = "localhost"; + } + } + + if (root.hasProxy()) { + this.clientHost = root.proxyHost(); + } else { + this.clientHost = this.host; + } + + this.container = config.container() != null ? config.container() : service.service; + if (config.port() == 0) { + throw new ISE("Must provide port"); + } + this.port = config.port(); + if (config.proxyPort() != 0) { + this.clientPort = config.proxyPort(); + } else { + this.clientPort = this.port; + } + } + + public ResolvedService service() + { + return ResolvedService.this; + } + + public String container() + { + return container; + } + + public String host() + { + return host; + } + + public String clientHost() + { + return clientHost; + } + + public String tag() + { + return tag; + } + + public int port() + { + return port; + } + + public int clientPort() + { + return clientPort; + } + } + + public static class ResolvedZk extends ResolvedService + { + private final int startTimeoutSecs; + + public ResolvedZk(ResolvedConfig root, ZKConfig config) + { + super(root, config, "zookeeper"); + this.startTimeoutSecs = config.startTimeoutSecs(); + } + + public int startTimeoutSecs() + { + return startTimeoutSecs; + } + + public String clientHosts() + { + List hosts = new ArrayList<>(); + for (ResolvedInstance instance : instances) { + hosts.add(formatHost(instance.clientHost(), instance.clientPort())); + } + return String.join(",", hosts); + } + + public String clusterHosts() + { + List hosts = new ArrayList<>(); + for (ResolvedInstance instance : instances) { + hosts.add(formatHost(instance.host(), instance.port())); + } + return String.join(",", hosts); + } + + private String formatHost(String host, int port) + { + return StringUtils.format("%s:%d", host, port); + } + + public Map toProperties() + { + /* + * We will use this instead of druid server's CuratorConfig, because CuratorConfig in + * a test cluster environment sees Zookeeper at localhost even if Zookeeper is elsewhere. + * We'll take the Zookeeper host from the configuration file instead. + */ + return ImmutableMap.of( + CuratorConfig.CONFIG_PREFIX + ".zkHosts", + clientHosts()); + } + } + + public static class ResolvedKafka extends ResolvedService + { + public ResolvedKafka(ResolvedConfig root, KafkaConfig config) + { + super(root, config, "kafka"); + } + + public String clientHost() + { + return instance().clientHost(); + } + + public String bootstrap() + { + ResolvedInstance instance = instance(); + return StringUtils.format("%s:%d", instance.clientHost(), instance.clientPort()); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java new file mode 100644 index 00000000000..08b07bc4ed8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class ServiceConfig +{ + protected final String service; + protected List instances; + + public ServiceConfig( + String service, + List instances + ) + { + this.service = service; + this.instances = instances; + } + + @JsonProperty("service") + @JsonInclude(Include.NON_NULL) + public String service() + { + return service; + } + + @JsonProperty("instances") + @JsonInclude(Include.NON_NULL) + public List instances() + { + return instances; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } + + /** + * YAML description of a ZK cluster. Converted to + * {@link org.apache.druid.curator.CuratorConfig} + */ + public static class ZKConfig extends ServiceConfig + { + /** + * Amount of time to wait for ZK to become ready. + * Defaults to 5 seconds. + */ + private final int startTimeoutSecs; + + @JsonCreator + public ZKConfig( + @JsonProperty("service") String service, + @JsonProperty("startTimeoutSecs") int startTimeoutSecs, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + this.startTimeoutSecs = startTimeoutSecs; + } + + @JsonProperty("startTimeoutSecs") + public int startTimeoutSecs() + { + return startTimeoutSecs; + } + } + + /** + * Represents a Druid service (of one or more instances) running + * in the test cluster. The service name comes from the key used + * in the {@code druid} map:

+   * druid:
+   *   broker:  # <-- key (service name)
+   *     instances:
+   *       ...
+   * 
+ */ + public static class DruidConfig extends ServiceConfig + { + @JsonCreator + public DruidConfig( + // Note: service is not actually used. + @JsonProperty("service") String service, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + } + } + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java new file mode 100644 index 00000000000..c9a7e0e3a87 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * One instance of a Druid or third-party service running on + * a host or in a container. + */ +public class ServiceInstance +{ + /** + * Name of the Docker container. Used in Docker commands against + * the container, such as starting and stopping. + */ + private final String container; + + /** + * Name of the host running the service as known to the cluster + * (which many not be visible to the host running the test.) + * Assumed to be {@code } or @{code -} + * if not explicitly set. + */ + private final String host; + + /** + * Tag used to identify a service when there are multiple + * instances. The host is assumed to be @{code-} if + * not explicitly set. + */ + private final String tag; + + /** + * The port exposed by the service on its host. May not be + * visible to the test. Required. + */ + + private final int port; + + /** + * The proxy port visible for the test for this service. Defaults + * to the same as the @{code port}. Define only if Docker is configured + * for port mapping other than identity. + */ + private final int proxyPort; + + @JsonCreator + public ServiceInstance( + @JsonProperty("container") String container, + @JsonProperty("host") String host, + @JsonProperty("tag") String tag, + @JsonProperty("port") int port, + @JsonProperty("proxyPort") int proxyPort + ) + { + this.container = container; + this.host = host; + this.tag = tag; + this.port = port; + this.proxyPort = proxyPort; + } + + @JsonProperty("container") + @JsonInclude(Include.NON_NULL) + public String container() + { + return container; + } + + @JsonProperty("host") + @JsonInclude(Include.NON_NULL) + public String host() + { + return host; + } + + @JsonProperty("tag") + @JsonInclude(Include.NON_NULL) + public String tag() + { + return tag; + } + + @JsonProperty("port") + @JsonInclude(Include.NON_DEFAULT) + public int port() + { + return port; + } + + @JsonProperty("proxyPort") + @JsonInclude(Include.NON_DEFAULT) + public int proxyPort() + { + return proxyPort; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java new file mode 100644 index 00000000000..8b909487c23 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; + +import java.util.Map; + +/** + * Utility functions related to test configuration. + */ +public class TestConfigs +{ + /** + * Converts a YAML-aware object to a YAML string, primarily + * for use in @{code toString()} methods. + */ + public static String toYaml(Object obj) + { + ObjectMapper mapper = new ObjectMapper( + new YAMLFactory() + .enable(YAMLGenerator.Feature.MINIMIZE_QUOTES)); + try { + return mapper.writeValueAsString(obj); + } + catch (JsonProcessingException e) { + return ""; + } + } + + public static void putProperty(Map properties, String key, Object value) + { + if (value == null) { + return; + } + properties.put(key, value); + } + + public static void putProperty(Map properties, String base, String key, Object value) + { + if (value == null) { + return; + } + properties.put(base + "." + key, value); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java new file mode 100644 index 00000000000..5b080cd7448 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java @@ -0,0 +1,502 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.FluentIterable; +import com.google.inject.Inject; +import org.apache.commons.io.IOUtils; +import org.apache.druid.indexer.partitions.SecondaryPartitionType; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionCardinalityTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.ClientInfoResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.SqlTestQueryHelper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.junit.Assert; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest +{ + public enum InputFormatDetails + { + AVRO("avro_ocf", ".avro", "/avro"), + CSV("csv", ".csv", "/csv"), + TSV("tsv", ".tsv", "/tsv"), + ORC("orc", ".orc", "/orc"), + JSON("json", ".json", "/json"), + PARQUET("parquet", ".parquet", "/parquet"); + + private final String inputFormatType; + private final String fileExtension; + private final String folderSuffix; + + InputFormatDetails(String inputFormatType, String fileExtension, String folderSuffix) + { + this.inputFormatType = inputFormatType; + this.fileExtension = fileExtension; + this.folderSuffix = folderSuffix; + } + + public String getInputFormatType() + { + return inputFormatType; + } + + public String getFileExtension() + { + return fileExtension; + } + + public String getFolderSuffix() + { + return folderSuffix; + } + } + + private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class); + + @Inject + protected IntegrationTestingConfig config; + @Inject + protected SqlTestQueryHelper sqlQueryHelper; + + @Inject + ClientInfoResourceTestClient clientInfoResourceTestClient; + + protected void doIndexTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath, + boolean waitForNewVersion, + boolean runTestQueries, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + doIndexTest( + dataSource, + indexTaskFilePath, + Function.identity(), + queryFilePath, + waitForNewVersion, + runTestQueries, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); + } + + protected void doIndexTest( + String dataSource, + String indexTaskFilePath, + Function taskSpecTransform, + String queryFilePath, + boolean waitForNewVersion, + boolean runTestQueries, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final String taskSpec = taskSpecTransform.apply( + StringUtils.replace( + getResourceAsString(indexTaskFilePath), + "%%DATASOURCE%%", + fullDatasourceName + ) + ); + + submitTaskAndWait( + taskSpec, + fullDatasourceName, + waitForNewVersion, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); + if (runTestQueries) { + doTestQuery(dataSource, queryFilePath); + } + } + + protected void doTestQuery(String dataSource, String queryFilePath) + { + try { + String queryResponseTemplate; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath); + queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", queryFilePath); + } + + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%DATASOURCE%%", + dataSource + config.getExtraDatasourceNameSuffix() + ); + queryHelper.testQueriesFromString(queryResponseTemplate); + + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + protected void doReindexTest( + String baseDataSource, + String reindexDataSource, + String reindexTaskFilePath, + String queryFilePath, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + doReindexTest( + baseDataSource, + reindexDataSource, + Function.identity(), + reindexTaskFilePath, + queryFilePath, + segmentAvailabilityConfirmationPair + ); + } + + void doReindexTest( + String baseDataSource, + String reindexDataSource, + Function taskSpecTransform, + String reindexTaskFilePath, + String queryFilePath, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + final String fullBaseDatasourceName = baseDataSource + config.getExtraDatasourceNameSuffix(); + final String fullReindexDatasourceName = reindexDataSource + config.getExtraDatasourceNameSuffix(); + + String taskSpec = StringUtils.replace( + getResourceAsString(reindexTaskFilePath), + "%%DATASOURCE%%", + fullBaseDatasourceName + ); + + taskSpec = StringUtils.replace( + taskSpec, + "%%REINDEX_DATASOURCE%%", + fullReindexDatasourceName + ); + + taskSpec = taskSpecTransform.apply(taskSpec); + + submitTaskAndWait( + taskSpec, + fullReindexDatasourceName, + false, + true, + segmentAvailabilityConfirmationPair + ); + try { + String queryResponseTemplate; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath); + queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", queryFilePath); + } + + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%DATASOURCE%%", + fullReindexDatasourceName + ); + + queryHelper.testQueriesFromString(queryResponseTemplate); + // verify excluded dimension is not reIndexed + final List dimensions = clientInfoResourceTestClient.getDimensions( + fullReindexDatasourceName, + "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z" + ); + Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot")); + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + void doIndexTestSqlTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath + ) throws IOException + { + doIndexTestSqlTest( + dataSource, + indexTaskFilePath, + queryFilePath, + Function.identity() + ); + } + void doIndexTestSqlTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath, + Function taskSpecTransform + ) throws IOException + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final String taskSpec = taskSpecTransform.apply( + StringUtils.replace( + getResourceAsString(indexTaskFilePath), + "%%DATASOURCE%%", + fullDatasourceName + ) + ); + + Pair dummyPair = new Pair<>(false, false); + submitTaskAndWait(taskSpec, fullDatasourceName, false, true, dummyPair); + try { + sqlQueryHelper.testQueriesFromFile(queryFilePath); + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + protected void submitTaskAndWait( + String taskSpec, + String dataSourceName, + boolean waitForNewVersion, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) + { + final List oldVersions = waitForNewVersion ? coordinator.getAvailableSegments(dataSourceName) : null; + + long startSubTaskCount = -1; + final boolean assertRunsSubTasks = taskSpec.contains("index_parallel"); + if (assertRunsSubTasks) { + startSubTaskCount = countCompleteSubTasks(dataSourceName, !taskSpec.contains("dynamic")); + } + + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for loading index task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + if (assertRunsSubTasks) { + final boolean perfectRollup = !taskSpec.contains("dynamic"); + final long newSubTasks = countCompleteSubTasks(dataSourceName, perfectRollup) - startSubTaskCount; + Assert.assertTrue( + StringUtils.format( + "The supervisor task [%s] didn't create any sub tasks. Was it executed in the parallel mode?", + taskID + ), + newSubTasks > 0 + ); + } + + if (segmentAvailabilityConfirmationPair.lhs != null && segmentAvailabilityConfirmationPair.lhs) { + TaskReport reportRaw = indexer.getTaskReport(taskID).get("ingestionStatsAndErrors"); + IngestionStatsAndErrorsTaskReport report = (IngestionStatsAndErrorsTaskReport) reportRaw; + IngestionStatsAndErrorsTaskReportData reportData = (IngestionStatsAndErrorsTaskReportData) report.getPayload(); + + // Confirm that the task waited longer than 0ms for the task to complete. + Assert.assertTrue(reportData.getSegmentAvailabilityWaitTimeMs() > 0); + + // Make sure that the result of waiting for segments to load matches the expected result + if (segmentAvailabilityConfirmationPair.rhs != null) { + Assert.assertEquals( + Boolean.valueOf(reportData.isSegmentAvailabilityConfirmed()), + segmentAvailabilityConfirmationPair.rhs + ); + } + } + + // IT*ParallelIndexTest do a second round of ingestion to replace segments in an existing + // data source. For that second round we need to make sure the coordinator actually learned + // about the new segments before waiting for it to report that all segments are loaded; otherwise + // this method could return too early because the coordinator is merely reporting that all the + // original segments have loaded. + if (waitForNewVersion) { + ITRetryUtil.retryUntilTrue( + () -> { + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + coordinator.getAvailableSegments(dataSourceName) + ); + + final List> holders = timeline.lookup(Intervals.ETERNITY); + return FluentIterable + .from(holders) + .transformAndConcat(TimelineObjectHolder::getObject) + .anyMatch( + chunk -> FluentIterable.from(oldVersions) + .anyMatch(oldSegment -> chunk.getObject().overshadows(oldSegment)) + ); + }, + "See a new version" + ); + } + + if (waitForSegmentsToLoad) { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load" + ); + } + } + + private long countCompleteSubTasks(final String dataSource, final boolean perfectRollup) + { + return indexer.getCompleteTasksForDataSource(dataSource) + .stream() + .filter(t -> { + if (!perfectRollup) { + return t.getType().equals(SinglePhaseSubTask.TYPE); + } else { + return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE) + || t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE) + || t.getType().equalsIgnoreCase(PartialDimensionCardinalityTask.TYPE) + || t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE) + || t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE); + } + }) + .count(); + } + + void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments) + { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()), + "Segment load check" + ); + ITRetryUtil.retryUntilTrue( + () -> { + List segments = coordinator.getAvailableSegments( + dataSource + config.getExtraDatasourceNameSuffix() + ); + int segmentCount = segments.size(); + LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments); + + return segmentCount == numExpectedSegments; + }, + "Segment count check" + ); + } + + void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments, int numExpectedTombstones) + { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()), + "Segment load check" + ); + ITRetryUtil.retryUntilTrue( + () -> { + List segments = coordinator.getAvailableSegments( + dataSource + config.getExtraDatasourceNameSuffix() + ); + int segmentCount = segments.size(); + LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments); + + int tombstoneCount = 0; + for (DataSegment segment : segments) { + if (segment.isTombstone()) { + tombstoneCount++; + } + } + + LOG.info("Current tombstone count: %d, expected: %d", tombstoneCount, numExpectedTombstones); + + return segmentCount == numExpectedSegments && tombstoneCount == numExpectedTombstones; + }, + "Segment count check" + ); + } + + void compactData(String dataSource, String compactionTask) throws Exception + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + intervalsBeforeCompaction.sort(null); + final String template = getResourceAsString(compactionTask); + String taskSpec = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); + + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for compaction task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(fullDatasourceName), + "Segment Compaction" + ); + ITRetryUtil.retryUntilTrue( + () -> { + final List actualIntervals = coordinator.getSegmentIntervals( + dataSource + config.getExtraDatasourceNameSuffix() + ); + actualIntervals.sort(null); + return actualIntervals.equals(intervalsBeforeCompaction); + }, + "Compaction interval check" + ); + } + + void verifySegmentsCompacted(String dataSource, int expectedCompactedSegmentCount) + { + List segments = coordinator.getFullSegmentsMetadata( + dataSource + config.getExtraDatasourceNameSuffix() + ); + List foundCompactedSegments = new ArrayList<>(); + for (DataSegment segment : segments) { + if (segment.getLastCompactionState() != null) { + foundCompactedSegments.add(segment); + } + } + Assert.assertEquals(foundCompactedSegments.size(), expectedCompactedSegmentCount); + for (DataSegment compactedSegment : foundCompactedSegments) { + Assert.assertNotNull(compactedSegment.getLastCompactionState()); + Assert.assertNotNull(compactedSegment.getLastCompactionState().getPartitionsSpec()); + Assert.assertEquals( + compactedSegment.getLastCompactionState().getPartitionsSpec().getType(), + SecondaryPartitionType.LINEAR + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java new file mode 100644 index 00000000000..5e53330e705 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.commons.io.IOUtils; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.clients.OverlordResourceTestClient; +import org.apache.druid.testing.clients.TaskResponseObject; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.TestQueryHelper; +import org.joda.time.Interval; + +import java.io.BufferedReader; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; + +public abstract class AbstractIndexerTest +{ + private static final Logger LOG = new Logger(AbstractIndexerTest.class); + + @Inject + protected CoordinatorResourceTestClient coordinator; + @Inject + protected OverlordResourceTestClient indexer; + @Inject + @Json + protected ObjectMapper jsonMapper; + @Inject + @Smile + protected ObjectMapper smileMapper; + @Inject + protected TestQueryHelper queryHelper; + + @Inject + protected IntegrationTestingConfig config; + + protected Closeable unloader(final String dataSource) + { + return () -> unloadAndKillData(dataSource); + } + + protected void unloadAndKillData(final String dataSource) + { + // Get all failed task logs + List allTasks = indexer.getCompleteTasksForDataSource(dataSource); + for (TaskResponseObject task : allTasks) { + if (task.getStatus().isFailure()) { + LOG.info("------- START Found failed task logging for taskId=" + task.getId() + " -------"); + LOG.info("Start failed task log:"); + LOG.info(indexer.getTaskLog(task.getId())); + LOG.info("End failed task log."); + LOG.info("Start failed task errorMsg:"); + LOG.info(indexer.getTaskErrorMessage(task.getId())); + LOG.info("End failed task errorMsg."); + LOG.info("------- END Found failed task logging for taskId=" + task.getId() + " -------"); + } + } + + List intervals = coordinator.getSegmentIntervals(dataSource); + + // each element in intervals has this form: + // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z + // we'll sort the list (ISO dates have lexicographic order) + // then delete segments from the 1st date in the first string + // to the 2nd date in the last string + Collections.sort(intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + unloadAndKillData(dataSource, first, last); + } + + protected String submitIndexTask(String indexTask, final String fullDatasourceName) throws Exception + { + String taskSpec = getResourceAsString(indexTask); + taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName); + taskSpec = StringUtils.replace( + taskSpec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for loading index task %s", taskID); + + return taskID; + } + + protected void loadData(String indexTask, final String fullDatasourceName) throws Exception + { + final String taskID = submitIndexTask(indexTask, fullDatasourceName); + indexer.waitUntilTaskCompletes(taskID); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(fullDatasourceName), + "Segment Load" + ); + } + + private void unloadAndKillData(final String dataSource, String start, String end) + { + // Wait for any existing index tasks to complete before disabling the datasource otherwise + // realtime tasks can get stuck waiting for handoff. https://github.com/apache/druid/issues/1729 + waitForAllTasksToCompleteForDataSource(dataSource); + Interval interval = Intervals.of(start + "/" + end); + coordinator.unloadSegmentsForDataSource(dataSource); + ITRetryUtil.retryUntilFalse( + new Callable() + { + @Override + public Boolean call() + { + return coordinator.areSegmentsLoaded(dataSource); + } + }, "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(dataSource, interval); + waitForAllTasksToCompleteForDataSource(dataSource); + } + + protected void waitForAllTasksToCompleteForDataSource(final String dataSource) + { + ITRetryUtil.retryUntilTrue( + () -> (indexer.getUncompletedTasksForDataSource(dataSource).size() == 0), + StringUtils.format("Waiting for all tasks of [%s] to complete", dataSource) + ); + } + + public static String getResourceAsString(String file) throws IOException + { + try (final InputStream inputStream = getResourceAsStream(file)) { + return IOUtils.toString(inputStream, StandardCharsets.UTF_8); + } + } + + public static InputStream getResourceAsStream(String resource) + { + return AbstractIndexerTest.class.getResourceAsStream(resource); + } + + public static List listResources(String dir) throws IOException + { + List resources = new ArrayList<>(); + + try ( + InputStream in = getResourceAsStream(dir); + BufferedReader br = new BufferedReader(new InputStreamReader(in, StringUtils.UTF8_STRING)) + ) { + String resource; + + while ((resource = br.readLine()) != null) { + resources.add(resource); + } + } + + return resources; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java new file mode 100644 index 00000000000..ce8a9f5c13c --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nonnull; +import java.io.Closeable; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +public abstract class AbstractLocalInputSourceParallelIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + + public void doIndexTest( + InputFormatDetails inputFormatDetails, + Pair segmentAvailabilityConfirmationPair + ) throws Exception + { + doIndexTest(inputFormatDetails, ImmutableMap.of(), segmentAvailabilityConfirmationPair); + } + + public void doIndexTest( + InputFormatDetails inputFormatDetails, + @Nonnull Map extraInputFormatMap, + Pair segmentAvailabilityConfirmationPair + ) throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + Map inputFormatMap = new ImmutableMap.Builder().putAll(extraInputFormatMap) + .put("type", inputFormatDetails.getInputFormatType()) + .build(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + final Function sqlInputSourcePropsTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + "*" + inputFormatDetails.getFileExtension() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index" + inputFormatDetails.getFolderSuffix() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + sqlInputSourcePropsTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + segmentAvailabilityConfirmationPair + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java new file mode 100644 index 00000000000..96e8d500fa6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.inject.Inject; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest +{ + // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle + // maxSplitSize of 1 properly. + private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; + private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_reindex_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; + private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test"; + private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json"; + private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test"; + private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json"; + + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = + CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = + CoordinatorDynamicConfig.builder().build(); + + @Inject + CoordinatorResourceTestClient coordinatorClient; + + @Test + public void testIndexData() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix()) + ) { + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + + // Index again, this time only choosing the second data file, and without explicit intervals chosen. + // The second datafile covers both day segments, so this should replace them, as reflected in the queries. + doIndexTest( + INDEX_DATASOURCE, + REINDEX_TASK, + rollupTransform, + REINDEX_QUERIES_RESOURCE, + true, + true, + true, + new Pair<>(false, false) + ); + + doReindexTest( + INDEX_DATASOURCE, + INDEX_INGEST_SEGMENT_DATASOURCE, + rollupTransform, + INDEX_INGEST_SEGMENT_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + + // with DruidInputSource instead of IngestSegmentFirehose + doReindexTest( + INDEX_DATASOURCE, + INDEX_DRUID_INPUT_SOURCE_DATASOURCE, + rollupTransform, + INDEX_DRUID_INPUT_SOURCE_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. This will confirm that the report for the task + * indicates segments were confirmed to be available on the cluster before finishing the ingestion job. + */ + @Test + public void testIndexDataVerifySegmentAvailability() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("600000") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. Setting the config value to 1 millis + * and pausing coordination to confirm that the task will still succeed even if the job was not able to confirm the + * segments were loaded by the time the timeout occurs. + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("1") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + false, + false, + new Pair<>(true, false) + ); + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java new file mode 100644 index 00000000000..8d394a0297e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; + + @Test + public void testIndexData() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function combiningFirehoseSpecTransform = spec -> { + try { + return StringUtils.replace( + spec, + "%%COMBINING_DATASOURCE%%", + INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doIndexTest( + COMBINING_INDEX_DATASOURCE, + COMBINING_INDEX_TASK, + combiningFirehoseSpecTransform, + COMBINING_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java new file mode 100644 index 00000000000..5ec9ea0dbc4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Map; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; + + @Test + public void testIndexData() throws Exception + { + Map inputFormatMap = new ImmutableMap + .Builder() + .put("type", "json") + .build(); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function combiningInputSourceSpecTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + "wikipedia_index_data*" + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index/json" + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%COMBINING_DATASOURCE%%", + INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + combiningInputSourceSpecTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doIndexTest( + COMBINING_INDEX_DATASOURCE, + COMBINING_INDEX_TASK, + combiningInputSourceSpecTransform, + COMBINING_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java new file mode 100644 index 00000000000..65b8dc0b1ac --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITIndexerTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String INDEX_WITH_TIMESTAMP_TASK = "/indexer/wikipedia_with_timestamp_index_task.json"; + // TODO: add queries that validate timestamp is different from the __time column since it is a dimension + // TODO: https://github.com/apache/druid/issues/9565 + private static final String INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_WITH_TIMESTAMP_DATASOURCE = "wikipedia_with_timestamp_index_test"; + + private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json"; + private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json"; + private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; + + private static final String MERGE_INDEX_TASK = "/indexer/wikipedia_merge_index_task.json"; + private static final String MERGE_INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json"; + private static final String MERGE_INDEX_DATASOURCE = "wikipedia_merge_index_test"; + + private static final String MERGE_REINDEX_TASK = "/indexer/wikipedia_merge_reindex_task.json"; + private static final String MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_merge_reindex_druid_input_source_task.json"; + private static final String MERGE_REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json"; + private static final String MERGE_REINDEX_DATASOURCE = "wikipedia_merge_reindex_test"; + + private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_TASK = "/indexer/wikipedia_index_with_merge_column_limit_task.json"; + private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE = "wikipedia_index_with_merge_column_limit_test"; + + private static final String GET_LOCKED_INTERVALS = "wikipedia_index_get_locked_intervals_test"; + + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = + CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = + CoordinatorDynamicConfig.builder().build(); + + @Inject + CoordinatorResourceTestClient coordinatorClient; + + @Test + public void testIndexData() throws Exception + { + final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testIndexData-druidInputSource"; + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + public void testReIndexDataWithTimestamp() throws Exception + { + final String reindexDatasource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp"; + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp-druidInputSource"; + try ( + final Closeable ignored1 = unloader(INDEX_WITH_TIMESTAMP_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + INDEX_WITH_TIMESTAMP_TASK, + INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + public void testReIndexWithNonExistingDatasource() throws Exception + { + Pair dummyPair = new Pair<>(false, false); + final String fullBaseDatasourceName = "nonExistingDatasource2904"; + final String fullReindexDatasourceName = "newDatasource123"; + + String taskSpec = StringUtils.replace( + getResourceAsString(REINDEX_TASK_WITH_DRUID_INPUT_SOURCE), + "%%DATASOURCE%%", + fullBaseDatasourceName + ); + taskSpec = StringUtils.replace( + taskSpec, + "%%REINDEX_DATASOURCE%%", + fullReindexDatasourceName + ); + + // This method will also verify task is successful after task finish running + // We expect task to be successful even if the datasource to reindex does not exist + submitTaskAndWait( + taskSpec, + fullReindexDatasourceName, + false, + false, + dummyPair + ); + } + + @Test + public void testMERGEIndexData() throws Exception + { + final String reindexDatasource = MERGE_REINDEX_DATASOURCE + "-testMergeIndexData"; + final String reindexDatasourceWithDruidInputSource = MERGE_REINDEX_DATASOURCE + "-testMergeReIndexData-druidInputSource"; + try ( + final Closeable ignored1 = unloader(MERGE_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + MERGE_INDEX_DATASOURCE, + MERGE_INDEX_TASK, + MERGE_INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + MERGE_INDEX_DATASOURCE, + reindexDatasource, + MERGE_REINDEX_TASK, + MERGE_REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + MERGE_INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + MERGE_INDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test that task reports indicate the ingested segments were loaded before the configured timeout expired. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailability() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("600000") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test that the task still succeeds if the segments do not become available before the configured wait timeout + * expires. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("1") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + false, + false, + new Pair<>(true, false) + ); + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" + ); + } + } + + + @Test + public void testIndexWithMergeColumnLimitData() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + doIndexTest( + INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE, + INDEX_WITH_MERGE_COLUMN_LIMIT_TASK, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } + + @Test + public void testGetLockedIntervals() throws Exception + { + final String datasourceName = GET_LOCKED_INTERVALS + config.getExtraDatasourceNameSuffix(); + try (final Closeable ignored = unloader(datasourceName)) { + // Submit an Indexing Task + submitIndexTask(INDEX_TASK, datasourceName); + + // Wait until it acquires a lock + final Map minTaskPriority = Collections.singletonMap(datasourceName, 0); + final Map> lockedIntervals = new HashMap<>(); + ITRetryUtil.retryUntilFalse( + () -> { + lockedIntervals.clear(); + lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority)); + return lockedIntervals.isEmpty(); + }, + "Verify Intervals are Locked" + ); + + // Verify the locked intervals for this datasource + Assert.assertEquals(lockedIntervals.size(), 1); + Assert.assertEquals( + lockedIntervals.get(datasourceName), + Collections.singletonList(Intervals.of("2013-08-31/2013-09-02")) + ); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(datasourceName), + "Segment Load" + ); + } + } + + @Test + public void testJsonFunctions() throws Exception + { + final String taskSpec = getResourceAsString("/indexer/json_path_index_task.json"); + + submitTaskAndWait( + taskSpec, + "json_path_index_test", + false, + true, + new Pair<>(false, false) + ); + + doTestQuery("json_path_index_test", "/indexer/json_path_index_queries.json"); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java new file mode 100644 index 00000000000..73fc73d42d8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_ALL_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries_only_data3.json"; + private static final InputFormatDetails INPUT_FORMAT_DETAILS = InputFormatDetails.JSON; + private static final String ALL_DATA = "*" + INPUT_FORMAT_DETAILS.getFileExtension(); + private static final String ONE_DAY_DATA = "wikipedia_index_data3" + INPUT_FORMAT_DETAILS.getFileExtension(); + + @Test + public void doIndexTestWithOverwriteAndDrop() throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + // Submit initial ingestion task + // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity + // dropExisting true or false does not matter as there is no existing segments + // Even though there are only 10 rows in the three input files and that maxRowsPerSegment is the default + // there will be four segments generated due to the distribution of rows in the input files and the + // fact that there are three sub-tasks (two of the tasks generate one segment each and the other + // task generates two segments) + submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, true); + verifySegmentsCountAndLoaded(indexDatasource, 4, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + // Submit overwrite ingestion task with drop existing + // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"), + // however, the input data only contains one file with one day of data, 2013-09-02 + // with dropExisting flag set to true, after the second ingestion, we should expect to + // have two segments one for the sub-task that ingested the single data file containing one day + // and one segment for the tombstone to cover the other segment previously generated by the first ingestion + submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, true); + verifySegmentsCountAndLoaded(indexDatasource, 2, 1); + doTestQuery(indexDatasource, INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE); + } + } + + @Test + public void doIndexTestWithOverwriteAndNoDrop() throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + // Submit initial ingestion task + // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity + // dropExisting true or false does not matter as there is no existing segments + submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, false); + verifySegmentsCountAndLoaded(indexDatasource, 4, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + // Submit overwrite ingestion task without drop existing + // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"), + // however, the input data only contains one day of data, 2013-09-02 (instead of 2 days of data) + // with dropExisting flag set to false, after the second ingestion, we should expect to have + // data from 2013-08-31/2013-09-01 remains unchanged and data for 2013-09-01/2013-09-02 from + // the second overwrite ingestion task + submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, false); + verifySegmentsCountAndLoaded(indexDatasource, 3, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + } + } + + private void submitIngestionTaskAndVerify( + String indexDatasource, + String fileFilter, + boolean dropExisting + ) throws Exception + { + Map inputFormatMap = new ImmutableMap.Builder() + .put("type", INPUT_FORMAT_DETAILS.getInputFormatType()) + .build(); + final Function sqlInputSourcePropsTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + fileFilter + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index" + INPUT_FORMAT_DETAILS.getFolderSuffix() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(dropExisting) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + sqlInputSourcePropsTransform, + null, + false, + false, + true, + new Pair<>(false, false) + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java new file mode 100644 index 00000000000..a5e4edc9580 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITSystemTableBatchIndexTaskTest extends AbstractITBatchIndexTest +{ + private static final Logger LOG = new Logger(ITSystemTableBatchIndexTaskTest.class); + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String SYSTEM_QUERIES_RESOURCE = "/indexer/sys_segment_batch_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + @Test + public void testIndexData() throws Exception + { + LOG.info("Starting batch index sys table queries"); + try ( + final Closeable ignored = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()) + ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTestSqlTest( + INDEX_DATASOURCE, + INDEX_TASK, + SYSTEM_QUERIES_RESOURCE, + transform + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java new file mode 100644 index 00000000000..61cb33fd2c6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.io.IOException; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITTransformTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json"; + private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json"; + private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json"; + private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; + + @Test + public void testIndexAndReIndexWithTransformSpec() throws IOException + { + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-druidInputSource"; + + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK_WITH_INPUT_SOURCE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + @Ignore + public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException + { + // TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed + // Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests! + final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK_WITH_INPUT_SOURCE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + @Ignore + public void testIndexWithFirehoseAndTransforms() throws IOException + { + // TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed + final String indexDatasource = INDEX_DATASOURCE + "-firehose"; + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + doIndexTest( + indexDatasource, + INDEX_TASK_WITH_FIREHOSE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java new file mode 100644 index 00000000000..735bc0a50ff --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.leadership; + +import com.google.inject.Inject; +import org.apache.druid.cli.CliCustomNodeRole; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.utils.SqlTestQueryHelper; +import org.apache.druid.tests.indexer.AbstractIndexerTest; +import org.apache.druid.testsEx.categories.HighAvailability; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.config.Initializer; +import org.apache.druid.testsEx.utils.DruidClusterAdminClient; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@RunWith(DruidTestRunner.class) +@Category(HighAvailability.class) +public class ITHighAvailabilityTest +{ + private static final Logger LOG = new Logger(ITHighAvailabilityTest.class); + private static final String SYSTEM_QUERIES_RESOURCE = Initializer.queryFile(HighAvailability.class, "sys.json"); + private static final int NUM_LEADERSHIP_SWAPS = 3; + + @Inject + private IntegrationTestingConfig config; + + @Inject + private DruidClusterAdminClient druidClusterAdminClient; + + @Inject + private DruidNodeDiscoveryProvider druidNodeDiscovery; + + @Inject + private SqlTestQueryHelper queryHelper; + + @Inject + @TestClient + private HttpClient httpClient; + + @Inject + private DruidClusterClient clusterClient; + + @Test + public void testLeadershipChanges() throws Exception + { + int runCount = 0; + String previousCoordinatorLeader = null; + String previousOverlordLeader = null; + // fetch current leaders, make sure queries work, then swap leaders and do it again + do { + String coordinatorLeader = getLeader("coordinator"); + String overlordLeader = getLeader("indexer"); + + // we expect leadership swap to happen + assertNotEquals(previousCoordinatorLeader, coordinatorLeader); + assertNotEquals(previousOverlordLeader, overlordLeader); + + previousCoordinatorLeader = coordinatorLeader; + previousOverlordLeader = overlordLeader; + + String queries = fillTemplate( + AbstractIndexerTest.getResourceAsString(SYSTEM_QUERIES_RESOURCE), + overlordLeader, + coordinatorLeader + ); + queryHelper.testQueriesFromString(queries); + + swapLeadersAndWait(coordinatorLeader, overlordLeader); + } while (runCount++ < NUM_LEADERSHIP_SWAPS); + } + + @Test + public void testDiscoveryAndSelfDiscovery() + { + // The cluster used here has an abbreviated set of services. + verifyRoleDiscovery(NodeRole.BROKER, 1); + verifyRoleDiscovery(NodeRole.COORDINATOR, 2); + verifyRoleDiscovery(NodeRole.OVERLORD, 2); + verifyRoleDiscovery(NodeRole.ROUTER, 1); + } + + public void verifyRoleDiscovery(NodeRole role, int expectedCount) + { + DruidNodeDiscovery discovered = druidNodeDiscovery.getForNodeRole(role); + try { + int count = 0; + for (DiscoveryDruidNode node : discovered.getAllNodes()) { + if (clusterClient.selfDiscovered(clusterClient.nodeUrl(node.getDruidNode()))) { + count++; + } + } + assertEquals(expectedCount, count); + } + catch (Exception e) { + LOG.error(e, "node discovery failed"); + fail(); + } + } + + @Test + public void testCustomDiscovery() + { + verifyRoleDiscovery(CliCustomNodeRole.NODE_ROLE, 1); + verifyCoordinatorCluster(); + } + + private void swapLeadersAndWait(String coordinatorLeader, String overlordLeader) + { + String coordUrl; + String coordLabel; + if (isCoordinatorOneLeader(coordinatorLeader)) { + druidClusterAdminClient.restartCoordinatorContainer(); + coordUrl = config.getCoordinatorUrl(); + coordLabel = "coordinator one"; + } else { + druidClusterAdminClient.restartCoordinatorTwoContainer(); + coordUrl = config.getCoordinatorTwoUrl(); + coordLabel = "coordinator two"; + } + + String overlordUrl; + String overlordLabel; + if (isOverlordOneLeader(overlordLeader)) { + druidClusterAdminClient.restartOverlordContainer(); + overlordUrl = config.getOverlordUrl(); + overlordLabel = "overlord one"; + } else { + druidClusterAdminClient.restartOverlordTwoContainer(); + overlordUrl = config.getOverlordTwoUrl(); + overlordLabel = "overlord two"; + } + clusterClient.waitForNodeReady(coordLabel, coordUrl); + clusterClient.waitForNodeReady(overlordLabel, overlordUrl); + } + + private String getLeader(String service) + { + return clusterClient.getLeader(service); + } + + private String fillTemplate(String template, String overlordLeader, String coordinatorLeader) + { + /* + {"host":"%%BROKER%%","server_type":"broker", "is_leader": %%NON_LEADER%%}, + {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, + {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, + {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, + {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, + {"host":"%%ROUTER%%","server_type":"router", "is_leader": %%NON_LEADER%%}, + */ + String working = template; + working = StringUtils.replace(working, "%%OVERLORD_ONE%%", config.getOverlordInternalHost()); + working = StringUtils.replace(working, "%%OVERLORD_TWO%%", config.getOverlordTwoInternalHost()); + working = StringUtils.replace(working, "%%COORDINATOR_ONE%%", config.getCoordinatorInternalHost()); + working = StringUtils.replace(working, "%%COORDINATOR_TWO%%", config.getCoordinatorTwoInternalHost()); + working = StringUtils.replace(working, "%%BROKER%%", config.getBrokerInternalHost()); + working = StringUtils.replace(working, "%%ROUTER%%", config.getRouterInternalHost()); + if (isOverlordOneLeader(overlordLeader)) { + working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "1"); + working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "0"); + } else { + working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "0"); + working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "1"); + } + if (isCoordinatorOneLeader(coordinatorLeader)) { + working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "1"); + working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "0"); + } else { + working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "0"); + working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "1"); + } + working = StringUtils.replace(working, "%%NON_LEADER%%", String.valueOf(NullHandling.defaultLongValue())); + return working; + } + + private boolean isCoordinatorOneLeader(String coordinatorLeader) + { + return coordinatorLeader.contains(transformHost(config.getCoordinatorInternalHost())); + } + + private boolean isOverlordOneLeader(String overlordLeader) + { + return overlordLeader.contains(transformHost(config.getOverlordInternalHost())); + } + + /** + * host + ':' which should be enough to distinguish subsets, e.g. 'druid-coordinator:8081' from + * 'druid-coordinator-two:8081' for example + */ + private static String transformHost(String host) + { + return StringUtils.format("%s:", host); + } + + private void verifyCoordinatorCluster() + { + // Verify the basics: 4 service types, excluding the custom node role. + // One of the two-node services has a size of 2. + // This endpoint includes an entry for historicals, even if none are running. + Map results = clusterClient.coordinatorCluster(); + assertEquals(5, results.size()); + @SuppressWarnings("unchecked") + List coordNodes = (List) results.get(NodeRole.COORDINATOR.getJsonName()); + assertEquals(2, coordNodes.size()); + @SuppressWarnings("unchecked") + List histNodes = (List) results.get(NodeRole.HISTORICAL.getJsonName()); + assertTrue(histNodes.isEmpty()); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java new file mode 100644 index 00000000000..60bd64ef847 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.utils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Container; +import com.github.dockerjava.core.DockerClientBuilder; +import com.github.dockerjava.core.command.ExecStartResultCallback; +import com.github.dockerjava.netty.NettyDockerCmdExecFactory; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.io.ByteArrayOutputStream; +import java.net.URL; +import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +public class DruidClusterAdminClient +{ + public static final String COORDINATOR_DOCKER_CONTAINER_NAME = "/coordinator-one"; + public static final String COORDINATOR_TWO_DOCKER_CONTAINER_NAME = "/coordinator-two"; + public static final String HISTORICAL_DOCKER_CONTAINER_NAME = "/historical"; + public static final String OVERLORD_DOCKER_CONTAINER_NAME = "/overlord-one"; + public static final String OVERLORD_TWO_DOCKER_CONTAINER_NAME = "/overlord-two"; + public static final String BROKER_DOCKER_CONTAINER_NAME = "/broker"; + public static final String ROUTER_DOCKER_CONTAINER_NAME = "/router"; + public static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "/middlemanager"; + + private static final Logger LOG = new Logger(DruidClusterAdminClient.class); + + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private IntegrationTestingConfig config; + + @Inject + DruidClusterAdminClient( + ObjectMapper jsonMapper, + @TestClient HttpClient httpClient, + IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.config = config; + } + + public void restartCoordinatorContainer() + { + restartDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME); + } + + public void restartCoordinatorTwoContainer() + { + restartDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME); + } + + public void restartHistoricalContainer() + { + restartDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME); + } + + public void restartOverlordContainer() + { + restartDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME); + } + + public void restartOverlordTwoContainer() + { + restartDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME); + } + + public void restartBrokerContainer() + { + restartDockerContainer(BROKER_DOCKER_CONTAINER_NAME); + } + + public void restartRouterContainer() + { + restartDockerContainer(ROUTER_DOCKER_CONTAINER_NAME); + } + + public void restartMiddleManagerContainer() + { + restartDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME); + } + + public void waitUntilCoordinatorReady() + { + waitUntilInstanceReady(config.getCoordinatorUrl()); + postDynamicConfig(CoordinatorDynamicConfig.builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .build()); + } + + public void waitUntilCoordinatorTwoReady() + { + waitUntilInstanceReady(config.getCoordinatorTwoUrl()); + postDynamicConfig(CoordinatorDynamicConfig.builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .build()); + } + + public void waitUntilOverlordTwoReady() + { + waitUntilInstanceReady(config.getOverlordTwoUrl()); + } + + public void waitUntilHistoricalReady() + { + waitUntilInstanceReady(config.getHistoricalUrl()); + } + + public void waitUntilIndexerReady() + { + waitUntilInstanceReady(config.getOverlordUrl()); + } + + public void waitUntilBrokerReady() + { + waitUntilInstanceReady(config.getBrokerUrl()); + } + + public void waitUntilRouterReady() + { + waitUntilInstanceReady(config.getRouterUrl()); + } + + public Pair runCommandInCoordinatorContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInCoordinatorTwoContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInHistoricalContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInOverlordContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInOverlordTwoContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInBrokerContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(BROKER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInRouterContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(ROUTER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInMiddleManagerContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInDockerContainer(String serviceName, String... cmd) throws Exception + { + DockerClient dockerClient = newClient(); + ByteArrayOutputStream stdout = new ByteArrayOutputStream(); + ByteArrayOutputStream stderr = new ByteArrayOutputStream(); + ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(findDockerContainer(dockerClient, serviceName)) + .withAttachStderr(true) + .withAttachStdout(true) + .withCmd(cmd) + .exec(); + dockerClient.execStartCmd(execCreateCmdResponse.getId()) + .exec(new ExecStartResultCallback(stdout, stderr)) + .awaitCompletion(); + + return new Pair<>(stdout.toString(StandardCharsets.UTF_8.name()), stderr.toString(StandardCharsets.UTF_8.name())); + } + + public void restartDockerContainer(String serviceName) + { + DockerClient dockerClient = newClient(); + dockerClient.restartContainerCmd(findDockerContainer(dockerClient, serviceName)).exec(); + } + + public void killAndRestartDockerContainer(String serviceName) + { + final DockerClient dockerClient = newClient(); + final String containerId = findDockerContainer(dockerClient, serviceName); + + dockerClient.killContainerCmd(containerId).withSignal("SIGKILL").exec(); + dockerClient.startContainerCmd(containerId).exec(); + } + + private static DockerClient newClient() + { + return DockerClientBuilder + .getInstance() + .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()).withConnectTimeout(10 * 1000)) + .build(); + } + + private String findDockerContainer(DockerClient dockerClient, String serviceName) + { + + List containers = dockerClient.listContainersCmd().exec(); + Optional containerName = containers + .stream() + .filter(container -> Arrays.asList(container.getNames()).contains(serviceName)) + .findFirst() + .map(Container::getId); + + if (!containerName.isPresent()) { + LOG.error("Cannot find docker container for " + serviceName); + throw new ISE("Cannot find docker container for " + serviceName); + } + return containerName.get(); + } + + private void waitUntilInstanceReady(final String host) + { + ITRetryUtil.retryUntilTrue( + () -> { + try { + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status/health", host))), + StatusResponseHandler.getInstance() + ).get(); + + LOG.info("%s %s", response.getStatus(), response.getContent()); + return response.getStatus().equals(HttpResponseStatus.OK); + } + catch (Throwable e) { + // + // suppress stack trace logging for some specific exceptions + // to reduce excessive stack trace messages when waiting druid nodes to start up + // + if (e.getCause() instanceof ChannelException) { + Throwable channelException = e.getCause(); + + if (channelException.getCause() instanceof ClosedChannelException) { + LOG.error("Channel Closed"); + } else if ("Channel disconnected".equals(channelException.getMessage())) { + // log message only + LOG.error("Channel disconnected"); + } else { + // log stack trace for unknown exception + LOG.error(e, "Error while waiting for [%s] to be ready", host); + } + } else { + // log stack trace for unknown exception + LOG.error(e, "Error while waiting for [%s] to be ready", host); + } + + return false; + } + }, + "Waiting for instance to be ready: [" + host + "]" + ); + } + + private void postDynamicConfig(CoordinatorDynamicConfig coordinatorDynamicConfig) + { + ITRetryUtil.retryUntilTrue( + () -> { + try { + String url = StringUtils.format("%s/druid/coordinator/v1/config", config.getCoordinatorUrl()); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.POST, new URL(url)).setContent( + "application/json", + jsonMapper.writeValueAsBytes(coordinatorDynamicConfig) + ), StatusResponseHandler.getInstance() + ).get(); + + LOG.info("%s %s", response.getStatus(), response.getContent()); + // if coordinator is not leader then it will return 307 instead of 200 + return response.getStatus().equals(HttpResponseStatus.OK) || response.getStatus().equals(HttpResponseStatus.TEMPORARY_REDIRECT); + } + catch (Throwable e) { + LOG.error(e, "Error while posting dynamic config"); + return false; + } + }, + "Posting dynamic config after startup" + ); + } +} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml new file mode 100644 index 00000000000..ac07bab9843 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml @@ -0,0 +1,58 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the batch index test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + indexer: + instances: + - port: 8091 + +# Properties to be set in the Properties object used in +# Guice configuration in lieu of the server-side runtime.properties +# file. +# +# druid.global.http.numMaxThreads avoids creating 40+ Netty threads. +# We only ever use 1. +# druid.test.config.dockerIp is used by some older test code. Remove +# it when that code is updated. +properties: + druid.global.http.numMaxThreads: 3 + druid.broker.http.numMaxThreads: 3 + druid.test.config.dockerIp: localhost + druid.test.config.cloudBucket: "new-it-framework" + druid.test.config.cloudPath: "" + docker.build.hadoop: true + start.hadoop.docker: true + override.config.path: "/Users/abhishekagrawal/pr_druid_it/druid/integration-tests-ex/it-azure-deep-storage/azure-config" diff --git a/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml new file mode 100644 index 00000000000..d676f530e90 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the batch index test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + indexer: + instances: + - port: 8091 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml new file mode 100644 index 00000000000..769c80989a4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Definition of the Kafka node used by some integration tests. +# Include this into test-specific configurations. + +kafka: + instances: + - port: 9092 + proxyPort: 9093 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml new file mode 100644 index 00000000000..fded303c113 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Defines the standard ZooKeeper and metadata storage (metastore) +# services used in all Druid test clusters. Include this in to +# test-specific configuration files. + +zk: + instances: + - port: 2181 +metastore: + connectURI: jdbc:mysql://:/druid + user: druid + password: diurd + instances: + - port: 3306 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml new file mode 100644 index 00000000000..5d99ffcbc52 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml @@ -0,0 +1,204 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the high-availability test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +# This test does not use ingest or query workers. +druid: + coordinator: + instances: + - tag: one + port: 8081 + - tag: two + port: 8081 + proxyPort: 18081 + overlord: + instances: + - tag: one + port: 8090 + - tag: two + port: 8090 + proxyPort: 18090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + custom-node-role: + instances: + - port: 9301 + +# Metastore initialization queries. +# REPLACE is used so that the statements are idempotent +# The fancy formatting is for human consumption, it is compacted internally +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', + 'twitterstream', + '2013-05-13T01:08:18.192Z', + '2013-01-01T00:00:00.000Z', + '2013-01-02T00:00:00.000Z', + 0, + '2013-01-02T04:13:41.980Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", + "version": "2013-01-02T04:13:41.980Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 445235220, + "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9', + 'twitterstream', + '2013-05-13T00:03:28.640Z', + '2013-01-02T00:00:00.000Z', + '2013-01-03T00:00:00.000Z', + 0, + '2013-01-03T03:44:58.791Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z", + "version": "2013-01-03T03:44:58.791Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 435325540, + "identifier": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9', + 'twitterstream', + '2013-05-13T00:03:48.807Z', + '2013-01-03T00:00:00.000Z', + '2013-01-04T00:00:00.000Z', + 0, + '2013-01-04T04:09:13.590Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z", + "version": "2013-01-04T04:09:13.590Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 411651320, + "identifier": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9', + 'wikipedia_editstream', + '2013-03-15T20:49:52.348Z', + '2012-12-29T00:00:00.000Z', + '2013-01-10T08:00:00.000Z', + 0, + '2013-01-10T08:13:47.830Z_v9', + 1, + '{"dataSource": "wikipedia_editstream", + "interval": "2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z", + "version": "2013-01-10T08:13:47.830Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip"}, + "dimensions": "anonymous,area_code,city,continent_code,country_name,dma_code,geo,language, + namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user", + "metrics": "added,count,deleted,delta,delta_hist,unique_users,variation", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 446027801, + "identifier": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', + 'wikipedia', + '2013-08-08T21:26:23.799Z', + '2013-08-01T00:00:00.000Z', + '2013-08-02T00:00:00.000Z', + 0, + '2013-08-08T21:22:48.989Z', + 1, + '{"dataSource": "wikipedia", + "interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z", + "version": "2013-08-08T21:22:48.989Z", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip" + }, + "dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace, + anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup", + "metrics": "count,delta,variation,added,deleted", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 24664730, + "identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json new file mode 100644 index 00000000000..d5d60d4f297 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json @@ -0,0 +1,39 @@ +[ + { + "description": "query sys.servers to make sure all expected servers are available", + "query": { + "query": "SELECT host, server_type, is_leader FROM sys.servers ORDER BY host" + }, + "expectedResults": [ + {"host":"%%BROKER%%","server_type":"broker", "is_leader": %%NON_LEADER%%}, + {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, + {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, + {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, + {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, + {"host":"%%ROUTER%%","server_type":"router", "is_leader": %%NON_LEADER%%} + ] + }, + { + "description": "query sys.segments which is fed via coordinator data", + "query": { + "query": "SELECT datasource, count(*) FROM sys.segments WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " + }, + "expectedResults": [ + { + "datasource": "wikipedia_editstream", + "EXPR$1": 1 + }, + { + "datasource": "twitterstream", + "EXPR$1": 3 + } + ] + }, + { + "description": "query sys.tasks which is fed via overlord", + "query": { + "query": "SELECT datasource, count(*) FROM sys.tasks WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " + }, + "expectedResults": [] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/config-test/test.yaml b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml new file mode 100644 index 00000000000..92745caea11 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml @@ -0,0 +1,85 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Definition of a simple dummy "cluster" for use in +# testing the configuration mechanism itself. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + - /cluster/Common/kafka.yaml + +druid: + overlord: + instances: + - port: 8090 + coordinator: + instances: + - port: 8081 + historical: + instances: + - port: 8083 + middlemanager: + instances: + - port: 8091 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + +properties: + druid.test.config.dockerIp: localhost + druid.test.config.cloudRegion: myRegion + my.test.property: howdy + +settings: + cloudBucket: myBucket + cloud_path: myPath + cloud_region: hidden + druid_s3_accessKey: secret + +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', + 'twitterstream', + '2013-05-13T01:08:18.192Z', + '2013-01-01T00:00:00.000Z', + '2013-01-02T00:00:00.000Z', + 0, + '2013-01-02T04:13:41.980Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", + "version": "2013-01-02T04:13:41.980Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name, + is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites, + user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 445235220, + "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json new file mode 100644 index 00000000000..20c3b162ea6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json @@ -0,0 +1,82 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%JOIN_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + { + "type": "long", + "name": "added" + }, + { + "type": "long", + "name": "deleted" + } + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "sum_added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "sum_deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "YEAR", + "queryGranularity": "second" + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "baseDir": "/resources/data/union_query/", + "filter": "wikipedia_index_data*" + }, + "appendToExisting": false, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "indexSpec": { + "segmentLoader": { + "type": "broadcastJoinableMMapSegmentFactory", + "keyColumns": ["user", "language", "added", "deleted"] + } + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json new file mode 100644 index 00000000000..1940cc6ea02 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json @@ -0,0 +1,49 @@ +[ + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "json_path_index_test", + "intervals": [ + "1000/3000" + ], + "aggregations": [ + { + "type": "longSum", + "name": "len", + "fieldName": "len" + }, + { + "type": "longSum", + "name": "max", + "fieldName": "max" + }, + { + "type": "longSum", + "name": "min", + "fieldName": "min" + }, + { + "type": "longSum", + "name": "sum", + "fieldName": "sum" + } + ], + "granularity": { + "type": "all" + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "sum": 10, + "min": 0, + "len": 5, + "max": 4 + } + } + ] + } +] + diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json new file mode 100644 index 00000000000..2fd6990b116 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json @@ -0,0 +1,80 @@ +{ + "type": "index", + "dataSource": "json_path_index_test", + "spec": { + "dataSchema": { + "dataSource": "json_path_index_test", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + { + "type": "long", + "name": "len" + }, + { + "type": "long", + "name": "min" + }, + { + "type": "long", + "name": "max" + }, + { + "type": "long", + "name": "sum" + } + ], + "dimensionExclusions": [ + "__time", + "timestamp" + ] + }, + "metricsSpec": [], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": { + "type": "none" + } + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "inline", + "data": "{\"timestamp\": \"2013-08-31T01:02:33Z\", \"values\": [0,1,2,3,4] }" + }, + "inputFormat": { + "type": "json", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "len", + "expr": "$.values.length()" + }, + { + "type": "path", + "name": "min", + "expr": "$.values.min()" + }, + { + "type": "path", + "name": "max", + "expr": "$.values.max()" + }, + { + "type": "path", + "name": "sum", + "expr": "$.values.sum()" + } + ] + } + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json new file mode 100644 index 00000000000..1868c4596d0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json @@ -0,0 +1,16 @@ +[ + { + "query": { + "queryType": "segmentMetadata", + "dataSource": "%%DATASOURCE%%", + "analysisTypes": [ + "%%ANALYSIS_TYPE%%" + ], + "intervals": [ + "%%INTERVALS%%" + ] + }, + "expectedResults": %%EXPECTED_QUERY_GRANULARITY%%, + "fieldsToTest": ["queryGranularity"] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json new file mode 100644 index 00000000000..193e69fbc9c --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json @@ -0,0 +1,80 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "minTime" : "2015-09-12T00:00:00.000Z", + "maxTime" : "2015-09-12T00:00:00.000Z" + } + } + ] + }, + { + "description": "scan, all", + "query": { + "queryType": "scan", + "dataSource": "%%DATASOURCE%%", + "intervals": [ + "2013-01-01/2020-01-02" + ], + "resultFormat":"compactedList" + }, + "expectedResults": %%EXPECTED_SCAN_RESULT%%, + "fieldsToTest": ["events"] + }, + { + "description": "roll up ratio", + "query": { + "queryType":"timeseries", + "dataSource":{ + "type":"table", + "name":"%%DATASOURCE%%" + }, + "intervals":{ + "type":"intervals", + "intervals":[ + "2013-01-01/2020-01-02" + ] + }, + "granularity":{ + "type":"all" + }, + "aggregations":[ + { + "type":"count", + "name":"a0" + }, + { + "type":"longSum", + "name":"a1", + "fieldName":"count", + "expression":null + } + ], + "postAggregations":[ + { + "type":"expression", + "name":"p0", + "expression":"((\"a0\" * 1.00) / \"a1\")", + "ordering":null + } + ] + }, + "expectedResults": [ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "a1" : %%EXPECTED_SUM_COUNT%%, + "p0" : %%EXPECTED_ROLLUP_RATIO%%, + "a0" : %%EXPECTED_NUM_ROW%% + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json new file mode 100644 index 00000000000..3a21a856ac6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json @@ -0,0 +1,57 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 3, + "maxTotalRows": 3 + }, + "maxRowsInMemory": 3 + }, + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "time", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "dimB", + "dimA", + "dimC", + "dimD", + "dimE", + "dimF" + ] + }, + "granularitySpec": { + "queryGranularity": "hour", + "rollup": true, + "segmentGranularity": "hour" + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "sum_metA", + "type": "longSum", + "fieldName": "metA" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json new file mode 100644 index 00000000000..9416a3f6bda --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json @@ -0,0 +1,19 @@ +{ + "type": "compact", + "dataSource": "%%DATASOURCE%%", + "dimensionsSpec": { + "dimensions": %%DIMENSION_NAMES%% + }, + "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", + "tuningConfig": { + "type": "index_parallel", + "maxRowsPerSegment": 3, + "maxRowsInMemory": 3, + "maxNumConcurrentSubTasks": 2, + "partitionsSpec": { + "type": "hashed", + "numShards": 1 + }, + "forceGuaranteedRollup": true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json new file mode 100644 index 00000000000..a149d7a2512 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json @@ -0,0 +1,16 @@ +{ + "type": "compact", + "dataSource": "%%DATASOURCE%%", + "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", + "tuningConfig": { + "type": "index_parallel", + "maxRowsPerSegment": 3, + "maxRowsInMemory": 3, + "maxNumConcurrentSubTasks": 2, + "partitionsSpec": { + "type": "hashed", + "numShards": 1 + }, + "forceGuaranteedRollup": true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json new file mode 100644 index 00000000000..931ad895778 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json @@ -0,0 +1,35 @@ +[ + { + "query": { + "query": "SELECT count(*) FROM sys.segments WHERE datasource LIKE 'wikipedia_index_test%'" + }, + "expectedResults": [ + { + "EXPR$0": 4 + } + ] + }, + { + "query": { + "query": "SELECT server_type FROM sys.servers WHERE tier IS NOT NULL AND server_type <> 'indexer'" + }, + "expectedResults": [ + { + "server_type":"historical" + }, + { + "server_type":"broker" + } + ] + }, + { + "query": { + "query": "SELECT status AS status FROM sys.tasks WHERE datasource LIKE 'wikipedia_index_test%' GROUP BY 1" + }, + "expectedResults": [ + { + "status": "SUCCESS" + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json new file mode 100644 index 00000000000..8c4242c91e2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json @@ -0,0 +1,87 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "%%INPUT_FORMAT_TYPE%%" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": %%PARTITIONS_SPEC%%, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json new file mode 100644 index 00000000000..742ab209f72 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json @@ -0,0 +1,81 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index" + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json new file mode 100644 index 00000000000..302d2fea284 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json @@ -0,0 +1,141 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query": { + "queryType": "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "minTime": "2013-08-31T01:02:33.000Z", + "maxTime": "2013-09-01T18:22:39.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity": "day", + "intervals": [ + "2013-09-01T00:00/2013-09-02T00:00" + ], + "filter": null, + "aggregations": [ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type": "thetaSketch", + "name": "approxCountTheta", + "fieldName": "thetaSketch", + "size": 16384, + "shouldFinalize": true, + "isInputThetaSketch": false, + "errorBoundsStdDev": null + }, + { + "type": "quantilesDoublesSketch", + "name": "quantilesSketch", + "fieldName": "quantilesDoublesSketch", + "k": 128 + } + ] + }, + "expectedResults": [ + { + "timestamp": "2013-09-01T00:00:00.000Z", + "result": { + "quantilesSketch": 6, + "approxCountTheta": 6.0, + "approxCountHLL": 6 + } + } + ] + }, + { + "description": "having spec on post aggregation", + "query": { + "queryType": "groupBy", + "dataSource": "%%DATASOURCE%%", + "granularity": "day", + "dimensions": [ + "page" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "zh" + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "added", + "name": "added_count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "added_count_times_ten", + "fn": "*", + "fields": [ + { + "type": "fieldAccess", + "name": "added_count", + "fieldName": "added_count" + }, + { + "type": "constant", + "name": "const", + "value": 10 + } + ] + } + ], + "having": { + "type": "greaterThan", + "aggregation": "added_count_times_ten", + "value": 9000 + }, + "intervals": [ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "added_count_times_ten": 9050.0, + "page": "Crimson Typhoon", + "added_count": 905, + "rows": 1 + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "added_count_times_ten": 9770.0, + "page": "Gypsy Danger", + "added_count": 977, + "rows": 1 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json new file mode 100644 index 00000000000..1e7deffc6b5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json @@ -0,0 +1,95 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "combining", + "delegates": [ + { + "type": "local", + "baseDir": "/resources/indexer", + "filter": "wikipedia_combining_index_data.json" + }, + { + "type": "ingestSegment", + "dataSource": "%%COMBINING_DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + ] + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json new file mode 100644 index 00000000000..8e1d0941bf0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "combining", + "delegates": [ + { + "type": "local", + "baseDir": "/resources/indexer", + "filter": "wikipedia_combining_index_data.json" + }, + { + "type": "druid", + "dataSource": "%%COMBINING_DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + ] + }, + "appendToExisting": %%APPEND_TO_EXISTING%%, + "inputFormat": %%INPUT_FORMAT%% + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 4, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json new file mode 100644 index 00000000000..fb620c11aa2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json @@ -0,0 +1,14 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json new file mode 100644 index 00000000000..4fddeaa8aa3 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json @@ -0,0 +1,15 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "granularitySpec": %%GRANULARITY_SPEC%%, + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json new file mode 100644 index 00000000000..7a2c016dc9d --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json @@ -0,0 +1,15 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "segmentGranularity": "%%SEGMENT_GRANULARITY%%", + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json new file mode 100644 index 00000000000..586da63e3db --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json @@ -0,0 +1,143 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 2 + } + } ] + }, + { + "description": "timeseries, count aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "count", + "name": "rows" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "rows":10 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json new file mode 100644 index 00000000000..eaa9592ca26 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json @@ -0,0 +1,143 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, count aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "count", + "name": "rows" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "rows":5 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json new file mode 100644 index 00000000000..f0cbb1cb859 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json @@ -0,0 +1,47 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : { + "minTime" : "2016-06-27T00:00:11.000Z", + "maxTime" : "2016-06-27T21:31:02.000Z" + } + } + ] + }, + { + "description": "simple aggr", + "query":{ + "queryType" : "topN", + "dataSource" : "%%DATASOURCE%%", + "intervals" : ["2016-06-27/2016-06-28"], + "granularity" : "all", + "dimension" : "page", + "metric" : "count", + "threshold" : 3, + "aggregations" : [ + { + "type" : "count", + "name" : "count" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : + [ + {"count":58,"page":"Copa América Centenario"}, + {"count":32,"page":"User:Cyde/List of candidates for speedy deletion/Subpage"}, + {"count":32,"page":"Wikipedia:Administrators' noticeboard/Incidents"} + ] + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json new file mode 100644 index 00000000000..0ac4e05af85 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json @@ -0,0 +1,74 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2016-06/P1M" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "http", + "uris": ["https://druid.apache.org/data/wikipedia.json.gz", "https://druid.apache.org/data/wikipedia.json.gz"] + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": { + "type": "dynamic" + }, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json new file mode 100644 index 00000000000..928effe65e9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json new file mode 100644 index 00000000000..6ed715b89eb --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:00:00.000Z", + "result" : { + "minTime" : "2013-08-31T01:00:00.000Z", + "maxTime" : "2013-09-01T12:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json new file mode 100644 index 00000000000..f0cfba67735 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json @@ -0,0 +1,62 @@ +[ + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page", + "city" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"language-zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"triple-added", + "name":"added_count" + }, + { + "type":"longSum", + "fieldName":"delta", + "name":"delta_sum" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 27150.0, + "page" : "Crimson Typhoon", + "city" : "Taiyuan", + "added_count" : 2715, + "delta_sum" : 900, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json new file mode 100644 index 00000000000..25f9203ac1b --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "minTime" : "2013-01-01T00:00:00.000Z", + "maxTime" : "2013-01-01T00:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-01-01T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "first_user":"masterYi", + "last_user":"speed" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json new file mode 100644 index 00000000000..77854ea0acf --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json @@ -0,0 +1,56 @@ +[ + { + "description": "rows count", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "en", + "extractionFn": null + }, + "aggregations":[ + { + "type": "count", + "name": "count" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "count":%%EXPECTED_COUNT_RESULT%% + } + } + ] + }, + { + "description": "scan with filter", + "query":{ + "queryType" : "scan", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "en", + "extractionFn": null + }, + "columns": [ + %%FIELD_TO_QUERY%% + ], + "resultFormat":"compactedList" + }, + "expectedResults": %%EXPECTED_SCAN_RESULT%%, + "fieldsToTest": ["events"] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json new file mode 100644 index 00000000000..00bf7721f2b --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json @@ -0,0 +1,89 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "local", + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json new file mode 100644 index 00000000000..1fa8b4eba32 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json new file mode 100644 index 00000000000..544d191d925 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json @@ -0,0 +1,71 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": %%GRANULARITYSPEC%%, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false} + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 10, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json new file mode 100644 index 00000000000..e9bcbfd0aae --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json @@ -0,0 +1,103 @@ +{ + "type" : "index", + "spec" : { + "dataSchema" : { + "dataSource" : "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "language", + "expression": "concat('language-', language)" + }, + { + "type": "expression", + "name": "triple-added", + "expression": "added * 3" + } + ] + } + }, + "ioConfig" : { + "type" : "index", + "inputSource" : { + "type" : "local", + "baseDir" : "/resources/data/batch_index/json", + "filter" : "wikipedia_index_data*" + }, + "inputFormat" : { + "type" : "json" + }, + "appendToExisting" : false + }, + "tuningConfig" : { + "type" : "index", + "maxRowsPerSegment" : 5000000, + "maxRowsInMemory" : 25000 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json new file mode 100644 index 00000000000..9f3128fb303 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json @@ -0,0 +1,103 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "language", + "expression": "concat('language-', language)" + }, + { + "type": "expression", + "name": "triple-added", + "expression": "added * 3" + } + ] + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json new file mode 100644 index 00000000000..35b115c9f19 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "maxColumnsToMerge" : 30 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json new file mode 100644 index 00000000000..ee0fd73021a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json @@ -0,0 +1,89 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "filter" : "%%INPUT_SOURCE_FILTER%%", + "baseDir": "%%INPUT_SOURCE_BASE_DIR%%" + }, + "appendToExisting": %%APPEND_TO_EXISTING%%, + "dropExisting": %%DROP_EXISTING%%, + "inputFormat": %%INPUT_FORMAT%% + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 4, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json new file mode 100644 index 00000000000..ab4674999b5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json @@ -0,0 +1,42 @@ +[ + { + "description": "groupby, stringFirst/stringLast rollup aggs, all", + "query":{ + "queryType" : "groupBy", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "continent" + ], + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":{ + "type":"selector", + "dimension":"continent", + "value":"Asia" + }, + "aggregations":[ + { + "type": "stringFirst", + "name": "earliest_user", + "fieldName": "first_user" + }, + { + "type":"stringLast", + "name":"latest_user", + "fieldName":"last_user" + } + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "continent":"Asia", + "earliest_user":"masterYi", + "latest_user":"stringer" + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json new file mode 100644 index 00000000000..43264a8c675 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json @@ -0,0 +1,70 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 5, + "maxRowsInMemory": 2 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json new file mode 100644 index 00000000000..9daae62c8d4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json @@ -0,0 +1,63 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "DAY", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "first_user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "last_user" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json new file mode 100644 index 00000000000..127461dd117 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json @@ -0,0 +1,65 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "first_user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "last_user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json new file mode 100644 index 00000000000..12b87977743 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json @@ -0,0 +1,82 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "properties": %%INPUT_SOURCE_CONFIG%%, + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index" + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json new file mode 100644 index 00000000000..46af17af598 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json @@ -0,0 +1,64 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + }, + "timestampSpec": { + "column": "ignored-see-ignoreTimestampSpecForDruidInputSource", + "format": "iso" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals": [ + "2013-08-31/2013-09-02" + ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "partitionsSpec": %%PARTITIONS_SPEC%%, + "splitHintSpec": { + "type":"segments", + "maxInputSegmentBytesPerTask": 1 + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json new file mode 100644 index 00000000000..9618ba9e9b6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json @@ -0,0 +1,71 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json new file mode 100644 index 00000000000..e83b1109da0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -0,0 +1,77 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxSplitSize": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%%, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json new file mode 100644 index 00000000000..b6b0d54c2aa --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json @@ -0,0 +1,88 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "sql", + "database": { + "type": "mysql", + "connectorConfig": { + "connectURI": "jdbc:mysql://druid-metadata-storage/sqlinputsource", + "user": "druid", + "password": "diurd" + } + }, + "sqls": %%SQL_QUERY%% + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json new file mode 100644 index 00000000000..e995a0f299e --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json @@ -0,0 +1,69 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals": [ + "2013-08-31/2013-09-02" + ] + }, + "parser": { + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-02", + "maxInputSegmentBytesPerTask": 1 + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json new file mode 100644 index 00000000000..bbbeca9b58a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json @@ -0,0 +1,18 @@ +[ + { + "description": "timeseries, 1 agg, all should only show data2", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T11:58:39.000Z", + "result" : { + "minTime" : "2013-08-31T11:58:39.000Z", + "maxTime" : "2013-09-01T01:02:33.000Z" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json new file mode 100644 index 00000000000..9ab153c6862 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json @@ -0,0 +1,75 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second" + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data2*" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json new file mode 100644 index 00000000000..46d5ec4395a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json @@ -0,0 +1,87 @@ +[ + { + "description": "timeBoundary", + "query": { + "queryType":"timeBoundary", + "dataSource":"%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", + "result": { + "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", + "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%" + } + } + ] + }, + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "aggregations": [ + {"type": "longSum", "fieldName": "count", "name": "edit_count"}, + {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} + ] + }, + "expectedResults": [ + { + "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result" : { + "chars_added" : 1642.0, + "edit_count" : 22 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"minute", + "dimensions":[ + "page" + ], + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json new file mode 100644 index 00000000000..9e773609cb7 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json @@ -0,0 +1,94 @@ +{ + "type": "index_realtime_appenderator", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "minute", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime_appenderator", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json new file mode 100644 index 00000000000..e5fe33b6b15 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json @@ -0,0 +1,147 @@ +[ + { + "description": "timeBoundary", + "query": { + "queryType":"timeBoundary", + "dataSource":"%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", + "result": { + "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", + "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%" + } + } + ] + }, + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "aggregations": [ + {"type": "longSum", "fieldName": "count", "name": "edit_count"}, + {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} + ] + }, + "expectedResults": [ + { + "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result" : { + "chars_added" : 1595.0, + "edit_count" : 21 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"minute", + "dimensions":[ + "page" + ], + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "topN, 1 agg, join to inline", + "query": { + "queryType": "topN", + "dataSource": { + "type": "join", + "left": "%%DATASOURCE%%", + "right": { + "type": "inline", + "columnNames": ["language", "lookupLanguage"], + "columnTypes": ["string", "string"], + "rows": [ + ["en", "inline join!"] + ] + }, + "rightPrefix": "j.", + "condition": "language == \"j.language\"", + "joinType": "LEFT" + }, + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "virtualColumns": [ + { + "type": "expression", + "name": "lookupLanguage", + "expression": "nvl(\"j.lookupLanguage\", \"language\")", + "outputType": "string" + } + ], + "aggregations": [ + { + "type": "longSum", + "name": "count", + "fieldName": "count" + } + ], + "dimension": "lookupLanguage", + "metric": "count", + "threshold": 3 + }, + "expectedResults": [ + { + "timestamp": "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result": [ + { + "lookupLanguage": "inline join!", + "count": 14 + }, + { + "lookupLanguage": "ja", + "count": 3 + }, + { + "lookupLanguage": "ru", + "count": 3 + } + ] + } + ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json new file mode 100644 index 00000000000..5f48162c488 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_realtime", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "minute", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M", + "windowPeriod": "PT1M", + "rejectionPolicy": { + "type": "serverTime" + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json new file mode 100644 index 00000000000..cf2415c2b45 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json @@ -0,0 +1,51 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "SECOND", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "millis" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json new file mode 100644 index 00000000000..2c2b0372a56 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json @@ -0,0 +1,106 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "SECOND", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "millis" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "page", + "newPage", + "anonymous", + "namespace", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "newPage", + "expression": "page" + }, + { + "type": "expression", + "name": "city", + "expression": "concat('city-', city)" + }, + { + "type": "expression", + "name": "one-plus-triple-added", + "expression": "\"triple-added\" + 1" + }, + { + "type": "expression", + "name": "delta", + "expression": "\"delta\" / 2" + }, + { + "type": "expression", + "name": "double-deleted", + "expression": "deleted * 2" + } + ] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "one-plus-triple-added", + "fieldName": "one-plus-triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "double-deleted", + "fieldName": "double-deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json new file mode 100644 index 00000000000..572e0f1b5db --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json @@ -0,0 +1,66 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json new file mode 100644 index 00000000000..40503121b92 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json @@ -0,0 +1,80 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "newPage", + "city" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"language-zh" + }, + "aggregations":[ + { + "type":"longSum", + "fieldName":"one-plus-triple-added", + "name":"added_count" + }, + { + "type":"longSum", + "fieldName":"double-deleted", + "name":"double_deleted_count" + }, + { + "type":"longSum", + "fieldName":"delta", + "name":"delta_overshadowed" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 27160.0, + "newPage" : "Crimson Typhoon", + "city" : "city-Taiyuan", + "double_deleted_count" : 10, + "delta_overshadowed" : 450, + "added_count" : 2716 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json new file mode 100644 index 00000000000..e277a9127f4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json @@ -0,0 +1,53 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json new file mode 100644 index 00000000000..029b136d441 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json @@ -0,0 +1,108 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "one-plus-triple-added", + "fieldName": "one-plus-triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "double-deleted", + "fieldName": "double-deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "page", + "newPage", + "anonymous", + "namespace", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "newPage", + "expression": "page" + }, + { + "type": "expression", + "name": "city", + "expression": "concat('city-', city)" + }, + { + "type": "expression", + "name": "one-plus-triple-added", + "expression": "\"triple-added\" + 1" + }, + { + "type": "expression", + "name": "delta", + "expression": "\"delta\" / 2" + }, + { + "type": "expression", + "name": "double-deleted", + "expression": "deleted * 2" + } + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json new file mode 100644 index 00000000000..75c1281fcd2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_realtime", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M", + "windowPeriod": "PT1M", + "rejectionPolicy": { + "type": "none" + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json new file mode 100644 index 00000000000..4f13b70cb72 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "timestamp" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json new file mode 100644 index 00000000000..d450c7b9458 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json @@ -0,0 +1,66 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "day", + "queryGranularity": "none", + "intervals": [ + "2015-09-12/2015-09-13" + ] + }, + "parser": { + "type": "hadoopyString", + "parseSpec": { + "format": "json", + "dimensionsSpec": { + "dimensions": [ + "channel", + "page", + "user" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "time" + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "added", + "type": "longSum", + "fieldName": "added" + }, + { + "name": "deleted", + "type": "longSum", + "fieldName": "deleted" + }, + { + "name": "delta", + "type": "longSum", + "fieldName": "delta" + } + ] + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/shared/wikiticker-it", + "filter": "wikiticker-2015-09-12-sampled.json.gz" + } + }, + "tuningConfig": { + "type": "index", + "targetPartitionSize" : 10000 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/check-results.sh b/integration-tests-ex/check-results.sh deleted file mode 100755 index 44aac0aa9f6..00000000000 --- a/integration-tests-ex/check-results.sh +++ /dev/null @@ -1,89 +0,0 @@ -#! /bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -#-------------------------------------------------------------------- - -# Run from Travis which has no good way to attach logs to a -# build. Instead, we check if any IT failed. If so, we append -# the last 100 lines of each server log to stdout. We have to -# stay wihtin the 4MB limit which Travis applies, so we only -# emit logs for the first failure, and only for servers that -# don't report normal completion. -# -# The only good way to check for test failures is to parse -# the Failsafe summary for each test located in -# /target/failsafe-reports/failsafe-summary.xml -# -# This directory has many subdirectories, some of which are -# tests. We rely on the fact that a test starts with "it-" AND -# contains a failsafe report. (Some projects start with "it-" -# but are not tests.) - -# Run in the docker-tests directory -cd $(dirname $0) - -# Scan for candidate projects -for PROJECT in it-* -do - # Check if a failsafe report exists. It will exist if the directory is - # a test project and failsafe ran on that directory. - REPORTS="$PROJECT/target/failsafe-reports/failsafe-summary.xml" - if [ -f "$REPORTS" ] - then - # OK, so Bash isn't the world's best text processing language... - ERRS=1 - FAILS=1 - while IFS= read -r line - do - if [ "$line" = " 0" ] - then - ERRS=0 - fi - if [ "$line" = " 0" ] - then - FAILS=0 - fi - done < "$REPORTS" - if [ $ERRS -eq 1 -o $FAILS -eq 1 ] - then - FOUND_LOGS=0 - echo "======= $PROJECT Failed ==========" - # All logs except zookeeper - for log in $(ls $PROJECT/target/shared/logs/[a-y]*.log) - do - # We assume that a successful exit includes a line with the - # following: - # Stopping lifecycle [module] stage [INIT] - tail -5 "$log" | grep -Fq 'Stopping lifecycle [module] stage [INIT]' - if [ $? -ne 0 ] - then - # Assume failure and report tail - echo $(basename $log) "logtail ========================" - tail -100 "$log" - FOUND_LOGS=1 - fi - done - - # Only emit the first failure to avoid output bloat - if [ $FOUND_LOGS -eq 1 ] - then - exit 0 - else - echo "All Druid services exited normally." - fi - fi - fi -done diff --git a/integration-tests-ex/docs/compose.md b/integration-tests-ex/docs/compose.md new file mode 100644 index 00000000000..01b41896adb --- /dev/null +++ b/integration-tests-ex/docs/compose.md @@ -0,0 +1,238 @@ + + +# Docker Compose Configuration + +The integration tests use Docker Compose to launch Druid clusters. Each +test defines its own cluster +depending on what is to be tested. Since a large amount of the definition is +common, we use inheritance to simplify cluster definition. + +Tests are split into categories so that they can run in parallel. Some of +these categories use the same cluster configuration. To further reduce +redundancy, test categories can share cluster configurations. + +See also: + +* [Druid configuration](druid-config.md) which is done via Compose. +* [Test configuration](test-config.md) which tells tests about the + cluster configuration. +* [Docker compose specification](https://github.com/compose-spec/compose-spec/blob/master/spec.md) + +## File Structure + +Docker Compose files live in the `druid-it-cases` module (`test-cases` folder) +in the `cluster` directory. There is a separate subdirectory for each cluster type +(subset of test categories), plus a `Common` folder for shared files. + +## Shared Directory + +Each test has a "shared" directory that is mounted into each container to hold things +like logs, security files, etc. The directory is known as `/shared` within the container, +and resides in `target/`. Even if two categories share a cluster configuration, +they will have separate local versions of the shared directory. This is important to +keep log files separate for each category. + +## Base Configurations + +Test clusters run some number of third-party "infrastructure" containers, +and some number of Druid service containers. For the most part, each of +these services (in Compose terms) is similar from test to test. Compose +provides [an inheritance feature]( +https://github.com/compose-spec/compose-spec/blob/master/spec.md#extends) +that we use to define base configurations. + +* `cluster/Common/dependencies.yaml` defines external dependencis (MySQL, Kafka, ZK + etc.) +* `cluster/Common/druid.yaml` defines typical settings for each Druid service. + +Test-specific configurations extend and customize the above. + +### Druid Configuration + +Docker compose passes information to Docker in the form of environment variables. +The test use a variation of the environment-variable-based configuration used in +the [public Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html). +That is, variables of the form `druid_my_config` are converted, by the image launch +script, into properties of the form `my.config`. These properties are then written +to a launch-specific `runtime.properties` file. + +Rather than have a test version of `runtime.properties`, instead we have a set of +files that define properties as environment variables. All are located in +`cases/cluster/Common/environment-configs`: + +* `common.env` - Properties common to all services. This is the test equivalent to + the `common.runtime.properties` file. +* `.env` - Properties unique to one service. This is the test equivalent to + the `service/runtime.properties` files. + +### Special Environment Variables + +Druid properties can be a bit awkward and verbose in a test environment. A number of +test-specific properties help: + +* `druid_standard_loadList` - Common extension load list for all tests, in the form + of a comma-delimited list of extensions (without the brackets.) Defined in + `common.env`. +* `druid_test_loadList` - A list of additional extensions to load for a specific test. + Defined in the `docker-compose.yaml` file for that test category. Do not include + quotes. + +Example test-specific list: + +```text +druid_test_loadList=druid-azure-extensions,my-extension +``` + +The launch script combines the two lists, and adds the required brackets and quotes. + +## Test-Specific Cluster + +Each test has a directory named `cluster/`. Docker Compose uses this name +as the cluster name which appears in the Docker desktop UI. The folder contains +the `docker-compose.yaml` file that defines the test cluster. + +In the simplest case, the file just lists the services to run as extensions +of the base services: + +```text +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + broker: + extends: + file: ../Common/compose/druid.yaml + service: broker +... +``` + +## Cluster Configuration + +If a test wants to run two of some service (say Coordinator), then it +can use the "standard" definition for only one of them and must fill in +the details (especially distinct port numbers) for the second. +(See `HighAvilability` for an example.) + +By default, the container and internal host name is the same as the service +name. Thus, a `broker` service resides in a `broker` container known as +host `broker` on the Docker overlay network. +The service name is also usually the log file name. Thus `broker` logs +to `/target//logs/broker.log`. + +An environment variable `DRUID_INSTANCE` adds a suffix to the service +name and causes the log file to be `broker-one.log` if the instance +is `one`. The service name should have the full name `broker-one`. + +Druid configuration comes from the common and service-specific environment +files in `/compose/environment-config`. A test-specific service configuration +can override any of these settings using the `environment` section. +(See [Druid Configuration](druid-config.md) for details.) +For special cases, the service can define its configuration in-line and +not load the standard settings at all. + +Each service can override the Java options. However, in practice, the +only options that actually change are those for memory. As a result, +the memory settings reside in `DRUID_SERVICE_JAVA_OPTS`, which you can +easily change on a service-by-service or test-by-test basis. + +Debugging is enabled on port 8000 in the container. Each service that +wishes to expose debugging must map that container port to a distinct +host port. + +The easiest way understand the above is to look at a few examples. + +## Service Names + +The Docker Compose file sets up an "overlay" network to connect the containers. +Each is known via a host name taken from the service name. Thus "zookeeper" is +the name of the ZK service and of the container that runs ZK. Use these names +in configuration within each container. + +### Host Ports + +Outside of the application network, containers are accessible only via the +host ports defined in the Docker Compose files. Thus, ZK is known as `localhost:2181` +to tests and other code running outside of Docker. + +## Test-Specific Configuration + +In addition to the Druid configuration discussed above, the framework provides +three ways to pass test-specific configuration to the tests. All of these methods +override any configuration in the `docker-compose` or cluster `env` files. + +The values here are passed into the Druid server as configuration values. The +values apply to all services. (This mechanism does not allow service-specific +values.) In all three approaches, use the `druid_` environment variable form. + +Precendence is in the order below with the user file lowest priority and environment +variables highest. + +### User-specific `~/druid-it/ + +# Test Conversion + +Here is the process to convert an existing `integration-test` +group to this new structure. + +The tests all go into the `druid-integration-test-cases` module +(sub-directory `test-cases`). Move the tests into the existing +`testsEx` name space so they do not collide with the existing +integration test namespace. + +## Cluster Definition + +Define a category for your tests. See [tests](tests.md) for the +details. The category is the name of the cluster definition by +default. + +Determine if you can use an existing cluster definition, or if you +need to define a new one. See [tests](tests.md) for how to share a +cluster definition. If you share a definition, update `cluster.sh` +to map from your category name to the shared cluster definition +name. + +To create a new defnition, +create a `druid-cluster/docker-compose.yaml` file by converting the +previous `docker/docker-compose-.yml` file. Carefully review +each service. Use existing files as a guide. + +In `integration-tests` all groups share a set of files with many +conditionals to work out what is to be done. In this system, each +test group stands alone: its Docker Compose file defines the cluster +for that one test. There is some detangling of the existing conditionals +to determine the setup used by each test group. + +Create the `yaml/docker.yaml` resource in `/src/test/resources` to +define your cluster for the Java tests. + +Determine if the test group populates the metadata store using queries +run in the Docker container. If so, copy those queries into the +`docker.yaml` file in the `metadataInit` section. (In the new structure, +these setup queries run in the test client, not in each Docker service.) +See the former `druid.sh` script to see what SQL was used previously. + +### + +## Test Runner + +ITs require a large amount of setup. All that code is encapsulated in the +`DruidTestRunner` class: + +```java +@RunWith(DruidTestRunner.class) +@Category(MyCategory.class) +public class ITMyTest +``` + +It is helpful to know what the test runner does: + +* Loads the cluster configuration from the `docker.yaml` file, and + resolves any includes. +* Builds up the set of Guice modules needed for the test. +* Creates the Guice injector. +* Uses the injector to inject dependencies into your test class. +* Starts the Druid lifecycle. +* Waits for each Druid service defined in `docker.yaml` to become + available. +* Runs your test methods. +* Ends the Druid lifecycle. + +You can customize the configuration for non-standard cases. See +[tests](tests.md) for details. + +## Tests + +Convert the individual tests. + +### Basics + +Copy the existing tests for the target group into the +`druid-it-cases`. For sanity, you may want to do one by one. + +When adding tests, leave the original tests in `integration-tests` for +now. (Until we have the new system running in Travis.) Once Travis +runs, you can move, rather than copy, the tests. + +While we are copying, copy to the `org.apache.druid.testsEx` package to +prevent name conficts with `org.apache.druid.tests`. + +### Maven Dependencies + +You may need to add dependencies to `pom.xml`. + +The `docker-tests/pom.xml` file includes Maven dependencies for the most +common Druid modules, which transitiviely include the third-party modules +which the Druid modules reference. You test sub-project may need addition +dependencies. To find them, review `integration-tests/pom.xml`. Careful, +however, as that file is a bit of a "kitchen sink" that includes every +possible dependency, even those already available transitively. + +If you feel the dependency is one used by multiple tests, go ahead and +add it to `docker-tests/pom.xml`. If, however, it is somehwat unique to +the test group, just add it to that sub-modules `pom.xml` file instead. + +Use the following to verify the `pom.xml`: + +```bash +mvn dependency:analyze -DoutputXML=true -DignoreNonCompile=true \ + -P skip-static-checks -Ddruid.console.skip=true -Dmaven.javadoc.skip=true \ + -P skip-tests +``` + +Doing it now will save build cycles when submitting your PR. + +### Resources and Test Data + +The existing tests use the `/src/test/resources/` directory to hold both +JSON specs used by the tests, as well as test data used by the cluster. +To make the data available to tests, we mount the `/src/test/resources` +folder into the Indexer at `/resources`. + +In the new version, we separate these two groups of files. Those used by +tests continue to reside in `/src/test/resources` for the individual +tests. Those shared by multiple tests can be in `base-test/src/test/resources`. +Copy the resource files from `integration-tests` into one of these +locations. Try to avoid doing a bulk copy: copy only the files used by +the particular test group being converted. + +Then, copy the data into `/data`, keeping the same path. See +`data/README.md` for more information. + +#### To Do + +It may turn out that data files are shared among tests. In that case, we'd +want to put them in a common location, keeping test-specific data in the +project for that test. But, we can't easily combine two directories into +a single volume mount. + +Instead, we can use the `target/shared` folder: create a new `data` +folder, copy in the required files, and mount that at `/resources`. +Or, if we feel energetic, just change the specs to read their data +from `/shared/data`, since `/shared` is already mounted. + +### Extensions + +You may see build or other code that passes a list of extensions to an old +integration test. Such configuration represents a misunderstanding of how tests (as +clients) actually work. Tests nave no visibility to a Druid installation directory. +As a result, the "extension" concept does not apply. Instead, tests are run from +Maven, and are subject to the usual Maven process for locating jar files. That +means that any extensions which the test wants to use should be listed as dependencies +in the `pom.xml` file, and will be available on the class path. There is no need for, +or use of, the `druid_extensions_loadList` for tests (or, indeed, for any client.) + +### Starter Test (Optional) + +An optional step is to ease into the new system by doing a simple +"starter test". +Create a ad-hoc test file, say `StarterTest` to hold one of the +tests to be converted. Copy any needed Guice injections. This will +be a JUnit test. + +Define your test class like this: + +```java +@RunWith(DruidTestRunner.class) +public class StarterTest +``` + +The test runner handles the required startup, Guice configuration, cluster +validation, and shutdown. Just add your own test cases. + +Determine if the test runs queries from `src/test/resources/queries`. +If so, copy those to the new sub-project. Do the same with any other +resources which the test requires. + +In this new structure, each group is its own sub-project, so resources +are separted out per test group (sub-project), whereas in +`integration-tests` the resources are all grouped together. If there +are shared resources, put those in the `docker-tests/src/test/resources` +folder so they can be shared. (This may require creating a test-jar. +As an alternative, they can be put in `base-test` which is already +available to all tests.) + +Run the one test. This will find bugs in the above. It will also likely +point out that you need Druid modules not in the base set defined by +`Initialization`. Add these modules via the `Builder.modules()` method. +Resolve the other issues which will inevitably appear. + +This starter test will ensure that most of the dependency and configuration +issues are resolved. + +### Revised Helper Classes + +The new test structure adopted shorter container and host names: +`coordinator` instead of `druid-coordinator` etc. This is safe because the +Docker application runs in isolation, we don't have to worry about a +potential `coordinator` from application X. + +To handle these changes, there are new versions of several helper classes. +Modify the tests to use the new versions: + +* `DruidClusterAdminClient` - interfaces with Docker using hard-coded + container names. + +The old versions are in `org.apache.druid.testing.utils` in +`integration-tests`, the new versions in `org.apache.druid.testing2.utils` +in this project. + +### Test Classes + +You can now convert the bulk of the tests. +One-by-one, convert existing classes: + +* Remove the TestNG annotations and includes. Substitute JUnit includes. +* Add the `@RunWith` annotation. +* Run the test in the debugger to ensure it works. + +The test class definition should look like this: + +```java +@RunWith(DruidTestRunner.class) +public class ITIndexerTest ... +{ +``` + +Run the entire suite from Maven in the sub-module directory. It should +start the cluster, run the tests, and shut down the cluster. + +## Improving Tests + +Once the tests work, an optional step is to improvement a bit beyond what +was already done. + +### Retries + +The `Initializer` takes upon itself the task of ensuring that all services +are up (at least enough that they each report that they are healthy.) So, +it is not necessary for each test case to retry endlessly to handle the case +that it is the first test run on a cluster still coming up. We can remove +retries that don't represent valid server behavior. For example, if the goal +is too ensure that the endpoint `/foo` returns `bar`, then there is no need +to retry: if the server is up, then its `/foo` endpoint should be working, and +so it should return `bar`, assuming that the server is deterministic. + +If `bar` represents something that takes time to compute (the result of a +task, say), then retry is valid. If `bar` is deterministic, then retrying won't +fix a bug that causes `bar` to be reported incorrectly. + +Use your judgement to determine when retries were added "just to be safe" +(and can thus be removed), and when the represent actual race conditions in +the operation under tests. + +### Cluster Client + +The tests obviously do a large number of API calls to the server. Some (most) +seem to spell out the code inline, resulting in much copy/paste. An improvement +is to use the cluster client instead: `ClusterClient`. Add methods for endpoints +not yet covered by copying the code from the test in question. (Better, refactor +that code to use the existing lower-level `get()` and similar methods. Then, +use the cluster client method in place of the copy/paste wad of code. + +The result is a test that is smaller, easier to undestand, easier to maintain, +and easier debug. Also, future tests are easier to write because they can reuse +the method you added to the cluster client. + +You can inject the cluster client into your test: + +```java + @Inject + private ClusterClient clusterClient; +``` + +You may find that by using the cluster client, some of the dependencies which +the test needed are now unused. Go ahead and remove them. diff --git a/integration-tests-ex/docs/debugging.md b/integration-tests-ex/docs/debugging.md new file mode 100644 index 00000000000..fb0da9aee37 --- /dev/null +++ b/integration-tests-ex/docs/debugging.md @@ -0,0 +1,159 @@ + + +# Debugging the Druid Image and Integration Tests + +The integration setup has as a primary goal the ability to quickly debug +the Druid image and any individual tests. A first step is to move the +image build into a separate project. A second step is to ensure each +test can run in JUnit in an IDE against a cluster you start by hand. + +This section discusses how to use the various debugging features. + +See: + +* [Docker Configuration](docker.md) for information on debugging + docker builds. + +## General Debug Process + +Ease of debugging is a key goal of the revised structure. + +* Rebuild the Docker image only when the Druid code changes. + Do a normal distribution build, then build a new image. +* Reuse the same image over and over if you only change tests + (such as when adding a new test.) +* Reuse the same `shared` directory when the test does not + make permanent changes. +* Change Druid configuration by changing the Docker compose + files, no need to rebuild the image. +* Work primarily in the IDE when debugging tests. +* To add more logging, change the `log4j2.xml` file in the shared + directory to increase the logging level. +* Remote debug Druid services if needed. + +## Exploring the Test Cluster + +When run in Docker Compose, the endpoints known to Druid nodes differ from +those needed by a client sitting outside the cluster. We could provide an +explicit mapping. Better is to use the +[Router](https://druid.apache.org/docs/latest/design/router.html#router-as-management-proxy) +to proxy requests. Fortunately, the Druid Console already does this. + +## Docker Build Output + +Modern Docker seems to hide the output of commands, which is a hassle to debug +a build. Oddly, the details appear for a failed build, but not for success. +Use the followig to see at least some output: + +```bash +export DOCKER_BUILDKIT=0 +``` + +Once the base container is built, you can run it, log in and poke around. First +identify the name. See the last line of the container build: + +```text +Successfully tagged org.apache.druid/test: +``` + +Or ask Docker: + +```bash +docker images +``` + +## Debug the Docker Image + +You can log into the Docker image and poke around to see what's what: + + +```bash +docker run --rm -it --entrypoint bash org.apache.druid/test: +``` + +Quite a few environment variables are provided by Docker and the setup scripts +to see them, within the container, use: + +```bash +env +``` + +## Debug an Integration Test + +To debug an integration test, you need a Docker image with the latest Druid. +To get that, you need a full Druid build. So, we break the debugging process +down into steps that depend on the state of your code. Assume `DRUID_DEV` +points to your Druid development area. + +### On Each Druid Build + +If you need to rebuild Druid (because you fixed something), do: + +* Do a distribution build of Druid: +* Build the test image. + +See [quickstart](quickstart.md) for the commands. + +### Start the Test Cluster + +* Pick a test "group" to use. +* Start a test cluster configured for this test. +* Run a test from the command line: + +Again, see [quickstart](quickstart.md) for the commands. + +### Debug the Test + +To run from your IDE, find the test to run and run it as a JUnit test (with the +cluster up.) + +Depending on the test, you may be able to run the test over and over against the +same cluster. (In fact, you should try to design your tests so that this is true: +clean up after each run.) + +The tests are just plain old JUnit tests that happen to reach out to the +test cluster and/or Docker to do their work. You can set breakpoints and debug +in the usual way. + +Each test will first verify that the cluster is fully up before it starts, so +you can launch your debug session immediately after starting the cluster: the tests +will wait as needed. + +### Stop the Test Cluster + +When done, stop the cluster: [quickstart](quickstart.md) again for details. + +## Typical Issues + +For the most part, you can stop and restart the Druid services as often +as you like and Druid will just work. There are a few combinations that +can lead to trouble, however. + +* Services won't start: When doing a new build, stop the existing cluster + before doing the build. The build removes and rebuilds the shared + directory: services can't survive that. +* Metastore failure: The metastore container will recreate the DB on + each restart. This will fail if your shared directory already contains + a DB. Do a `rm -r target//db` before restarting the DB container. +* Coordinator fails with DB errors. The Coordinator will create the Druid + tables when it starts. This means the DB has to be created. If the DB + is removed after the Coordinator starts (to fix the above issue, say) + then you have to restart the Coordinator so it can create the needed + tables. diff --git a/integration-tests-ex/docs/dependencies.md b/integration-tests-ex/docs/dependencies.md new file mode 100644 index 00000000000..90047449f5c --- /dev/null +++ b/integration-tests-ex/docs/dependencies.md @@ -0,0 +1,239 @@ + + +# Dependencies + +The Docker tests have a number of dependencies which are important to understand +when making changes or debugging problems. + +## Third-Party Libraries + +As described in the [Docker](docker.md) section, the Docker image contains Druid +plus three external dependencies: + +* The MySQL client library +* The MariaDB client library +* The Kafka protobuf provider + +These libraries are not shipped with Druid itself. Instead, we add them to the +image as follows: + +* Dependencies are listed in the `test-image/pom.xml` file. +* Maven fetches the dependencides from an upstream repo and places them + into the local Maven cache. +* The `test-image/pom.xml` file uses the `maven-dependency-plugin` + to copy these dependencies from the local repo into the + `target/docker` directory. +* The `Dockerfile` copies the dependencies into the `/usr/local/druid/lib` + directory after `build-image.sh` has unpacked the Druid distribution + into `/usr/local/druid`. + +The key benefit is that the dependencies are downloaded once and are +served from the local repo afterwards. + +## Third-Party Servers + +As described in the [Docker](docker.md) section, we use third-party +"official" images for three of our external server dependencies: + +* [MySQL](https://hub.docker.com/_/mysql). This image is configured + to create the Druid database and user upon startup. +* [ZooKeeper](https://hub.docker.com/_/zookeeper). +* [Kafka](https://hub.docker.com/r/bitnami/kafka/). There is no + "official" image so we use the one from Bitnami. + +See `compose/dependencies.yaml` for the Docker Compose configuration +for each of these services. + +Other integration tests use additional servers such as Hadoop. +We will want to track down official images for those as well. + +## Guice and Lifecycle + +Nothing will consume more of your time than fighting with Druid's +Guice and Lifecycle mechanisms. These mechanisms are designed to do +exactly one thing: configure the Druid server. They are a nightmare +to use in other configurations such as unit or integration tests. + +### Guice Modules + +Druid has *many* Guice modules. There is no documentation to explain +which components are available from which modules, or their dependencies. +So, if one needs component X, one has to hunt through the source to +find the module that provides X. (Or, one has to "just know.") There +is no trick other than putting in the time to do the research, watching +things fail, and trying harder. + +In addition, modules have implicit dependencies: to use module Y you +must also include module Z. Again, there is no documentation, you have +to know or figure it out. + +The modules are designed to work only in the server, so they assume +the entire server is avaialble. Once we have a way that the modules +work in the server, we don't mess with it. But, in tests, we want +to use a subset because tests are clients, not a server. So, we end +up fighting to reuse a system that was designed for exactly one use +case: the server. The result is either a huge amount of time fiddling +to get things right or (as in the original integration tests), we just +include everything and pretend we are a server. + +There is no obvious solution, it is just a massive time sink at +present. + +### Druid Modules + +Many of the modules we want to use in integration test are +`DruidModule`s. These go beyond the usual Guice modules to provide +extra functionality, some of which is vital in tests: + +* The modules have depenencies injected from the "startup injector." +* The modules provide Jackson modules needed to serialized JSON. + +The `Initialization` class provides the mechanisms needed to work +with `DruidModule`s, but only when creating a server: that same class +has a strong opinion about which modules to include based on the +assumption that the process is a server (or a Druid tool which acts +like a server.) + +The code here refactored `Initialization` a bit to allow us to +use the functionality without being forced to accept all the default +server modules. The upside is that we don't end up having to fake the +tests to look like servers. The downside is the issue above: we have to +deal with the dependency nightmare. + +### Lifecycle Race Conditions + +Druid uses the `Lifecycle` class to manage object initialization. The +Lifecycle expects instances to be registered before the lifecycle +starts, after which it is impossible to register new instances. + +The lifecycle works out startup order based on Guice injection +dependencies. Thus, if a constructor is `X(Y y, Z y)` Guice knows +to create an `Y` and `Z` before creating `X`. `Lifecycle` leverages +this knowledge to start `Y` and `Z` before starting `X`. + +This works only if, during module creation, something has a +dependency on `X`. Else, if `X` is a `LazySingleton` it won't be +created until it is first used. But, by then, the `Lifecycle` will have +started and you'll get the dreaded "It doesn't work that way" error. + +### Guice and Lifecycle in Tests + +In the server, this works fine: there is exactly one usage of each +singleton, and the various modules have appearently been tweaked to +ensure every lifecycle-aware object is referenced (thus created, +this registerd in the lifecycle) by some other module. + +In tests, however, this system breaks down. Maven runs a series of +tests (via `failsafe`), each of which has any number of test methods. +The test driver is free to create any number of test class instances. + +When using the `Lifecycle` mechanism in tests, we would prefer to +set up the injector, and run the lifecycle, once per test class. This +is easy to do with the JUnit `@BeforeClass` annotation. But, when we +try this, the livecycle race condition issue slams us hard. + +Tests want to reference certain components, such as `DruidNodeDiscoveryProvider` +which require `CuratorFramework` which is provided by a module that +registers a component with the lifecycle. Because of the lazy singleton +pattern, `DruidNodeDiscoveryProvider` (and hence its dependenencies) +are created when first referenced, which occurs when JUnit instantiates +the test class, which happens after the Guice/Lifecycle setup in +`@BeforeClass`. And, we get our "It doesn't work that way" error. + +We can then try to move Guice/Lifecycle creation into the test class +constuctor, but then we'll watch as JUnit creates multiple instances +and we end up running initialization over and over. Further, it seems +there are race conditions when we do that (haven't figure out the +details), and we get strange errors. Further, we end up thrashing +the very complex initializaiton logic (which is a great stress test, +but we need to it only once, not on every test.) + +A hacky compromise is to add a caching layer: do the initialization in +the constructor, so we can inject the member variables, which creates +references, which causes the comonents to be created, which causes them +to register with the `Lifecycle` at the proper time. In the second +constructor call, we reuse the injector created in the first call. +Since we simply reuse the same singletons, we should not run into +Livecycle race conditions. The `@AfterClass` JUnit annotation is pressed +into service to shut down the lifecycle after all tests run. + +## Testing Tools And the Custom Node Role + +The Druid extension `druid-testing-tools` (Maven project +`extensions-core/testing-tools` provides an extension to be loaded +into the Druid image along with the Druid distribution and third-party +libraries. + +The `integration-tests` provides additional components (such as the +custom node role) that must be placed in the image, but uses an +entirely different mechanism. + +There is no documentation to explain why we do `testing-tools` one +way, the custom node role a different way. Is there a reason other than +the items were created by different people at different times who chose +to use different approaches? + +In an ideal world, `testing-tools` would contain the custom node role: +there would be a single way to provide test-only extensions. However, +since we must maintain backward compatibility with `integration-tests`, +and that module is a nightmare to modify, we must use a short-term +compromise. + +For now, we punt: we make a copy of `druid-testing-tools`, add the +`integraton-tools` custom node role, and call it `testing-tools-ex`. +See [`testing-tools/README`](../testing-tools/README.md) for the +details. + +## Integration Tests and `base-test` + +The `integration-tests` project contains the set of existing TestNG-based +tests as well as a large number of utilities used by the tests. +The revised framework adds its own utilities. + +The utilities speicfic to the new tests resides in the `base-test` +sub-project. We include the `integration-test` project to reusse its +utilities. + +This does create a potential conflict: as we convert tests, the tests +here will have the same name as tests in the `integration-test` +package, which causes duplicate class names on the class path: never +a good thing. + +The ideal solution would be to move the test utilities to a new +sub-project within `integration-tests` and have both the new and old test +projects include the resulting jar. + +For now, we use a "shadow" approach, we use the `org.apache.druid.testsEx` +package name for new tests so names do not conflict with the +`org.apache.druid.tests` name used in `integration-tests`. Eventually, +if all tests are renamed, we can rename the `testsEx` package back +to `tests`. + +In a few cases, the utilitiy classes make asumptions about the test +setup which does not match the new setup. In this case, we make a copy +of the class and apply needed changes. At present, only one class has this +issue: + +* `DruidClusterAdminClient` - interfaces with Docker using hard-coded + container names. + +The old versions are in `org.apache.druid.testing.utils` in +`integration-tests`, the new versions in `org.apache.druid.testing2.utils` +in this project. diff --git a/integration-tests-ex/docs/docker.md b/integration-tests-ex/docs/docker.md new file mode 100644 index 00000000000..afbc1064687 --- /dev/null +++ b/integration-tests-ex/docs/docker.md @@ -0,0 +1,302 @@ + + +# Docker Test Image for Druid + +Integration tests need a Druid cluster. While some tests support using +Kubernetes for the Quickstart cluster, most need a cluster with some +test-specific configuration. We use Docker Compose to create that cluster, +based on a test-oriented Docker image built by the `it-image` Maven module +(activated by the `test-image` profile.) +The image contains the Druid distribution, +unpacked, along with the MySQL and MariaDB client libaries and +and the Kafka protobuf dependency. Docker Compose is +used to pass configuration specific to each service. + +In addition to the Druid image, we use "official" images for dependencies such +as ZooKeeper, MySQL and Kafka. + +The image here is distinct from the +["retail" image](https://druid.apache.org/docs/latest/tutorials/docker.html) +used for getting started. The test image: + +* Uses a shared directory to hold logs and some configuration. +* Uses "official" images for dependencies. +* Assumes the wrapper Docker compose scripts. +* Has some additional test-specific extensions as defind in `it-tools`. + +## Build Process + +Assuming `DRUID_DEV` points to your Druid build directory, +to build the image (only): + +```bash +cd $DRUID_DEV/docker-tests/it-image +mvn -P test-image install +``` + +Building of the image occurs in four steps: + +* The Maven `pom.xml` file gathers versions and other information from the build. + It also uses the normal Maven dependency mechanism to download the MySQL, + MariaDB and + Kafka client libraries, then copies them to the `target/docker` directory. + It then invokes the `build-image.sh` script. +* `build-image.sh` adds the Druid build tarball from `distribution/target`, + copies the contents of `test-image/docker` to `target/docker` and + then invokes the `docker build` command. +* `docker build` uses `target/docker` as the context, and thus + uses the `Dockerfile` to build the image. The `Dockerfile` copies artifacts into + the image, then defers to the `test-setup.sh` script. +* The `test-setup.sh` script is copied into the image and run. This script does + the work of installing Druid. + +The resulting image is named `org.apache.druid/test:`. + +### Clean + +A normal `mvn clean` won't remove the Docker image because that is often not +what you want. Instead, do: + +```bash +mvn clean -P test-image +``` + +You can also remove the image using Docker or the Docker desktop. + +### `target/docker` + +Docker requires that all build resources be within the current directory. We don't want +to change the source directory: in Maven, only the target directories should contain +build artifacts. So, the `pom.xml` file builds up a `target/docker` directory. The +`pom.xml` file then invokes the `build-image.sh` script to complete the setup. The +resulting directory structure is: + +```text +/target/docker +|- Dockerfile (from docker/) +|- scripts (from docker/) +|- apache-druid--bin.tar.gz (from distribution, by build-image.sh) +|- MySQL client (done by pom.xml) +|- MariaDB client (done by pom.xml) +|- Kafka protobuf client (done by pom.xml) +``` + +Then, we invoke `docker build` to build our test image. The `Dockerfile` copies +files into the image. Actual setup is done by the `test-setup.sh` script copied +into the image. + +Many Dockerfiles issue Linux commands inline. In some cases, this can speed up +subsequent builds because Docker can reuse layers. However, such Dockerfiles are +tedious to debug. It is far easier to do the detailed setup in a script within +the image. With this approach, you can debug the script by loading it into +the image, but don't run it in the Dockerfile. Instead, launch the image with +a `bash` shell and run the script by hand to debug. Since our build process +is quick, we don't lose much by reusing layers. + +### Manual Image Rebuilds + +You can quick rebuild the image if you've previously run a Maven image build. +Assume `DRUID_DEV` points to your Druid development root. Start with a +Maven build: + +```bash +cd $DRUID_DEV/docker/test-image +mvn -P test-image install +``` + +Maven is rather slow to do its part. Let it grind away once to populate +`target/docker`. Then, as you debug the `Dockerfile`, or `test-setup.sh`, +you can build faster: + +```bash +cd $DRUID_DEV/docker/test-image +./rebuild.sh +``` + +This works because the Maven build creates a file `target/env.sh` that +contains the Maven-defined environment. `rebuild.sh` reads that +environment, then proceeds as would the Maven build. +Image build time shrinks from about a minute to just a few seconds. +`rebuild.sh` will fail if `target/env.sh` is missing, which reminds +you to do the full Maven build that first time. + +Remember to do a full Maven build if you change the actual Druid code. +You'll need Maven to rebuild the affected jar file and to recreate the +distribution image. You can do this the slow way by doing a full rebuild, +or, if you are comfortable with maven, you can selectively run just the +one module build followed by just the distribution build. + +## Image Contents + +The Druid test image adds the following to the base image: + +* A Debian base image with the target JDK installed. +* Druid in `/usr/local/druid` +* Script to run Druid: `/usr/local/launch.sh` +* Extra libraries (Kafka, MySQL, MariaDB) placed in the Druid `lib` directory. + +The specific "bill of materials" follows. `DRUID_HOME` is the location of +the Druid install and is set to `/usr/local/druid`. + +| Variable or Item | Source | Destination | +| -------- | ------ | ----- | +| Druid build | `distribution/target` | `$DRUID_HOME` | +| MySQL Connector | Maven repo | `$DRUID_HOME/lib` | +| Kafka Protobuf | Maven repo | `$DRUID_HOME/lib` | +| Druid launch script | `docker/launch.sh` | `/usr/local/launch.sh` | +| Env-var-to-config script | `docker/druid.sh` | `/usr/local/druid.sh` | + +Several environment variables are defined. `DRUID_HOME` is useful at +runtime. + +| Name | Description | +| ---- | ----------- | +| `DRUID_HOME` | Location of the Druid install | +| `DRUID_VERSION` | Druid version used to build the image | +| `JAVA_HOME` | Java location | +| `JAVA_VERSION` | Java version | +| `MYSQL_VERSION` | MySQL version (DB, connector) (not actually used) | +| `MYSQL_DRIVER_CLASSNAME` | Name of the MySQL driver (not actually used) | +| `CONFLUENT_VERSION` | Kafka Protobuf library version (not actually used) | + +## Shared Directory + +The image assumes a "shared" directory passes in additional configuration +information, and exports logs and other items for inspection. + +* Location in the container: `/shared` +* Location on the host: `/target/shared` + +This means that each test group has a distinct shared directory, +populated as needed for that test. + +Input items: + +| Item | Description | +| ---- | ----------- | +| `conf/` | `log4j.xml` config (optional) | +| `hadoop-xml/` | Hadoop configuration (optional) | +| `hadoop-dependencies/` | Hadoop dependencies (optional) | +| `lib/` | Extra Druid class path items (optional) | + +Output items: + +| Item | Description | +| ---- | ----------- | +| `logs/` | Log files from each service | +| `tasklogs/` | Indexer task logs | +| `kafka/` | Kafka persistence | +| `db/` | MySQL database | +| `druid/` | Druid persistence, etc. | + +Note on the `db` directory: the MySQL container creates this directory +when it starts. If you start, then restart the MySQL container, you *must* +remove the `db` directory before restart or MySQL will fail due to existing +files. + +### Third-Party Logs + +The three third-party containers are configured to log to the `/shared` +directory rather than to Docker: + +* Kafka: `/shared/logs/kafka.log` +* ZooKeeper: `/shared/logs/zookeeper.log` +* MySQL: `/shared/logs/mysql.log` + +## Entry Point + +The container launches the `launch.sh` script which: + +* Converts environment variables to config files. +* Assembles the Java command line arguments, including those + explained above, and the just-generated config files. +* Launches Java as "pid 1" so it will receive signals. + +### Run Configuration + +The "raw" Java environment variables are a bit overly broad and result +in copy/paste when a test wants to customize only part of the option, such +as JVM arguments. To assist, the image breaks configuration down into +smaller pieces, which it assembles prior to launch. + +| Enviornment Viable | Description | +| ------------------ | ----------- | +| `DRUID_SERVICE` | Name of the Druid service to run in the `server $DRUID_SERVICE` option | +| `DRUID_INSTANCE` | Suffix added to the `DRUID_SERVICE` to create the log file name. +Use when running more than one of the same service. | +| `DRUID_COMMON_JAVA_OPTS` | Java options common to all services | +| `DRUID_SERVICE_JAVA_OPTS` | Java options for this one service or instance | +| `DEBUG_OPTS` | Optional debugging Java options | +| `LOG4J_CONFIG` | Optional Log4J configuration used in `-Dlog4j.configurationFile=$LOG4J_CONFIG` | +| `DRUID_CLASSPATH` | Optional extra Druid class path | + +In addition, three other shared directories are added to the class path if they exist: + +* `/shared/hadoop-xml` - included itself +* `/shared/lib` - Included as `/shared/lib/*` to include extra jars +* `/shared/resources` - included itself to hold extra class-path resources + +### `init` Process + +Middle Manager launches Peon processes which must be reaped. +Add [the following option](https://docs.docker.com/compose/compose-file/compose-file-v2/#init) +to the Docker Compose configuration for this service: + +```text + init: true +``` + +## Extensions + +The following extensions are installed in the image: + +```text +druid-avro-extensions +druid-aws-rds-extensions +druid-azure-extensions +druid-basic-security +druid-bloom-filter +druid-datasketches +druid-ec2-extensions +druid-google-extensions +druid-hdfs-storage +druid-histogram +druid-kafka-extraction-namespace +druid-kafka-indexing-service +druid-kerberos +druid-kinesis-indexing-service +druid-kubernetes-extensions +druid-lookups-cached-global +druid-lookups-cached-single +druid-orc-extensions +druid-pac4j +druid-parquet-extensions +druid-protobuf-extensions +druid-ranger-security +druid-s3-extensions +druid-stats +it-tools +mysql-metadata-storage +postgresql-metadata-storage +simple-client-sslcontext +``` + +If more are needed, they should be added during the image build. + diff --git a/integration-tests-ex/docs/druid-config.md b/integration-tests-ex/docs/druid-config.md new file mode 100644 index 00000000000..5d715f43ca0 --- /dev/null +++ b/integration-tests-ex/docs/druid-config.md @@ -0,0 +1,138 @@ + + +# Druid Configuration + +In a normal install, Druid obtains configuration from properties files: + +* `/_common/common.runtime.properties` +* `//runtime.properties` + +In the container, Druid uses the same mechanism, though the common properties +file is empty. The container could simply mount the `runtime.properties` file. +However, doing so runs into the normal issues with Druid configuration: Druid +provides no form of inheritance: we'd have to copy/paste the same properties +over and over, which would be a maintenance headache. + +Instead, the images use the same technique as the +[production Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html): +we pass in a large number of environment variables. + +The test configuration extends the production set to include extra +variables. Thus there are two kinds: + +* General configuration (capitalized) +* Druid configuration file settings (lower case) + +## Configuration Flow + +We use `docker-compose` to gather up the variables. From most specific +(highest priority) to most general, configuration comes from: + +* An environment variable set by the script which launches Docker Compose. + (Use sparingly, only for different test "modes" such as choosing the + DB driver, when we will use a different mode across diffrerent test runs.) +* As in-line settings in the `environment` section in the Docker Compose + definition for each service. +* In the service-specific `compose/environment-configs/.env` file. +* In the common `compose/environment-configs/common.env` file. + +Make test-specific changes in the test-specific Docker compose file. Make +changes to the `*.env` files only if you are certain that the change should +apply to all tests. An example is when we change something in our product +configs. + +The set of defined environment variables starts with the +`druid/conf/single-server/micro-quickstart` settings. It would be great to generate +these files directly from the latest quickstart files. For now, it is a manual +process to keep the definitions in sync. + +These are defined in a hierarchy: + +* `common.env` - roughly equivalent to the `_common` configuration area in Druid: + contains definitions common to all Druid services. Services can override any + of the definitions. +* `.env` - base definitions for each service, assume it runs stand-alone. + Adjust if test cluster runs multiple instances. Rougly equivalent to the + service-specific `runtime.properties` file. +* `docker-compose.yaml` - test-specific settings. + +The `launch.sh` script converts these variables to config files in +`/tmp/conf/druid`. Those files are then added to the class path. + +## Druid Config Settings + +To set a Druid config variable, replace dots in the name with underscores. + +In the usual properties file: + +```text +druid.auth.basic.common.maxSyncRetries=20 +``` + +In an environment config file: + +```text +druid_auth_basic_common_maxSyncRetries=20 +``` + +```text + environment: + - druid_auth_basic_common_maxSyncRetries=20 +``` + +For everyone's sanity, please include a comment to explain the reason for +the setting if it differs from the Quickstart defaults. + +## Special Config Variables + +The test configuration goes beyond the production Druid image configuration +to add several extensions specfically for tests. These are variables which +handle some specific part of the configuration to avoid what would otherwise +require redundant copy/paste. See the [Docker section](docker.md) for the +details. + +## Shared Directory + +Druid configuration includes not just the config files, but also items +on the Druid class path. These are provided via a `shared` directory mounted +into the container at `/shared`. +The shared directory is built in the `target/` folder for each test +category. + +The `launch.sh` script fills in a number of implicit configuration items: + +| Item | Description | +| ---- | ----------- | +| Heap dump path | Set to `${SHARED}/logs/` | +| Log4J config | Optional at `${SHARED}/conf/log4j.xml` | +| Hadoop config | Optional at `${SHARED}/hadoop-xml` | +| Extra libraries | Optional at `${SHARED}/lib` | +| Extra resources | Optional at `${SHARED}/resources` | + +`${SHARED}/resources` is the place to put things like a custom `log4j2.xml` +file. + +## Security Setup + +Tests can run with or without security enabled. (Security setup is a work in progress, +the prior integration tests enabled security for all tests.) + +* `auth.env` - Additional definitions to create a secure cluster. Also requires that + the client certificates be created. Add this to tests which test security. diff --git a/integration-tests-ex/docs/guide.md b/integration-tests-ex/docs/guide.md new file mode 100644 index 00000000000..795a725e690 --- /dev/null +++ b/integration-tests-ex/docs/guide.md @@ -0,0 +1,241 @@ + + +# Test Creation Guide + +You've played with the existing tests and you are ready to create a new test. +This section walks you through the process. If you are converting an existing +test, then see the [conversion guide](conversion.md) instead. The details +of each step are covered in other files, we'll link them from here. + +## Category + +The first quesetion is: should your new test go into an existing category, +or should you create a new one? + +You should use an existing category if: + +* Your test is a new case within an obviously-existing category. +* Your test needs the same setup as an existing category, and is quick + to run. Using the existing category avoids the need to fire up a + Docker cluster just for your test. + +You should create a new category if: + +* Your test uses a customized setup: set of services, service + configuration, set of external dependencies, instead. +* Your test will run for an extended time, and is best run in + parallel with other tests in a build envrionment. Your test + can share a cluster configuration with an existing test, but + the new category allows the test to run by itself. + +When your test *can* reuse an existing cluser definition, then the question is +about time. It takes significan time (minutes) to start a Docker cluster. We clearly +don't want to pay that cost for a test that runs for seconds, if we could just add the +test to another category. On the other hand, if you've gone crazy and added a huge +suite of tests that take 20 minutes to run, then there is a huge win to be had by +running the tests in parallel, even if they reuse an existing cluster configuration. +Use your best judgment. + +The existing categories are listed in the +`org.apache.druid.testsEx.categories` package. The classes there represent +[JUnit categories]( +https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html). +See [Test Category](tests.md#Test+Category) for details. + +If you create a new category, but want to reuse the configuration of +an existing category, add the `@Cluster` annotation as described in the above +link. Note: be sure to link to a "base" category, not to a category that, itself, +has a `@Cluster` annotation. + +If you use the `@Cluster` annotation, you must also add a mapping in the +`cluster.sh` file. See the top of the file for an example. + +## Cluster Configuration + +If you create a new category, you must define a new cluster. There are two parts: + +* Docker compose +* Test configuration + +### Docker Compose + +Create a new folder: `custer/`, then create a `docker-compose.yaml` file +in that folder. Define your cluster by borrowing heavily from existing files. +See [compose](compose.md) for details. + +The only trick is if you want to include a new external dependency. The preferred +approach is to use an "official" image. If you must, you can create a custom image +in the `it-image` module. (We've not yet done that, so if you need a custom image, +let us know and we'll figure it out.) + +### Test Configuration + +Tests need a variety of configuration information. This is, at present, more +complex than we might like. You will at least need: + +* Describe the Docker Compose cluster +* Provide test-specific properties + +You may also need: + +* Test-specific Guice modules +* Environment variable bindings to various properties +* MySQL statements to pre-populate the Druid metastore DB +* And so on. + +### Test Config File + +The cluster and properties are defined in a config file. Create a folder +`src/test/resources/cluster/`. Then add a file called `docker.yaml`. +Crib the contents from the same category from which you borrowed the Docker +Compose definitions. Strip out properties and metastore statements you don't +need. Add those you do need. See [Test Configuration](test-config.md) for the +gory details of this file. + +### Test Config Code + +You may also want to customize Guice, environment variable bindings, etc. +This is done in the [test setup](tests.md#Initialization) method in your test. + +## Start Simple + +There are *many* things that can go wrong. It is best to start simple. + +### Verify the Cluster + +Start by ensuring your cluster works. + +* Define your cluster as described above. Or, pick one to reuse. +* Verify the cluster using `it.sh up `. +* Look at the Docker desktop UI to ensure the cluster says up. if not, + track down what went wrong. Look at both the Docker (stdout) and + Druid (`target//logs/.log`) files. + +### Starter Test + +Next, create your test file as described above and in [Tests](tests.md). + +* Create the test class. +* Add the required annotations. +* Create a simple test function that just prints "hello, world". +* Create your `docker.yaml` file as decribed above. +* Start your cluster, as described above, if not already started. +* Run the test from your IDE. +* Verify that the test "passes" (that is, it prints the message.) + +If so, then this means that your test connected to your custer and +verified the health of all the services declared in your `docker.yaml` file. + +If something goes wrong, you'll know it is in the basics. Check your +cluster status. Double-check the `docker.yaml` structure. Check ports. +Etc. + +### Client + +Every test is a Druid client. Determine which service API you need. Find an +existing test client. The `DruidClusterAdminClient` is the "modern" way to +interact with the cluster, but thus far has a limited set of methods. There +are older clients as well, but they tend to be quirky. Feel free to extend +`DruidClusterAdminClient`, or use the older one: whatever works. + +Inject the client into your test. See existing tests for how this is done. + +Revise your "starter" test to do some trivial operation using the client. +Retest to ensure things work. + +### Test Cases + +From here, you can start writing tests. Explore the existing mechanisms +(including those in the original `druid-integration-tests` module which may +not yet have been ported to the new framework yet.) For example, there are +ways to store specs as files and parameterize them in tests. There is a +syntax for running queries and specifying expected results. + +You may have to create a new tool to help with your test. If you do, +try to use the new mechanisms, such as `ResolvedClusterConfig` rather than +using the old, cumbersome ones. Post questions in Slack so we can help. + +### Extensions + +Your test may need a "non-default" extension. See [Special Environment Variables]( +compose.md#Special+Environment+Variables) for how to specify test-specific +extensions. (Hint: don't copy/paste the full load list!) + +Extensions have two aspects in ITs. They act like extensions in the Druid servers +running in Docker. So, the extension must be avaialble in the Docker image. All +standard Druid extensions which are available in the Druid distribution, are also +available in the image. The may not be enabled, however. Hence the need to define +the custom load list. + +Your test may use code from the extension. To the *tests*, however, the extension +is just another jar: it must be listed in the `pom.xml` file. There is no such +thing as a "Druid extensions" to the tests themselves. + +If you test an extension that is *not* part of the Druid distributeion, then it +has to get into the image. Reach out on the slack mailing list so we can discuss +solutions (such as mounting a directory that contains the extension). + +### Retries + +The old IT framework was very liberal in its use of retries. Retires were +used to handle: + +* the time lag in starting a cluster, +* the latency inherent in events propagaing through a distributed system + (such as when segments get published), +* random network failures, +* flaky tests. + +The new framework takes a stricter view. The framework itself will ensure +service are ready (using the Druid API for that purpose.) If a server reports +itself ready, but still fails on one of your API calls, then we've got a bug +to fix. Don't use retries to work around this issue because users won't know +to do this. + +In the new framwork, tests should not be flaky. Flaky tests are a drag on +development; they waste time. If your test is flaky, please fix it. Don't count +on the amount of times things take: a busy build system will run much slower than +your dedicated laptop. And so on. + +Ideally, Druid would provide a way to positively confirm that an action has +occurred. Perhaps this might be a test-only API. Otherwise, a retry is fine, but +should be coded into your test. (Or, better, implemented in a client.) Do this only +if we document that, for that API, users should poll. Otherwise, again, users of +the API under test won't know to retry, and so the test shouldn't do so either. + +This leaves random failures. The right place to handle those is in the client, +since they are independent of the usage of the API. + +The result of the above is that you should not need (or use) the `ITRetryUtil` +mechanism. No reason for your test to retry 240 times if something is really wrong +or your test is flaky. + +This is an area under development. If you see a reason to retry, lets discuss it +and put it in the proper place. + +### Travis + +Run your tests in the IDE. Try them using `it.sh test `. If that passes +add the test to Travis. The details on how to do so are still being worked out. +Likely, you will just copy/paste an existing test "stanza" to define your new +test. Your test will run in parallel with all other IT categories, which is why +we offered the advice above: the test has to have a good reason to fire up yet +another build task. + diff --git a/integration-tests-ex/docs/history.md b/integration-tests-ex/docs/history.md new file mode 100644 index 00000000000..1a35be30323 --- /dev/null +++ b/integration-tests-ex/docs/history.md @@ -0,0 +1,96 @@ + + + +## History + +This revision of the integration test Docker scripts is based on a prior +integration test version, which is, in turn, based on +the build used for the public Docker image used in the Druid tutorial. If you are familiar +with the prior structure, here are some of the notable changes. + +* Use of "official" images for third-party dependencies, rather than adding them + to the Druid image. (Results in *far* faster image builds.) +* This project splits the prior `druid-integration-tests` project into several parts. This + project holds the Druid Docker image, while sibling projects hold the cluster definition + and test for each test group. + This allows the projects to better utilize the standard Maven build phases, and allows + better partial build support. +* The prior approach built the Docker image in the `pre-integration-test` phase. Here, since + the project is separate, we can use the Maven `install` phase. +* The prior structure ran *before* the Druid `distribution` module, hence the Druid artifacts + were not available, and the scripts did its own build, which could end up polluting the + Maven build cache. This version runs after `distribution` so it can reuse the actual build + artifacts. +* The `pom.xml` file in this project does some of the work that that `build_run_cluster.sh` + previously did, such as passing Maven versions into Docker. +* The work from the prior Dockerfile and `base-setup.sh` are combined into the revised + `base-setup.sh` here so that the work is done in the target container. +* Since the prior approach was "all-in-one", it would pass test configuration options into + the container build process so that the container is specific to the test options. This + project attempts to create a generic container and instead handle test-specific options + at container run time. +* The detailed launch commands formerly in the Dockerfile now reside in + `$DRUID_HOME/launch.sh`. +* The prior version used a much-extended version of the public launch script. Those + extensions moved into `launch.sh` with the eventual goal of using the same launch + scripts in both cases. +* The various `generate_*_cert.sh` scripts wrote into the source directory. The revised + scripts write into `target/shared/tls`. +* The shared directory previously was in `~/shared`, but that places the directory outside + of the Maven build tree. The new location is `$DRUID_DEV/docker/base-docker/target/shared`. + As a result, the directory is removed and rebuild on each Maven build. The old location was + removed via scripts, but the new one is very clearly a Maven artifact, and thus to be + removed on a Maven `clean` operation. +* The prior approach had security enabled for all tests, which makes debugging hard. + This version makes security optional, it should be enabled for just a security test. +* The orginal design was based on TestNG. Revised tests are based on JUnit. +* The original tests had "test groups" within the single directory. This version splits + the former groups into projects, so each can have its own tailored cluster definition. +* Prior images would set up MySQL inline in the container by starting the MySQL engine. + This led to some redundancy (all images would do the same thing) and also some lost + work (since the DBs in each container are not those used when running.) Here, MySQL + is in its own image. Clients can update MySQL as needed using JDBC. +* Prior code used Supervisor to launch tasks. This version uses Docker directly and + runs one process per container (except for Middle Manager, which runs Peons.) + +## History + +The current work builds on the prior integration tests, with changes to +simplify and speed up the process. + +* The prior tests required a separate Docker build for each test "group" + Here, the former groups are sub-projects. All use the same Docker image. +* The prior code used the long-obsolte TestNG. Tests here use JUnit. +* The prior test used a TestNG suite to create test intances and inject + various items using Guice. This version uses an `Initializer` class to + do roughly the same job. +* The prior tests required test configuration be passed in on the command + line, which is tedious when debugging. This version uses a cluster + configuation file instead. +* The prior version perfomed MySQL initialization in the Docker container. + But, since each test would launch multiple containers, that work was + done multiple times. Here the work is done by the test itself. +* The prior version had a single "shared" directory for all tests in + `~/shared`. This version creates a separate shared folder for each + test module, in `/target/shared`. This ensures that Maven will + delete everything between test runs. +* This version removes many of the `druid-` prefixes on the container + names. We assume that the cluster runs as the only Docker app locally, + so the extra naming just clutters things. \ No newline at end of file diff --git a/integration-tests-ex/docs/maven.md b/integration-tests-ex/docs/maven.md new file mode 100644 index 00000000000..6d17aebe452 --- /dev/null +++ b/integration-tests-ex/docs/maven.md @@ -0,0 +1,255 @@ + + +# Maven Structure + +The integration tests are built and run as part of Druid's Maven script. +Maven itself is used by hand, and as part of the [Travis](travis.md) build +proces. Running integration tests in maven is a multi-part process. + +* Build the product `distribution`. +* Build the test image. The tests run against the Maven-created Druid build, + and so appear in the root `pom.xml` file *after* the `distribution` + project which builds the Druid tarball. +* Run one or more ITs. Each Maven run includes a single test category and its + required Druid cluster. + +Travis orchestrates the above process to run the ITs in parallel. When you +run tests locally, you do the above steps one by one. You can, of course, reuse +the same disribution for multiple image builds, and the same image for multiple +test runs. + +## Build the Distribution and Image + +Use the following command to run the ITs, assuming `DRUID_DEV` points +to your Druid development directory: + +```bash +cd $DRUID_DEV +mvn clean package -P dist,test-image,skip-static-checks \ + -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +The various pieces are: + +* `clean`: Remove any existing artifacts, and any existing Docker image. +* `install`: Build the Druid code and write it to the local Maven repo. +* `-P dist`: Create the Druid distribution tarball by pulling jars from + the local Maven repo. +* `-P test-image`: Build the Docker images by grabbing the Druid tarball + and pulling additional dependencies into the local repo, then stage them + for Docker. +* Everything else: ignore parts of the build not needed for the ITs, such + as static checks, unit tests, Javadoc, etc. + +Once you've done the above once, you can do just the specific part you want +to repeat during development. See below for details. + +See [quickstart](quickstart.md) for how to run the two steps separately. + +## Run Each Integration Test Category + +Each pass through Maven runs a single test category. Running a test category +has three parts, spelled out in Maven: + +* Launch the required cluster. +* Run the test category. +* Shut down the cluster. + +Again, see [quickstart](quickstart.md) for how to run the three steps separately, +and how to run the tests in an IDE. + +To do the task via Maven: + +```bash +cd $DRUID_DEV +mvn verify -P docker-tests,skip-static-checks,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +The various pieces are: + +* `verify`: Run the steps up to the one that checks the output of the ITs. Because of + the extra cluster step in an IT, the build does not fail if an IT failse. Instead, + it continues on to clean up the cluster, and only after that does it check test + sucess in the `verify` step. +* `` selects the category to run. + +The profiles allow you to build the test image once during debugging, +and reuse it across multiple test runs. (See [Debugging](debugging.md).) + +## Dependencies + +The Docker image inclues three third-party dependencies not included in the +Druid build: + +* MySQL connector +* MariaDB connector +* Kafka Protobuf provider + +We use dependency rules in the `test-image/pom.xml` file to cause Maven to download +these dependencies into the Maven cache, then we use the +`maven-dependency-plugin` to copy those dependencies into a Docker directory, +and we use Docker to copy the files into the image. This approach avoids the need +to pull the dependency from a remote repository into the image directly, and thus +both speeds up the build, and is kinder to the upstream repositories. + +If you add additional dependencies, please follow the above process. See the +`pom.xml` files for examples. + +## Environment Variables + +The build environment users environment variables to pass information to Maven. +Maven communicates with Docker and Docker Compose via environment variables +set in the `exec-maven-plugin` of various `pom.xml` files. The environment +variables then flow into either the Docker build script (`Dockerfile`) or the +various Docker Compose scripts (`docker-compose.yaml`). It can be tedious to follow +this flow. A quick outline: + +* The build environment (such as Travis) sets environment variables, or passes values + to maven via the `-d= + +# Future Work + +The present version establishes the new IT framework. Work completed to +date includes: + +* Restructure the Docker images to use the Druid produced from the + Maven build. Use "official" images for dependencies. +* Restructure the Docker compose files. +* Create the cluster configuration mechanisms. +* Convert one "test group" to a sub-module: "high-availability". +* Create the `pom.xml`, scripts and other knick-knacks needed to tie + everything together. +* Create the initial test without using security settings to aid + debugging. + +However, *much* work remains: + +* Convert remaining tests. +* Decide when we need full security. Convert the many certificate + setup scripts. +* Support cluster types other than Docker. + +## Open Tasks + +The following detail items are open: + +* Disable a test if the `disabled` type is set in the test configuration + file. Apply it to disable the HA tests for all but Docker. +* Handle missing config files: generate a "dummy" that is disabled. +* When launching a container build or test run from Maven, write + environment variables to a `target/env.sh` file so that the user + doesn't have to find them manually to run the helper scripts. +* There is some redundancy in each test group project. Figure out + solutions: + * The `cluster.sh` script + * Boilerplate in the `pom.xml` file. +* Move test data from `/resources` to `/shared/data`. Build up the + data directory from multiple sources during cluster launch. +* Sort out which data and spec files are actually used. Remove those + which are not used. Sort the files by test-specific and shared + across tests by moving them into different directories. + +## Later Tasks + +The "public" and "integration test" versions of the Docker images have diverged significantly, +which makes it harder to "test what we ship." Differences include: + +* Different base image +* Different ways to set up dependencies. +* Different paths within the container. +* Different launch scripts. +* The test images place Druid in `/usr/local`, the public images in `/opt`. + +The tests do want to do things beyond what the "public" image does. However, this should +not require a fork of the builds. To address this issue: + +* Extend this project to create a base common to the "public" and integration test images. +* Extend the integration test image to build on top of the public image. diff --git a/integration-tests-ex/docs/quickstart.md b/integration-tests-ex/docs/quickstart.md new file mode 100644 index 00000000000..1a3fcb22f7b --- /dev/null +++ b/integration-tests-ex/docs/quickstart.md @@ -0,0 +1,201 @@ + + +# Quickstart + +If you just need to know how to build, run and use the tests, this +is the place. You can refer to the detailed material later as you +add new tests or work to improve the tests. + +## Example Test + +When first learning the framework, you can try thing out using the +`HighAvailability` test. Of the tests converted thus far, it is the +one that runs for the shortest time (on the order of a minute or two.) + +## Working with Individual Tests + +To work with tests for development and debugging, you can break the +above all-in-one step into a number of sub-steps. + +* [Build Druid](#Build Druid). +* [Build the Docker image](#Build the Docker Image). +* [Start a cluster](#Start a Cluster). +* [Run a test from the command line](#Run a Test from the Command Line). +* [Run a test from an IDE](#Run a Test from an IDE). +* [Stop the cluster](#Stop the Cluster). +* [Clean up](#Clean Up). + +## Build Druid + +The integration tests start with a Druid distribution in `distribution/target`, +which you can build using your preferred Maven command line. Simplest: + +``` +cd $DRUID_DEV +./it.sh dist +``` + +Or, in detail: + +For example: +To make the text a bit simpler, define a variable for the standard settings: + +```bash +export MAVEN_IGNORE=-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true +``` + +```bash +mvn clean package -P dist $MAVEN_IGNORE -T1.0C +``` + +Hint: put this into a script somewhere, such as a `~/bin` directory and +add that to your `PATH`. A good name is `bdru` (Build DRUid). +Here is the full script: + +```bash +#! /bin/bash + +mvn clean package -P dist,skip-static-checks,skip-tests \ + -Dmaven.javadoc.skip=true -T1.0C $* +``` + +Now you can run the any Druid build with the above script. To resume a +build: + +```bash +> bdru -rf :foo +``` + +## Build the Docker Image + +You must rebuild the Docker image whenever you rebuild the Druid distribution, +since the image includes the distribution. You also will want to rebuild the +image if you change the `it-image` project which contains the build scripts. + +```bash +./it.sh image +``` + +In detail, and assuming `DRUID_DEV` points to your Druid build directory, +to build the image (only): + +```bash +cd $DRUID_DEV/integration-tests-ex/image +mvn install -P test-image $MAVEN_IGNORE +``` + +The above has you `cd` into the project to avoid the need to disable all the +unwanted bits of the Maven build. + +See [this page](docker.md) for more information. + +## Run an IT from the Command Line + +```bash +./it.sh test +``` + +Or, in detail: + +```bash +mvn verify -P docker-tests,IT- -pl :druid-it-cases \ + -P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +Where `` is one of the test categories. + +Or + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +mvn verify -P skip-static-checks,docker-tests,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases +``` + +If the test fails, find the Druid logs in `target/shared/logs` within the +test group project. + +## Start a Cluster + +The previous generation of tests were organized into TestNG groups. This +revision moves those groups into Maven modules. Each group has a distinct +cluster configuration. (In fact, it is the cluster configuration which defines +the group: we combine all tests with the same configuration into the same module.) +So, to start a cluster, you have to pick a group to run. See +[this list](maven.md#Modules) for the list of groups. + +```bash +./it.sh up +``` + +Or, in detail: + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +./cluster.sh up +``` + +You can use Docker Desktop to monitor the cluster. Give things about 30 seconds +or a minute: if something is going to fail, it will happen during starup and you'll +see that one or more containers exited unexpectedly. + +Remember to first shut down any Druid cluster you may already be running on +your machine. + +See [this page](docker.md) for more information. + +## Run a Test from an IDE + +To run an IT in your IDE: + +* Find the IT to run. +* Run it as a JUnit test. + +The tests are specifically designed to require no command-line setup: you can +just run them directly. + +## Stop the Cluster + +Once you are done with your cluster, you can stop it as follows: + + +```bash +./it.sh down +``` + +Or, in detail: + +```bash +cd $DRUID_DEV/integration-tests-ex/ +./cluster.sh down +``` + +## Clean Up + +You can remove the Docker image when you no longer need it: + +```bash +cd $DRUID_DEV +mvn clean -P test-image +``` + +It is usually fine to skip this step: the next image build will replace +the current one anyway. diff --git a/integration-tests-ex/docs/runtime.md b/integration-tests-ex/docs/runtime.md new file mode 100644 index 00000000000..c276eee207e --- /dev/null +++ b/integration-tests-ex/docs/runtime.md @@ -0,0 +1,105 @@ + + +## Test Runtime Behavior + +This section explains how the various configuration pieces come together +to run tests. + +See also: + +* [Docker setup](docker.md) +* [Docker Compose setup](compose.md) +* [Test setup](test-config.md) + +## Shared Resources + +This module has a number of folders that are used by all tests: + +* `compose`: A collection of Docker Compose scripts that define the basics + of the cluster. Each test "inherits" the bits that it needs. +* `compose/environment-configs`: Files which define, as environment variables, + the runtime properties for each service in the cluster. (See below + for details.) +* `assets`: The `log4j2.xml` file used by images for logging. + +## Container Runtime Structure + +The container itself is a bit of a hybrid. The Druid distribution, along +with some test-specific extensions, is reused. The container also contains +libraries for Kafka, MySQL and MariaDB. + +Druid configuration is passed into the container as environment variables, +and then converted to a `runtime.properties` file by the container launch +script. Though a bit of a [Rube Goldberg](https://en.wikipedia.org/wiki/Rube_Goldberg) +mechanism, it does have one important advantage over the usual Druid configs: +we can support inheritance and overrides. The various `.env` files +provide the standard configurations. Test-specific Docker Compose files can +modify any setting. + +The container mounts a shared volume, defined in the `target/shared` directory +of each test module. This volume can provide extra libraries and class path +items. The one made available by default is `log4j2.xml`, but tests can add +more as needed. + +Container "output" also goes into the shared folder: logs, "cold storage" +and so on. + +Each container exposes the Java debugger on port 8000, mapped to a different +host port for each container. + +Each container exposes the usual Druid ports so you can work with the +container as you would a local cluster. Two handy tools are the Druid +console and the scriptable [Python client](https://github.com/paul-rogers/druid-client). + +## Test Execution + +Tests run using the Maven [failsafe](https://maven.apache.org/surefire/maven-failsafe-plugin/) +plugin which is designed for integration tests. The Maven phases are: + +* `pre-integration-test`: Starts the test cluster with `cluster.sh up` using Docker Compose. +* `integration-test`: Runs tests that start or end with `IT`. +* `post-integration-test`: Stops the test cluster using `cluster.sh down` +* `verify`: Checks the result of the integration tests. + +See [this example](https://maven.apache.org/surefire/maven-failsafe-plugin/examples/junit.html) +for JUnit setup with failsafe. + +The basic process for running a test group (sub-module) is: + +* Cluser startup builds a `target/shared` directory with items to be mounted + into the containers, such as the `log4j2.xml` file, sample data, etc. + The shared directory also holds log files, Druid persistent storage, + the metastore (MySQL) DB, etc. See `test-image/README.md` for details. +* The test is configured via a `druid-cluster/compose.yaml` file. + This file defines the services to run and their configuration. +* The `cluster.sh up` script builds the shared directory, loads the env vars + defined when the image was created and starts the cluster. +* Tests run on the local host within JUnit. +* The `Initialization` class loads the cluster configuration (see below), + optionally populates the Druid metadata storage, and is used to + inject instances into the test. +* The individual tests run. +* The `cluster.sh down` script shuts down the cluster. + +`cluster.sh` uses the generated `test-image/target/env.sh` for versions and +and other environment variables. This ensures that tests run with the same +versions used to build the image. It also simplifies the Maven boilerplate to +be copy/pasted into each test sub-project. + diff --git a/integration-tests-ex/docs/scripts.md b/integration-tests-ex/docs/scripts.md new file mode 100644 index 00000000000..2c9ceb2a874 --- /dev/null +++ b/integration-tests-ex/docs/scripts.md @@ -0,0 +1,74 @@ + + +# Scripts + +The IT framework uses a number of scripts and it can be a bit of a mystery +what each one does. This guide lists each of them. + +## `integration-tests-ex` + +* `it.sh` - Utility to perform many IT-related actions such as building Druid, + running ITs, starting a cluster, etc. Use `it.sh help` to see the list of commands. + +### `it-image` + +* `build-image.sh` - Internal script to (you guessed it), build the image. + Creates the `target/env.sh` file above, then invokes `docker-build.sh`. +* `rebuild.sh` - Rebuilds the image after the above script has created the + `env.sh` file. Used to debug changes to the image build itself: use `rebuild.sh` + rather than waiting for Maven to do its thing. +* `docker-build.sh` - Internal script to set up the needed environment + variables and invoke Docker to build the image. Modify this if you ned to change + the information passed into Docker via the command line. +* `docker/Dockerfile` - Docker script to build the image. +* `docker/test-setup.sh` - Script copied into the image at build time and run to + set up the image. Keeps the `Dockerfile` simpler (and, it is somewhat easier to + debug this script than the `Dockerfile` itself.) +* `docker/launch.sh` - Container entrypoint which runs inside the container. + Sets up configuration and calls `druid.sh` to launch Druid itself. +* `druid.sh` - Creates a Druid configuration file from environment variables, + then runs Druid within the container. +* `target/env.sh` - Created when the image is built. Provides environment + variables for things like the image name, versions and so on. Used to + quickly rebuild the image (see [Maven configuration](docs/maven.md)) and + to launch tests. + +### `test-cases` + +* `cluster//*.yaml` - Base Docker Compose scripts that define the "standard" + Druid cluster. Tests use these files to avoid redundant copy/past of the + standard items. +* `cluster.sh` - Launches or tears down a cluster for a test. Called from Maven + and `it.sh`. Can be used manually. See below. + +The options for `cluster.sh` are: + +```bash +cluster.sh [-h|help|up|down|status|compose-cmd] category +``` + +* `up` - starts the cluster. +* `down` - shuts down the cluster. +* `status` - displays cluster status for debugging. Expecially useful for debugging + issues in Travis where we cannot directly inspect the Docker cluster itself. +* `help` - repeats the usage line. +* Others - passes the command on to Docker Compose. +* `category` - the test category to launch. Performs mapping from the category name + to cluster name when categories share definitions. diff --git a/integration-tests-ex/docs/test-config.md b/integration-tests-ex/docs/test-config.md new file mode 100644 index 00000000000..4a5c7a583f7 --- /dev/null +++ b/integration-tests-ex/docs/test-config.md @@ -0,0 +1,588 @@ + + +## Test Configuration + +Tests typically need to understand how the cluster is structured. +To create a test, you must supply at least three key components: + +* A `cluster//docker-compose.yaml` file that launches the desired cluster. + (The folder name `` becomes the application name in Docker.) +* A `src/test/resources/cluster//docker.yaml` file that describes the cluster + for tests. This file can also include Metastore SQL statements needed to + populate the metastore. +* The test itself, as a JUnit test that uses the `Initializer` class to + configure the tests to match the cluster. + +This section explains the test configuration file which defines the test +cluster. + +Note that you can create multiple versions of the `docker.yaml` file. For example, +you might want to create one that lists hosts and credentials unique to your +debugging environment. You then use your custom version in place of the standard +one. + +## Cluster Types + +The integration tests can run in a variety of cluster types, depending +on the details of the test: + +* Docker Compose: the normal configuration that all tests support. +* Micro Quickstart: allows for a manual cluster setup, if, say, you + want to run services in your IDE. Supported by a subset of tests. +* Kubernetes: (Details needed.) + +Each cluster type has its own quirks. The job of the tests's cluster configuration +file is to communicate those quirks to the test. + +Docker and Kubernetes use proxies to communicate to the cluster. Thus, the host +known to the tests is different than the hosts known within the cluster. Ports +may also are mapped differently "outside" than "inside." + +Clusters outside of Docker don't provide a good way to start and stop +services, so tests that want to do that (to, say, test high availability) +can't run except in a Docker cluster. + +### Specify the Cluster Type + +To reflect this, tests provide named configuration files. The configuration +itself is passed in via the environment: + +```bash +export TEST_CONFIG=quickstart +``` + +```bash +java ... -DtestConfig=quickstart +``` + +The system property taskes precedence over the environment variable. +If neither are set, `docker` is the default. The configuration file +itself is assumed to be a resource named `/yaml/.yaml`. + +As a debug aide, a test can specify and ad-hoc file in the file system +to load for one-off special cases. See `Initialization.Builder` for +details. + +## Cluster Configuration Files + +Cluster configuration is specified in a file for ease of debugging. Since +configuration is in a file (resource), and not in environment variables or +system properties, you +should need no special launch setup in your IDE to run a test that uses +the standard Docker Compose cluster for that test. + +The configuration file has the same name as the cluster type and resides on +the class path at `/yaml/.yaml` and in the source tree at +`/src/test/resources/yaml/.yaml`. The standard names are: + +* `docker.yaml`: the default and required for all tests. Describes a Docker + Compose based test. +* `k8s.yaml`: a test cluster running in Kubernetes. (Details needed.) +* `local.yaml`: a local cluser such as Micro Quickstart cluster. (Details needed.) +* `.yaml`: custom cluster configuration. + +Configuration files support include files. Most of the boiler-plate configuration +should appear in commmon files. As a result, you should only need to specify +test-specific differences in your `docker.yaml` file, with all else obtained +from the included files. + +## Configuration File Syntax + +The configuration is a [YAML](https://yaml.org/spec/1.2.2/) file that +has a few top-level properties and an entry for each service in your +cluster. + +### `type` + +```yaml +type: docker|k8s|local|disabled +``` + +The type explains the infrastructure that runs the cluster: + +* `docker`: a cluster launched in Docker, typically via Docker Compose. + A proxy host is needed. (See below.) +* `k8s`: a cluster run in Kubernets. (Details needed). A proxy host + is needed. +* `local`: a cluster running as processes on a network directly reachable + by the tests. Example: a micro-quickstart cluster running locally. +* `disabled`: the configuration is not supported by the test. + +The `disabled` type is handy for tests that require Docker: you can say that +the test is not available when the cluster is local. + +If the test tries to load a cluster name that does not exist, a "dummy" +configuration is loaded instead with the type set to `disabled`. + +The type is separate from the cluster name (as explained earlier): there +may be multiple names for the same type. For example, you might have two +or three local cluster setups you wish to test. + +### `include` + +```yaml: +include: + - +``` + +Allows including any number of other files. Similar to inheritance for +Docker Compose. The inheritance rules are: + +* Properties set later in the list take precedence over properties set in + files earlier in the list. +* Properties set in the file take precedence over properties set in + included files. +* Includes can nest to any level. + +Merging occurs as follows: + +* Top level scalars: newer values replace older values. +* Services: newer values replace all older settings for that service. +* Metastore init: newer values add more queries to any list defined + by an earlier file. +* Properties: newer values replace values defined by earlier files. + +The files are assumed to be resources (on the class path) and require +the full path name. Example: `/cluster/Commmon/base.yaml` + +### `proxyHost` + +```yaml +proxyHost: +``` + +When tests run in either Docker or Kubernetes, the test communicate with +a proxy, which forwards requests to the cluster hosts and ports. In +Docker, the proxy host is the machine that runs Docker. In Kubernetes, +the proxy host is the host running the Kubernetes proxy service. + +There is no proxy host for clusters running directly on a machine. + +If the proxy host is omitted for Docker, `localhost` is assumed. + +### `datasourceSuffix` + +```yaml +datasourceSuffix: +``` + +Suffix to append to data source names in indexer tests. The default +is the empty string. + +### `zk` + +```yaml +zk: + +``` + +Specifies the ZooKeeper instances. + +#### `startTimeoutSecs` + +```yaml +startTimeoutSecs: +``` + +Specifies the amount of time to wait for ZK to become available when using the +test client. Optional. + +### `metastore` + +```yaml +metastore: + +``` + +Describes the Druid "metadata storage" (metastore) typically +hosted in the offical MySql container. See `MetastoreConfig` for +configuration options. + +#### `driver` + +```yaml +driver: +``` + +The Driver to use to work with the metastore. The driver must be +available on the tests's class path. + +#### `connectURI` + +```yaml: +connectURI: +``` + +The JDBC connetion URL. Example: + +```text +jdbc:mysql://:/druid +``` + +The config system supports two special fields: `` and ``. +A string of form `` will be replaced by the resolved host name +(proxy host for Docker) and `` with the resolved port number. + +#### `user` + +```yaml +user: +``` + +The MySQL user name. + + +#### `password` + +```yaml +user: +``` + +The MySQL password. + +#### `properties` + +```yaml +properties: + : +``` + +Optional map of additional key/value pairs to pass to the JDBC driver. + +### `kafka` + +```yaml +zk: + +``` + +Describes the optional Kafka service. + +### `druid` + +```yaml +druid: + : + +``` + +Describes the set of Druid services using the `ServiceConfig` object. +Each service is keyed by the standard service name: the same name used +by the Druid `server` option. + +When using inheritance, overrides replace entire services: it is not possible +to override individual instances of the service. That is, an include file might +define `coordinator`, but a test-specific file might override this with a +definition of two Coordinators. + +### `properties` + +```yaml +properties: + : +``` + +Optional set of properties to use to configuration the Druid components loaded +by tests. This is the test-specific form of the standard Druid `common.runtime.properties` +and `runtime.properties` files. Because the test runs as a client, the server +files are not available, and might not even make sense. (The client is not +a "service", for example.) Technically, the properties listed here are added to +Guice as the one and only `Properties` object. + +Typically most components work using the default values. Tests are free to change +any of these values for a given test scenario. The properties are +the same for all tests within a category. However, they can be changed via environment +variables via the environment variable "binding" mechanism described in +[tests](tests.md). + +The "JSON configuration" mechanism wants all properties to be strings. YAML +will deserialize number-like properties as numbers. To avoid confusion, all +properties are converted to strings before being passed to Druid. + +When using inheritance, later properties override earlier properties. Environment +variables, if bound, override the defaults specified in this section. Command-line +settings, if provided, have the highest priority. + +A number of test-specific properties are avilable: + +* `druid.test.config.cloudBucket` +* `druid.test.config.cloudPath` + +### `settings` + +The settings section is much like the properties section, and, indeed, are converted +to properties internally. Settings are a fixed set of values that map to the config +files used in the prior tests. Keys include: + +| Setting | Property | Environment Variable | +| `druid_storage_type` | - | - | +| `druid_storage_bucket` | `druid.test.config.cloudBucket` | `DRUID_STORAGE_BUCKET` | +| `druid_storage_baseKey` | `druid.test.config.cloudPath` | `DRUID_STORAGE_BASEKEY` | +| `druid_s3_accessKey` | - | `AWS_ACCESS_KEY_ID` | +| `druid_s3_secretKey` | - | AWS_SECRET_ACCESS_KEY` | + +The above replaces the config file mechanism from the older tests. In general, when a +setting is fixed for a test category, list it in the `docker.yaml` configuration file. +When it varies, pass it in as an environment variable. As a result, the prior configuration +file is not needed. As a result, the prior `override.config.path` property is not supported. + +### `metastoreInit` + +```yaml +metastoreInit: + - sql: | + +``` + +A set of MySQL statements to be run against the +metadata storage before the test starts. Queries run in the +order specified. Ensure each is idempotent to +allow running tests multiple times against the same database. + +To be kind to readers, please format the statements across multiple lines. +The code will compress out extra spaces before submitting the query so +that JSON payloads are as compact as possible. + +The `sql` keyword is the only one supported at present. The idea is that +there may need to be context for some queries in some tests. (To be +enhanced as query conversion proceeds.) + +When using inheritance, the set of queries is the union of all queries +in all configuration files. Base statements appear first, then included +statements. + +### `metastoreInitDelaySec` + +```yaml +metastoreInitDelaySec: +``` + +The default value is 6 seconds. + +The metastore init section issues queries to the MySQL DB read by the +Coordinator. For performance, the Coordinator *does not* directly query +the database: instead, it queries an in-memory cache. This leads to the +following behavior: + +* The Coordinator starts, checks the DB, and records the poll time. +* The test starts and updates the DB. +* The test runs and issues a query that needs the DB contents. +* The Coordinator checks that its poll timeout has not yet occurred + and returns the (empty) contents of the cache. +* The test checks the empty contents against the expected contents, + notices the results differ, and fails the test. + +To work around this, we must change _two_ settings. First, change +the following Druid configuration for the Coordinator: + +```yaml + - druid_manager_segments_pollDuration=PT5S +``` + +Second, change the `metastoreInitDelaySec` to be a bit longer: + +```yaml +metastoreInitDelaySec: 6 +``` + +The result is that the test will sit idle for 6 seconds, but that is better +than random failures. + +**Note:** a better fix would be for the Coordinator to have an API that causes +it to flush its cache. Since some tests run two coordinators, the message must be +sent to both. An even better fix would be fore the Coordinator to detect such +changes itself somehow. + +### Service Object + +Generic object to describe Docker Compose services. + +#### `service` + +```yaml +service: +``` + +Name of the service as known to Docker Compose. Defaults to be +the same as the service name used in this configuration file. + +#### `instances` + +```yaml +instances: + - +``` + +Describes the instances of the service as `ServiceInstance` objects. +Each service requires at least one instance. If more than one, then +each instance must define a `tag` that is a suffix that distinguishes +the instances. + +### Service Instance Object + +The service sections all allow multiple instances of each service. Service +instances define each instance of a service and provide a number of properties: + +#### `tag` + +When a service has more than one instance, the instances must have unique +names. The name is made up of the a base name (see below) with the tag +appended. Thus, if the service is `cooordinator` and the tag is `one`, +then the instance name is `coordinator-one`. + +The tag is required when there is more than one instance of a service, +and is optional if there is only one instance. The tag corresponds to the +`DRUID_INSTANCE` environment variable passed into the container. + +#### `container` + +```yaml +container: +``` + +Name of the Docker container. If omitted, defaults to: + +* `-` if a `tag` is provided (see below.) +* The name of the service (if there is only one instance). + +#### `host` + +```yaml +host: +``` + +The host name or IP address on which the instance runs. This is +the host name known to the _cluster_: the name inside a Docker overlay network. +Has the same defaults as `container`. + +#### `port` + +```yaml +port: +``` + +The port number of the service on the container as seen by other +services running within Docker. Required. + +(TODO: If TLS is enabled, this is the TLS port.) + +#### `proxyPort` + +```yaml +proxyPort: +``` + +The port number for the service as exposed on the proxy host. +Defaults to the same as `port`. You must specify a value if +you run multiple instances of the same service. + +## Conversion Guide + +In prior tests, a config file, and the `ConfigFileConfigProvider` class, +provided test configuration. In this version, the file described here +provides configuration. This section presents a mapping from the old to +the new form. + +The `IntegrationTestingConfig` class, which the above class used to provide, +is reimplemented to provide the same information +to tests as before; only the source of the information has changed. + +The new framework assumes that each Druid node is configured either for +plain text or for TLS. (If this assumption is wrong, we'll change the config +file to match.) + +Many of the properties are derived from information in the configuration file. +For example, host names (within Docker) are those given in the `druid` section, +and ports (within the cluster and for the client) are given in `druid..intances.port`, +from which the code computes the URL. + +The old system hard-codes the idea that there are two coordinators or overlords. The +new system allows any number of instances. + +| Method | Old Property | New Format | +| ------ | ------------ | ---------- | +| Router | | | +| `getRouterHost()` | `router_host` | `'router'` | +| `getRouterUrl()` | `router_url` | `'router'` & `instances.port` | +| `getRouterTLSUrl()` | `router_tls_url` | " | +| `getPermissiveRouterUrl()` | `router_permissive_url` | " | +| `getPermissiveRouterTLSUrl()` | `router_permissive_tls_url` | " | +| `getNoClientAuthRouterUrl()` | `router_no_client_auth_url` | " | +| `getNoClientAuthRouterTLSUrl()` | `router_no_client_auth_tls_url` | " | +| `getCustomCertCheckRouterUrl()` | | " | +| `getCustomCertCheckRouterTLSUrl()` | | " | +| Broker | | | +| `getBrokerHost()` | `broker_host` | `'broker'` | +| `getBrokerUrl()` | `broker_url` | `'broker'` & `instances.port` | +| `getBrokerTLSUrl()` | `broker_tls_url` | " | +| Coordinator | | | +| `getCoordinatorHost()` | `coordinator_host` | `'coordinator'` + `tag` | +| `getCoordinatorTwoHost()` | `coordinator_two_host` | " | +| `getCoordinatorUrl()` | `coordinator_url` | host & `instances.port` | +| `getCoordinatorTLSUrl()` | `coordinator_tls_url` | " | +| `getCoordinatorTwoUrl()` | `coordinator_two_url` | " | +| `getCoordinatorTwoTLSUrl()` | `coordinator_two_tls_url` | " | +| Overlord | | | +| `getOverlordUrl()` | ? | `'overlord'` + `tag` | +| `getOverlordTwoHost()` | `overlord_two_host` | " | +| `getOverlordTwoUrl()` | `overlord_two_url` | host & `instances.port` | +| `getOverlordTLSUrl()` | ? | " | +| `getOverlordTwoTLSUrl()` | `overlord_two_tls_url` | " | +| Overlord | | | +| `getHistoricalHost()` | `historical_host` | `historical'` | +| `getHistoricalUrl()` | `historical_url` | `'historical'` & `instances.port` | +| `getHistoricalTLSUrl()` | `historical_tls_url` | " | +| Overlord | | | +| `getMiddleManagerHost()` | `middlemanager_host` | `'middlemanager'` | +| Dependencies | | | +| `getZookeeperHosts()` | `zookeeper_hosts` | `'zk'` | +| `getKafkaHost()` | `kafka_host` | '`kafka`' | +| `getSchemaRegistryHost()` | `schema_registry_host` | ? | +| `getProperty()` | From config file | From `settings` | +| `getProperties()` | " | " | +| `getUsername()` | `username` | Setting | +| `getPassword()` | `password` | Setting | +| `getCloudBucket()` | `cloud_bucket` | Setting | +| `getCloudPath()` | `cloud_path` | Setting | +| `getCloudRegion()` | `cloud_region` | Setting | +| `getS3AssumeRoleWithExternalId()` | `s3_assume_role_with_external_id` | Setting | +| `getS3AssumeRoleExternalId()` | `s3_assume_role_external_id` | Setting | +| `getS3AssumeRoleWithoutExternalId()` | `s3_assume_role_without_external_id` | Setting | +| `getAzureKey()` | `azureKey` | Setting | +| `getHadoopGcsCredentialsPath()` | `hadoopGcsCredentialsPath` | Setting | +| `getStreamEndpoint()` | `stream_endpoint` | Setting | +| `manageKafkaTopic()` | ? | ? | +| `getExtraDatasourceNameSuffix()` | ? | ? | + +Pre-defined environment bindings: + +| Setting | Env. Var. | +| `cloudBucket` | `DRUID_CLOUD_BUCKET` | +| `cloudPath` | `DRUID_CLOUD_PATH` | +| `s3AccessKey` | `AWS_ACCESS_KEY_ID` | +| `s3SecretKey` | `AWS_SECRET_ACCESS_KEY` | +| `azureContainer` | `AZURE_CONTAINER` | +| `azureAccount` | `AZURE_ACCOUNT` | +| `azureKey` | `AZURE_KEY` | +| `googleBucket` | `GOOGLE_BUCKET` | +| `googlePrefix` | `GOOGLE_PREFIX` | + +Others can be added in `Initializer.Builder`. + diff --git a/integration-tests-ex/docs/tests.md b/integration-tests-ex/docs/tests.md new file mode 100644 index 00000000000..1ef74a77338 --- /dev/null +++ b/integration-tests-ex/docs/tests.md @@ -0,0 +1,411 @@ + + +# Test Structure + +The structure of these integration tests is heavily influenced by the existing +integration test structure. In that previous structure: + +* Each test group ran as separate Maven build. +* Each would build an image, start a cluster, run the test, and shut down the cluster. +* Tests were created using [TestNG](https://testng.org/doc/), a long-obsolete + test framework. +* A `IntegrationTestingConfig` is created from system properties (passed in from + Maven via `-D=` options). +* A TestNG test runner uses a part of the Druid Guice configuration to inject + test objects into the tests. +* The test then runs. + +To minimize test changes, we try to keep much of the "interface" while changing +the "implementation". Basically: + +* The same Docker image is used for all tests. +* Each test defines its own test cluster using Docker Compose. +* Tests are grouped into categories, represented by [JUnit categories]( + https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html). +* Maven runs one selected category, starting and stopping the test-specific cluster + for each. +* A cluster-specific directory contains the `docker-compose.yaml` file that defines + that cluster. Each of these files imports from common definitions. +* Each test is annotated with the `DruidTestRunner` to handle initialization, and + JUnit `Category` to group the test into a category. +* Categories can share cluster configuration to reduce redundant definitions. +* A `docker.yaml` file defines the test configuration and creates the + `IntegrationTestingConfig` object. +* Tests run as JUnit tests. + +The remainder of this section describes the test internals. + +## Test Name + +Due to the way the [Failsafe]( +https://maven.apache.org/surefire/maven-failsafe-plugin/integration-test-mojo.html) +Maven plugin works, it will look for ITs with +names of the form "IT*.java". This is the preferred form for Druid ITs. That is, +name your test "ITSomething", not "SomethingTest" or "IntegTestSomething", etc. +Many tests are called "ITSomethingTest", but this is a bit repetitious and redundant +since "IT" stands for "Integration Test". + +## Cluster Configuration + +A test must have a [cluster configuration](compose.md) to define the cluster. +There is a many-to-one relationship between test categories and test clusters. + +## Test Configuration + +See [Test Configuration](test-config.md) for details on the `docker.yaml` file +that you create for each test module to tell the tests about the cluster you +have defined. + +Test configuration allows inheritance so, as in Docker Compose, we define +standard bits in one place, just providing test-specific information in each +tests `docker.yaml` file. + +The test code assumes that the test configuration file is in +`src/test/resources/cluster//docker.yaml`, where `` is +the test category. The test runner loads the configuration file into +(or, specifically that it is on the class path at `/yaml/docker.yaml`) +a `ClusterConfig` instance. + +The `ClusterConfig` instance provides the backward-compatible +`IntegrationTestingConfig` instance tha that most existing test cases use. +New tests may want to work with `ClusterConfig` directly as the older interface +is a bit of a muddle in several areas. + +## Test Category + +Each test is associated with a cluster definition. Maven starts the required +cluster, runs a group of tests, and shuts down the cluster. We use the JUnit +`Category` to identify the category for each test: + +```java +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITIndexerTest extends AbstractITBatchIndexTest +{ + ... +``` + +The category is a trivial class that exists just to provide the category name. +It can also hold annotations, which will use in a moment. When adding tests, use +and existing category, or define a new one if you want your tests to run in +parallel with other categories. + +The `test-cases` module contains all integration tests. However, +Maven can run only one category per Maven run. You specify the category using a +profile of the same name, but with "IT-" prefixed. Thus the Maven profile for the +above `BatchIndex` category is `IT-BatchIndex`. + +Test categories may share the same cluster definition. We mark this by adding an +annotation to the category (_not_ test) class. The test class itself: + +```java +@RunWith(DruidTestRunner.class) +@Category(InputFormat.class) +public class ITLocalInputSourceAllInputFormatTest extends AbstractLocalInputSourceParallelIndexTest +{ + ... +``` + +The test category class: + +```java +@Cluster(BatchIndex.class) +public class InputFormat +{ +} +``` + +This says that the test above is in the `InputFormat` category, and tests in that +category use the same cluster definition as the `BatchIndex` category. Specifically, +to look for the cluster definition in the `BatchIndex` folders. + +### Defined Categories + +At present, the following test categories are fully or partly converted: + +| Category | Test NG Group | Description | +| -------- | ------------- | ----------- | +| HighAvailability | high-availability | Cluster failover tests | +| BatchIndex | batch-index | Batch indexing tsets | +| InputFormat | input-format | Input format tests | + +The new names correspond to class names. The Test NG names were strings. + +## Test Runner + +The ITs are JUnit test, but use a special test runner to handle configuration. +Test configuration is complex. The easiest way to configure, once the configuration +files are set, is to use the `DruidTestRunner` class: + +```java +@RunWith(DruidTestRunner.class) +@Category(MyCategory.class) +public class MyTest +{ + @Inject + private SomeObject myObject; + ... + + @Test + public void myTest() + { + ... +``` + +The test runner loads the configuration files, configures Guice, starts the +Druid lifecycle, and injects the requested values into the class each time +a test method runs. For simple tests, this is all you need. + +The test runner validates that the test has a category, and handles the +above mapping from category to cluster definition. + +### Parameterization + +The `DruidTestRunner` extends `JUnitParamsRunner` to allow parameterized tests. +This class stays discretely out of the way if you don't care about parameters. +To use parameters, see the `CalciteJoinQueryTest` class for an example. + +## Initialization + +The JUnit-based integration tests are designed to be as simple as possible +to debug. Each test class uses annotations and configuration files to provide +all the information needed to run a test. Once the customer is started +(using `cluster.sh` as described [here](quickstart.md)), each test can +be run from the command line or IDE with no additional command-line parameters. +To do that, we use a `docker.yaml` configuration file that defines all needed +parameters, etc. + +A test needs both configuration and a Guice setup. The `DruidTestRunner` , +along with a number ofm support classes, mostly hide the details from the tests. +However, you should know what's being done so you can debug. + +* JUnit uses the annotation to notice that we've provided a custom + test runner. (When converting tests, remember to add the required + annotation.) +* JUnit calls the test class constructor one or more times per test class. +* On the first creation of the test class, `DruidTestRunner` creates an + instance of the `Initializer` class, via its `Builder` to + load test configuration, create the Guice injector, + inject dependencies into the class instanance, and + start the Druid lifecycle. +* JUnit calls one of the test methods in the class. +* On the second creation of the test class in the same JVM, `DruidTestRunner` + reuses the existing injector to inject dependencies into the test, + which avoids the large setup overhead. +* During the first configuration, `DruidTestRunner` causes initialization + to check the health of each service prior to starting the tests. +* The test is now configured just as it would be from TestNG, and is ready to run. +* `DruidTestRunner` ends the lifecycle after the last test within this class runs. + +See [this explanation](dependencies.md) for the gory details. + +`DruidTestRunner` loads the basic set of Druid modules to run the basic client +code. Tests may wish to load additional modules specific to that test. + +## Custom Configuration + +There are times when a test needs additional Guice modules beyond what the +`Initializer` provides. In such cases, you can add a method to customize +configuration. + +### Guice Modules + +If your test requires additional Guice modules, add them as follows: + +```java +@Configure +public static void configure(Initializer.Builder builder) +{ + builder.modules( + new MyExtraModule(), + new AnotherModule() + ); +} +``` + +### Properties + +Druid makes heavy use of properties to configure objects via the 'JsonConfigProvider` +mechanism. Integration tests don't read the usual `runtime.properties` files: there +is no such file to read. Instead, properties are set in the test configuration +file. There are times, however, when it makes more sense to hard-code a property +value. This is done in the `@Configure` method: + +```java + builder.property(key, value); +``` + +You can also bind a property to an environment variable. This value is used when +the environment variable is set. You should also bind a default value: + +```java + builder.property("druid.my.property", 42); + builder.propertyEnvVarBinding("druid.my.property", "ULTIMATE_ANSWER"); +``` + +A property can also be passed in as either a system property or an environment +variable of the "Docker property environment variable form": + +```bash +druid_property_a=foo +./it.sh Category test +``` + +Or, directly on the command line: + +```text +-Ddruid_property_b=bar +``` + +Property precedence is: + +* Properties set in code, as above. +* Properties from the configuration file. +* Properties bound to environment variables, and the environment variable is set. +* Properties from the command line. + +The test properties can also be seen as default values for properties provided +in config files or via the command line. + +## Resolving Lifecycle Issues + +If your test get the dreaded "it doesn't work that way" message, it means that +an injected property in your test is asking Guice to instantiate a lifecycle-managed +class after the lifecycle itself was started. This typically happens if the class +in question is bound via the polymorphic `PolyBind` mechanism which doesn't support +"eager singletons". (If the class in question is not created via `PolyBind`, change +its Guice binding to include `.asEagerSingleton()` rather than `.as(LazySingleton.class)`. +See [this reference](https://github.com/google/guice/wiki/Scopes#eager-singletons). + +A quick workaround is to tell the initializer to create an instance before the +lifecycle starts. The easy way to do that is simply to inject the object into a +field in your class. Otherwise, give the builder a hint: + +```java + builder.eagerInstance(ThePeskyComponent.class); +``` + +## Test Operation + +When working with tests, it is helpful to know a bit more about the "magic" +behind `DruidTestRunner`. + +Druid's code is designed to run in a server, not a client. Yet, the tests are +clients. This means that tests want to run code in a way that it was not +intended to be run. The existing ITs have mostly figured out how to make that +happen, but result is not very clean. This is an opportunity for improvement. + +Druid introduced a set of "injector builders" to organize Guice initialization +a bit. The builders normally build the full server Guice setup. For the ITs, +the builders also allow us to pick and choose which modules to use to define +a client. The `Initializer` class in `it-base` uses the injector builders to +define the "client" modules needed to run tests. + +Druid uses the `Lifecycle` class to start and stop services. For this to work, +the managed instance must be created *before* the lifecycle starts. There are +a few items that are lazy singletons. When run in the server, they work fine. +But, when run in tests, we run into a race condition: we want to start the +lifecycle once before the tests start, the inject dependencies into each test +class instance as tests run. But, those injections create the insteance we want +the lifecycle to manage, resulting in a muddle. This is why the `DruidTestRunner` +has that odd "first test. vs. subsequent test" logic. + +The prior ITs would start running tests immediately. But, it can take up to a +minute or more for a Druid cluster to stabilize as all the services start +running simultaneously. The previous ITs would use a generic retry up to 240 +times to work around the fact that any given test could fail due to the cluster +not being ready. This version does that startup check as part if `DruidTestRunner`. +By the time the tests run, the cluster is up and has reported itself healthy. +That is, your tests can assume a healthy cluster. If a test fails: it indicates +an actual error or race condition. + +Specifically, if tests still randomly fail, those tests are telling you something: something +in Druid itself is non-deterministic (such as the delay to see changes to the DB, etc.), +or the tests are making invalid assumptions such as assuming an ordering when there +is none, using a time delay to try to synchronize actions when there should be +some specific synchronization, etc. This means that, in general, you should avoid +the use of the generic retry facility: if you have to retry to get your tests to +work, then the Druid user has to also retry. Unless we document the need to retry +in the API documentation, then having to retry should be considered a bug to be fixed +(perhaps by documenting the need to retry, perhaps by fixing a bug, perhaps by adding +a synchronization API.) + +Another benefit of the startup check is that the startup and health-check costs are +paid once per test class. This allows you to structure your +tests as a large number of small tests rather than a few big tests. + +## `ClusterConfig` and `ResolvedClusterConfig` + +The `ClusterConfig` class is the Java representation of the +[test configuration](test-config.md). The instance is available from the +`Initializer` and by Guice injection. + +It is a Jackson-serialized class that handles the "raw" form of +configuration. + +The `ClusterConfig.resolve()` method expands includes, applies defaults, +validates values, and returns a `ResolvedClusterConfig` instance used +by tests. `ResolvedClusterConfig` is available via Guice injection. +In most cases, however, you'll use it indirecty via the various clients +described below. Each of those uses `IntegrationTestingConfig` class, an +instance of which is created to read from `ResolvedClusterConfig`. + +Remember that each host has two names and two ports: + +* The external (or "proxy") host and port, as seen by the machine running + the tests. +* The internal host and port, as seen by the service itself running + in the Docker cluster. + +The various [config files](test-config.md) provide configurations for +the Docker, K8s and local cluster cases. This means that `resolveProxyHost()` +will resolve to the proxy for Docker, but the actual host for a local cluster. + +The original test setup was designed before Druid introduced the router. +A good future improvement is to modify the code to use the router to do the +routing rather than doing it "by hand" in the tests. This means that each +test would use the router port and router API for things like the Overlord +and Coordinator. Then, configuration need only specify the router, not the +other services. + +It is also possible to use Router APIs to obtain the server list dynamically +rather than hard-coding the services and ports. If we find cases where tests +must use the APIs directly, then we could either extend the Router API or +implement client-side service lookup. + +## `ClusterClient` + +The integration tests make many REST calls to the Druid cluster. The tests +contain much copy/paste code to make these calls. The `ClusterClient` class +is intended to gather up these calls so we have a single implementation +rather than many copies. Add methods as needed for additional APIs. + +The cluster client is "test aware": it uses the information in +`ClusterConfig` to know how to send the requested API. The methods handle +JSON deserialization, so tests can focus simply on making a call and +checking the results. + +## `org.apache.druid.testing.clients` + +This package in `integration-tests` has clients for most other parts of +Druid. For example, `CoordinatorResourceTestClient` is a +client for Coordinator calls. These clients are also aware of the test +configuration, by way of the `IntegrationTestingConfig` class, an +instance of which is created to read from `ResolvedClusterConfig`. diff --git a/integration-tests-ex/docs/travis.md b/integration-tests-ex/docs/travis.md new file mode 100644 index 00000000000..f4b454b6df2 --- /dev/null +++ b/integration-tests-ex/docs/travis.md @@ -0,0 +1,168 @@ + + +# Travis Integration + +Apache Druid uses Travis to manage builds, including running the integration +tests. You can find the Travis build file at `$DRUID_DEV/.travis.yml`, where +`DRUID_DEV` is the root of your Druid development directory. Information +about Travis can be found at: + +* [Documentation](https://docs.travis-ci.com/) +* [Job lifecycle](https://docs.travis-ci.com/user/job-lifecycle/) +* [Environment variables](https://docs.travis-ci.com/user/environment-variables/) +* [Travis file reference](https://config.travis-ci.com/) +* [Travis YAML](https://docs.travis-ci.com/user/build-config-yaml) + +## Running ITs In Travis + +Travis integration is still experimental. The latest iteration is: + +```yaml + - name: "experimental docker tests" + stage: Tests - phase 1 + script: ${MVN} install -P test-image,docker-tests -rf :it-tools ${MAVEN_SKIP} -DskipUTs=true + after_failure: + - docker-tests/check-results.sh +``` + +The above is a Travis job definition. The job "inherits" an `install` task defined +earlier in the file. That install task builds all of Druid and creates the distribution +tarball. Since the tests are isolated in specialized Maven profiles, the `install` +task does not build any of the IT-related artifacts. + +We've placed the test run in "Phase 1" for debugging convenience. Later, the tests +will run in "Phase 2" along with the other ITs. Once conversion is complete, the +"previous generation" IT tests will be replaced by the newer revisions. + +The `script` runs the ITs. The components of the command line are: + +* `install` - Run Maven though the install [lifecycle phase]( + https://maven.apache.org/guides/introduction/introduction-to-the-lifecycle.html) + for each module. This allows us to build and install the "testing tools" + (see the [Maven notes](maven.md)). The test image is also built during the + `install` phase. The tests themselves only need the `verify` phase, which occurs + before `install`. `install` does nothing for ITs. +* `-P test-image,docker-tests` - activates the image to build the image + (`test-image`) and then runs the ITs (`docker-tests`). +* `-rf :it-tools` - The `it-tools` module is the first of the IT modules: it contains + the "testing tools" added into the image. Using `-rf` skips all the other projects + which we already built in the Travis `install` step. Doing so saves the time + otherwise required for Maven to figure out it has nothing to do for those modules. +* `${MAVEN_SKIP}` - Omits the static checks: they are not needed for ITs. +* `-DskipUTs=true` - The ITs use the [Maven Failsafe plugin]( + https://maven.apache.org/surefire/maven-failsafe-plugin/index.html) + which shares code with the [Maven Surefire plugin]( + https://maven.apache.org/surefire/maven-surefire-plugin/index.html). We don't want + to run unit tests. If we did the usual `-DskipTests`, then we'd also disable the + ITs. The `-DskipUTs=true` uses a bit of [Maven trickery]( + https://stackoverflow.com/questions/6612344/prevent-unit-tests-but-allow-integration-tests-in-maven) + to skip only the Surefire, but not Faisafe tests. + +## Travis Diagnostics + +A common failure when running ITs is that they uncover a bug in a Druid service; +typically in the code you added that you want to test. Or, if you are changing the +Docker or Docker Compose infratructure, then the tests will often fail because the +Druid services are mis-configured. (Bad configuration tends to result in services +that don't start, or start and immediately exit.) + +The standard way to diagnose such failures is to look at the Druid logs. However, +Travis provides no support for attaching files to a build. The best alternative +seems to be to upload the files somewhere else. As a compromise, the Travis build +will append to the build log a subset of the Druid logs. + +Travis has a limit of 4MB per build log, so we can't append the entire log for +every Druid service for every IT. We have to be selective. In most cases, we only +care about the logs for ITs that fail. + +Now, it turns out to be *very hard* indeed to capture failues! Eventually, we want +Maven to run many ITs for each test run: we need to know which failed. Each IT +creates its own "shared" directory, so to find the logs, we need to know which IT +failed. Travis does not have this information: Travis only knows that Maven itself +exited with a non-zero status. Maven doesn't know: it only knows that Failsafe +failed the build. Failsafe is designed to run all ITs, then check the results in +the `verify` state, so Maven doesn't even know about the failures. + +### Failsafe Error Reports + +To work around all this, we mimic Failsafe: we look at the Failsafe error report +in `$DRUID_DEV/docker-tests//target/failsafe-reports/failsafe-summary.xml` +which looks like this: + +```xml + + 3 + 1 + 0 + 0 + + +``` + +The above shows one error and no failures. A successful run will show 0 for the +`errors` tag. This example tells us "something didn't work". The corresponding +Druid service logs are candidates for review. + +### Druid Service Failures + +The Druid logs are in `$DRUID_DEV/docker-tests//target/shared/logs`. +We could append all of them, but recall the 4MB limit. We generally are +interested only in those services that failed. So, we look at the logs and +see that a successful run is indicated by a normal Lifecycle shutdown: + +```text +2022-04-16T20:54:37,997 INFO [Thread-56] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [INIT] +``` + +The key bit of text is: + +```text +Stopping lifecycle [module] stage [INIT] +``` + +This says that 1) we're shutting down the lifecycle (which means no exception was thrown), +and 2) that we got all the way to the end (`[INIT]`). Since Druid emits no final +"exited normally" message, we take the above as the next-best thing. + +So, we only care about logs that *don't* have the above line. For those, we want to +append the log to the build output. Or, because of the size limit, we append the +last 100 lines. + +All of this is encapsulated in the `docker-tests/check-results.sh` script which +is run if the build fails (in the `after_failure`) tag. + +### Druid Log Output + +For a failed test, the build log will end with something like this: + +```text +======= it-high-availability Failed ========== +broker.log logtail ======================== +022-04-16T03:53:10,492 INFO [CoordinatorRuleManager-Exec--0] org.apache.druid.discovery.DruidLeaderClient - Request[http://coordinator-one:8081/druid/coordinator/v1/rules] received redirect response to location [http://coordinator-two:8081/druid/coordinator/v1/rules]. +... +``` + +To keep below the limit, on the first failed test is reported. + +The above won't catch all cases: maybe the service exited normally, but might still have +log lines of interest. Since all tests run, those lines could be anywhere in the file +and the scripts can't know which might be of interest. To handle that, we either +have to upload all logs somewhere, or you can use the convenience of the new +IT framework to rerun the tests on your development machine. diff --git a/integration-tests-ex/it-image/build-image.sh b/integration-tests-ex/image/build-image.sh similarity index 100% rename from integration-tests-ex/it-image/build-image.sh rename to integration-tests-ex/image/build-image.sh diff --git a/integration-tests-ex/it-image/docker-build.sh b/integration-tests-ex/image/docker-build.sh similarity index 100% rename from integration-tests-ex/it-image/docker-build.sh rename to integration-tests-ex/image/docker-build.sh diff --git a/integration-tests-ex/it-image/docker/Dockerfile b/integration-tests-ex/image/docker/Dockerfile similarity index 100% rename from integration-tests-ex/it-image/docker/Dockerfile rename to integration-tests-ex/image/docker/Dockerfile diff --git a/integration-tests-ex/it-image/docker/druid.sh b/integration-tests-ex/image/docker/druid.sh similarity index 100% rename from integration-tests-ex/it-image/docker/druid.sh rename to integration-tests-ex/image/docker/druid.sh diff --git a/integration-tests-ex/it-image/docker/launch.sh b/integration-tests-ex/image/docker/launch.sh similarity index 76% rename from integration-tests-ex/it-image/docker/launch.sh rename to integration-tests-ex/image/docker/launch.sh index 4b8d0293c8c..1f64b4e14df 100644 --- a/integration-tests-ex/it-image/docker/launch.sh +++ b/integration-tests-ex/image/docker/launch.sh @@ -33,14 +33,38 @@ cd / # TODO: enable only for security-related tests? #/tls/generate-server-certs-and-keystores.sh -. /druid.sh # The image contains both the MySQL and MariaDB JDBC drivers. # The MySQL driver is selected by the Docker Compose file. # Set druid.metadata.mysql.driver.driverClassName to the preferred # driver. +# Test-specific way to define extensions. Compose defines two test-specific +# variables. We combine these to create the final form converted to a property. +if [ -n "$druid_extensions_loadList" ]; then + echo "Using the provided druid_extensions_loadList=$druid_extensions_loadList" +else + mkdir -p /tmp/conf + EXTNS_FILE=/tmp/conf/extns + echo $druid_standard_loadList | tr "," "\n" > $EXTNS_FILE + if [ -n "$druid_test_loadList" ]; then + echo $druid_test_loadList | tr "," "\n" >> $EXTNS_FILE + fi + druid_extensions_loadList="[" + delim="" + while read -r line; do + druid_extensions_loadList="$druid_extensions_loadList$delim\"$line\"" + delim="," + done < $EXTNS_FILE + export druid_extensions_loadList="${druid_extensions_loadList}]" + unset druid_standard_loadList + unset druid_test_loadList + rm $EXTNS_FILE + echo "Effective druid_extensions_loadList=$druid_extensions_loadList" +fi + # Create druid service config files with all the config variables +. /druid.sh setupConfig # Export the service config file path to use in supervisord conf file @@ -92,7 +116,9 @@ fi LOG_FILE=$LOG_DIR/${INSTANCE_NAME}.log echo "" >> $LOG_FILE -echo "--- Service runtime.properties ---" >> $LOG_FILE +echo "--- env ---" >> $LOG_FILE +env >> $LOG_FILE +echo "--- runtime.properties ---" >> $LOG_FILE cat $DRUID_SERVICE_CONF_DIR/*.properties >> $LOG_FILE echo "---" >> $LOG_FILE echo "" >> $LOG_FILE diff --git a/integration-tests-ex/it-image/docker/test-setup.sh b/integration-tests-ex/image/docker/test-setup.sh similarity index 100% rename from integration-tests-ex/it-image/docker/test-setup.sh rename to integration-tests-ex/image/docker/test-setup.sh diff --git a/integration-tests-ex/it-image/pom.xml b/integration-tests-ex/image/pom.xml similarity index 98% rename from integration-tests-ex/it-image/pom.xml rename to integration-tests-ex/image/pom.xml index cd0d4e4e272..e352aa41793 100644 --- a/integration-tests-ex/it-image/pom.xml +++ b/integration-tests-ex/image/pom.xml @@ -38,8 +38,9 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven pom - it-image - it-image + org.apache.druid.integration-tests + druid-it-image + druid-it-image Build the Docker image for integration tests. @@ -199,15 +200,15 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven - ${project.version} ${mysql.version} - ${mysql.image.version} ${mariadb.version} - ${apache.kafka.version} + com.mysql.jdbc.Driver + ${mysql.image.version} ${confluent-version} + ${apache.kafka.version} ${zookeeper.version} ${hadoop.compile.version} - com.mysql.jdbc.Driver + ${project.version} ${druid.it.image-name} ${project.build.directory} runtime diff --git a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java index f2ac334b913..de8970b20ee 100644 --- a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java +++ b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java @@ -23,6 +23,7 @@ import com.github.rvesse.airline.builder.CliBuilder; public class CustomNodeRoleCommandCreator implements CliCommandCreator { + @SuppressWarnings({ "rawtypes", "unchecked" }) @Override public void addCommands(CliBuilder builder) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java index e39c630cf28..8e3c5a83e8d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java @@ -221,7 +221,7 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide overlordTwoTLSUrl = StringUtils.format("https://%s:%s", overlordTwoHost, props.get("overlord_two_tls_port")); } } - + middleManagerHost = props.get("middlemanager_host"); zookeeperHosts = props.get("zookeeper_hosts"); @@ -259,7 +259,6 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide { return new IntegrationTestingConfig() { - @Override public String getCoordinatorUrl() { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java index 3ce1dd21f68..a815391fb22 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java @@ -30,4 +30,5 @@ import com.google.inject.Provider; }) public interface IntegrationTestingConfigProvider extends Provider { + String PROPERTY_BASE = "druid.test.config"; } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java index 3df3d8a49d5..e3f2a98dc63 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java @@ -323,6 +323,8 @@ public class OverlordResourceTestClient { TaskState status = getTaskStatus(taskID).getStatusCode(); if (status == TaskState.FAILED) { + LOG.error("Task failed: %s", taskID); + LOG.error("Message: %s", getTaskErrorMessage(taskID)); throw new ISE("Indexer task FAILED"); } return status == TaskState.SUCCESS; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java index 1e239ae57db..b0bef034083 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java @@ -51,7 +51,7 @@ public class DruidTestModule implements Module binder.bind(IntegrationTestingConfig.class) .toProvider(IntegrationTestingConfigProvider.class) .in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.test.config", IntegrationTestingConfigProvider.class); + JsonConfigProvider.bind(binder, IntegrationTestingConfigProvider.PROPERTY_BASE, IntegrationTestingConfigProvider.class); binder.bind(CuratorConfig.class).to(IntegrationTestingCuratorConfig.class); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index 0c88619bc12..1bb6e8e12c8 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -165,6 +166,9 @@ public abstract class AbstractIndexerTest public static String getResourceAsString(String file) throws IOException { try (final InputStream inputStream = getResourceAsStream(file)) { + if (inputStream == null) { + throw new ISE("Failed to load resource: [%s]", file); + } return IOUtils.toString(inputStream, StandardCharsets.UTF_8); } } diff --git a/it.sh b/it.sh new file mode 100755 index 00000000000..6cf0b54a631 --- /dev/null +++ b/it.sh @@ -0,0 +1,130 @@ +#! /bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#-------------------------------------------------------------------- + +# Utility script for running the new integration tests, since the Maven +# commands are unwieldy. + +export DRUID_DEV=$(cd $(dirname $0) && pwd) + +function usage +{ + cat < + start the cluster for category + down + stop the cluster for category + test + start the cluster, run the test for category, and stop the cluster + tail + show the last 20 lines of each container log + travis + run one IT in Travis (build dist, image, run test, tail logs) + prune + prune Docker volumes +EOF +} + +function tail_logs +{ + category=$1 + cd integration-tests-ex/cases/target/$category/logs + ls *.log | while read log; + do + echo "----- $category/$log -----" + tail -20 $log + done +} + +CMD=$1 +shift +MAVEN_IGNORE="-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true" + +case $CMD in + "help" ) + usage + ;; + "build" ) + mvn clean package -P dist $MAVEN_IGNORE -T1.0C + ;; + "dist" ) + mvn package -P dist $MAVEN_IGNORE -pl :distribution + ;; + "image" ) + cd $DRUID_DEV/integration-tests-ex/image + mvn install -P test-image $MAVEN_IGNORE + ;; + "up" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + ./cluster.sh up $1 + ;; + "down" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + ./cluster.sh down $1 + ;; + "test" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + mvn verify -P skip-static-checks,docker-tests,IT-$1 \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases + ;; + "tail" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + tail_logs $1 + ;; + "travis" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + $0 dist + $0 image + $0 test $1 + $0 tail $1 + ;; + "prune" ) + # Caution: this removes all volumes, which is generally what you + # want when testing. + docker system prune --volumes + ;; + * ) + usage + exit -1 + ;; +esac diff --git a/pom.xml b/pom.xml index 1eb1979a31b..7dede645166 100644 --- a/pom.xml +++ b/pom.xml @@ -214,8 +214,9 @@ distribution - integration-tests-ex/it-tools - integration-tests-ex/it-image + integration-tests-ex/tools + integration-tests-ex/image + integration-tests-ex/cases @@ -1205,7 +1206,8 @@ org/apache/druid/benchmark/**/* org/apache/druid/**/*Benchmark* org/apache/druid/testing/**/* - org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.class org/apache/druid/guice/SleepModule.class @@ -1523,7 +1525,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M6 + 3.0.0-M7 @@ -1878,10 +1880,11 @@ conf/** docker/*.conf docker/service-supervisords/*.conf - target/** + **/target/** licenses/** - **/test/resources/** - **/data/data/** + **/test/resources/**/* + **/docker/client_tls/* + resources/data/**/* **/derby.log **/jvm.config **/*.avsc @@ -1901,14 +1904,17 @@ .github/pull_request_template.md .github/dependabot.yml git.version - node_modules/** + website/node_modules/** src/**/*.snap examples/conf/** .asf.yaml **/dependency-reduced-pom.xml .editorconfig **/hadoop.indexer.libs.version - **/codegen/** + **/codegen/**/* + **/.settings/**/* + **/.classpath + **/.project diff --git a/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java b/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java index b9dcb7f3053..81e28431e2d 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java @@ -87,13 +87,27 @@ public class DruidInjectorBuilder /** * Add an arbitrary set of modules. - * - * @see #add(Object) */ public DruidInjectorBuilder add(Object...input) { for (Object o : input) { - add(o); + addInput(o); + } + return this; + } + + public DruidInjectorBuilder addModules(Module...inputs) + { + for (Object o : inputs) { + addInput(o); + } + return this; + } + + public DruidInjectorBuilder addAll(List inputs) + { + for (Object o : inputs) { + addInput(o); } return this; } @@ -106,11 +120,9 @@ public class DruidInjectorBuilder * modules have visibility only to objects defined in the base * injector, but not to objects defined in the injector being built. */ - public DruidInjectorBuilder add(Object input) + public DruidInjectorBuilder addInput(Object input) { - if (input instanceof DruidModule) { - return addDruidModule((DruidModule) input); - } else if (input instanceof Module) { + if (input instanceof Module) { return addModule((Module) input); } else if (input instanceof Class) { return addClass((Class) input); @@ -119,23 +131,15 @@ public class DruidInjectorBuilder } } - public DruidInjectorBuilder addDruidModule(DruidModule module) - { - if (!acceptModule(module.getClass())) { - return this; - } - baseInjector.injectMembers(module); - registerJacksonModules(module); - modules.add(module); - return this; - } - public DruidInjectorBuilder addModule(Module module) { if (!acceptModule(module.getClass())) { return this; } baseInjector.injectMembers(module); + if (module instanceof DruidModule) { + registerJacksonModules((DruidModule) module); + } modules.add(module); return this; } diff --git a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java index ccff6dd2870..4a508826f2b 100644 --- a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java @@ -31,8 +31,8 @@ public class MetadataConfigModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.metadata.storage.tables", MetadataStorageTablesConfig.class); - JsonConfigProvider.bind(binder, "druid.metadata.storage.connector", MetadataStorageConnectorConfig.class); + JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class); + JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segments", SegmentsMetadataManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); diff --git a/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java index f030f62bc72..08614e2979f 100644 --- a/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java @@ -39,7 +39,7 @@ public class ExtensionInjectorBuilder extends DruidInjectorBuilder this.serviceBuilder = serviceBuilder; ExtensionsLoader extnLoader = ExtensionsLoader.instance(baseInjector); for (DruidModule module : extnLoader.getFromExtensions(DruidModule.class)) { - addDruidModule(module); + addModule(module); } } diff --git a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java index 9f694e45755..e20d64da51e 100644 --- a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java @@ -109,11 +109,11 @@ public abstract class GuiceRunnable implements Runnable } log.info( - "Starting up with processors[%,d], memory[%,d], maxMemory[%,d]%s. Properties follow.", + "Starting up with processors [%,d], memory [%,d], maxMemory [%,d]%s. Properties follow.", JvmUtils.getRuntimeInfo().getAvailableProcessors(), JvmUtils.getRuntimeInfo().getTotalHeapSizeBytes(), JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), - directSizeBytes != null ? StringUtils.format(", directMemory[%,d]", directSizeBytes) : "" + directSizeBytes != null ? StringUtils.format(", directMemory [%,d]", directSizeBytes) : "" ); if (startupLoggingConfig.isLogProperties()) {