diff --git a/.github/workflows/reusable-revised-its.yml b/.github/workflows/reusable-revised-its.yml index d9237a52abab..1aa29f64cb5b 100644 --- a/.github/workflows/reusable-revised-its.yml +++ b/.github/workflows/reusable-revised-its.yml @@ -57,6 +57,19 @@ on: AWS_SECRET_ACCESS_KEY: required: false type: string + BACKWARD_COMPATIBILITY_IT_ENABLED: + required: false + type: string + default: false + DRUID_PREVIOUS_VERSION: + required: false + type: string + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: + required: false + type: string + DRUID_PREVIOUS_IT_IMAGE_NAME: + required: false + type: string env: MYSQL_DRIVER_CLASSNAME: ${{ inputs.mysql_driver }} # Used by tests to connect to metadata store directly. @@ -106,6 +119,15 @@ jobs: ./druid-container-jdk${{ inputs.build_jdk }}.tar.gz ./integration-tests-ex/image/target/env.sh + - name: Retrieve previous version cached docker image + id: docker-restore-previous-version + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + uses: actions/cache/restore@v4 + with: + key: druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz-${{ github.sha }} + path: | + ./druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + - name: Maven build if: steps.maven-restore.outputs.cache-hit != 'true' || ( steps.docker-restore.outputs.cache-hit != 'true' && steps.targets-restore.outputs.cache-hit != 'true' ) run: | @@ -115,6 +137,10 @@ jobs: if: steps.docker-restore.outputs.cache-hit != 'true' || steps.maven-restore.outputs.cache-hit != 'true' env: docker-restore: ${{ toJson(steps.docker-restore.outputs) }} + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ inputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ inputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} run: | ./it.sh image source ./integration-tests-ex/image/target/env.sh @@ -122,6 +148,15 @@ jobs: echo $DRUID_IT_IMAGE_NAME docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}.tar.gz + - name: Save previous version docker image + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' && (steps.docker-restore.outputs.cache-hit != 'true' || steps.maven-restore.outputs.cache-hit != 'true') }} + env: + docker-restore: ${{ toJson(steps.docker-restore.outputs) }} + run: | + docker tag ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }}-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo ${DRUID_PREVIOUS_IT_IMAGE_NAME} + docker save "${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }}" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + - name: Stop and remove docker containers run: | echo "Force stopping all containers and pruning" @@ -133,9 +168,21 @@ jobs: docker load --input druid-container-jdk${{ inputs.build_jdk }}.tar.gz docker images + - name: Load previous version docker image + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + run: | + docker load --input druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + docker images + - name: Run IT id: run-it - run: ${{ inputs.script }} + env: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ inputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ inputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} + run: | + ${{ inputs.script }} - name: Collect docker logs on failure if: ${{ failure() && steps.run-it.conclusion == 'failure' }} diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 069562bf7bd3..412a3e469737 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -18,6 +18,24 @@ name: "Revised ITs workflow" on: workflow_call: + inputs: + BACKWARD_COMPATIBILITY_IT_ENABLED: + description: "Flag for backward compatibility IT" + required: false + default: false + type: string + DRUID_PREVIOUS_VERSION: + description: "Previous druid versions to run the test against." + required: false + type: string + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: + description: "URL to download the previous druid version." + required: false + type: string + DRUID_PREVIOUS_IT_IMAGE_NAME: + description: "Druid previous version image name." + required: false + type: string workflow_dispatch: jobs: @@ -79,3 +97,24 @@ jobs: AWS_REGION: us-east-1 AWS_ACCESS_KEY_ID: admin AWS_SECRET_ACCESS_KEY: miniopassword + + backward-compatibility-it: + needs: changes + uses: ./.github/workflows/reusable-revised-its.yml + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' && (needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true') }} + with: + build_jdk: 8 + runtime_jdk: 8 + use_indexer: middleManager + script: ./it.sh github BackwardCompatibilityMain + it: BackwardCompatibilityMain + mysql_driver: com.mysql.jdbc.Driver + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ inputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ inputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} + DRUID_CLOUD_BUCKET: druid-qa + DRUID_CLOUD_PATH: aws-${{ github.run_id }}-${{ github.run_attempt }} + AWS_REGION: us-east-1 + AWS_ACCESS_KEY_ID: admin + AWS_SECRET_ACCESS_KEY: miniopassword diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 9651a56b8cb7..544787e79a3c 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -49,7 +49,30 @@ env: SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 jobs: + set-env-var: + name: Set env var + runs-on: ubuntu-latest + outputs: + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ steps.image_name.outputs.image_name }} + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ steps.it_enabled.outputs.enabled }} + DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + steps: + - name: Set image name env var + id: image_name + run: | + echo "::set-output name=image_name::org.apache.druid.integration-tests/test:${{ env.DRUID_PREVIOUS_VERSION }}" + - name: Set env for enabling backward compatibility it + id: it_enabled + run: | + if [ -n "${{ env.DRUID_PREVIOUS_VERSION }}" ]; then + echo "::set-output name=enabled::true" + else + echo "::set-output name=enabled::false" + fi + build: + needs: set-env-var name: "build (jdk${{ matrix.jdk }})" strategy: fail-fast: false @@ -94,12 +117,25 @@ jobs: ./druid-container-jdk${{ matrix.jdk }}.tar.gz ./integration-tests-ex/image/target/env.sh + - name: Cache previous version image + id: docker_container_previous_version + uses: actions/cache@v4 + with: + key: druid-container-jdk${{ matrix.jdk }}-version${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }}.tar.gz-${{ github.sha }} + path: | + ./druid-container-jdk${{ matrix.jdk }}-version${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }}.tar.gz + - name: Maven build id: maven_build run: | ./it.sh ci - name: Container build + env: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ needs.set-env-var.outputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} run: | ./it.sh image source ./integration-tests-ex/image/target/env.sh @@ -111,6 +147,13 @@ jobs: echo $DRUID_IT_IMAGE_NAME docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz + - name: Save previous version docker image + if: ${{ needs.set-env-var.outputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + run: | + docker tag ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }}-jdk${{ matrix.jdk }}-version${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }} + echo ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} + docker save "${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }}" | gzip > druid-container-jdk${{ matrix.jdk }}-version${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }}.tar.gz + unit-tests-phase2: strategy: fail-fast: false @@ -142,6 +185,11 @@ jobs: uses: ./.github/workflows/standard-its.yml revised-its: - needs: unit-tests + needs: [unit-tests, set-env-var] if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml + with: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ needs.set-env-var.outputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ needs.set-env-var.outputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} diff --git a/.gitignore b/.gitignore index 0828365f2a72..adaf4d14c2bd 100644 --- a/.gitignore +++ b/.gitignore @@ -27,7 +27,7 @@ README .pmdruleset.xml .java-version integration-tests/gen-scripts/ -/bin/ +**/bin/ *.hprof **/.ipynb_checkpoints/ website/.yarn/ diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java index c3ebbe5122f9..ee1ec34dc3b0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; @@ -68,7 +68,7 @@ public class GroupByDeserializationBenchmark static { NullHandling.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); AggregatorsModule.registerComplexMetricsAndSerde(); } @@ -93,7 +93,7 @@ public class GroupByDeserializationBenchmark public void setup() throws JsonProcessingException { final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper(); - undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList()); + undecoratedMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); undecoratedMapper.registerModule(new AggregatorsModule()); final Pair sqlQueryAndResultRow = sqlQueryAndResultRow( numDimensions, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java index a864bcd3419f..a8b90e56eea1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java @@ -37,7 +37,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -85,7 +85,7 @@ public class FrameChannelMergerBenchmark { static { NullHandling.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final String KEY = "key"; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java index ce01324116f9..046da2574235 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/InPlanningBenchmark.java @@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -202,7 +203,8 @@ public void setup() throws JsonProcessingException new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); String prefix = ("explain plan for select long1 from foo where long1 in "); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 3441e31c0498..06ea2aa9e19d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -89,6 +89,7 @@ import org.apache.druid.sql.calcite.util.LookylooModule; import org.apache.druid.sql.calcite.util.QueryFrameworkUtils; import org.apache.druid.sql.calcite.util.testoperator.CalciteTestOperatorModule; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -674,7 +675,8 @@ public static Pair createSqlSystem( new CalciteRulesManager(ImmutableSet.of()), new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(injector)), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); return Pair.of(plannerFactory, engine); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index 8b2172182a07..d351b40f2dbf 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -59,6 +59,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -364,7 +365,8 @@ public void setup() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java index 80b6647a0eee..1f58f97b8e8a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlGroupByBenchmark.java @@ -24,7 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -57,6 +57,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -89,7 +90,7 @@ public class SqlGroupByBenchmark static { NullHandling.initializeForTests(); ExpressionProcessing.initializeForTests(); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final Logger log = new Logger(SqlGroupByBenchmark.class); @@ -331,7 +332,7 @@ public void setup() // Hacky and pollutes global namespace, but it is fine since benchmarks are run in isolation. Wasn't able // to work up a cleaner way of doing it by modifying the injector. - CalciteTests.getJsonMapper().registerModules(NestedDataModule.getJacksonModulesList()); + CalciteTests.getJsonMapper().registerModules(BuiltInTypesModule.getJacksonModulesList()); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); @@ -347,7 +348,8 @@ public void setup() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index c329e9da30ec..b896f8df52fc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -63,6 +63,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -402,7 +403,8 @@ public void setup() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index bffc03469e89..2150b2100fb0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -132,7 +133,8 @@ public void setup() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); groupByQuery = GroupByQuery .builder() diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 98c27c4b2b8c..87e92ab6fb1c 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -92,21 +92,12 @@ public void setup() final String dataSource = DATA_SOURCE_PREFIX + i; compactionConfigs.put( dataSource, - new DataSourceCompactionConfig( - dataSource, - 0, - inputSegmentSizeBytes, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + DataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withTaskPriority(0) + .withInputSegmentSizeBytes(inputSegmentSizeBytes) + .build() ); } diff --git a/dev/quidem b/dev/quidem index a68e08b3cfc1..1a7d3f0ec96a 100755 --- a/dev/quidem +++ b/dev/quidem @@ -28,13 +28,13 @@ OPTS+=" -Dsurefire.rerunFailingTestsCount=0" OPTS+=" -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugin.surefire.SurefirePlugin=INFO" [[ $@ =~ "-q" ]] && OPTS+=" -Dsurefire.trimStackTrace=true" -OPTS+=" -pl sql -Dtest=SqlQuidemTest" +OPTS+=" -pl quidem-ut -Dtest=QTest" OPTS+=" org.apache.maven.plugins:maven-surefire-plugin:test" case "$1" in -h|--help) cat << EOF -Run SqlQuidemTest tests. +Run QTest tests. -q quiet (recommened) -Dquidem.overwrite enables overwrite mode -Dquidem.filter=*join* runs only tests matching path expression diff --git a/docs/api-reference/sql-api.md b/docs/api-reference/sql-api.md index bf58d2364262..e1fb53bc6eba 100644 --- a/docs/api-reference/sql-api.md +++ b/docs/api-reference/sql-api.md @@ -629,10 +629,21 @@ Retrieves information about the query associated with the given query ID. The re - `sizeInBytes`: the size of the page. - `id`: the page number that you can use to reference a specific page when you get query results. +If the optional query parameter `detail` is supplied, then the response also includes the following: +- A `stages` object that summarizes information about the different stages being used for query execution, such as stage number, phase, start time, duration, input and output information, processing methods, and partitioning. +- A `counters` object that provides details on the rows, bytes, and files processed at various stages for each worker across different channels, along with sort progress. +- A `warnings` object that provides details about any warnings. + #### URL `GET` `/druid/v2/sql/statements/{queryId}` +#### Query parameters +* `detail` (optional) + * Type: Boolean + * Default: false + * Fetch additional details about the query, which includes the information about different stages, counters for each stage, and any warnings. + #### Responses @@ -672,7 +683,7 @@ The following example retrieves the status of a query with specified ID `query-9 ```shell -curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804" +curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true" ``` @@ -680,7 +691,7 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-4 ```HTTP -GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804 HTTP/1.1 +GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT ``` @@ -835,7 +846,422 @@ Host: http://ROUTER_IP:ROUTER_PORT "sizeInBytes": 375 } ] - } + }, + "stages": [ + { + "stageNumber": 0, + "definition": { + "id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_0", + "input": [ + { + "type": "table", + "dataSource": "wikipedia", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + ], + "filter": { + "type": "equals", + "column": "user", + "matchValueType": "STRING", + "matchValue": "BlueMoon2662" + }, + "filterFields": [ + "user" + ] + } + ], + "processor": { + "type": "scan", + "query": { + "queryType": "scan", + "dataSource": { + "type": "inputNumber", + "inputNumber": 0 + }, + "intervals": { + "type": "intervals", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + ] + }, + "virtualColumns": [ + { + "type": "expression", + "name": "v0", + "expression": "'BlueMoon2662'", + "outputType": "STRING" + } + ], + "resultFormat": "compactedList", + "limit": 1001, + "filter": { + "type": "equals", + "column": "user", + "matchValueType": "STRING", + "matchValue": "BlueMoon2662" + }, + "columns": [ + "__time", + "added", + "channel", + "cityName", + "comment", + "commentLength", + "countryIsoCode", + "countryName", + "deleted", + "delta", + "deltaBucket", + "diffUrl", + "flags", + "isAnonymous", + "isMinor", + "isNew", + "isRobot", + "isUnpatrolled", + "metroCode", + "namespace", + "page", + "regionIsoCode", + "regionName", + "v0" + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "enableWindowing": true, + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "33b53acb-7533-4880-a81b-51c16c489eab", + "scanSignature": "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"commentLength\",\"type\":\"LONG\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"deltaBucket\",\"type\":\"LONG\"},{\"name\":\"diffUrl\",\"type\":\"STRING\"},{\"name\":\"flags\",\"type\":\"STRING\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"STRING\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"STRING\"}]", + "sqlOuterLimit": 1001, + "sqlQueryId": "33b53acb-7533-4880-a81b-51c16c489eab", + "sqlStringifyArrays": false + }, + "columnTypes": [ + "LONG", + "LONG", + "STRING", + "STRING", + "STRING", + "LONG", + "STRING", + "STRING", + "LONG", + "LONG", + "LONG", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING", + "STRING" + ], + "granularity": { + "type": "all" + }, + "legacy": false + } + }, + "signature": [ + { + "name": "__boost", + "type": "LONG" + }, + { + "name": "__time", + "type": "LONG" + }, + { + "name": "added", + "type": "LONG" + }, + { + "name": "channel", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "comment", + "type": "STRING" + }, + { + "name": "commentLength", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "deleted", + "type": "LONG" + }, + { + "name": "delta", + "type": "LONG" + }, + { + "name": "deltaBucket", + "type": "LONG" + }, + { + "name": "diffUrl", + "type": "STRING" + }, + { + "name": "flags", + "type": "STRING" + }, + { + "name": "isAnonymous", + "type": "STRING" + }, + { + "name": "isMinor", + "type": "STRING" + }, + { + "name": "isNew", + "type": "STRING" + }, + { + "name": "isRobot", + "type": "STRING" + }, + { + "name": "isUnpatrolled", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "STRING" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "page", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + }, + { + "name": "v0", + "type": "STRING" + } + ], + "shuffleSpec": { + "type": "mix" + }, + "maxWorkerCount": 1 + }, + "phase": "FINISHED", + "workerCount": 1, + "partitionCount": 1, + "shuffle": "mix", + "output": "localStorage", + "startTime": "2024-07-31T15:20:21.255Z", + "duration": 103 + }, + { + "stageNumber": 1, + "definition": { + "id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_1", + "input": [ + { + "type": "stage", + "stage": 0 + } + ], + "processor": { + "type": "limit", + "limit": 1001 + }, + "signature": [ + { + "name": "__boost", + "type": "LONG" + }, + { + "name": "__time", + "type": "LONG" + }, + { + "name": "added", + "type": "LONG" + }, + { + "name": "channel", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "comment", + "type": "STRING" + }, + { + "name": "commentLength", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "deleted", + "type": "LONG" + }, + { + "name": "delta", + "type": "LONG" + }, + { + "name": "deltaBucket", + "type": "LONG" + }, + { + "name": "diffUrl", + "type": "STRING" + }, + { + "name": "flags", + "type": "STRING" + }, + { + "name": "isAnonymous", + "type": "STRING" + }, + { + "name": "isMinor", + "type": "STRING" + }, + { + "name": "isNew", + "type": "STRING" + }, + { + "name": "isRobot", + "type": "STRING" + }, + { + "name": "isUnpatrolled", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "STRING" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "page", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + }, + { + "name": "v0", + "type": "STRING" + } + ], + "shuffleSpec": { + "type": "maxCount", + "clusterBy": { + "columns": [ + { + "columnName": "__boost", + "order": "ASCENDING" + } + ] + }, + "partitions": 1 + }, + "maxWorkerCount": 1 + }, + "phase": "FINISHED", + "workerCount": 1, + "partitionCount": 1, + "shuffle": "globalSort", + "output": "localStorage", + "startTime": "2024-07-31T15:20:21.355Z", + "duration": 10, + "sort": true + } + ], + "counters": { + "0": { + "0": { + "input0": { + "type": "channel", + "rows": [ + 24433 + ], + "bytes": [ + 7393933 + ], + "files": [ + 22 + ], + "totalFiles": [ + 22 + ] + } + } + }, + "1": { + "0": { + "sortProgress": { + "type": "sortProgress", + "totalMergingLevels": -1, + "levelToTotalBatches": {}, + "levelToMergedBatches": {}, + "totalMergersForUltimateLevel": -1, + "triviallyComplete": true, + "progressDigest": 1 + } + } + } + }, + "warnings": [] } ``` diff --git a/docs/ingestion/kafka-ingestion.md b/docs/ingestion/kafka-ingestion.md index 5d14b1ad6baf..5a3ac7f29c41 100644 --- a/docs/ingestion/kafka-ingestion.md +++ b/docs/ingestion/kafka-ingestion.md @@ -254,7 +254,6 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `json` * `kafka` * `avro_stream` -* `avro_ocf` * `protobuf` You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. diff --git a/docs/ingestion/kinesis-ingestion.md b/docs/ingestion/kinesis-ingestion.md index fb4bfde235a0..3b4c5de8548d 100644 --- a/docs/ingestion/kinesis-ingestion.md +++ b/docs/ingestion/kinesis-ingestion.md @@ -143,7 +143,6 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `tvs` * `json` * `avro_stream` -* `avro_ocf` * `protobuf` You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index b5be94ba4ba7..ec97f44fe391 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -77,6 +77,8 @@ Most metric values reset each emission period, as specified in `druid.monitoring |`metadatacache/schemaPoll/time`|Time taken for coordinator polls to fetch datasource schema.||| |`serverview/sync/healthy`|Sync status of the Broker with a segment-loading server such as a Historical or Peon. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled. This metric can be used in conjunction with `serverview/sync/unstableTime` to debug slow startup of Brokers.|`server`, `tier`|1 for fully synced servers, 0 otherwise| |`serverview/sync/unstableTime`|Time in milliseconds for which the Broker has been failing to sync with a segment-loading server. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled.|`server`, `tier`|Not emitted for synced servers.| +|`subquery/rows`|Number of rows materialized by the subquery's results. |`id`, `subqueryId`| Varies | +|`subquery/bytes`|Number of bytes materialized by the subquery's results. This metric is only emitted if the query uses [byte-based subquery guardrails](https://druid.apache.org/docs/latest/configuration/#guardrails-for-materialization-of-subqueries) |`id`, `subqueryId` | Varies | |`subquery/rowLimit/count`|Number of subqueries whose results are materialized as rows (Java objects on heap).|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| | |`subquery/byteLimit/count`|Number of subqueries whose results are materialized as frames (Druid's internal byte representation of rows).|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| | |`subquery/fallback/count`|Number of subqueries which cannot be materialized as frames|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| | @@ -297,6 +299,8 @@ If the JVM does not support CPU time measurement for the current thread, `ingest |`worker/taskSlot/used/count`|Number of busy task slots on the reporting worker per emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.| `category`, `workerVersion`|Varies| |`worker/task/assigned/count`|Number of tasks assigned to an indexer per emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| |`worker/task/completed/count`|Number of tasks completed by an indexer per emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| +|`worker/task/failed/count`|Number of tasks that failed on an indexer during the emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| +|`worker/task/success/count`|Number of tasks that succeeded on an indexer during the emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| |`worker/task/running/count`|Number of tasks running on an indexer per emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| ## Shuffle metrics (Native parallel task) @@ -311,7 +315,7 @@ See [Enabling metrics](../configuration/index.md#enabling-metrics) for more deta ## Coordination -These metrics are for the Druid Coordinator and are reset each time the Coordinator runs the coordination logic. +These metrics are emitted by the Druid Coordinator in every run of the corresponding coordinator duty. |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| @@ -325,6 +329,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina |`segment/dropSkipped/count`|Number of segments that could not be dropped from any server.|`dataSource`, `tier`, `description`|Varies| |`segment/loadQueue/size`|Size in bytes of segments to load.|`server`|Varies| |`segment/loadQueue/count`|Number of segments to load.|`server`|Varies| +|`segment/loading/rateKbps`|Current rate of segment loading on a server in kbps (1000 bits per second). The rate is calculated as a moving average over the last 10 GiB or more of successful segment loads on that server.|`server`|Varies| |`segment/dropQueue/count`|Number of segments to drop.|`server`|Varies| |`segment/loadQueue/assigned`|Number of segments assigned for load or drop to the load queue of a server.|`dataSource`, `server`|Varies| |`segment/loadQueue/success`|Number of segment assignments that completed successfully.|`dataSource`, `server`|Varies| diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index b04664ac90e3..2ec056290b6e 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -70,11 +70,26 @@ Returns the following: ## ACOS -`ACOS()` +Calculates the arc cosine (arccosine) of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ACOS(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the arc cosine of `0`. + +```sql +SELECT ACOS(0) AS "arc_cosine" +``` +Returns the following: -Calculates the arc cosine of a numeric expression. +| `arc_cosine` | +| -- | +| `1.5707963267948966` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ANY_VALUE @@ -280,27 +295,72 @@ Joins all elements of `arr` by the delimiter specified by `str`. ## ASIN -`ASIN()` +Calculates the arc sine (arcsine) of a numeric expression. + +* **Syntax:** `ASIN(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the arc sine of `1`. + +```sql +SELECT ASIN(1) AS "arc_sine" +``` +Returns the following: -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +| `arc_sine` | +| -- | +| `1.5707963267948966` | +
-Calculates the arc sine of a numeric expression. +[Learn more](sql-scalar.md#numeric-functions) ## ATAN -`ATAN()` +Calculates the arc tangent (arctangent) of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ATAN(expr)` +* **Function type:** Scalar, numeric + +
Example -Calculates the arc tangent of a numeric expression. +The following example calculates the arc tangent of `1`. + +```sql +SELECT ATAN(1) AS "arc_tangent" +``` +Returns the following: + +| `arc_tangent` | +| -- | +| `0.7853981633974483` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ATAN2 -`ATAN2(, )` +Calculates the arc tangent (arctangent) of a specified x and y coordinate. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ATAN2(x, y)` +* **Function type:** Scalar, numeric -Calculates the arc tangent of the two arguments. +
Example + +The following example calculates the arc tangent of the coordinate `(1, -1)` + +```sql +SELECT ATAN2(1,-1) AS "arc_tangent_2" +``` +Returns the following: + +| `arc_tangent_2` | +| -- | +| `2.356194490192345` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## AVG @@ -336,67 +396,188 @@ Performs a bitwise XOR operation on all input values. ## BITWISE_AND -`BITWISE_AND(expr1, expr2)` +Returns the bitwise AND between two expressions: `expr1 & expr2`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_AND(expr1, expr2)` +* **Function type:** Scalar, numeric + +
Example -Returns the bitwise AND between the two expressions, that is, `expr1 & expr2`. +The following example performs the bitwise AND operation `12 & 10`. + +```sql +SELECT BITWISE_AND(12, 10) AS "bitwise_and" +``` +Returns the following: + +| `bitwise_and` | +| -- | +| 8 | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_COMPLEMENT -`BITWISE_COMPLEMENT(expr)` +Returns the bitwise complement (bitwise not) for the expression: `~expr`. + +* **Syntax:** `BITWISE_COMPLEMENT(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example performs the bitwise complement operation `~12`. + +```sql +SELECT BITWISE_COMPLEMENT(12) AS "bitwise_complement" +``` +Returns the following: -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +| `bitwise_complement` | +| -- | +| -13 | +
-Returns the bitwise NOT for the expression, that is, `~expr`. +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_CONVERT_DOUBLE_TO_LONG_BITS -`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` +Converts the bits of an IEEE 754 floating-point double value to long. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:**`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` +* **Function type:** Scalar, numeric -Converts the bits of an IEEE 754 floating-point double value to a long. +
Example -## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE +The following example returns the IEEE 754 floating-point double representation of `255` as a long. -`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` +```sql +SELECT BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(255) AS "ieee_754_double_to_long" +``` +Returns the following: + +| `ieee_754_double_to_long` | +| -- | +| `4643176031446892544` | +
+ +[Learn more](sql-scalar.md#numeric-functions) -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + +## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE Converts a long to the IEEE 754 floating-point double specified by the bits stored in the long. +* **Syntax:**`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example returns the long representation of `4643176031446892544` as an IEEE 754 floating-point double. + +```sql +SELECT BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(4643176031446892544) AS "long_to_ieee_754_double" +``` +Returns the following: + +| `long_to_ieee_754_double` | +| -- | +| `255` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## BITWISE_OR -`BITWISE_OR(expr1, expr2)` +Returns the bitwise OR between the two expressions: `expr1 | expr2`. + +* **Syntax:** `BITWISE_OR(expr1, expr2)` +* **Function type:** Scalar, numeric + +
Example + +The following example performs the bitwise OR operation `12 | 10`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +```sql +SELECT BITWISE_OR(12, 10) AS "bitwise_or" +``` +Returns the following: -Returns the bitwise OR between the two expressions, that is, `expr1 | expr2`. +| `bitwise_or` | +| -- | +| `14` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_SHIFT_LEFT -`BITWISE_SHIFT_LEFT(expr1, expr2)` +Returns the bitwise left shift by x positions of an expr: `expr << x`. + +* **Syntax:** `BITWISE_SHIFT_LEFT(expr, x)` +* **Function type:** Scalar, numeric + +
Example -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +The following example performs the bitwise SHIFT operation `2 << 3`. + +```sql +SELECT BITWISE_SHIFT_LEFT(2, 3) AS "bitwise_shift_left" +``` +Returns the following: -Returns a bitwise left shift of expr1, that is, `expr1 << expr2`. +| `bitwise_shift_left` | +| -- | +| `16` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_SHIFT_RIGHT -`BITWISE_SHIFT_RIGHT(expr1, expr2)` +Returns the bitwise right shift by x positions of an expr: `expr >> x`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `BITWISE_SHIFT_RIGHT(expr, x)` +* **Function type:** Scalar, numeric -Returns a bitwise right shift of expr1, that is, `expr1 >> expr2`. +
Example + +The following example performs the bitwise SHIFT operation `16 >> 3`. + +```sql +SELECT BITWISE_SHIFT_RIGHT(16, 3) AS "bitwise_shift_right" +``` +Returns the following: + +| `bitwise_shift_right` | +| -- | +| `2` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BITWISE_XOR -`BITWISE_XOR(expr1, expr2)` +Returns the bitwise exclusive OR between the two expressions: `expr1 ^ expr2`. + +* **Syntax:** `BITWISE_XOR(expr1, expr2)` +* **Function type:** Scalar, numeric + +
Example + +The following example performs the bitwise XOR operation `12 ^ 10`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +```sql +SELECT BITWISE_XOR(12, 10) AS "bitwise_xor" +``` +Returns the following: -Returns the bitwise exclusive OR between the two expressions, that is, `expr1 ^ expr2`. +| `bitwise_xor` | +| -- | +| `6` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## BLOOM_FILTER @@ -514,20 +695,50 @@ Finds whether a string is in a given expression, case-sensitive. ## COS -`COS()` +Calculates the trigonometric cosine of an angle expressed in radians. + +* **Syntax:** `COS(expr)` +* **Function type:** Scalar, numeric -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +
Example -Calculates the trigonometric cosine of an angle expressed in radians. +The following example calculates the cosine of angle `PI/3` radians. -## COT +```sql +SELECT COS(PI / 3) AS "cosine" +``` +Returns the following: -`COT()` +| `cosine` | +| -- | +| `0.5000000000000001` | +
-**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +[Learn more](sql-scalar.md#numeric-functions) + +## COT Calculates the trigonometric cotangent of an angle expressed in radians. +* **Syntax:** `COT(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the cotangent of angle `PI/3` radians. + +```sql +SELECT COT(PI / 3) AS "cotangent" +``` +Returns the following: + +| `cotangent` | +| -- | +| `0.577350269189626` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## COUNT `COUNT([DISTINCT] expr)` @@ -589,11 +800,26 @@ Decodes a Base64-encoded string into a UTF-8 encoded string. ## DEGREES -`DEGREES()` +Converts an angle from radians to degrees. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `DEGREES(expr)` +* **Function type:** Scalar, numeric -Converts an angle from radians to degrees. +
Example + +The following example converts an angle of `PI` radians to degrees + +```sql +SELECT DEGREES(PI) AS "degrees" +``` +Returns the following: + +| `degrees` | +| -- | +| `180` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## DENSE_RANK @@ -605,11 +831,36 @@ Returns the rank for a row within a window without gaps. For example, if two row ## DIV -`DIV(x, y)` +Returns the result of integer division of `x` by `y`. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `DIV(x, y)` +* **Function type:** Scalar, numeric -Returns the result of integer division of `x` by `y`. + + +:::info + + The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md/#safe_divide) instead. + +::: + +[Learn more](sql-scalar.md#numeric-functions) ## DS_CDF @@ -866,28 +1117,78 @@ Returns a union of HLL sketches. ## HUMAN_READABLE_BINARY_BYTE_FORMAT -`HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` +Converts an integer byte size into human-readable [IEC](https://en.wikipedia.org/wiki/Binary_prefix) format. + +* **Syntax:** `HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric + +
Example -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + The following example converts `1000000` into IEC format. -Converts an integer byte size into human-readable IEC format. + ```sql + SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(1000000, 2) AS "iec_format" + ``` + + Returns the following: + + | `iec_format` | + | -- | + | `976.56 KiB` | + +
+ +[Learn more](sql-scalar.md#numeric-functions) ## HUMAN_READABLE_DECIMAL_BYTE_FORMAT -`HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` +Converts a byte size into human-readable [SI](https://en.wikipedia.org/wiki/Binary_prefix) format. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric -Converts a byte size into human-readable SI format. +
Example -## HUMAN_READABLE_DECIMAL_FORMAT +The following example converts `1000000` into SI format. -`HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` +```sql +SELECT HUMAN_READABLE_DECIMAL_BYTE_FORMAT(1000000, 2) AS "si_format" +``` -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +Returns the following: + +|`si_format`| +|--| +|`1.00 MB`| + +
+ +[Learn more](sql-scalar.md#numeric-functions) + +## HUMAN_READABLE_DECIMAL_FORMAT Converts a byte size into human-readable SI format with single-character units. +* **Syntax:** `HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` +* **Function type:** Scalar, numeric + +
Example + + The following example converts `1000000` into single character SI format. + +```sql +SELECT HUMAN_READABLE_DECIMAL_FORMAT(1000000, 2) AS "single_character_si_format" +``` + +Returns the following: + +|`single_character_si_format`| +|--| +|`1.00 M`| +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## ICONTAINS_STRING `ICONTAINS_STRING(, str)` @@ -1374,11 +1675,26 @@ Returns the following: ## RADIANS -`RADIANS(expr)` +Converts an angle from degrees to radians. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `RADIANS(expr)` +* **Function type:** Scalar, numeric -Converts an angle from degrees to radians. +
Example + +The following example converts an angle of `180` degrees to radians + +```sql +SELECT RADIANS(180) AS "radians" +``` +Returns the following: + +| `radians` | +| -- | +| `3.141592653589793` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## RANK @@ -1499,20 +1815,52 @@ Trims characters from the trailing end of an expression. ## SAFE_DIVIDE -`SAFE_DIVIDE(x, y)` +Returns `x` divided by `y`, guarded on division by 0. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `SAFE_DIVIDE(x, y)` +* **Function type:** Scalar, numeric -Returns `x` divided by `y`, guarded on division by 0. +
Example -## SIN +The following example calculates divisions of integer `78` by integer `10`. + +```sql +SELECT SAFE_DIVIDE(78, 10) AS "safe_division" +``` + +Returns the following: + +|`safe_division`| +|--| +| `7` | -`SIN(expr)` +
-**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +[Learn more](sql-scalar.md#numeric-functions) + +## SIN Calculates the trigonometric sine of an angle expressed in radians. +* **Syntax:** `SIN(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the sine of angle `PI/3` radians. + +```sql +SELECT SIN(PI / 3) AS "sine" +``` +Returns the following: + +| `sine` | +| -- | +| `0.8660254037844386` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## SQRT Calculates the square root of a numeric expression. @@ -1635,11 +1983,26 @@ Calculates the sum of a set of values. ## TAN -`TAN(expr)` +Calculates the trigonometric tangent of an angle expressed in radians. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `TAN(expr)` +* **Function type:** Scalar, numeric -Calculates the trigonometric tangent of an angle expressed in radians. +
Example + +The following example calculates the tangent of angle `PI/3` radians. + +```sql +SELECT TAN(PI / 3) AS "tangent" +``` +Returns the following: + +| `tangent` | +| -- | +| `1.7320508075688767` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## TDIGEST_GENERATE_SKETCH diff --git a/docs/tutorials/tutorial-rollup.md b/docs/tutorials/tutorial-rollup.md index b2c74d7e5b37..464197d551c9 100644 --- a/docs/tutorials/tutorial-rollup.md +++ b/docs/tutorials/tutorial-rollup.md @@ -24,18 +24,21 @@ sidebar_label: Aggregate data with rollup --> -Apache Druid can summarize raw data at ingestion time using a process we refer to as "rollup". Rollup is a first-level aggregation operation over a selected set of columns that reduces the size of stored data. +Apache Druid® can summarize raw data at ingestion time using a process known as "rollup." [Rollup](../multi-stage-query/concepts.md#rollup) is a first-level aggregation operation over a selected set of columns that reduces the size of stored data. -This tutorial will demonstrate the effects of rollup on an example dataset. +This tutorial demonstrates how to apply rollup during ingestion and highlights its effects during query execution. The examples in the tutorial use the [multi-stage query (MSQ)](../multi-stage-query/index.md) task engine to execute SQL statements. -For this tutorial, we'll assume you've already downloaded Druid as described in -the [single-machine quickstart](index.md) and have it running on your local machine. +## Prerequisites -It will also be helpful to have finished [Load a file](../tutorials/tutorial-batch.md) and [Query data](../tutorials/tutorial-query.md) tutorials. +Before proceeding, download Druid as described in [Quickstart (local)](index.md) and have it running on your local machine. You don't need to load any data into the Druid cluster. -## Example data +You should be familiar with data querying in Druid. If you haven't already, go through the [Query data](../tutorials/tutorial-query.md) tutorial first. -For this tutorial, we'll use a small sample of network flow event data, representing packet and byte counts for traffic from a source to a destination IP address that occurred within a particular second. + +## Load the example data + +For this tutorial, you use a small sample of network flow event data representing IP traffic. +The data contains packet and byte counts from a source IP address to a destination IP address. ```json {"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024} @@ -49,102 +52,63 @@ For this tutorial, we'll use a small sample of network flow event data, represen {"timestamp":"2018-01-02T21:35:45Z","srcIP":"7.7.7.7", "dstIP":"8.8.8.8","packets":12,"bytes":2818} ``` -A file containing this sample input data is located at `quickstart/tutorial/rollup-data.json`. - -We'll ingest this data using the following ingestion task spec, located at `quickstart/tutorial/rollup-index.json`. - -```json -{ - "type" : "index_parallel", - "spec" : { - "dataSchema" : { - "dataSource" : "rollup-tutorial", - "dimensionsSpec" : { - "dimensions" : [ - "srcIP", - "dstIP" - ] - }, - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "metricsSpec" : [ - { "type" : "count", "name" : "count" }, - { "type" : "longSum", "name" : "packets", "fieldName" : "packets" }, - { "type" : "longSum", "name" : "bytes", "fieldName" : "bytes" } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "week", - "queryGranularity" : "minute", - "intervals" : ["2018-01-01/2018-01-03"], - "rollup" : true - } - }, - "ioConfig" : { - "type" : "index_parallel", - "inputSource" : { - "type" : "local", - "baseDir" : "quickstart/tutorial", - "filter" : "rollup-data.json" - }, - "inputFormat" : { - "type" : "json" - }, - "appendToExisting" : false - }, - "tuningConfig" : { - "type" : "index_parallel", - "partitionsSpec": { - "type": "dynamic" - }, - "maxRowsInMemory" : 25000 - } - } -} +Load the sample dataset using the [`INSERT INTO`](../multi-stage-query/reference.md/#insert) statement and the [`EXTERN`](../multi-stage-query/reference.md/#extern-function) function to ingest the data inline. In the [Druid web console](../operations/web-console.md), go to the **Query** view and run the following query: + +```sql +INSERT INTO "rollup_tutorial" +WITH "inline_data" AS ( + SELECT * + FROM TABLE(EXTERN('{ + "type":"inline", + "data":"{\"timestamp\":\"2018-01-01T01:01:35Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":20,\"bytes\":9024}\n{\"timestamp\":\"2018-01-01T01:02:14Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-01T01:01:59Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":11,\"bytes\":5780}\n{\"timestamp\":\"2018-01-01T01:01:51Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":255,\"bytes\":21133}\n{\"timestamp\":\"2018-01-01T01:02:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":377,\"bytes\":359971}\n{\"timestamp\":\"2018-01-01T01:03:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":49,\"bytes\":10204}\n{\"timestamp\":\"2018-01-02T21:33:14Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-02T21:33:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":123,\"bytes\":93999}\n{\"timestamp\":\"2018-01-02T21:35:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":12,\"bytes\":2818}"}', + '{"type":"json"}')) + EXTEND ("timestamp" VARCHAR, "srcIP" VARCHAR, "dstIP" VARCHAR, "packets" BIGINT, "bytes" BIGINT) +) +SELECT + FLOOR(TIME_PARSE("timestamp") TO MINUTE) AS __time, + "srcIP", + "dstIP", + SUM("bytes") AS "bytes", + SUM("packets") AS "packets", + COUNT(*) AS "count" +FROM "inline_data" +GROUP BY 1, 2, 3 +PARTITIONED BY DAY ``` -Rollup has been enabled by setting `"rollup" : true` in the `granularitySpec`. +Note the following aspects of the ingestion statement: +* You transform the timestamp field using the `FLOOR` function to round timestamps down to the minute. +* You group by the dimensions `timestamp`, `srcIP`, and `dstIP`. +* You create the `bytes` and `packets` metrics, which are summed from their respective input fields. +* You also create the `count` metric that records the number of rows that get rolled-up per each row in the datasource. -Note that we have `srcIP` and `dstIP` defined as dimensions, a longSum metric is defined for the `packets` and `bytes` columns, and the `queryGranularity` has been defined as `minute`. +With rollup, Druid combines rows with identical timestamp and dimension values after the timestamp truncation. Druid computes and stores the metric values using the specified aggregation function over each set of rolled-up rows. -We will see how these definitions are used after we load this data. +After the ingestion completes, you can query the data. -## Load the example data +## Query the example data -From the apache-druid-{{DRUIDVERSION}} package root, run the following command: +In the web console, open a new tab in the **Query** view. Run the following query to view the ingested data: -```bash -bin/post-index-task --file quickstart/tutorial/rollup-index.json --url http://localhost:8081 +```sql +SELECT * FROM "rollup_tutorial" ``` -After the script completes, we will query the data. +Returns the following: -## Query the example data +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` | +| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` | +| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` | +| `2018-01-02T21:33:00.000Z` | `7.7.7.7` | `8.8.8.8` | `100,288` | `2` | `161` | +| `2018-01-02T21:35:00.000Z` | `7.7.7.7` | `8.8.8.8` | `2,818` | `1` | `12` | -Let's run `bin/dsql` and issue a `select * from "rollup-tutorial";` query to see what data was ingested. - -```bash -$ bin/dsql -Welcome to dsql, the command-line client for Druid SQL. -Type "\h" for help. -dsql> select * from "rollup-tutorial"; -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │ -│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │ -│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │ -│ 2018-01-02T21:33:00.000Z │ 100288 │ 2 │ 8.8.8.8 │ 161 │ 7.7.7.7 │ -│ 2018-01-02T21:35:00.000Z │ 2818 │ 1 │ 8.8.8.8 │ 12 │ 7.7.7.7 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -Retrieved 5 rows in 1.18s. - -dsql> -``` +Notice there are only five rows as opposed to the nine rows in the example data. In the next section, you explore the components of the rolled-up rows. -Let's look at the three events in the original input data that occurred during `2018-01-01T01:01`: +## View rollup in action + +Consider the three events in the original input data that occur over the course of minute `2018-01-01T01:01`: ```json {"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024} @@ -152,47 +116,47 @@ Let's look at the three events in the original input data that occurred during ` {"timestamp":"2018-01-01T01:01:59Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":11,"bytes":5780} ``` -These three rows have been "rolled up" into the following row: +Druid combines the three rows into one during rollup: -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` | -The input rows have been grouped by the timestamp and dimension columns `{timestamp, srcIP, dstIP}` with sum aggregations on the metric columns `packets` and `bytes`. +Before the grouping occurs, the `FLOOR(TIME_PARSE("timestamp") TO MINUTE)` expression buckets (floors) the timestamp column of the original input by minute. -Before the grouping occurs, the timestamps of the original input data are bucketed/floored by minute, due to the `"queryGranularity":"minute"` setting in the ingestion spec. +The input rows are grouped because they have the same values for their dimension columns `{timestamp, srcIP, dstIP}`. The metric columns calculate the sum aggregation of the grouped rows for `packets` and `bytes`. The `count` metric shows how many rows from the original input data contributed to the final rolled-up row. -Likewise, these two events that occurred during `2018-01-01T01:02` have been rolled up: +Now, consider the two events in the original input data that occur over the course of minute `2018-01-01T01:02`: ```json {"timestamp":"2018-01-01T01:02:14Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":38,"bytes":6289} {"timestamp":"2018-01-01T01:02:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":377,"bytes":359971} ``` -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +The rows are grouped into the following during rollup: -For the last event recording traffic between 1.1.1.1 and 2.2.2.2, no rollup took place, because this was the only event that occurred during `2018-01-01T01:03`: +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` | + +In the original input data, only one event occurs over the course of minute `2018-01-01T01:03`: ```json {"timestamp":"2018-01-01T01:03:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":49,"bytes":10204} ``` -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +Therefore, no rollup takes place: + +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` | + + +## Learn more + +See the following topics for more information: -Note that the `count` metric shows how many rows in the original input data contributed to the final "rolled up" row. +* [Data rollup](../ingestion/rollup.md) for suggestions and best practices when performing rollup. +* [SQL-based ingestion concepts](../multi-stage-query/concepts.md#rollup) for information on rollup using SQL-based ingestion. +* [SQL-based ingestion query examples](../multi-stage-query/examples.md#insert-with-rollup) for another example of data rollup. +* [Druid schema model](../ingestion/schema-model.md) to learn about the primary timestamp, dimensions, and metrics. diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index e2df5b995317..2fca74ef55b1 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -35,7 +35,7 @@ UTF-8 - 2.5.0 + 2.6.0 diff --git a/extensions-contrib/compressed-bigdecimal/pom.xml b/extensions-contrib/compressed-bigdecimal/pom.xml index 33b368b255eb..d6c27d888129 100644 --- a/extensions-contrib/compressed-bigdecimal/pom.xml +++ b/extensions-contrib/compressed-bigdecimal/pom.xml @@ -166,6 +166,11 @@ com.fasterxml.jackson.core jackson-annotations + + org.reflections + reflections + test + diff --git a/extensions-contrib/opentelemetry-emitter/pom.xml b/extensions-contrib/opentelemetry-emitter/pom.xml index b48e8837effe..71a67270966a 100644 --- a/extensions-contrib/opentelemetry-emitter/pom.xml +++ b/extensions-contrib/opentelemetry-emitter/pom.xml @@ -39,7 +39,7 @@ 32.0.1-jre - 1.57.2 + 1.65.1 diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json index ad065c63d39a..91d7c4c4abdf 100644 --- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -74,8 +74,8 @@ "worker/task/assigned/count" : { "dimensions" : ["dataSource"], "type" : "count" }, "worker/task/running/count" : { "dimensions" : ["dataSource"], "type" : "count" }, "worker/task/completed/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "worker/task/failed/count" : { "dimensions" : ["category", "workerVersion"], "type" : "count" }, - "worker/task/success/count" : { "dimensions" : ["category", "workerVersion"], "type" : "count" }, + "worker/task/failed/count" : { "dimensions" : ["category", "workerVersion", "dataSource"], "type" : "count" }, + "worker/task/success/count" : { "dimensions" : ["category", "workerVersion", "dataSource"], "type" : "count" }, "worker/taskSlot/idle/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" }, "worker/taskSlot/total/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" }, "worker/taskSlot/used/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" }, diff --git a/extensions-contrib/tdigestsketch/pom.xml b/extensions-contrib/tdigestsketch/pom.xml index 489b0b995565..4c192d627462 100644 --- a/extensions-contrib/tdigestsketch/pom.xml +++ b/extensions-contrib/tdigestsketch/pom.xml @@ -164,6 +164,11 @@ junit-vintage-engine test + + org.reflections + reflections + test + org.easymock easymock diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index ee423909d192..75ff5a0868e0 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -170,6 +170,11 @@ junit-jupiter-params test + + org.reflections + reflections + test + org.junit.vintage junit-vintage-engine diff --git a/extensions-core/druid-bloom-filter/pom.xml b/extensions-core/druid-bloom-filter/pom.xml index 8ce419095de8..b57ede7b16b1 100644 --- a/extensions-core/druid-bloom-filter/pom.xml +++ b/extensions-core/druid-bloom-filter/pom.xml @@ -139,6 +139,11 @@ junit-vintage-engine test + + org.reflections + reflections + test + org.apache.druid druid-processing diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml index a48a7f1e7b80..c5bafbeb8dc1 100644 --- a/extensions-core/druid-catalog/pom.xml +++ b/extensions-core/druid-catalog/pom.xml @@ -177,6 +177,11 @@ junit-vintage-engine test + + org.reflections + reflections + test + org.easymock easymock diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index c252cfb87d33..a1c1f91e379c 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -123,6 +123,11 @@ junit-vintage-engine test + + org.reflections + reflections + test + org.apache.druid druid-processing diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java index 41c2c0a03258..53369cc6ad60 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java @@ -33,8 +33,6 @@ * key, and timestamp. *

* NOTE: Any records with null values will be skipped, even if they contain non-null keys, or headers - *

- * This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions. */ public class KafkaRecordEntity extends ByteEntity { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java new file mode 100644 index 000000000000..7d97fffed378 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputFormat.java @@ -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. + */ + +package org.apache.druid.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.DateTimes; + +import javax.annotation.Nullable; + +import java.io.File; +import java.util.Objects; + +/** + * Kinesis aware InputFormat. Allows for reading kinesis specific values that are stored in the {@link Record}. At + * this time, this input format only supports reading data from the following record components + *

+ * - {@link Record#data} + * - {@link Record#approximateArrivalTimestamp} + * - {@link Record#partitionKey} + *

+ * This class can be extended easily to read other fields available in the kinesis record. + */ +public class KinesisInputFormat implements InputFormat +{ + private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kinesis.timestamp"; + private static final String DEFAULT_PARTITION_KEY_COLUMN_NAME = "kinesis.partitionKey"; + + // Since KinesisInputFormat blends data from record properties, and payload, timestamp spec can be pointing to an + // attribute within one of these 2 sections. To handle scenarios where there is no timestamp value in the payload, we + // induce an artificial timestamp value to avoid unnecessary parser barf out. Users in such situations can use the + // inputFormat's kinesis record timestamp as its primary timestamp. + public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp"; + private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH); + + private final InputFormat valueFormat; + private final String timestampColumnName; + private final String partitionKeyColumnName; + + public KinesisInputFormat( + @JsonProperty("valueFormat") InputFormat valueFormat, + @JsonProperty("partitionKeyColumnName") @Nullable String partitionKeyColumnName, + @JsonProperty("timestampColumnName") @Nullable String timestampColumnName + ) + { + this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null"); + Preconditions.checkState( + !(timestampColumnName != null && timestampColumnName.equals(partitionKeyColumnName)), + "timestampColumnName and partitionKeyColumnName must be different" + ); + this.partitionKeyColumnName = partitionKeyColumnName != null + ? partitionKeyColumnName + : DEFAULT_PARTITION_KEY_COLUMN_NAME; + this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory) + { + final SettableByteEntity settableByteEntitySource; + if (source instanceof SettableByteEntity) { + settableByteEntitySource = (SettableByteEntity) source; + } else { + settableByteEntitySource = new SettableByteEntity<>(); + settableByteEntitySource.setEntity((KinesisRecordEntity) source); + } + InputRowSchema newInputRowSchema = new InputRowSchema( + dummyTimestampSpec, + inputRowSchema.getDimensionsSpec(), + inputRowSchema.getColumnsFilter(), + inputRowSchema.getMetricNames() + ); + return new KinesisInputReader( + inputRowSchema, + settableByteEntitySource, + JsonInputFormat.withLineSplittable(valueFormat, false).createReader( + newInputRowSchema, + source, + temporaryDirectory + ), + partitionKeyColumnName, + timestampColumnName + ); + } + + @JsonProperty + public InputFormat getValueFormat() + { + return valueFormat; + } + + @Nullable + @JsonProperty + public String getTimestampColumnName() + { + return timestampColumnName; + } + + @Nullable + @JsonProperty + public String getPartitionKeyColumnName() + { + return partitionKeyColumnName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KinesisInputFormat that = (KinesisInputFormat) o; + return Objects.equals(valueFormat, that.valueFormat) + && Objects.equals(timestampColumnName, that.timestampColumnName) + && Objects.equals(partitionKeyColumnName, that.partitionKeyColumnName); + } + + @Override + public int hashCode() + { + return Objects.hash(valueFormat, timestampColumnName, partitionKeyColumnName); + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java new file mode 100644 index 000000000000..d0c30280a2b7 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisInputReader.java @@ -0,0 +1,256 @@ +/* + * 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.data.input.kinesis; + +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.joda.time.DateTime; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class KinesisInputReader implements InputEntityReader +{ + + private final InputRowSchema inputRowSchema; + private final SettableByteEntity source; + private final InputEntityReader valueParser; + private final String partitionKeyColumnName; + private final String timestampColumnName; + + public KinesisInputReader( + InputRowSchema inputRowSchema, + SettableByteEntity source, + InputEntityReader valueParser, + String partitionKeyColumnName, + String timestampColumnName + ) + { + this.inputRowSchema = inputRowSchema; + this.source = source; + this.valueParser = valueParser; + this.partitionKeyColumnName = partitionKeyColumnName; + this.timestampColumnName = timestampColumnName; + + } + + @Override + public CloseableIterator read() throws IOException + { + final KinesisRecordEntity record = source.getEntity(); + final Map mergedHeaderMap = extractHeaders(record); + + if (record.getRecord().getData() != null) { + return buildBlendedRows(valueParser, mergedHeaderMap); + } else { + return CloseableIterators.withEmptyBaggage(buildInputRowsForMap(mergedHeaderMap).iterator()); + } + } + + @Override + public CloseableIterator sample() throws IOException + { + final KinesisRecordEntity record = source.getEntity(); + InputRowListPlusRawValues headers = extractHeaderSample(record); + if (record.getRecord().getData() != null) { + return buildBlendedRowsSample(valueParser, headers.getRawValues()); + } else { + final List rows = Collections.singletonList(headers); + return CloseableIterators.withEmptyBaggage(rows.iterator()); + } + } + + private Map extractHeaders(KinesisRecordEntity record) + { + final Map mergedHeaderMap = new HashMap<>(); + mergedHeaderMap.put(timestampColumnName, record.getRecord().getApproximateArrivalTimestamp().getTime()); + mergedHeaderMap.put(partitionKeyColumnName, record.getRecord().getPartitionKey()); + return mergedHeaderMap; + } + + private CloseableIterator buildBlendedRows( + InputEntityReader valueParser, + Map headerKeyList + ) throws IOException + { + return valueParser.read().map( + r -> { + final HashSet newDimensions = new HashSet<>(r.getDimensions()); + final Map event = buildBlendedEventMap(r::getRaw, newDimensions, headerKeyList); + newDimensions.addAll(headerKeyList.keySet()); + // Remove the dummy timestamp added in KinesisInputFormat + newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); + + final DateTime timestamp = MapInputRowParser.parseTimestamp(inputRowSchema.getTimestampSpec(), event); + return new MapBasedInputRow( + timestamp, + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + newDimensions + ), + event + ); + } + ); + } + + private InputRowListPlusRawValues extractHeaderSample(KinesisRecordEntity record) + { + Map mergedHeaderMap = extractHeaders(record); + return InputRowListPlusRawValues.of(buildInputRowsForMap(mergedHeaderMap), mergedHeaderMap); + } + + private CloseableIterator buildBlendedRowsSample( + InputEntityReader valueParser, + Map headerKeyList + ) throws IOException + { + return valueParser.sample().map( + rowAndValues -> { + if (rowAndValues.getParseException() != null) { + return rowAndValues; + } + List newInputRows = Lists.newArrayListWithCapacity(rowAndValues.getInputRows().size()); + List> newRawRows = Lists.newArrayListWithCapacity(rowAndValues.getRawValues().size()); + + for (Map raw : rowAndValues.getRawValuesList()) { + newRawRows.add(buildBlendedEventMap(raw::get, raw.keySet(), headerKeyList)); + } + for (InputRow r : rowAndValues.getInputRows()) { + if (r != null) { + final HashSet newDimensions = new HashSet<>(r.getDimensions()); + final Map event = buildBlendedEventMap( + r::getRaw, + newDimensions, + headerKeyList + ); + newDimensions.addAll(headerKeyList.keySet()); + // Remove the dummy timestamp added in KinesisInputFormat + newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING); + newInputRows.add( + new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(event), + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + newDimensions + ), + event + ) + ); + } + } + return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, null); + } + ); + } + + private List buildInputRowsForMap(Map headerKeyList) + { + return Collections.singletonList( + new MapBasedInputRow( + inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList), + MapInputRowParser.findDimensions( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + headerKeyList.keySet() + ), + headerKeyList + ) + ); + } + + private Map buildBlendedEventMap( + Function getRowValue, + Set rowDimensions, + Map fallback + ) + { + final Set keySet = new HashSet<>(fallback.keySet()); + keySet.addAll(rowDimensions); + + return new AbstractMap() + { + @Override + public Object get(Object key) + { + final String skey = (String) key; + final Object val = getRowValue.apply(skey); + if (val == null) { + return fallback.get(skey); + } + return val; + } + + @Override + public Set keySet() + { + return keySet; + } + + @Override + public Set> entrySet() + { + return keySet().stream() + .map( + field -> new Entry() + { + @Override + public String getKey() + { + return field; + } + + @Override + public Object getValue() + { + return get(field); + } + + @Override + public Object setValue(final Object value) + { + throw new UnsupportedOperationException(); + } + } + ) + .collect(Collectors.toCollection(LinkedHashSet::new)); + } + }; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java new file mode 100644 index 000000000000..a490fd8f4c36 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/data/input/kinesis/KinesisRecordEntity.java @@ -0,0 +1,51 @@ +/* + * 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.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; + +/** + * A {@link ByteEntity} generated by {@link KinesisRecordSupplier} and fed to any {@link InputFormat} used by kinesis + * indexing tasks. + *

+ * It can be used as a regular ByteEntity, in which case the kinesis record value is returned, but the {@link #getRecord} + * method also allows Kinesis-aware {@link InputFormat} implementations to read the full kinesis record, including + * timestamp, encrytion key, patition key, and sequence number + *

+ * NOTE: Any records with null values will be returned as records with just only kinesis properties and no data payload + */ +public class KinesisRecordEntity extends ByteEntity +{ + private final Record record; + + public KinesisRecordEntity(Record record) + { + super(record.getData()); + this.record = record; + } + + public Record getRecord() + { + return record; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index fb019f10030b..bea69d96c3d2 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -27,7 +27,7 @@ import com.google.common.base.Preconditions; import com.google.inject.name.Named; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; @@ -46,7 +46,7 @@ import java.util.Map; import java.util.Set; -public class KinesisIndexTask extends SeekableStreamIndexTask +public class KinesisIndexTask extends SeekableStreamIndexTask { private static final String TYPE = "index_kinesis"; @@ -100,7 +100,7 @@ public TaskStatus runTask(TaskToolbox toolbox) } @Override - protected SeekableStreamIndexTaskRunner createTaskRunner() + protected SeekableStreamIndexTaskRunner createTaskRunner() { //noinspection unchecked return new KinesisIndexTaskRunner( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 75f23da0e1f6..72e61635912c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; @@ -49,7 +49,7 @@ import java.util.TreeMap; import java.util.concurrent.ConcurrentMap; -public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner +public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class); private static final long POLL_TIMEOUT = 100; @@ -81,8 +81,8 @@ protected String getNextStartOffset(String sequenceNumber) @Nonnull @Override - protected List> getRecords( - RecordSupplier recordSupplier, TaskToolbox toolbox + protected List> getRecords( + RecordSupplier recordSupplier, TaskToolbox toolbox ) { return recordSupplier.poll(POLL_TIMEOUT); @@ -119,7 +119,7 @@ protected OrderedSequenceNumber createSequenceNumber(String sequenceNumb @Override protected void possiblyResetDataSourceMetadata( TaskToolbox toolbox, - RecordSupplier recordSupplier, + RecordSupplier recordSupplier, Set> assignment ) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index 0cce1a7e6983..5ac9022d001d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -26,6 +26,7 @@ import com.google.inject.Binder; import com.google.inject.name.Names; import org.apache.druid.common.aws.AWSCredentialsConfig; +import org.apache.druid.data.input.kinesis.KinesisInputFormat; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; @@ -50,7 +51,8 @@ public List getJacksonModules() new NamedType(KinesisIndexTaskIOConfig.class, SCHEME), new NamedType(KinesisSupervisorTuningConfig.class, SCHEME), new NamedType(KinesisSupervisorSpec.class, SCHEME), - new NamedType(KinesisSamplerSpec.class, SCHEME) + new NamedType(KinesisSamplerSpec.class, SCHEME), + new NamedType(KinesisInputFormat.class, SCHEME) ) ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 36047ce429db..07a0da32a954 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -26,6 +26,7 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.model.DescribeStreamRequest; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; @@ -49,7 +50,7 @@ import org.apache.druid.common.aws.AWSClientUtil; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -69,7 +70,6 @@ import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.reflect.Method; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -94,7 +94,7 @@ * This class implements a local buffer for storing fetched Kinesis records. Fetching is done * in background threads. */ -public class KinesisRecordSupplier implements RecordSupplier +public class KinesisRecordSupplier implements RecordSupplier { private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; @@ -210,7 +210,7 @@ private Runnable fetchRecords() // used for retrying on InterruptedException GetRecordsResult recordsResult = null; - OrderedPartitionableRecord currRecord; + OrderedPartitionableRecord currRecord; long recordBufferOfferWaitMillis; try { @@ -248,7 +248,7 @@ private Runnable fetchRecords() // list will come back empty if there are no records for (Record kinesisRecord : recordsResult.getRecords()) { - final List data; + final List data; if (deaggregateHandle == null || getDataHandle == null) { throw new ISE("deaggregateHandle or getDataHandle is null!"); @@ -256,15 +256,15 @@ private Runnable fetchRecords() data = new ArrayList<>(); - final List userRecords = (List) deaggregateHandle.invokeExact( + final List userRecords = (List) deaggregateHandle.invokeExact( Collections.singletonList(kinesisRecord) ); int recordSize = 0; - for (Object userRecord : userRecords) { - ByteEntity byteEntity = new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord)); - recordSize += byteEntity.getBuffer().array().length; - data.add(byteEntity); + for (UserRecord userRecord : userRecords) { + KinesisRecordEntity kinesisRecordEntity = new KinesisRecordEntity(userRecord); + recordSize += kinesisRecordEntity.getBuffer().array().length; + data.add(kinesisRecordEntity); } @@ -408,7 +408,7 @@ private long getPartitionTimeLag() private final ConcurrentMap, PartitionResource> partitionResources = new ConcurrentHashMap<>(); - private MemoryBoundLinkedBlockingQueue> records; + private MemoryBoundLinkedBlockingQueue> records; private final boolean backgroundFetchEnabled; private volatile boolean closed = false; @@ -615,12 +615,12 @@ public String getPosition(StreamPartition partition) @Nonnull @Override - public List> poll(long timeout) + public List> poll(long timeout) { start(); try { - List>> polledRecords = new ArrayList<>(); + List>> polledRecords = new ArrayList<>(); records.drain( polledRecords, @@ -1040,7 +1040,7 @@ private void filterBufferAndResetBackgroundFetch(Set> pa } // filter records in buffer and only retain ones whose partition was not seeked - MemoryBoundLinkedBlockingQueue> newQ = + MemoryBoundLinkedBlockingQueue> newQ = new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes); records.stream() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index a142f4147627..2f00c8c16cc9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; @@ -74,7 +74,7 @@ * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kinesis sequences. */ -public class KinesisSupervisor extends SeekableStreamSupervisor +public class KinesisSupervisor extends SeekableStreamSupervisor { private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class); @@ -150,7 +150,7 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( } @Override - protected List> createIndexTasks( + protected List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, @@ -164,7 +164,7 @@ protected List> createIndexT final Map context = createBaseTaskContexts(); context.put(CHECKPOINTS_CTX_KEY, checkpoints); - List> taskList = new ArrayList<>(); + List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); taskList.add(new KinesisIndexTask( @@ -183,7 +183,7 @@ protected List> createIndexT @Override - protected RecordSupplier setupRecordSupplier() throws RuntimeException + protected RecordSupplier setupRecordSupplier() throws RuntimeException { KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java new file mode 100644 index 000000000000..130f31681dee --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java @@ -0,0 +1,940 @@ +/* + * 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.data.input.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.seekablestream.SettableByteEntity; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.List; + +public class KinesisInputFormatTest +{ + static { + NullHandling.initializeForTests(); + } + + + private static final String KINESIS_APPROXIMATE_TIME_DATE = "2024-07-29"; + private static final long KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS = DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis(); + private static final String DATA_TIMSTAMP_DATE = "2024-07-30"; + private static final String PARTITION_KEY = "partition_key_1"; + + private static final byte[] SIMPLE_JSON_VALUE_BYTES = StringUtils.toUtf8( + TestUtils.singleQuoteToStandardJson( + "{" + + " 'timestamp': '" + DATA_TIMSTAMP_DATE + "'," + + " 'bar': null," + + " 'foo': 'x'," + + " 'baz': 4," + + " 'o': {'mg': 1}" + + "}" + ) + ); + + private KinesisInputFormat format; + + @Before + public void setUp() + { + format = new KinesisInputFormat( + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + KinesisInputFormat kif = new KinesisInputFormat( + // Value Format + new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + Assert.assertEquals(format, kif); + + final byte[] formatBytes = mapper.writeValueAsBytes(format); + final byte[] kifBytes = mapper.writeValueAsBytes(kif); + Assert.assertArrayEquals(formatBytes, kifBytes); + } + + @Test + public void testTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testRawSample() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.sample()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRowListPlusRawValues rawValues = iterator.next(); + Assert.assertEquals(1, rawValues.getInputRows().size()); + InputRow row = rawValues.getInputRows().get(0); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testProcessesSampleTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.sample()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRowListPlusRawValues rawValues = iterator.next(); + Assert.assertEquals(1, rawValues.getInputRows().size()); + InputRow row = rawValues.getInputRows().get(0); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS)), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithMultipleMixedRecordsTimestampFromHeader() throws IOException + { + final byte[][] values = new byte[5][]; + for (int i = 0; i < values.length; i++) { + values[i] = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2024-07-2" + i + "\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"index\": " + i + ",\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}" + ); + } + + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + settableByteEntity, + null + ); + + for (int i = 0; i < values.length; i++) { + KinesisRecordEntity inputEntity = makeInputEntity(values[i], DateTimes.of("2024-07-1" + i).getMillis()); + settableByteEntity.setEntity(inputEntity); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Payload verification + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + Assert.assertEquals(DateTimes.of("2024-07-1" + i), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of("2024-07-1" + i).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + } + + @Test + public void testTimestampFromData() throws IOException + { + KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithMultipleMixedRecordsTimestampFromData() throws IOException + { + final byte[][] values = new byte[5][]; + for (int i = 0; i < values.length; i++) { + values[i] = StringUtils.toUtf8( + "{\n" + + " \"timestamp\": \"2024-07-2" + i + "\",\n" + + " \"bar\": null,\n" + + " \"foo\": \"x\",\n" + + " \"baz\": 4,\n" + + " \"index\": " + i + ",\n" + + " \"o\": {\n" + + " \"mg\": 1\n" + + " }\n" + + "}" + ); + } + + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + settableByteEntity, + null + ); + + for (int i = 0; i < values.length; i++) { + KinesisRecordEntity inputEntity = makeInputEntity(values[i], KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + settableByteEntity.setEntity(inputEntity); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + // Payload verification + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + Assert.assertEquals(DateTimes.of("2024-07-2" + i), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of("2024-07-29").getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + } + + @Test + public void testMissingTimestampThrowsException() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("time", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + try (CloseableIterator iterator = reader.read()) { + while (iterator.hasNext()) { + Throwable t = Assert.assertThrows(ParseException.class, iterator::next); + Assert.assertTrue( + t.getMessage().startsWith("Timestamp[null] is unparseable! Event: {") + ); + } + } + } + + @Test + public void testWithSchemaDiscoveryKinesisTimestampExcluded() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .setDimensionExclusions(ImmutableList.of("kinesis.newts.timestamp")) + .build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + List expectedDimensions = Arrays.asList( + "foo", + "root_baz", + "o", + "bar", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2", + "kinesis.newts.partitionKey" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithSchemaDiscoveryTimestampFromHeader() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("kinesis.newts.timestamp", "iso", null), + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + List expectedDimensions = Arrays.asList( + "foo", + "timestamp", + "root_baz", + "o", + "bar", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2", + "kinesis.newts.partitionKey" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testValueInCsvFormat() throws IOException + { + format = new KinesisInputFormat( + // Value Format + new CsvInputFormat( + Arrays.asList("foo", "bar", "timestamp", "baz"), + null, + false, + false, + 0 + ), + "kinesis.newts.partitionKey", + "kinesis.newts.timestamp" + ); + + KinesisRecordEntity inputEntity = + makeInputEntity(StringUtils.toUtf8("x,,2024-07-30,4"), KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey" + ) + ) + ), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + Assert.assertEquals( + Arrays.asList( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey" + ), + row.getDimensions() + ); + // Payload verifications + // this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec + // but test reading them anyway since it isn't technically illegal + + Assert.assertEquals(DateTimes.of("2024-07-30"), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertTrue(row.getDimension("bar").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + public void testWithPartialDeclarationSchemaDiscovery() throws IOException + { + KinesisRecordEntity inputEntity = + makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + DimensionsSpec.builder().setDimensions( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar")) + ).useSchemaDiscovery(true).build(), + ColumnsFilter.all() + ), + newSettableByteEntity(inputEntity), + null + ); + + final int numExpectedIterations = 1; + try (CloseableIterator iterator = reader.read()) { + int numActualIterations = 0; + while (iterator.hasNext()) { + + final InputRow row = iterator.next(); + + List expectedDimensions = Arrays.asList( + "bar", + "foo", + "kinesis.newts.timestamp", + "kinesis.newts.partitionKey", + "root_baz", + "o", + "path_omg", + "jq_omg", + "jq_omg2", + "baz", + "root_baz2", + "path_omg2" + ); + Collections.sort(expectedDimensions); + Collections.sort(row.getDimensions()); + Assert.assertEquals( + expectedDimensions, + row.getDimensions() + ); + + // Payload verifications + Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp()); + Assert.assertEquals( + String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()), + Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp")) + ); + Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey"))); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o")); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + + numActualIterations++; + } + + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @Test + @SuppressWarnings("ResultOfMethodCallIgnored") + public void testValidInputFormatConstruction() + { + InputFormat valueFormat = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ); + // null partitionKeyColumnName and null timestampColumnName is valid + new KinesisInputFormat(valueFormat, null, null); + + // non-null partitionKeyColumnName and null timestampColumnName is valid + new KinesisInputFormat(valueFormat, "kinesis.partitionKey", null); + + // null partitionKeyColumnName and non-null timestampColumnName is valid + new KinesisInputFormat(valueFormat, null, "kinesis.timestamp"); + + // non-null partitionKeyColumnName and non-null timestampColumnName is valid + new KinesisInputFormat(valueFormat, "kinesis.partitionKey", "kinesis.timestamp"); + + } + + @Test + @SuppressWarnings("ResultOfMethodCallIgnored") + public void testInvalidInputFormatConstruction() + { + // null value format is invalid + Assert.assertThrows( + "valueFormat must not be null", + NullPointerException.class, + () -> new KinesisInputFormat(null, null, null) + ); + + InputFormat valueFormat = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null, + null, + false, + false + ); + + // partitionKeyColumnName == timestampColumnName is invalid + Assert.assertThrows( + "timestampColumnName and partitionKeyColumnName must be different", + IllegalStateException.class, + () -> new KinesisInputFormat(valueFormat, "kinesis.timestamp", "kinesis.timestamp") + ); + } + + private KinesisRecordEntity makeInputEntity( + byte[] payload, + long kinesisTimestampMillis) + { + return new KinesisRecordEntity( + new Record().withData(ByteBuffer.wrap(payload)) + .withApproximateArrivalTimestamp(new Date(kinesisTimestampMillis)) + .withPartitionKey(PARTITION_KEY) + ); + } + + private SettableByteEntity newSettableByteEntity(KinesisRecordEntity kinesisRecordEntity) + { + SettableByteEntity settableByteEntity = new SettableByteEntity<>(); + settableByteEntity.setEntity(kinesisRecordEntity); + return settableByteEntity; + } +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 527a6738ffe9..80bded2031d4 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.services.kinesis.model.Record; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -41,6 +42,7 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.report.IngestionStatsAndErrors; @@ -127,39 +129,39 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase private static final String SHARD_ID0 = "0"; private static final List RECORDS = Arrays.asList( - createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "5", jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), - createRecord("1", "6", new ByteEntity(StringUtils.toUtf8("unparseable"))), - createRecord("1", "7", new ByteEntity(StringUtils.toUtf8(""))), - createRecord("1", "8", new ByteEntity(StringUtils.toUtf8("{}"))), - createRecord("1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")), - createRecord("1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")), - createRecord("1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")), - createRecord("1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")), - createRecord("0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")), - createRecord("0", "1", jb("2011", "h", "y", "10", "20.0", "1.0")) + createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", kjb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), + createRecord("1", "6", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("unparseable")).getBuffer()))), + createRecord("1", "7", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("")).getBuffer()))), + createRecord("1", "8", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("{}")).getBuffer()))), + createRecord("1", "9", kjb("2013", "f", "y", "10", "20.0", "1.0")), + createRecord("1", "10", kjb("2049", "f", "y", "notanumber", "20.0", "1.0")), + createRecord("1", "11", kjb("2049", "f", "y", "10", "notanumber", "1.0")), + createRecord("1", "12", kjb("2049", "f", "y", "10", "20.0", "notanumber")), + createRecord("0", "0", kjb("2012", "g", "y", "10", "20.0", "1.0")), + createRecord("0", "1", kjb("2011", "h", "y", "10", "20.0", "1.0")) ); private static final List SINGLE_PARTITION_RECORDS = Arrays.asList( - createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")), - createRecord("1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")), - createRecord("1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")), - createRecord("1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")), - createRecord("1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")), - createRecord("1", "14", jb("2013", "e", "y", "10", "20.0", "1.0")) + createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", kjb("2012", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "6", kjb("2013", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "7", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "8", kjb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "9", kjb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "10", kjb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "11", kjb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "12", kjb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "13", kjb("2012", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "14", kjb("2013", "e", "y", "10", "20.0", "1.0")) ); private static KinesisRecordSupplier recordSupplier; @@ -272,12 +274,12 @@ private static KinesisRecord clone(KinesisRecord record) record.getPartitionId(), record.getSequenceNumber(), record.getData().stream() - .map(entity -> new ByteEntity(entity.getBuffer())) + .map(entity -> new KinesisRecordEntity(new Record().withData(entity.getBuffer()))) .collect(Collectors.toList()) ); } - private static List> clone( + private static List> clone( List records, int start, int end @@ -289,14 +291,14 @@ private static List> clon /** * Records can only be read once, hence we must use fresh records every time. */ - private static List> clone( + private static List> clone( List records ) { return records.stream().map(KinesisIndexTaskTest::clone).collect(Collectors.toList()); } - private static KinesisRecord createRecord(String partitionId, String sequenceNumber, ByteEntity entity) + private static KinesisRecord createRecord(String partitionId, String sequenceNumber, KinesisRecordEntity entity) { return new KinesisRecord(STREAM, partitionId, sequenceNumber, Collections.singletonList(entity)); } @@ -1697,7 +1699,7 @@ public void testRestoreAfterPersistingSequences() throws Exception maxRowsPerSegment = 2; maxRecordsPerPoll = 1; maxBytesPerPoll = 1_000_000; - List> records = + List> records = clone(SINGLE_PARTITION_RECORDS); recordSupplier.assign(EasyMock.anyObject()); @@ -2148,7 +2150,7 @@ public void testEndOfShard() throws Exception recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - List> eosRecord = ImmutableList.of( + List> eosRecord = ImmutableList.of( new OrderedPartitionableRecord<>(STREAM, SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER, null) ); @@ -2454,6 +2456,18 @@ private boolean isTaskReading(KinesisIndexTask task) return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.READING; } + private static KinesisRecordEntity kjb( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) + { + return new KinesisRecordEntity(new Record().withData(jb(timestamp, dim1, dim2, dimLong, dimFloat, met1).getBuffer())); + } + @JsonTypeName("index_kinesis") private static class TestableKinesisIndexTask extends KinesisIndexTask { @@ -2497,15 +2511,15 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) /** * Utility class to keep the test code more readable. */ - private static class KinesisRecord extends OrderedPartitionableRecord + private static class KinesisRecord extends OrderedPartitionableRecord { - private final List data; + private final List data; public KinesisRecord( String stream, String partitionId, String sequenceNumber, - List data + List data ) { super(stream, partitionId, sequenceNumber, data); @@ -2514,7 +2528,7 @@ public KinesisRecord( @Nonnull @Override - public List getData() + public List getData() { return data; } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 5fcf81139eb6..7c59ad61ac09 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -39,6 +39,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; @@ -99,34 +100,26 @@ public class KinesisRecordSupplierTest extends EasyMockSupport new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"), new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9") ); - private static final List> ALL_RECORDS = ImmutableList.>builder() - .addAll(SHARD0_RECORDS.stream() - .map(x -> new OrderedPartitionableRecord<>( - STREAM, - SHARD_ID0, - x.getSequenceNumber(), - Collections - .singletonList( - new ByteEntity( - x.getData())) - )) - .collect( - Collectors - .toList())) - .addAll(SHARD1_RECORDS.stream() - .map(x -> new OrderedPartitionableRecord<>( - STREAM, - SHARD_ID1, - x.getSequenceNumber(), - Collections - .singletonList( - new ByteEntity( - x.getData())) - )) - .collect( - Collectors - .toList())) - .build(); + private static final List> ALL_RECORDS = ImmutableList.>builder() + .addAll(SHARD0_RECORDS.stream() + .map(x -> new OrderedPartitionableRecord<>( + STREAM, + SHARD_ID0, + x.getSequenceNumber(), + Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer()))) + )) + .collect( + Collectors + .toList())) + .addAll(SHARD1_RECORDS.stream() + .map(x -> new OrderedPartitionableRecord<>( + STREAM, + SHARD_ID1, + x.getSequenceNumber(), + Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer()))) + )) + .collect(Collectors.toList())) + .build(); private static ByteBuffer jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) @@ -316,7 +309,7 @@ private static GetRecordsRequest generateGetRecordsWithLimitReq(String shardIter } // filter out EOS markers - private static List> cleanRecords(List> records) + private static List> cleanRecords(List> records) { return records.stream() .filter(x -> !x.getSequenceNumber() @@ -398,7 +391,7 @@ public void testPollWithKinesisInternalFailure() throws InterruptedException Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -457,7 +450,7 @@ public void testPollWithKinesisNonRetryableFailure() throws InterruptedException } Assert.assertFalse(recordSupplier.isAnyFetchActive()); - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -531,7 +524,7 @@ public void testSeek() Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); @@ -687,7 +680,7 @@ public void testPollAfterSeek() Thread.sleep(100); } - OrderedPartitionableRecord firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); + OrderedPartitionableRecord firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); Assert.assertEquals( ALL_RECORDS.get(7), @@ -705,7 +698,7 @@ public void testPollAfterSeek() } - OrderedPartitionableRecord record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); + OrderedPartitionableRecord record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0); Assert.assertEquals(ALL_RECORDS.get(9), record2); // only one partition in this test. second results come from getRecordsResult0, which has SHARD0_LAG_MILLIS @@ -776,7 +769,7 @@ public void testPollDeaggregate() throws InterruptedException Thread.sleep(100); } - List> polledRecords = cleanRecords(recordSupplier.poll( + List> polledRecords = cleanRecords(recordSupplier.poll( POLL_TIMEOUT_MILLIS)); verifyAll(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index b0ba730a3502..63144c6a9353 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.services.kinesis.model.Record; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -27,7 +28,6 @@ import org.apache.druid.client.indexing.SamplerSpec; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.InputRowParser; @@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; @@ -63,6 +64,7 @@ import org.junit.Test; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -99,7 +101,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport private final KinesisRecordSupplier recordSupplier = mock(KinesisRecordSupplier.class); - private static List> generateRecords(String stream) + private static List> generateRecords(String stream) { return ImmutableList.of( new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), @@ -115,9 +117,9 @@ private static List> gene stream, "1", "6", - Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable"))) + Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))))) ), - new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}")))) + new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}")))))) ); } @@ -428,19 +430,19 @@ private void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useIn Assert.assertFalse(it.hasNext()); } - private static List jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1) + private static List jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { - return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes( + return Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes( ImmutableMap.builder() - .put("timestamp", ts) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - ))); + .put("timestamp", ts) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ))))); } catch (Exception e) { throw new RuntimeException(e); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9001f148e99b..e6ed27c9cecc 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -27,12 +27,12 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskInfoProvider; @@ -5656,7 +5656,7 @@ public String generateSequenceName( } @Override - protected RecordSupplier setupRecordSupplier() + protected RecordSupplier setupRecordSupplier() { return supervisorRecordSupplier; } diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index 9e637acff009..ab23a26e36fa 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -233,6 +233,11 @@ junit-jupiter-params test + + org.reflections + reflections + test + org.easymock easymock diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java index 8936e104bd69..dce2fe7ac3ac 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/CounterSnapshotsTree.java @@ -108,4 +108,14 @@ private void putAll(final Map> otherMap) } } } + + @Override + public String toString() + { + synchronized (snapshotsMap) { + return "CounterSnapshotsTree{" + + "snapshotsMap=" + snapshotsMap + + '}'; + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java index f04286dd7c42..d2370b057935 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -117,9 +117,9 @@ void resultsComplete( ); /** - * Returns the current list of task ids, ordered by worker number. The Nth task has worker number N. + * Returns the current list of worker IDs, ordered by worker number. The Nth worker has worker number N. */ - List getTaskIds(); + List getWorkerIds(); @Nullable TaskReport.ReportMap liveReports(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java index 405ff4fb9026..428ce59cd8fa 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java @@ -23,20 +23,25 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.List; /** - * Client for the multi-stage query controller. Used by a Worker task. + * Client for the multi-stage query controller. Used by a {@link Worker}. Each instance is specific to a single query, + * meaning it communicates with a single controller. */ -public interface ControllerClient extends AutoCloseable +public interface ControllerClient extends Closeable { /** - * Client side method to update the controller with partial key statistics information for a particular stage and worker. - * Controller's implementation collates all the information for a stage to fetch key statistics from workers. + * Client side method to update the controller with partial key statistics information for a particular stage + * and worker. The controller collates all the information for a stage to fetch key statistics from workers. + * + * Only used when {@link StageDefinition#mustGatherResultKeyStatistics()}. */ void postPartialKeyStatistics( StageId stageId, @@ -86,11 +91,16 @@ void postWorkerError( /** * Client side method to inform the controller about the warnings generated by the given worker. */ - void postWorkerWarning( - List MSQErrorReports - ) throws IOException; + void postWorkerWarning(List MSQErrorReports) throws IOException; - List getTaskList() throws IOException; + /** + * Client side method for retrieving the list of worker IDs from the controller. These IDs can be passed to + * {@link WorkerClient} methods to communicate with other workers. Not necessary when the {@link WorkOrder} has + * {@link WorkOrder#getWorkerIds()} set. + * + * @see Controller#getWorkerIds() for the controller side + */ + List getWorkerIds() throws IOException; /** * Close this client. Idempotent. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a30e96860875..839839db4e42 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.exec; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -1172,7 +1171,7 @@ private List generateSegmentIdsWithShardSpecsForReplace( } @Override - public List getTaskIds() + public List getWorkerIds() { if (workerManager == null) { return Collections.emptyList(); @@ -1261,7 +1260,7 @@ private void contactWorkersForStage( { // Sorted copy of target worker numbers to ensure consistent iteration order. final List workersCopy = Ordering.natural().sortedCopy(workers); - final List workerIds = getTaskIds(); + final List workerIds = getWorkerIds(); final List> workerFutures = new ArrayList<>(workersCopy.size()); try { @@ -1357,7 +1356,10 @@ private void postResultPartitionBoundariesForStage( * Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()}, * also drop all other segments within the replacement intervals. */ - private void publishAllSegments(final Set segments) throws IOException + private void publishAllSegments( + final Set segments, + Function, Set> compactionStateAnnotateFunction + ) throws IOException { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); @@ -1413,7 +1415,7 @@ private void publishAllSegments(final Set segments) throws IOExcept } performSegmentPublish( context.taskActionClient(), - createOverwriteAction(taskLockType, segmentsWithTombstones) + createOverwriteAction(taskLockType, compactionStateAnnotateFunction.apply(segmentsWithTombstones)) ); } } else if (!segments.isEmpty()) { @@ -1486,7 +1488,7 @@ private List findIntervalsToDrop(final Set publishedSegme private CounterSnapshotsTree getCountersFromAllTasks() { final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); - final List taskList = getTaskIds(); + final List taskList = getWorkerIds(); final List> futures = new ArrayList<>(); @@ -1506,7 +1508,7 @@ private CounterSnapshotsTree getCountersFromAllTasks() private void postFinishToAllTasks() { - final List taskList = getTaskIds(); + final List taskList = getWorkerIds(); final List> futures = new ArrayList<>(); @@ -1543,6 +1545,7 @@ private void handleQueryResults( if (MSQControllerTask.isIngestion(querySpec)) { // Publish segments if needed. final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); + Function, Set> compactionStateAnnotateFunction = Function.identity(); @SuppressWarnings("unchecked") Set segments = (Set) queryKernel.getResultObjectForStage(finalStageId); @@ -1553,7 +1556,7 @@ private void handleQueryResults( Tasks.DEFAULT_STORE_COMPACTION_STATE ); - if (!segments.isEmpty() && storeCompactionState) { + if (storeCompactionState) { DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); if (!destination.isReplaceTimeChunks()) { // Store compaction state only for replace queries. @@ -1565,20 +1568,21 @@ private void handleQueryResults( DataSchema dataSchema = ((SegmentGeneratorFrameProcessorFactory) queryKernel .getStageDefinition(finalStageId).getProcessorFactory()).getDataSchema(); - ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec(); + ShardSpec shardSpec = segments.isEmpty() ? null : segments.stream().findFirst().get().getShardSpec(); + ClusterBy clusterBy = queryKernel.getStageDefinition(finalStageId).getClusterBy(); - Function, Set> compactionStateAnnotateFunction = addCompactionStateToSegments( + compactionStateAnnotateFunction = addCompactionStateToSegments( querySpec, context.jsonMapper(), dataSchema, shardSpec, + clusterBy, queryDef.getQueryId() ); - segments = compactionStateAnnotateFunction.apply(segments); } } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); - publishAllSegments(segments); + publishAllSegments(segments, compactionStateAnnotateFunction); } else if (MSQControllerTask.isExport(querySpec)) { // Write manifest file. ExportMSQDestination destination = (ExportMSQDestination) querySpec.getDestination(); @@ -1624,33 +1628,49 @@ private static Function, Set> addCompactionStateTo MSQSpec querySpec, ObjectMapper jsonMapper, DataSchema dataSchema, - ShardSpec shardSpec, + @Nullable ShardSpec shardSpec, + @Nullable ClusterBy clusterBy, String queryId ) { final MSQTuningConfig tuningConfig = querySpec.getTuningConfig(); PartitionsSpec partitionSpec; - if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) { - List partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions(); + // shardSpec is absent in the absence of segments, which happens when only tombstones are generated by an + // MSQControllerTask. + if (shardSpec != null) { + if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) { + List partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions(); + partitionSpec = new DimensionRangePartitionsSpec( + tuningConfig.getRowsPerSegment(), + null, + partitionDimensions, + false + ); + } else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) { + // MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE. + partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); + } else { + // SingleDimenionShardSpec and other shard specs are never created in MSQ. + throw new MSQException( + UnknownFault.forMessage( + StringUtils.format( + "Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].", + queryId, + shardSpec.getType() + ))); + } + } else if (clusterBy != null && !clusterBy.getColumns().isEmpty()) { partitionSpec = new DimensionRangePartitionsSpec( tuningConfig.getRowsPerSegment(), null, - partitionDimensions, + clusterBy.getColumns() + .stream() + .map(KeyColumn::columnName).collect(Collectors.toList()), false ); - } else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) { - // MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE. - partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); } else { - // SingleDimenionShardSpec and other shard specs are never created in MSQ. - throw new MSQException( - UnknownFault.forMessage( - StringUtils.format( - "Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].", - queryId, - shardSpec.getType() - ))); + partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); } Granularity segmentGranularity = ((DataSourceMSQDestination) querySpec.getDestination()) @@ -1671,13 +1691,26 @@ private static Function, Set> addCompactionStateTo : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); - List metricsSpec = dataSchema.getAggregators() == null - ? null - : jsonMapper.convertValue( - dataSchema.getAggregators(), - new TypeReference>() {} - ); - + List metricsSpec = Collections.emptyList(); + + if (querySpec.getQuery() instanceof GroupByQuery) { + // For group-by queries, the aggregators are transformed to their combining factories in the dataschema, resulting + // in a mismatch between schema in compaction spec and the one in compaction state. Sourcing the original + // AggregatorFactory definition for aggregators in the dataSchema, therefore, directly from the querySpec. + GroupByQuery groupByQuery = (GroupByQuery) querySpec.getQuery(); + // Collect all aggregators that are part of the current dataSchema, since a non-rollup query (isRollup() is false) + // moves metrics columns to dimensions in the final schema. + Set aggregatorsInDataSchema = Arrays.stream(dataSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect( + Collectors.toSet()); + metricsSpec = new ArrayList<>( + groupByQuery.getAggregatorSpecs() + .stream() + .filter(aggregatorFactory -> aggregatorsInDataSchema.contains(aggregatorFactory.getName())) + .collect(Collectors.toList()) + ); + } IndexSpec indexSpec = tuningConfig.getIndexSpec(); @@ -2930,7 +2963,7 @@ private void startQueryResultsReader() } final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); - final List taskIds = getTaskIds(); + final List taskIds = getWorkerIds(); final InputChannelFactory inputChannelFactory; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java index 8e6fc72b6aa7..2ab016e10e48 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java @@ -91,7 +91,8 @@ public static ControllerMemoryParameters createProductionInstance( memoryIntrospector.totalMemoryInJvm(), usableMemoryInJvm, numControllersInJvm, - memoryIntrospector.numProcessorsInJvm() + memoryIntrospector.numProcessorsInJvm(), + 0 ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java new file mode 100644 index 000000000000..ebaad0763872 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ListeningOutputChannelFactory.java @@ -0,0 +1,74 @@ +/* + * 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.msq.exec; + +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.PartitionedOutputChannel; + +import java.io.IOException; + +/** + * Decorator for {@link OutputChannelFactory} that notifies a {@link Listener} whenever a channel is opened. + */ +public class ListeningOutputChannelFactory implements OutputChannelFactory +{ + private final OutputChannelFactory delegate; + private final Listener listener; + + public ListeningOutputChannelFactory(final OutputChannelFactory delegate, final Listener listener) + { + this.delegate = delegate; + this.listener = listener; + } + + @Override + public OutputChannel openChannel(final int partitionNumber) throws IOException + { + return notifyListener(delegate.openChannel(partitionNumber)); + } + + + @Override + public OutputChannel openNilChannel(final int partitionNumber) + { + return notifyListener(delegate.openNilChannel(partitionNumber)); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel( + final String name, + final boolean deleteAfterRead + ) + { + throw new UnsupportedOperationException("Listening to partitioned channels is not supported"); + } + + private OutputChannel notifyListener(OutputChannel channel) + { + listener.channelOpened(channel); + return channel; + } + + public interface Listener + { + void channelOpened(OutputChannel channel); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java index 7e7fc3d3d6f3..f42d558a76ce 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/OutputChannelMode.java @@ -32,9 +32,12 @@ public enum OutputChannelMode { /** - * In-memory output channels. Stage shuffle data does not hit disk. This mode requires a consumer stage to run - * at the same time as its corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the - * logic that determines when we can use in-memory channels. + * In-memory output channels. Stage shuffle data does not hit disk. In-memory channels do not fully buffer stage + * output. They use a blocking queue; see {@link RunWorkOrder#makeStageOutputChannelFactory()}. + * + * Because stage output is not fully buffered, this mode requires a consumer stage to run at the same time as its + * corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the logic that + * determines when we can use in-memory channels. */ MEMORY("memory"), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java new file mode 100644 index 000000000000..0173979efeed --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java @@ -0,0 +1,1051 @@ +/* + * 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.msq.exec; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.frame.processor.ComposingOutputChannelFactory; +import org.apache.druid.frame.processor.FileOutputChannelFactory; +import org.apache.druid.frame.processor.FrameChannelHashPartitioner; +import org.apache.druid.frame.processor.FrameChannelMixer; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.PartitionedOutputChannel; +import org.apache.druid.frame.processor.SuperSorter; +import org.apache.druid.frame.processor.SuperSorterProgressTracker; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.processor.manager.ProcessorManagers; +import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.indexing.CountingOutputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.InputSlices; +import org.apache.druid.msq.input.MapInputSliceReader; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.NilInputSliceReader; +import org.apache.druid.msq.input.external.ExternalInputSlice; +import org.apache.druid.msq.input.external.ExternalInputSliceReader; +import org.apache.druid.msq.input.inline.InlineInputSlice; +import org.apache.druid.msq.input.inline.InlineInputSliceReader; +import org.apache.druid.msq.input.lookup.LookupInputSlice; +import org.apache.druid.msq.input.lookup.LookupInputSliceReader; +import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.stage.StageInputSliceReader; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.SegmentsInputSliceReader; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.ShuffleSpec; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; +import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.utils.CloseableUtils; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * Main worker logic for executing a {@link WorkOrder} in a {@link FrameProcessorExecutor}. + */ +public class RunWorkOrder +{ + private final WorkOrder workOrder; + private final InputChannelFactory inputChannelFactory; + private final CounterTracker counterTracker; + private final FrameProcessorExecutor exec; + private final String cancellationId; + private final int parallelism; + private final WorkerContext workerContext; + private final FrameContext frameContext; + private final RunWorkOrderListener listener; + private final boolean reindex; + private final boolean removeNullBytes; + private final ByteTracker intermediateSuperSorterLocalStorageTracker; + private final AtomicBoolean started = new AtomicBoolean(); + + @MonotonicNonNull + private InputSliceReader inputSliceReader; + @MonotonicNonNull + private OutputChannelFactory workOutputChannelFactory; + @MonotonicNonNull + private OutputChannelFactory shuffleOutputChannelFactory; + @MonotonicNonNull + private ResultAndChannels workResultAndOutputChannels; + @MonotonicNonNull + private SettableFuture stagePartitionBoundariesFuture; + @MonotonicNonNull + private ListenableFuture stageOutputChannelsFuture; + + public RunWorkOrder( + final WorkOrder workOrder, + final InputChannelFactory inputChannelFactory, + final CounterTracker counterTracker, + final FrameProcessorExecutor exec, + final String cancellationId, + final WorkerContext workerContext, + final FrameContext frameContext, + final RunWorkOrderListener listener, + final boolean reindex, + final boolean removeNullBytes + ) + { + this.workOrder = workOrder; + this.inputChannelFactory = inputChannelFactory; + this.counterTracker = counterTracker; + this.exec = exec; + this.cancellationId = cancellationId; + this.parallelism = workerContext.threadCount(); + this.workerContext = workerContext; + this.frameContext = frameContext; + this.listener = listener; + this.reindex = reindex; + this.removeNullBytes = removeNullBytes; + this.intermediateSuperSorterLocalStorageTracker = + new ByteTracker( + frameContext.storageParameters().isIntermediateStorageLimitConfigured() + ? frameContext.storageParameters().getIntermediateSuperSorterStorageMaxLocalBytes() + : Long.MAX_VALUE + ); + } + + /** + * Start execution of the provided {@link WorkOrder} in the provided {@link FrameProcessorExecutor}. + * + * Execution proceeds asynchronously after this method returns. The {@link RunWorkOrderListener} passed to the + * constructor of this instance can be used to track progress. + */ + public void start() throws IOException + { + if (started.getAndSet(true)) { + throw new ISE("Already started"); + } + + final StageDefinition stageDef = workOrder.getStageDefinition(); + + try { + makeInputSliceReader(); + makeWorkOutputChannelFactory(); + makeShuffleOutputChannelFactory(); + makeAndRunWorkProcessors(); + + if (stageDef.doesShuffle()) { + makeAndRunShuffleProcessors(); + } else { + // No shuffling: work output _is_ stage output. Retain read-only versions to reduce memory footprint. + stageOutputChannelsFuture = + Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); + } + + setUpCompletionCallbacks(); + } + catch (Throwable t) { + // If start() has problems, cancel anything that was already kicked off, and close the FrameContext. + try { + exec.cancel(cancellationId); + } + catch (Throwable t2) { + t.addSuppressed(t2); + } + + CloseableUtils.closeAndSuppressExceptions(frameContext, t::addSuppressed); + throw t; + } + } + + /** + * Settable {@link ClusterByPartitions} future for global sort. Necessary because we don't know ahead of time + * what the boundaries will be. The controller decides based on statistics from all workers. Once the controller + * decides, its decision is written to this future, which allows sorting on workers to proceed. + */ + @Nullable + public SettableFuture getStagePartitionBoundariesFuture() + { + return stagePartitionBoundariesFuture; + } + + private void makeInputSliceReader() + { + if (inputSliceReader != null) { + throw new ISE("inputSliceReader already created"); + } + + final String queryId = workOrder.getQueryDefinition().getQueryId(); + + final InputChannels inputChannels = + new InputChannelsImpl( + workOrder.getQueryDefinition(), + InputSlices.allReadablePartitions(workOrder.getInputs()), + inputChannelFactory, + () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), + exec, + cancellationId, + removeNullBytes + ); + + inputSliceReader = new MapInputSliceReader( + ImmutableMap., InputSliceReader>builder() + .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) + .put(StageInputSlice.class, new StageInputSliceReader(queryId, inputChannels)) + .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir("external"))) + .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) + .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) + .put(SegmentsInputSlice.class, new SegmentsInputSliceReader(frameContext, reindex)) + .build() + ); + } + + private void makeWorkOutputChannelFactory() + { + if (workOutputChannelFactory != null) { + throw new ISE("processorOutputChannelFactory already created"); + } + + final OutputChannelFactory baseOutputChannelFactory; + + if (workOrder.getStageDefinition().doesShuffle()) { + // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame + // size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames. + // Otherwise, use the standard frame size. + final int frameSize; + + if (workOrder.getStageDefinition().getShuffleSpec().kind().isSort()) { + frameSize = frameContext.memoryParameters().getLargeFrameSize(); + } else { + frameSize = frameContext.memoryParameters().getStandardFrameSize(); + } + + baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize); + } else { + // Writing stage output. + baseOutputChannelFactory = makeStageOutputChannelFactory(); + } + + workOutputChannelFactory = new CountingOutputChannelFactory( + baseOutputChannelFactory, + counterTracker.channel(CounterNames.outputChannel()) + ); + } + + private void makeShuffleOutputChannelFactory() + { + shuffleOutputChannelFactory = + new CountingOutputChannelFactory( + makeStageOutputChannelFactory(), + counterTracker.channel(CounterNames.shuffleChannel()) + ); + } + + /** + * Use {@link FrameProcessorFactory#makeProcessors} to create {@link ProcessorsAndChannels}. Executes the + * processors using {@link #exec} and sets the output channels in {@link #workResultAndOutputChannels}. + * + * @param type of {@link StageDefinition#getProcessorFactory()} + * @param return type of {@link FrameProcessor} created by the manager + * @param result type of {@link ProcessorManager#result()} + * @param type of {@link WorkOrder#getExtraInfo()} + */ + private , ProcessorReturnType, ManagerReturnType, ExtraInfoType> void makeAndRunWorkProcessors() + throws IOException + { + if (workResultAndOutputChannels != null) { + throw new ISE("workResultAndOutputChannels already set"); + } + + @SuppressWarnings("unchecked") + final FactoryType processorFactory = (FactoryType) workOrder.getStageDefinition().getProcessorFactory(); + + @SuppressWarnings("unchecked") + final ProcessorsAndChannels processors = + processorFactory.makeProcessors( + workOrder.getStageDefinition(), + workOrder.getWorkerNumber(), + workOrder.getInputs(), + inputSliceReader, + (ExtraInfoType) workOrder.getExtraInfo(), + workOutputChannelFactory, + frameContext, + parallelism, + counterTracker, + listener::onWarning, + removeNullBytes + ); + + final ProcessorManager processorManager = processors.getProcessorManager(); + + final int maxOutstandingProcessors; + + if (processors.getOutputChannels().getAllChannels().isEmpty()) { + // No output channels: run up to "parallelism" processors at once. + maxOutstandingProcessors = Math.max(1, parallelism); + } else { + // If there are output channels, that acts as a ceiling on the number of processors that can run at once. + maxOutstandingProcessors = + Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); + } + + final ListenableFuture workResultFuture = exec.runAllFully( + processorManager, + maxOutstandingProcessors, + frameContext.processorBouncer(), + cancellationId + ); + + workResultAndOutputChannels = new ResultAndChannels<>(workResultFuture, processors.getOutputChannels()); + } + + private void makeAndRunShuffleProcessors() + { + if (stageOutputChannelsFuture != null) { + throw new ISE("stageOutputChannelsFuture already set"); + } + + final ShuffleSpec shuffleSpec = workOrder.getStageDefinition().getShuffleSpec(); + + final ShufflePipelineBuilder shufflePipeline = new ShufflePipelineBuilder( + workOrder, + counterTracker, + exec, + cancellationId, + frameContext + ); + + shufflePipeline.initialize(workResultAndOutputChannels); + shufflePipeline.gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded(); + + switch (shuffleSpec.kind()) { + case MIX: + shufflePipeline.mix(shuffleOutputChannelFactory); + break; + + case HASH: + shufflePipeline.hashPartition(shuffleOutputChannelFactory); + break; + + case HASH_LOCAL_SORT: + final OutputChannelFactory hashOutputChannelFactory; + + if (shuffleSpec.partitionCount() == 1) { + // Single partition; no need to write temporary files. + hashOutputChannelFactory = + new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize()); + } else { + // Multi-partition; write temporary files and then sort each one file-by-file. + hashOutputChannelFactory = + new FileOutputChannelFactory( + frameContext.tempDir("hash-parts"), + frameContext.memoryParameters().getStandardFrameSize(), + null + ); + } + + shufflePipeline.hashPartition(hashOutputChannelFactory); + shufflePipeline.localSort(shuffleOutputChannelFactory); + break; + + case GLOBAL_SORT: + shufflePipeline.globalSort(shuffleOutputChannelFactory, makeGlobalSortPartitionBoundariesFuture()); + break; + + default: + throw new UOE("Cannot handle shuffle kind [%s]", shuffleSpec.kind()); + } + + stageOutputChannelsFuture = shufflePipeline.build(); + } + + private ListenableFuture makeGlobalSortPartitionBoundariesFuture() + { + if (workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + if (stagePartitionBoundariesFuture != null) { + throw new ISE("Cannot call 'makeGlobalSortPartitionBoundariesFuture' twice"); + } + + return (stagePartitionBoundariesFuture = SettableFuture.create()); + } else { + // Result key stats aren't needed, so the partition boundaries are knowable ahead of time. Compute them now. + final ClusterByPartitions boundaries = + workOrder.getStageDefinition() + .generatePartitionBoundariesForShuffle(null) + .valueOrThrow(); + + return Futures.immediateFuture(boundaries); + } + } + + private void setUpCompletionCallbacks() + { + Futures.addCallback( + Futures.allAsList( + Arrays.asList( + workResultAndOutputChannels.getResultFuture(), + stageOutputChannelsFuture + ) + ), + new FutureCallback>() + { + @Override + public void onSuccess(final List workerResultAndOutputChannelsResolved) + { + final Object resultObject = workerResultAndOutputChannelsResolved.get(0); + final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1); + + if (workOrder.getOutputChannelMode() != OutputChannelMode.MEMORY) { + // In non-MEMORY output channel modes, call onOutputChannelAvailable when all work is done. + // (In MEMORY mode, we would have called onOutputChannelAvailable when the channels were created.) + for (final OutputChannel channel : outputChannels.getAllChannels()) { + listener.onOutputChannelAvailable(channel); + } + } + + if (workOrder.getOutputChannelMode().isDurable()) { + // In DURABLE_STORAGE output channel mode, write a success file once all work is done. + writeDurableStorageSuccessFile(); + } + + listener.onSuccess(resultObject); + } + + @Override + public void onFailure(final Throwable t) + { + listener.onFailure(t); + } + }, + Execs.directExecutor() + ); + } + + /** + * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. + */ + private void writeDurableStorageSuccessFile() + { + final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory = + makeDurableStorageOutputChannelFactory( + frameContext.tempDir("durable"), + frameContext.memoryParameters().getStandardFrameSize(), + workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + try { + durableStorageOutputChannelFactory.createSuccessFile(workerContext.workerId()); + } + catch (IOException e) { + throw new ISE( + e, + "Unable to create success file at location[%s]", + durableStorageOutputChannelFactory.getSuccessFilePath() + ); + } + } + + private OutputChannelFactory makeStageOutputChannelFactory() + { + // Use the standard frame size, since we assume this size when computing how much is needed to merge output + // files from different workers. + final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); + final OutputChannelMode outputChannelMode = workOrder.getOutputChannelMode(); + + switch (outputChannelMode) { + case MEMORY: + // Use ListeningOutputChannelFactory to capture output channels as they are created, rather than when + // work is complete. + return new ListeningOutputChannelFactory( + new BlockingQueueOutputChannelFactory(frameSize), + listener::onOutputChannelAvailable + ); + + case LOCAL_STORAGE: + final File fileChannelDirectory = + frameContext.tempDir(StringUtils.format("output_stage_%06d", workOrder.getStageNumber())); + return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); + + case DURABLE_STORAGE_INTERMEDIATE: + case DURABLE_STORAGE_QUERY_RESULTS: + return makeDurableStorageOutputChannelFactory( + frameContext.tempDir("durable"), + frameSize, + outputChannelMode == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + default: + throw DruidException.defensive("No handling for outputChannelMode[%s]", outputChannelMode); + } + } + + private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory(final File tmpDir) + { + final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); + final File fileChannelDirectory = + new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", workOrder.getStageNumber())); + final FileOutputChannelFactory fileOutputChannelFactory = + new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterLocalStorageTracker); + + if (workOrder.getOutputChannelMode().isDurable() + && frameContext.storageParameters().isIntermediateStorageLimitConfigured()) { + final boolean isQueryResults = + workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS; + return new ComposingOutputChannelFactory( + ImmutableList.of( + fileOutputChannelFactory, + makeDurableStorageOutputChannelFactory(tmpDir, frameSize, isQueryResults) + ), + frameSize + ); + } else { + return fileOutputChannelFactory; + } + } + + private DurableStorageOutputChannelFactory makeDurableStorageOutputChannelFactory( + final File tmpDir, + final int frameSize, + final boolean isQueryResults + ) + { + return DurableStorageOutputChannelFactory.createStandardImplementation( + workOrder.getQueryDefinition().getQueryId(), + workOrder.getWorkerNumber(), + workOrder.getStageNumber(), + workerContext.workerId(), + frameSize, + MSQTasks.makeStorageConnector(workerContext.injector()), + tmpDir, + isQueryResults + ); + } + + /** + * Helper for {@link RunWorkOrder#makeAndRunShuffleProcessors()}. Builds a {@link FrameProcessor} pipeline to + * handle the shuffle. + */ + private class ShufflePipelineBuilder + { + private final WorkOrder workOrder; + private final CounterTracker counterTracker; + private final FrameProcessorExecutor exec; + private final String cancellationId; + private final FrameContext frameContext; + + // Current state of the pipeline. It's a future to allow pipeline construction to be deferred if necessary. + private ListenableFuture> pipelineFuture; + + public ShufflePipelineBuilder( + final WorkOrder workOrder, + final CounterTracker counterTracker, + final FrameProcessorExecutor exec, + final String cancellationId, + final FrameContext frameContext + ) + { + this.workOrder = workOrder; + this.counterTracker = counterTracker; + this.exec = exec; + this.cancellationId = cancellationId; + this.frameContext = frameContext; + } + + /** + * Start the pipeline with the outputs of the main processor. + */ + public void initialize(final ResultAndChannels resultAndChannels) + { + if (pipelineFuture != null) { + throw new ISE("already initialized"); + } + + pipelineFuture = Futures.immediateFuture(resultAndChannels); + } + + /** + * Add {@link FrameChannelMixer}, which mixes all current outputs into a single channel from the provided factory. + */ + public void mix(final OutputChannelFactory outputChannelFactory) + { + // No sorting or statistics gathering, just combining all outputs into one big partition. Use a mixer to get + // everything into one file. Note: even if there is only one output channel, we'll run it through the mixer + // anyway, to ensure the data gets written to a file. (httpGetChannelData requires files.) + + push( + resultAndChannels -> { + final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + + final FrameChannelMixer mixer = + new FrameChannelMixer( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + outputChannel.getWritableChannel() + ); + + return new ResultAndChannels<>( + exec.runFully(mixer, cancellationId), + OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly())) + ); + } + ); + } + + /** + * Add {@link KeyStatisticsCollectionProcessor} if {@link StageDefinition#mustGatherResultKeyStatistics()}. + * + * Calls {@link RunWorkOrderListener#onDoneReadingInput(ClusterByStatisticsSnapshot)} when statistics are gathered. + * If statistics were not needed, calls the listener immediately. + */ + public void gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded() + { + push( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final OutputChannels channels = resultAndChannels.getOutputChannels(); + + if (channels.getAllChannels().isEmpty()) { + // No data coming out of this stage. Report empty statistics, if the kernel is expecting statistics. + if (stageDefinition.mustGatherResultKeyStatistics()) { + listener.onDoneReadingInput(ClusterByStatisticsSnapshot.empty()); + } else { + listener.onDoneReadingInput(null); + } + + // Generate one empty channel so the next part of the pipeline has something to do. + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + channel.writable().close(); + + final OutputChannel outputChannel = OutputChannel.readOnly( + channel.readable(), + FrameWithPartition.NO_PARTITION + ); + + return new ResultAndChannels<>( + Futures.immediateFuture(null), + OutputChannels.wrap(Collections.singletonList(outputChannel)) + ); + } else if (stageDefinition.mustGatherResultKeyStatistics()) { + return gatherResultKeyStatistics(channels); + } else { + // Report "done reading input" when the input future resolves. + // No need to add any processors to the pipeline. + resultAndChannels.resultFuture.addListener( + () -> listener.onDoneReadingInput(null), + Execs.directExecutor() + ); + return resultAndChannels; + } + } + ); + } + + /** + * Add a {@link SuperSorter} using {@link StageDefinition#getSortKey()} and partition boundaries + * from {@code partitionBoundariesFuture}. + */ + public void globalSort( + final OutputChannelFactory outputChannelFactory, + final ListenableFuture partitionBoundariesFuture + ) + { + pushAsync( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + + final File sorterTmpDir = frameContext.tempDir("super-sort"); + FileUtils.mkdirp(sorterTmpDir); + if (!sorterTmpDir.isDirectory()) { + throw new IOException("Cannot create directory: " + sorterTmpDir); + } + + final WorkerMemoryParameters memoryParameters = frameContext.memoryParameters(); + final SuperSorter sorter = new SuperSorter( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + stageDefinition.getFrameReader(), + stageDefinition.getSortKey(), + partitionBoundariesFuture, + exec, + outputChannelFactory, + makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir), + memoryParameters.getSuperSorterMaxActiveProcessors(), + memoryParameters.getSuperSorterMaxChannelsPerProcessor(), + -1, + cancellationId, + counterTracker.sortProgress(), + removeNullBytes + ); + + return FutureUtils.transform( + sorter.run(), + sortedChannels -> new ResultAndChannels<>(Futures.immediateFuture(null), sortedChannels) + ); + } + ); + } + + /** + * Add a {@link FrameChannelHashPartitioner} using {@link StageDefinition#getSortKey()}. + */ + public void hashPartition(final OutputChannelFactory outputChannelFactory) + { + pushAsync( + resultAndChannels -> { + final ShuffleSpec shuffleSpec = workOrder.getStageDefinition().getShuffleSpec(); + final int partitions = shuffleSpec.partitionCount(); + + final List outputChannels = new ArrayList<>(); + + for (int i = 0; i < partitions; i++) { + outputChannels.add(outputChannelFactory.openChannel(i)); + } + + final FrameChannelHashPartitioner partitioner = new FrameChannelHashPartitioner( + resultAndChannels.getOutputChannels().getAllReadableChannels(), + outputChannels.stream().map(OutputChannel::getWritableChannel).collect(Collectors.toList()), + workOrder.getStageDefinition().getFrameReader(), + workOrder.getStageDefinition().getClusterBy().getColumns().size(), + FrameWriters.makeRowBasedFrameWriterFactory( + new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()), + workOrder.getStageDefinition().getSignature(), + workOrder.getStageDefinition().getSortKey(), + removeNullBytes + ) + ); + + final ListenableFuture partitionerFuture = exec.runFully(partitioner, cancellationId); + + final ResultAndChannels retVal = + new ResultAndChannels<>(partitionerFuture, OutputChannels.wrap(outputChannels)); + + if (retVal.getOutputChannels().areReadableChannelsReady()) { + return Futures.immediateFuture(retVal); + } else { + return FutureUtils.transform(partitionerFuture, ignored -> retVal); + } + } + ); + } + + /** + * Add a sequence of {@link SuperSorter}, operating on each current output channel in order, one at a time. + */ + public void localSort(final OutputChannelFactory outputChannelFactory) + { + pushAsync( + resultAndChannels -> { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final OutputChannels channels = resultAndChannels.getOutputChannels(); + final List> sortedChannelFutures = new ArrayList<>(); + + ListenableFuture nextFuture = Futures.immediateFuture(null); + + for (final OutputChannel channel : channels.getAllChannels()) { + final File sorterTmpDir = frameContext.tempDir( + StringUtils.format("hash-parts-super-sort-%06d", channel.getPartitionNumber()) + ); + + FileUtils.mkdirp(sorterTmpDir); + + // SuperSorter will try to write to output partition zero; we remap it to the correct partition number. + final OutputChannelFactory partitionOverrideOutputChannelFactory = new OutputChannelFactory() + { + @Override + public OutputChannel openChannel(int expectedZero) throws IOException + { + if (expectedZero != 0) { + throw new ISE("Unexpected part [%s]", expectedZero); + } + + return outputChannelFactory.openChannel(channel.getPartitionNumber()); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) + { + throw new UnsupportedOperationException(); + } + + @Override + public OutputChannel openNilChannel(int expectedZero) + { + if (expectedZero != 0) { + throw new ISE("Unexpected part [%s]", expectedZero); + } + + return outputChannelFactory.openNilChannel(channel.getPartitionNumber()); + } + }; + + // Chain futures so we only sort one partition at a time. + nextFuture = Futures.transformAsync( + nextFuture, + ignored -> { + final SuperSorter sorter = new SuperSorter( + Collections.singletonList(channel.getReadableChannel()), + stageDefinition.getFrameReader(), + stageDefinition.getSortKey(), + Futures.immediateFuture(ClusterByPartitions.oneUniversalPartition()), + exec, + partitionOverrideOutputChannelFactory, + makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir), + 1, + 2, + -1, + cancellationId, + + // Tracker is not actually tracked, since it doesn't quite fit into the way we report counters. + // There's a single SuperSorterProgressTrackerCounter per worker, but workers that do local + // sorting have a SuperSorter per partition. + new SuperSorterProgressTracker(), + removeNullBytes + ); + + return FutureUtils.transform(sorter.run(), r -> Iterables.getOnlyElement(r.getAllChannels())); + }, + MoreExecutors.directExecutor() + ); + + sortedChannelFutures.add(nextFuture); + } + + return FutureUtils.transform( + Futures.allAsList(sortedChannelFutures), + sortedChannels -> new ResultAndChannels<>( + Futures.immediateFuture(null), + OutputChannels.wrap(sortedChannels) + ) + ); + } + ); + } + + /** + * Return the (future) output channels for this pipeline. + */ + public ListenableFuture build() + { + if (pipelineFuture == null) { + throw new ISE("Not initialized"); + } + + return Futures.transformAsync( + pipelineFuture, + resultAndChannels -> + Futures.transform( + resultAndChannels.getResultFuture(), + (Function) input -> { + sanityCheckOutputChannels(resultAndChannels.getOutputChannels()); + return resultAndChannels.getOutputChannels(); + }, + Execs.directExecutor() + ), + Execs.directExecutor() + ); + } + + /** + * Adds {@link KeyStatisticsCollectionProcessor}. Called by {@link #gatherResultKeyStatisticsAndReportDoneReadingInputIfNeeded()}. + */ + private ResultAndChannels gatherResultKeyStatistics(final OutputChannels channels) + { + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final List retVal = new ArrayList<>(); + final List processors = new ArrayList<>(); + + for (final OutputChannel outputChannel : channels.getAllChannels()) { + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + retVal.add(OutputChannel.readOnly(channel.readable(), outputChannel.getPartitionNumber())); + + processors.add( + new KeyStatisticsCollectionProcessor( + outputChannel.getReadableChannel(), + channel.writable(), + stageDefinition.getFrameReader(), + stageDefinition.getClusterBy(), + stageDefinition.createResultKeyStatisticsCollector( + frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() + ) + ) + ); + } + + final ListenableFuture clusterByStatisticsCollectorFuture = + exec.runAllFully( + ProcessorManagers.of(processors) + .withAccumulation( + stageDefinition.createResultKeyStatisticsCollector( + frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() + ), + ClusterByStatisticsCollector::addAll + ), + // Run all processors simultaneously. They are lightweight and this keeps things moving. + processors.size(), + Bouncer.unlimited(), + cancellationId + ); + + Futures.addCallback( + clusterByStatisticsCollectorFuture, + new FutureCallback() + { + @Override + public void onSuccess(final ClusterByStatisticsCollector result) + { + listener.onDoneReadingInput(result.snapshot()); + } + + @Override + public void onFailure(Throwable t) + { + listener.onFailure( + new ISE(t, "Failed to gather clusterBy statistics for stage[%s]", stageDefinition.getId()) + ); + } + }, + Execs.directExecutor() + ); + + return new ResultAndChannels<>( + clusterByStatisticsCollectorFuture, + OutputChannels.wrap(retVal) + ); + } + + /** + * Update the {@link #pipelineFuture}. + */ + private void push(final ExceptionalFunction, ResultAndChannels> fn) + { + pushAsync( + channels -> + Futures.immediateFuture(fn.apply(channels)) + ); + } + + /** + * Update the {@link #pipelineFuture} asynchronously. + */ + private void pushAsync(final ExceptionalFunction, ListenableFuture>> fn) + { + if (pipelineFuture == null) { + throw new ISE("Not initialized"); + } + + pipelineFuture = FutureUtils.transform( + Futures.transformAsync( + pipelineFuture, + fn::apply, + Execs.directExecutor() + ), + resultAndChannels -> new ResultAndChannels<>( + resultAndChannels.getResultFuture(), + resultAndChannels.getOutputChannels().readOnly() + ) + ); + } + + /** + * Verifies there is exactly one channel per partition. + */ + private void sanityCheckOutputChannels(final OutputChannels outputChannels) + { + for (int partitionNumber : outputChannels.getPartitionNumbers()) { + final List outputChannelsForPartition = + outputChannels.getChannelsForPartition(partitionNumber); + + Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber); + Preconditions.checkState( + outputChannelsForPartition.size() == 1, + "Expected one channel for partition [%s], but got [%s]", + partitionNumber, + outputChannelsForPartition.size() + ); + } + } + } + + private static class ResultAndChannels + { + private final ListenableFuture resultFuture; + private final OutputChannels outputChannels; + + public ResultAndChannels( + ListenableFuture resultFuture, + OutputChannels outputChannels + ) + { + this.resultFuture = resultFuture; + this.outputChannels = outputChannels; + } + + public ListenableFuture getResultFuture() + { + return resultFuture; + } + + public OutputChannels getOutputChannels() + { + return outputChannels; + } + } + + private interface ExceptionalFunction + { + R apply(T t) throws Exception; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java new file mode 100644 index 000000000000..19c3c6570fe9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java @@ -0,0 +1,57 @@ +/* + * 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.msq.exec; + +import org.apache.druid.frame.processor.OutputChannel; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; + +import javax.annotation.Nullable; + +/** + * Listener for various things that may happen during execution of {@link RunWorkOrder#start()}. Listener methods are + * fired in processing threads, so they must be thread-safe, and it is important that they run quickly. + */ +public interface RunWorkOrderListener +{ + /** + * Called when done reading input. If key statistics were gathered, they are provided. + */ + void onDoneReadingInput(@Nullable ClusterByStatisticsSnapshot snapshot); + + /** + * Called when an output channel becomes available for reading by downstream stages. + */ + void onOutputChannelAvailable(OutputChannel outputChannel); + + /** + * Called when the work order has succeeded. + */ + void onSuccess(Object resultObject); + + /** + * Called when a non-fatal exception is encountered. Work continues after this listener fires. + */ + void onWarning(Throwable t); + + /** + * Called when the work order has failed. + */ + void onFailure(Throwable t); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java index cc5f0fae1732..a90068060d81 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java @@ -19,40 +19,44 @@ package org.apache.druid.msq.exec; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.indexer.TaskStatus; import org.apache.druid.msq.counters.CounterSnapshotsTree; -import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import javax.annotation.Nullable; -import java.io.IOException; import java.io.InputStream; +/** + * Interface for a multi-stage query (MSQ) worker. Workers are long-lived and are able to run multiple {@link WorkOrder} + * prior to exiting. + * + * @see WorkerImpl the production implementation + */ public interface Worker { /** - * Unique ID for this worker. + * Identifier for this worker. Same as {@link WorkerContext#workerId()}. */ String id(); /** - * The task which this worker runs. + * Runs the worker in the current thread. Surrounding classes provide the execution thread. */ - MSQWorkerTask task(); + void run(); /** - * Runs the worker in the current thread. Surrounding classes provide - * the execution thread. + * Terminate the worker upon a cancellation request. Causes a concurrently-running {@link #run()} method in + * a separate thread to cancel all outstanding work and exit. Does not block. Use {@link #awaitStop()} if you + * would like to wait for {@link #run()} to finish. */ - TaskStatus run() throws Exception; + void stop(); /** - * Terminate the worker upon a cancellation request. + * Wait for {@link #run()} to finish. */ - void stopGracefully(); + void awaitStop(); /** * Report that the controller has failed. The worker must cease work immediately. Cleanup then exit. @@ -63,20 +67,20 @@ public interface Worker // Controller-to-worker, and worker-to-worker messages /** - * Called when the worker chat handler receives a request for a work order. Accepts the work order and schedules it for - * execution + * Called when the worker receives a new work order. Accepts the work order and schedules it for execution. */ void postWorkOrder(WorkOrder workOrder); /** * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under - * PARALLEL OR AUTO modes. + * {@link ClusterStatisticsMergeMode#PARALLEL} OR {@link ClusterStatisticsMergeMode#AUTO} modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId); /** * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk. - * This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes. + * This is called from {@link WorkerSketchFetcher} under {@link ClusterStatisticsMergeMode#SEQUENTIAL} or + * {@link ClusterStatisticsMergeMode#AUTO} modes. */ ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk); @@ -84,26 +88,30 @@ public interface Worker * Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber * and queryId */ - boolean postResultPartitionBoundaries( - ClusterByPartitions stagePartitionBoundaries, - String queryId, - int stageNumber - ); + boolean postResultPartitionBoundaries(StageId stageId, ClusterByPartitions stagePartitionBoundaries); /** * Returns an InputStream of the worker output for a particular queryId, stageNumber and partitionNumber. * Offset indicates the number of bytes to skip the channel data, and is used to prevent re-reading the same data - * during retry in case of a connection error + * during retry in case of a connection error. + * + * The returned future resolves when at least one byte of data is available, or when the channel is finished. + * If the channel is finished, an empty {@link InputStream} is returned. + * + * With {@link OutputChannelMode#MEMORY}, once this method is called with a certain offset, workers are free to + * delete data prior to that offset. (Already-requested offsets will not be re-requested, because + * {@link OutputChannelMode#MEMORY} requires a single reader.) In this mode, if an already-requested offset is + * re-requested for some reason, an error future is returned. * - * Returns a null if the workerOutput for a particular queryId, stageNumber, and partitionNumber is not found. + * The returned future resolves to null if stage output for a particular queryId, stageNumber, and + * partitionNumber is not found. * - * @throws IOException when the worker output is found but there is an error while reading it. + * Throws an exception when worker output is found, but there is an error while reading it. */ - @Nullable - InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) throws IOException; + ListenableFuture readStageOutput(StageId stageId, int partitionNumber, long offset); /** - * Returns the snapshot of the worker counters + * Returns a snapshot of counters. */ CounterSnapshotsTree getCounters(); @@ -115,7 +123,7 @@ boolean postResultPartitionBoundaries( void postCleanupStage(StageId stageId); /** - * Called when the work required for the query has been finished + * Called when the worker is no longer needed, and should shut down. */ void postFinish(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java index f5e86039c23f..666115d774cf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -21,11 +21,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; -import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.FrameProcessorFactory; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.server.DruidNode; import java.io.File; @@ -33,10 +34,21 @@ /** * Context used by multi-stage query workers. * - * Useful because it allows test fixtures to provide their own implementations. + * Each context is scoped to a {@link Worker} and is shared across all {@link WorkOrder} run by that worker. */ public interface WorkerContext { + /** + * Query ID for this context. + */ + String queryId(); + + /** + * Identifier for this worker that enables the controller, and other workers, to find it. For tasks this is the + * task ID from {@link MSQWorkerTask#getId()}. For persistent servers, this is the server URI. + */ + String workerId(); + ObjectMapper jsonMapper(); // Using an Injector directly because tasks do not have a way to provide their own Guice modules. @@ -49,9 +61,15 @@ public interface WorkerContext void registerWorker(Worker worker, Closer closer); /** - * Creates and fetches the controller client for the provided controller ID. + * Maximum number of {@link WorkOrder} that a {@link Worker} with this context will be asked to execute + * simultaneously. + */ + int maxConcurrentStages(); + + /** + * Creates a controller client. */ - ControllerClient makeControllerClient(String controllerId); + ControllerClient makeControllerClient(); /** * Creates and fetches a {@link WorkerClient}. It is independent of the workerId because the workerId is passed @@ -60,24 +78,24 @@ public interface WorkerContext WorkerClient makeWorkerClient(); /** - * Fetch a directory for temporary outputs + * Directory for temporary outputs. */ File tempDir(); - FrameContext frameContext(QueryDefinition queryDef, int stageNumber); + /** + * Create a context with useful objects required by {@link FrameProcessorFactory#makeProcessors}. + */ + FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode); + /** + * Number of available processing threads. + */ int threadCount(); /** - * Fetch node info about self + * Fetch node info about self. */ DruidNode selfNode(); - Bouncer processorBouncer(); DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); - - default File tempDir(int stageNumber, String id) - { - return new File(StringUtils.format("%s/stage_%02d/%s", tempDir(), stageNumber, id)); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 61939d823731..7d2964eb2f8c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -19,111 +19,58 @@ package org.apache.druid.msq.exec; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.AsyncFunction; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; -import it.unimi.dsi.fastutil.bytes.ByteArrays; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntObjectPair; import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.frame.allocation.ArenaMemoryAllocator; -import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; -import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import org.apache.druid.frame.channel.ByteTracker; -import org.apache.druid.frame.channel.FrameWithPartition; -import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.channel.ReadableFrameChannel; -import org.apache.druid.frame.channel.ReadableNilFrameChannel; -import org.apache.druid.frame.file.FrameFile; -import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; -import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.frame.processor.ComposingOutputChannelFactory; -import org.apache.druid.frame.processor.FileOutputChannelFactory; -import org.apache.druid.frame.processor.FrameChannelHashPartitioner; -import org.apache.druid.frame.processor.FrameChannelMixer; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; -import org.apache.druid.frame.processor.OutputChannels; -import org.apache.druid.frame.processor.PartitionedOutputChannel; -import org.apache.druid.frame.processor.SuperSorter; -import org.apache.druid.frame.processor.SuperSorterProgressTracker; -import org.apache.druid.frame.processor.manager.ProcessorManager; -import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.frame.util.DurableStorageUtils; -import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.indexing.CountingOutputChannelFactory; import org.apache.druid.msq.indexing.InputChannelFactory; -import org.apache.druid.msq.indexing.InputChannelsImpl; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; -import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportPublisher; import org.apache.druid.msq.indexing.error.MSQWarningReportSimplePublisher; import org.apache.druid.msq.indexing.error.MSQWarnings; -import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.InputSlices; -import org.apache.druid.msq.input.MapInputSliceReader; -import org.apache.druid.msq.input.NilInputSlice; -import org.apache.druid.msq.input.NilInputSliceReader; -import org.apache.druid.msq.input.external.ExternalInputSlice; -import org.apache.druid.msq.input.external.ExternalInputSliceReader; -import org.apache.druid.msq.input.inline.InlineInputSlice; -import org.apache.druid.msq.input.inline.InlineInputSliceReader; -import org.apache.druid.msq.input.lookup.LookupInputSlice; -import org.apache.druid.msq.input.lookup.LookupInputSliceReader; -import org.apache.druid.msq.input.stage.InputChannels; import org.apache.druid.msq.input.stage.ReadablePartition; -import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.input.stage.StageInputSliceReader; -import org.apache.druid.msq.input.table.SegmentsInputSlice; -import org.apache.druid.msq.input.table.SegmentsInputSliceReader; import org.apache.druid.msq.kernel.FrameContext; -import org.apache.druid.msq.kernel.FrameProcessorFactory; -import org.apache.druid.msq.kernel.ProcessorsAndChannels; -import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils; import org.apache.druid.msq.kernel.worker.WorkerStageKernel; import org.apache.druid.msq.kernel.worker.WorkerStagePhase; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; +import org.apache.druid.msq.shuffle.input.MetaInputChannelFactory; import org.apache.druid.msq.shuffle.input.WorkerInputChannelFactory; -import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; -import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; +import org.apache.druid.msq.shuffle.input.WorkerOrLocalInputChannelFactory; +import org.apache.druid.msq.shuffle.output.StageOutputHolder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; import org.apache.druid.msq.util.DecoratedExecutorService; @@ -132,23 +79,14 @@ import org.apache.druid.query.PrioritizedRunnable; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.rpc.ServiceClosedException; import org.apache.druid.server.DruidNode; import javax.annotation.Nullable; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; +import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.io.RandomAccessFile; -import java.nio.channels.Channels; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -156,9 +94,9 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -166,105 +104,93 @@ /** * Interface for a worker of a multi-stage query. + * + * Not scoped to any particular query. There is one of these per {@link MSQWorkerTask}, and one per server for + * long-lived workers. */ public class WorkerImpl implements Worker { private static final Logger log = new Logger(WorkerImpl.class); + /** + * Task object, if this {@link WorkerImpl} was launched from a task. Ideally, this would not be needed, and we + * would be able to get everything we need from {@link WorkerContext}. + */ + @Nullable private final MSQWorkerTask task; private final WorkerContext context; private final DruidNode selfDruidNode; - private final Bouncer processorBouncer; - private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); - private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); - private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); - private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>(); - private final ByteTracker intermediateSuperSorterLocalStorageTracker; - private final boolean durableStageStorageEnabled; - private final WorkerStorageParameters workerStorageParameters; - private final boolean isRemoveNullBytes; + private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>(); + private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); /** - * Only set for select jobs. + * Pair of {workerNumber, stageId} -> counters. */ - @Nullable - private final MSQSelectDestination selectDestination; + private final ConcurrentHashMap, CounterTracker> stageCounters = new ConcurrentHashMap<>(); + + /** + * Atomic that is set to true when {@link #run()} starts (or when {@link #stop()} is called before {@link #run()}). + */ + private final AtomicBoolean didRun = new AtomicBoolean(); /** - * Set once in {@link #runTask} and never reassigned. + * Future that resolves when {@link #run()} completes. + */ + private final SettableFuture runFuture = SettableFuture.create(); + + /** + * Set once in {@link #run} and never reassigned. This is in a field so {@link #doCancel()} can close it. */ private volatile ControllerClient controllerClient; /** - * Set once in {@link #runTask} and never reassigned. Used by processing threads so we can contact other workers + * Set once in {@link #runInternal} and never reassigned. Used by processing threads so we can contact other workers * during a shuffle. */ private volatile WorkerClient workerClient; /** - * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runTask} method to exit promptly. + * Set to false by {@link #controllerFailed()} as a way of enticing the {@link #runInternal} method to exit promptly. */ private volatile boolean controllerAlive = true; - public WorkerImpl(MSQWorkerTask task, WorkerContext context) - { - this( - task, - context, - WorkerStorageParameters.createProductionInstance( - context.injector(), - MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())) - // If Durable Storage is enabled, then super sorter intermediate storage can be enabled. - ) - ); - } - - @VisibleForTesting - public WorkerImpl(MSQWorkerTask task, WorkerContext context, WorkerStorageParameters workerStorageParameters) + public WorkerImpl(@Nullable final MSQWorkerTask task, final WorkerContext context) { this.task = task; this.context = context; this.selfDruidNode = context.selfNode(); - this.processorBouncer = context.processorBouncer(); - QueryContext queryContext = QueryContext.of(task.getContext()); - this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(queryContext); - this.selectDestination = MultiStageQueryContext.getSelectDestinationOrNull(queryContext); - this.isRemoveNullBytes = MultiStageQueryContext.removeNullBytes(queryContext); - this.workerStorageParameters = workerStorageParameters; - - long maxBytes = workerStorageParameters.isIntermediateStorageLimitConfigured() - ? workerStorageParameters.getIntermediateSuperSorterStorageMaxLocalBytes() - : Long.MAX_VALUE; - this.intermediateSuperSorterLocalStorageTracker = new ByteTracker(maxBytes); } @Override public String id() { - return task.getId(); + return context.workerId(); } @Override - public MSQWorkerTask task() + public void run() { - return task; - } + if (!didRun.compareAndSet(false, true)) { + throw new ISE("already run"); + } - @Override - public TaskStatus run() throws Exception - { try (final Closer closer = Closer.create()) { + final KernelHolders kernelHolders = KernelHolders.create(context, closer); + controllerClient = kernelHolders.getControllerClient(); + + Throwable t = null; Optional maybeErrorReport; try { - maybeErrorReport = runTask(closer); + maybeErrorReport = runInternal(kernelHolders, closer); } catch (Throwable e) { + t = e; maybeErrorReport = Optional.of( MSQErrorReport.fromException( - id(), - MSQTasks.getHostFromSelfNode(selfDruidNode), + context.workerId(), + MSQTasks.getHostFromSelfNode(context.selfNode()), null, e ) @@ -273,203 +199,112 @@ public TaskStatus run() throws Exception if (maybeErrorReport.isPresent()) { final MSQErrorReport errorReport = maybeErrorReport.get(); - final String errorLogMessage = MSQTasks.errorReportToLogMessage(errorReport); - log.warn(errorLogMessage); + final String logMessage = MSQTasks.errorReportToLogMessage(errorReport); + log.warn("%s", logMessage); - closer.register(() -> { - if (controllerAlive && controllerClient != null && selfDruidNode != null) { - controllerClient.postWorkerError(id(), errorReport); - } - }); + if (controllerAlive) { + controllerClient.postWorkerError(context.workerId(), errorReport); + } - return TaskStatus.failure(id(), MSQFaultUtils.generateMessageWithErrorCode(errorReport.getFault())); - } else { - return TaskStatus.success(id()); + if (t != null) { + Throwables.throwIfInstanceOf(t, MSQException.class); + throw new MSQException(t, maybeErrorReport.get().getFault()); + } else { + throw new MSQException(maybeErrorReport.get().getFault()); + } } } + catch (IOException e) { + throw new RuntimeException(e); + } + finally { + runFuture.set(null); + } } /** * Runs worker logic. Returns an empty Optional on success. On failure, returns an error report for errors that * happened in other threads; throws exceptions for errors that happened in the main worker loop. */ - public Optional runTask(final Closer closer) throws Exception + private Optional runInternal(final KernelHolders kernelHolders, final Closer workerCloser) + throws Exception { - this.controllerClient = context.makeControllerClient(task.getControllerTaskId()); - closer.register(controllerClient::close); - closer.register(context.dataServerQueryHandlerFactory()); - context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized - - this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient()); - closer.register(workerClient::close); - - final KernelHolder kernelHolder = new KernelHolder(); - final String cancellationId = id(); - + context.registerWorker(this, workerCloser); + workerCloser.register(context.dataServerQueryHandlerFactory()); + this.workerClient = workerCloser.register(new ExceptionWrappingWorkerClient(context.makeWorkerClient())); final FrameProcessorExecutor workerExec = new FrameProcessorExecutor(makeProcessingPool()); - // Delete all the stage outputs - closer.register(() -> { - for (final StageId stageId : stageOutputs.keySet()) { - cleanStageOutput(stageId, false); - } - }); - - // Close stage output processors and running futures (if present) - closer.register(() -> { - try { - workerExec.cancel(cancellationId); - } - catch (InterruptedException e) { - // Strange that cancellation would itself be interrupted. Throw an exception, since this is unexpected. - throw new RuntimeException(e); - } - }); + final long maxAllowedParseExceptions; - long maxAllowedParseExceptions = Long.parseLong(task.getContext().getOrDefault( - MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, - Long.MAX_VALUE - ).toString()); + if (task != null) { + maxAllowedParseExceptions = + Long.parseLong(task.getContext() + .getOrDefault(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, Long.MAX_VALUE) + .toString()); + } else { + maxAllowedParseExceptions = 0; + } - long maxVerboseParseExceptions; + final long maxVerboseParseExceptions; if (maxAllowedParseExceptions == -1L) { maxVerboseParseExceptions = Limits.MAX_VERBOSE_PARSE_EXCEPTIONS; } else { maxVerboseParseExceptions = Math.min(maxAllowedParseExceptions, Limits.MAX_VERBOSE_PARSE_EXCEPTIONS); } - Set criticalWarningCodes; + final Set criticalWarningCodes; if (maxAllowedParseExceptions == 0) { criticalWarningCodes = ImmutableSet.of(CannotParseExternalDataFault.CODE); } else { criticalWarningCodes = ImmutableSet.of(); } - final MSQWarningReportPublisher msqWarningReportPublisher = new MSQWarningReportLimiterPublisher( - new MSQWarningReportSimplePublisher( - id(), - controllerClient, - id(), - MSQTasks.getHostFromSelfNode(selfDruidNode) - ), - Limits.MAX_VERBOSE_WARNINGS, - ImmutableMap.of(CannotParseExternalDataFault.CODE, maxVerboseParseExceptions), - criticalWarningCodes, - controllerClient, - id(), - MSQTasks.getHostFromSelfNode(selfDruidNode) - ); - - closer.register(msqWarningReportPublisher); - - final Map> partitionBoundariesFutureMap = new HashMap<>(); + // Delay removal of kernels so we don't interfere with iteration of kernelHolders.getAllKernelHolders(). + final Set kernelsToRemove = new HashSet<>(); - final Map stageFrameContexts = new HashMap<>(); - - while (!kernelHolder.isDone()) { + while (!kernelHolders.isDone()) { boolean didSomething = false; - for (final WorkerStageKernel kernel : kernelHolder.getStageKernelMap().values()) { + for (final KernelHolder kernelHolder : kernelHolders.getAllKernelHolders()) { + final WorkerStageKernel kernel = kernelHolder.kernel; final StageDefinition stageDefinition = kernel.getStageDefinition(); - if (kernel.getPhase() == WorkerStagePhase.NEW) { - - log.info("Processing work order for stage [%d]" + - (log.isDebugEnabled() - ? StringUtils.format( - " with payload [%s]", - context.jsonMapper().writeValueAsString(kernel.getWorkOrder()) - ) : ""), stageDefinition.getId().getStageNumber()); - - // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and - // so we need to avoid the memoization in baseInputChannelFactory. - final InputChannelFactory inputChannelFactory = makeBaseInputChannelFactory(closer); - - // Compute memory parameters for all stages, even ones that haven't been assigned yet, so we can fail-fast - // if some won't work. (We expect that all stages will get assigned to the same pool of workers.) - for (final StageDefinition stageDef : kernel.getWorkOrder().getQueryDefinition().getStageDefinitions()) { - stageFrameContexts.computeIfAbsent( - stageDef.getId(), - stageId -> context.frameContext( - kernel.getWorkOrder().getQueryDefinition(), - stageId.getStageNumber() - ) - ); - } - - // Start working on this stage immediately. - kernel.startReading(); - - final RunWorkOrder runWorkOrder = new RunWorkOrder( - kernel, - inputChannelFactory, - stageCounters.computeIfAbsent(stageDefinition.getId(), ignored -> new CounterTracker()), + // Workers run all work orders they get. There is not (currently) any limit on the number of concurrent work + // orders; we rely on the controller to avoid overloading workers. + if (kernel.getPhase() == WorkerStagePhase.NEW + && kernelHolders.runningKernelCount() < context.maxConcurrentStages()) { + handleNewWorkOrder( + kernelHolder, + controllerClient, workerExec, - cancellationId, - context.threadCount(), - stageFrameContexts.get(stageDefinition.getId()), - msqWarningReportPublisher + criticalWarningCodes, + maxVerboseParseExceptions ); - - runWorkOrder.start(); - - final SettableFuture partitionBoundariesFuture = - runWorkOrder.getStagePartitionBoundariesFuture(); - - if (partitionBoundariesFuture != null) { - if (partitionBoundariesFutureMap.put(stageDefinition.getId(), partitionBoundariesFuture) != null) { - throw new ISE("Work order collision for stage [%s]", stageDefinition.getId()); - } - } - + logKernelStatus(kernelHolders.getAllKernels()); didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); } - if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) { - if (controllerAlive) { - PartialKeyStatisticsInformation partialKeyStatisticsInformation = - kernel.getResultKeyStatisticsSnapshot() - .partialKeyStatistics(); - - controllerClient.postPartialKeyStatistics( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber(), - partialKeyStatisticsInformation - ); - } - kernel.startPreshuffleWaitingForResultPartitionBoundaries(); - + if (kernel.getPhase() == WorkerStagePhase.READING_INPUT + && handleReadingInput(kernelHolder, controllerClient)) { didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); + logKernelStatus(kernelHolders.getAllKernels()); } - logKernelStatus(kernelHolder.getStageKernelMap().values()); if (kernel.getPhase() == WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES - && kernel.hasResultPartitionBoundaries()) { - partitionBoundariesFutureMap.get(stageDefinition.getId()).set(kernel.getResultPartitionBoundaries()); - kernel.startPreshuffleWritingOutput(); - + && handleWaitingForResultPartitionBoundaries(kernelHolder)) { didSomething = true; - logKernelStatus(kernelHolder.getStageKernelMap().values()); + logKernelStatus(kernelHolders.getAllKernels()); } - if (kernel.getPhase() == WorkerStagePhase.RESULTS_READY - && kernel.addPostedResultsComplete(Pair.of( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber() - ))) { - if (controllerAlive) { - controllerClient.postResultsComplete( - stageDefinition.getId(), - kernel.getWorkOrder().getWorkerNumber(), - kernel.getResultObject() - ); - } + if (kernel.getPhase() == WorkerStagePhase.RESULTS_COMPLETE + && handleResultsReady(kernelHolder, controllerClient)) { + didSomething = true; + logKernelStatus(kernelHolders.getAllKernels()); } if (kernel.getPhase() == WorkerStagePhase.FAILED) { - // Better than throwing an exception, because we can include the stage number. + // Return an error report when a work order fails. This is better than throwing an exception, because we can + // include the stage number. return Optional.of( MSQErrorReport.fromException( id(), @@ -479,17 +314,37 @@ public Optional runTask(final Closer closer) throws Exception ) ); } + + if (kernel.getPhase().isTerminal()) { + handleTerminated(kernelHolder); + kernelsToRemove.add(stageDefinition.getId()); + } + } + + for (final StageId stageId : kernelsToRemove) { + kernelHolders.removeKernel(stageId); } - if (!didSomething && !kernelHolder.isDone()) { - Consumer nextCommand; + kernelsToRemove.clear(); + if (!didSomething && !kernelHolders.isDone()) { + Consumer nextCommand; + + // Run the next command, waiting for it if necessary. Post counters to the controller every 5 seconds + // while waiting. do { - postCountersToController(); + postCountersToController(kernelHolders.getControllerClient()); } while ((nextCommand = kernelManipulationQueue.poll(5, TimeUnit.SECONDS)) == null); - nextCommand.accept(kernelHolder); - logKernelStatus(kernelHolder.getStageKernelMap().values()); + nextCommand.accept(kernelHolders); + + // Run all pending commands after that one. Helps avoid deep queues. + // After draining the command queue, move on to the next iteration of the worker loop. + while ((nextCommand = kernelManipulationQueue.poll()) != null) { + nextCommand.accept(kernelHolders); + } + + logKernelStatus(kernelHolders.getAllKernels()); } } @@ -497,123 +352,288 @@ public Optional runTask(final Closer closer) throws Exception return Optional.empty(); } - @Override - public void stopGracefully() + /** + * Handle a kernel in state {@link WorkerStagePhase#NEW}. The kernel is transitioned to + * {@link WorkerStagePhase#READING_INPUT} and a {@link RunWorkOrder} instance is created to start executing work. + */ + private void handleNewWorkOrder( + final KernelHolder kernelHolder, + final ControllerClient controllerClient, + final FrameProcessorExecutor workerExec, + final Set criticalWarningCodes, + final long maxVerboseParseExceptions + ) throws IOException { - // stopGracefully() is called when the containing process is terminated, or when the task is canceled. - log.info("Worker task[%s] canceled.", task.getId()); - doCancel(); - } + final WorkerStageKernel kernel = kernelHolder.kernel; + final WorkOrder workOrder = kernel.getWorkOrder(); + final StageDefinition stageDefinition = workOrder.getStageDefinition(); + final String cancellationId = cancellationIdFor(stageDefinition.getId()); + + log.info( + "Processing work order for stage[%s]%s", + stageDefinition.getId(), + (log.isDebugEnabled() + ? StringUtils.format(", payload[%s]", context.jsonMapper().writeValueAsString(workOrder)) : "") + ); - @Override - public void controllerFailed() - { - log.info("Controller task[%s] for worker task[%s] failed. Canceling.", task.getControllerTaskId(), task.getId()); - doCancel(); + final FrameContext frameContext = kernelHolder.processorCloser.register( + context.frameContext( + workOrder.getQueryDefinition(), + stageDefinition.getStageNumber(), + workOrder.getOutputChannelMode() + ) + ); + kernelHolder.processorCloser.register(() -> { + try { + workerExec.cancel(cancellationId); + } + catch (InterruptedException e) { + // Strange that cancellation would itself be interrupted. Log and suppress. + log.warn(e, "Cancellation interrupted for stage[%s]", stageDefinition.getId()); + Thread.currentThread().interrupt(); + } + }); + + // Set up cleanup functions for this work order. + kernelHolder.resultsCloser.register(() -> FileUtils.deleteDirectory(frameContext.tempDir())); + kernelHolder.resultsCloser.register(() -> removeStageOutputChannels(stageDefinition.getId())); + + // Create separate inputChannelFactory per stage, because the list of tasks can grow between stages, and + // so we need to avoid the memoization of controllerClient.getWorkerIds() in baseInputChannelFactory. + final InputChannelFactory inputChannelFactory = + makeBaseInputChannelFactory(workOrder, controllerClient, kernelHolder.processorCloser); + + // Start working on this stage immediately. + kernel.startReading(); + + final QueryContext queryContext = task != null ? QueryContext.of(task.getContext()) : QueryContext.empty(); + final RunWorkOrder runWorkOrder = new RunWorkOrder( + workOrder, + inputChannelFactory, + stageCounters.computeIfAbsent( + IntObjectPair.of(workOrder.getWorkerNumber(), stageDefinition.getId()), + ignored -> new CounterTracker() + ), + workerExec, + cancellationId, + context, + frameContext, + makeRunWorkOrderListener(workOrder, controllerClient, criticalWarningCodes, maxVerboseParseExceptions), + MultiStageQueryContext.isReindex(queryContext), + MultiStageQueryContext.removeNullBytes(queryContext) + ); + + runWorkOrder.start(); + kernelHolder.partitionBoundariesFuture = runWorkOrder.getStagePartitionBoundariesFuture(); } - @Override - public InputStream readChannel( - final String queryId, - final int stageNumber, - final int partitionNumber, - final long offset + /** + * Handle a kernel in state {@link WorkerStagePhase#READING_INPUT}. + * + * If the worker has finished generating result key statistics, they are posted to the controller and the kernel is + * transitioned to {@link WorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}. + * + * @return whether kernel state changed + */ + private boolean handleReadingInput( + final KernelHolder kernelHolder, + final ControllerClient controllerClient ) throws IOException { - final StageId stageId = new StageId(queryId, stageNumber); - final StagePartition stagePartition = new StagePartition(stageId, partitionNumber); - final ConcurrentHashMap partitionOutputsForStage = stageOutputs.get(stageId); + final WorkerStageKernel kernel = kernelHolder.kernel; + if (kernel.hasResultKeyStatisticsSnapshot()) { + if (controllerAlive) { + PartialKeyStatisticsInformation partialKeyStatisticsInformation = + kernel.getResultKeyStatisticsSnapshot() + .partialKeyStatistics(); + + controllerClient.postPartialKeyStatistics( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber(), + partialKeyStatisticsInformation + ); + } - if (partitionOutputsForStage == null) { - return null; + kernel.startPreshuffleWaitingForResultPartitionBoundaries(); + return true; + } else if (kernel.isDoneReadingInput() + && kernel.getStageDefinition().doesSortDuringShuffle() + && !kernel.getStageDefinition().mustGatherResultKeyStatistics()) { + // Skip postDoneReadingInput when context.maxConcurrentStages() == 1, for backwards compatibility. + // See Javadoc comment on ControllerClient#postDoneReadingInput. + if (controllerAlive && context.maxConcurrentStages() > 1) { + controllerClient.postDoneReadingInput( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber() + ); + } + + kernel.startPreshuffleWritingOutput(); + return true; + } else { + return false; + } + } + + /** + * Handle a kernel in state {@link WorkerStagePhase#PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES}. + * + * If partition boundaries have become available, the {@link KernelHolder#partitionBoundariesFuture} is updated and + * the kernel is transitioned to state {@link WorkerStagePhase#PRESHUFFLE_WRITING_OUTPUT}. + * + * @return whether kernel state changed + */ + private boolean handleWaitingForResultPartitionBoundaries(final KernelHolder kernelHolder) + { + if (kernelHolder.kernel.hasResultPartitionBoundaries()) { + kernelHolder.partitionBoundariesFuture.set(kernelHolder.kernel.getResultPartitionBoundaries()); + kernelHolder.kernel.startPreshuffleWritingOutput(); + return true; + } else { + return false; } - final ReadableFrameChannel channel = partitionOutputsForStage.get(partitionNumber); + } - if (channel == null) { - return null; + /** + * Handle a kernel in state {@link WorkerStagePhase#RESULTS_COMPLETE}. If {@link ControllerClient#postResultsComplete} + * has not yet been posted to the controller, it is posted at this time. Otherwise nothing happens. + * + * @return whether kernel state changed + */ + private boolean handleResultsReady(final KernelHolder kernelHolder, final ControllerClient controllerClient) + throws IOException + { + final WorkerStageKernel kernel = kernelHolder.kernel; + final boolean didNotPostYet = + kernel.addPostedResultsComplete(kernel.getStageDefinition().getId(), kernel.getWorkOrder().getWorkerNumber()); + + if (controllerAlive && didNotPostYet) { + controllerClient.postResultsComplete( + kernel.getStageDefinition().getId(), + kernel.getWorkOrder().getWorkerNumber(), + kernel.getResultObject() + ); } - if (channel instanceof ReadableNilFrameChannel) { - // Build an empty frame file. - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close(); + return didNotPostYet; + } - final ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray()); + /** + * Handle a kernel in state where {@link WorkerStagePhase#isTerminal()} is true. + */ + private void handleTerminated(final KernelHolder kernelHolder) + { + final WorkerStageKernel kernel = kernelHolder.kernel; + removeStageOutputChannels(kernel.getStageDefinition().getId()); - //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream. - in.skip(offset); + if (kernelHolder.kernel.getWorkOrder().getOutputChannelMode().isDurable()) { + removeStageDurableStorageOutput(kernel.getStageDefinition().getId()); + } + } - return in; - } else if (channel instanceof ReadableFileFrameChannel) { - // Close frameFile once we've returned an input stream: no need to retain a reference to the mmap after that, - // since we aren't using it. - try (final FrameFile frameFile = ((ReadableFileFrameChannel) channel).newFrameFileReference()) { - final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r"); + @Override + public void stop() + { + // stopGracefully() is called when the containing process is terminated, or when the task is canceled. + log.info("Worker id[%s] canceled.", context.workerId()); - if (offset >= randomAccessFile.length()) { - randomAccessFile.close(); - return new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY); - } else { - randomAccessFile.seek(offset); - return Channels.newInputStream(randomAccessFile.getChannel()); - } - } + if (didRun.compareAndSet(false, true)) { + // run() hasn't been called yet. Set runFuture so awaitStop() still works. + runFuture.set(null); } else { - String errorMsg = StringUtils.format( - "Returned server error to client because channel for [%s] is not nil or file-based (class = %s)", - stagePartition, - channel.getClass().getName() - ); - log.error(StringUtils.encodeForFormat(errorMsg)); - - throw new IOException(errorMsg); + doCancel(); } } + @Override + public void awaitStop() + { + FutureUtils.getUnchecked(runFuture, false); + } + + @Override + public void controllerFailed() + { + log.info( + "Controller task[%s] for worker[%s] failed. Canceling.", + task != null ? task.getControllerTaskId() : null, + id() + ); + doCancel(); + } + + @Override + public ListenableFuture readStageOutput( + final StageId stageId, + final int partitionNumber, + final long offset + ) + { + return getOrCreateStageOutputHolder(stageId, partitionNumber).readRemotelyFrom(offset); + } + @Override public void postWorkOrder(final WorkOrder workOrder) { - log.info("Got work order for stage [%d]", workOrder.getStageNumber()); - if (task.getWorkerNumber() != workOrder.getWorkerNumber()) { - throw new ISE("Worker number mismatch: expected [%d]", task.getWorkerNumber()); + log.info( + "Got work order for stage[%s], workerNumber[%s]", + workOrder.getStageDefinition().getId(), + workOrder.getWorkerNumber() + ); + + if (task != null && task.getWorkerNumber() != workOrder.getWorkerNumber()) { + throw new ISE( + "Worker number mismatch: expected workerNumber[%d], got[%d]", + task.getWorkerNumber(), + workOrder.getWorkerNumber() + ); + } + + final OutputChannelMode outputChannelMode; + + // This stack of conditions can be removed once we can rely on OutputChannelMode always being in the WorkOrder. + // (It will be there for newer controllers; this is a backwards-compatibility thing.) + if (workOrder.hasOutputChannelMode()) { + outputChannelMode = workOrder.getOutputChannelMode(); + } else { + final MSQSelectDestination selectDestination = + task != null + ? MultiStageQueryContext.getSelectDestination(QueryContext.of(task.getContext())) + : MSQSelectDestination.TASKREPORT; + + outputChannelMode = ControllerQueryKernelUtils.getOutputChannelMode( + workOrder.getQueryDefinition(), + workOrder.getStageNumber(), + selectDestination, + task != null && MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())), + false + ); } - // Do not add to queue if workerOrder already present. + final WorkOrder workOrderToUse = workOrder.withOutputChannelMode(outputChannelMode); kernelManipulationQueue.add( - kernelHolder -> - kernelHolder.getStageKernelMap().putIfAbsent( - workOrder.getStageDefinition().getId(), - WorkerStageKernel.create(workOrder) - ) + kernelHolders -> + kernelHolders.addKernel(WorkerStageKernel.create(workOrderToUse)) ); } @Override public boolean postResultPartitionBoundaries( - final ClusterByPartitions stagePartitionBoundaries, - final String queryId, - final int stageNumber + final StageId stageId, + final ClusterByPartitions stagePartitionBoundaries ) { - final StageId stageId = new StageId(queryId, stageNumber); - kernelManipulationQueue.add( - kernelHolder -> { - final WorkerStageKernel stageKernel = kernelHolder.getStageKernelMap().get(stageId); + kernelHolders -> { + final WorkerStageKernel stageKernel = kernelHolders.getKernelFor(stageId); if (stageKernel != null) { if (!stageKernel.hasResultPartitionBoundaries()) { stageKernel.setResultPartitionBoundaries(stagePartitionBoundaries); } else { // Ignore if partition boundaries are already set. - log.warn( - "Stage[%s] already has result partition boundaries set. Ignoring the latest partition boundaries recieved.", - stageId - ); + log.warn("Stage[%s] already has result partition boundaries set. Ignoring new ones.", stageId); } - } else { - // Ignore the update if we don't have a kernel for this stage. - log.warn("Ignored result partition boundaries call for unknown stage [%s]", stageId); } } ); @@ -623,167 +643,230 @@ public boolean postResultPartitionBoundaries( @Override public void postCleanupStage(final StageId stageId) { - log.info("Cleanup order for stage [%s] received", stageId); - kernelManipulationQueue.add( - holder -> { - cleanStageOutput(stageId, true); - // Mark the stage as FINISHED - WorkerStageKernel stageKernel = holder.getStageKernelMap().get(stageId); - if (stageKernel == null) { - log.warn("Stage id [%s] non existent. Unable to mark the stage kernel for it as FINISHED", stageId); - } else { - stageKernel.setStageFinished(); - } - } - ); + log.debug("Received cleanup order for stage[%s].", stageId); + kernelManipulationQueue.add(holder -> { + holder.finishProcessing(stageId); + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setStageFinished(); + } + }); } @Override public void postFinish() { - log.info("Finish received for task [%s]", task.getId()); - kernelManipulationQueue.add(KernelHolder::setDone); + log.debug("Received finish call."); + kernelManipulationQueue.add(KernelHolders::setDone); } @Override public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId) { - log.info("Fetching statistics for stage [%d]", stageId.getStageNumber()); - if (stageKernelMap.get(stageId) == null) { - throw new ISE("Requested statistics snapshot for non-existent stageId %s.", stageId); - } else if (stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot() == null) { - throw new ISE( - "Requested statistics snapshot is not generated yet for stageId [%s]", - stageId - ); - } else { - return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot(); - } + log.debug("Fetching statistics for stage[%s]", stageId); + final SettableFuture snapshotFuture = SettableFuture.create(); + kernelManipulationQueue.add( + holder -> { + try { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + final ClusterByStatisticsSnapshot snapshot = kernel.getResultKeyStatisticsSnapshot(); + if (snapshot == null) { + throw new ISE("Requested statistics snapshot is not generated yet for stage [%s]", stageId); + } + + snapshotFuture.set(snapshot); + } else { + snapshotFuture.setException(new ISE("Stage[%s] has terminated", stageId)); + } + } + catch (Throwable t) { + snapshotFuture.setException(t); + } + } + ); + return FutureUtils.getUnchecked(snapshotFuture, true); } @Override public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk) { - log.debug( - "Fetching statistics for stage [%d] with time chunk [%d] ", - stageId.getStageNumber(), - timeChunk - ); - if (stageKernelMap.get(stageId) == null) { - throw new ISE("Requested statistics snapshot for non-existent stageId [%s].", stageId); - } else if (stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot() == null) { - throw new ISE( - "Requested statistics snapshot is not generated yet for stageId [%s]", - stageId - ); - } else { - return stageKernelMap.get(stageId) - .getResultKeyStatisticsSnapshot() - .getSnapshotForTimeChunk(timeChunk); - } - + return fetchStatisticsSnapshot(stageId).getSnapshotForTimeChunk(timeChunk); } - @Override public CounterSnapshotsTree getCounters() { final CounterSnapshotsTree retVal = new CounterSnapshotsTree(); - for (final Map.Entry entry : stageCounters.entrySet()) { - retVal.put(entry.getKey().getStageNumber(), task().getWorkerNumber(), entry.getValue().snapshot()); + for (final Map.Entry, CounterTracker> entry : stageCounters.entrySet()) { + retVal.put( + entry.getKey().right().getStageNumber(), + entry.getKey().leftInt(), + entry.getValue().snapshot() + ); } return retVal; } - private InputChannelFactory makeBaseInputChannelFactory(final Closer closer) + /** + * Create a {@link RunWorkOrderListener} for {@link RunWorkOrder} that hooks back into the {@link KernelHolders} + * in the main loop. + */ + private RunWorkOrderListener makeRunWorkOrderListener( + final WorkOrder workOrder, + final ControllerClient controllerClient, + final Set criticalWarningCodes, + final long maxVerboseParseExceptions + ) { - final Supplier> workerTaskList = Suppliers.memoize( - () -> { - try { - return controllerClient.getTaskList(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - )::get; + final StageId stageId = workOrder.getStageDefinition().getId(); + final MSQWarningReportPublisher msqWarningReportPublisher = new MSQWarningReportLimiterPublisher( + new MSQWarningReportSimplePublisher( + id(), + controllerClient, + id(), + MSQTasks.getHostFromSelfNode(selfDruidNode) + ), + Limits.MAX_VERBOSE_WARNINGS, + ImmutableMap.of(CannotParseExternalDataFault.CODE, maxVerboseParseExceptions), + criticalWarningCodes, + controllerClient, + id(), + MSQTasks.getHostFromSelfNode(selfDruidNode) + ); - if (durableStageStorageEnabled) { - return DurableStorageInputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - MSQTasks.makeStorageConnector(context.injector()), - closer, - false - ); - } else { - return new WorkerOrLocalInputChannelFactory(workerTaskList); - } - } + return new RunWorkOrderListener() + { + @Override + public void onDoneReadingInput(@Nullable ClusterByStatisticsSnapshot snapshot) + { + kernelManipulationQueue.add( + holder -> { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setResultKeyStatisticsSnapshot(snapshot); + } + } + ); + } - private OutputChannelFactory makeStageOutputChannelFactory( - final FrameContext frameContext, - final int stageNumber, - boolean isFinalStage - ) - { - // Use the standard frame size, since we assume this size when computing how much is needed to merge output - // files from different workers. - final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); - - if (durableStageStorageEnabled || (isFinalStage - && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination))) { - return DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameSize, - MSQTasks.makeStorageConnector(context.injector()), - context.tempDir(), - (isFinalStage && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination)) - ); - } else { - final File fileChannelDirectory = - new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber)); + @Override + public void onOutputChannelAvailable(OutputChannel channel) + { + ReadableFrameChannel readableChannel = null; - return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); - } - } + try { + readableChannel = channel.getReadableChannel(); + getOrCreateStageOutputHolder(stageId, channel.getPartitionNumber()) + .setChannel(readableChannel); + } + catch (Exception e) { + if (readableChannel != null) { + try { + readableChannel.close(); + } + catch (Throwable e2) { + e.addSuppressed(e2); + } + } - private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( - final FrameContext frameContext, - final int stageNumber, - final File tmpDir - ) - { - final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); - final File fileChannelDirectory = - new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); - final FileOutputChannelFactory fileOutputChannelFactory = - new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterLocalStorageTracker); - - if (durableStageStorageEnabled && workerStorageParameters.isIntermediateStorageLimitConfigured()) { - return new ComposingOutputChannelFactory( - ImmutableList.of( - fileOutputChannelFactory, - DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameSize, - MSQTasks.makeStorageConnector(context.injector()), - tmpDir, - false - ) - ), - frameSize - ); - } else { - return fileOutputChannelFactory; - } + kernelManipulationQueue.add(holder -> { + throw new RE(e, "Worker completion callback error for stage [%s]", stageId); + }); + } + } + + @Override + public void onSuccess(Object resultObject) + { + kernelManipulationQueue.add( + holder -> { + // Call finishProcessing prior to transitioning to RESULTS_COMPLETE, so the FrameContext is closed + // and resources are released. + holder.finishProcessing(stageId); + + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.setResultsComplete(resultObject); + } + } + ); + } + + @Override + public void onWarning(Throwable t) + { + msqWarningReportPublisher.publishException(stageId.getStageNumber(), t); + } + + @Override + public void onFailure(Throwable t) + { + kernelManipulationQueue.add( + holder -> { + final WorkerStageKernel kernel = holder.getKernelFor(stageId); + if (kernel != null) { + kernel.fail(t); + } + } + ); + } + }; + } + + private InputChannelFactory makeBaseInputChannelFactory( + final WorkOrder workOrder, + final ControllerClient controllerClient, + final Closer closer + ) + { + return MetaInputChannelFactory.create( + InputSlices.allStageSlices(workOrder.getInputs()), + workOrder.getOutputChannelMode(), + outputChannelMode -> { + switch (outputChannelMode) { + case MEMORY: + case LOCAL_STORAGE: + final Supplier> workerIds; + + if (workOrder.getWorkerIds() != null) { + workerIds = workOrder::getWorkerIds; + } else { + workerIds = Suppliers.memoize( + () -> { + try { + return controllerClient.getWorkerIds(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); + } + + return new WorkerOrLocalInputChannelFactory( + id(), + workerIds, + new WorkerInputChannelFactory(workerClient, workerIds), + this::getOrCreateStageOutputHolder + ); + + case DURABLE_STORAGE_INTERMEDIATE: + case DURABLE_STORAGE_QUERY_RESULTS: + return DurableStorageInputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + MSQTasks.makeStorageConnector(context.injector()), + closer, + outputChannelMode == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS + ); + + default: + throw DruidException.defensive("No handling for output channel mode[%s]", outputChannelMode); + } + } + ); } /** @@ -846,69 +929,75 @@ public void run() /** * Posts all counters for this worker to the controller. */ - private void postCountersToController() throws IOException + private void postCountersToController(final ControllerClient controllerClient) throws IOException { final CounterSnapshotsTree snapshotsTree = getCounters(); if (controllerAlive && !snapshotsTree.isEmpty()) { - try { - controllerClient.postCounters(id(), snapshotsTree); - } - catch (IOException e) { - if (e.getCause() instanceof ServiceClosedException) { - // Suppress. This can happen if the controller goes away while a postCounters call is in flight. - log.debug(e, "Ignoring failure on postCounters, because controller has gone away."); - } else { - throw e; - } - } + controllerClient.postCounters(id(), snapshotsTree); } } /** - * Cleans up the stage outputs corresponding to the provided stage id. It essentially calls {@code doneReading()} on - * the readable channels corresponding to all the partitions for that stage, and removes it from the {@code stageOutputs} - * map + * Removes and closes all output channels for a stage from {@link #stageOutputs}. */ - private void cleanStageOutput(final StageId stageId, boolean removeDurableStorageFiles) + private void removeStageOutputChannels(final StageId stageId) { // This code is thread-safe because remove() on ConcurrentHashMap will remove and return the removed channel only for // one thread. For the other threads it will return null, therefore we will call doneReading for a channel only once - final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId); + final ConcurrentHashMap partitionOutputsForStage = stageOutputs.remove(stageId); // Check for null, this can be the case if this method is called simultaneously from multiple threads. if (partitionOutputsForStage == null) { return; } for (final int partition : partitionOutputsForStage.keySet()) { - final ReadableFrameChannel output = partitionOutputsForStage.remove(partition); - if (output == null) { - continue; + final StageOutputHolder output = partitionOutputsForStage.remove(partition); + if (output != null) { + output.close(); } - output.close(); } + } + + /** + * Remove outputs from durable storage for a particular stage. + */ + private void removeStageDurableStorageOutput(final StageId stageId) + { // One caveat with this approach is that in case of a worker crash, while the MM/Indexer systems will delete their // temp directories where intermediate results were stored, it won't be the case for the external storage. // Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external. // We currently take care of this in the controller. - if (durableStageStorageEnabled && removeDurableStorageFiles) { - final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName( - task.getControllerTaskId(), - stageId.getStageNumber(), - task.getWorkerNumber(), - task.getId() - ); - try { - MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(folderName); - } - catch (Exception e) { - // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup - log.warn(e, "Error while cleaning up folder at path " + folderName); - } + final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName( + task.getControllerTaskId(), + stageId.getStageNumber(), + task.getWorkerNumber(), + context.workerId() + ); + try { + MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(folderName); + } + catch (Exception e) { + // If an error is thrown while cleaning up a file, log it and try to continue with the cleanup + log.warn(e, "Error while cleaning up durable storage path[%s].", folderName); } } + private StageOutputHolder getOrCreateStageOutputHolder(final StageId stageId, final int partitionNumber) + { + return stageOutputs.computeIfAbsent(stageId, ignored1 -> new ConcurrentHashMap<>()) + .computeIfAbsent(partitionNumber, ignored -> new StageOutputHolder()); + } + /** - * Called by {@link #stopGracefully()} (task canceled, or containing process shut down) and + * Returns cancellation ID for a particular stage, to be used in {@link FrameProcessorExecutor#cancel(String)}. + */ + private static String cancellationIdFor(final StageId stageId) + { + return stageId.toString(); + } + + /** + * Called by {@link #stop()} (task canceled, or containing process shut down) and * {@link #controllerFailed()}. */ private void doCancel() @@ -935,15 +1024,15 @@ private void doCancel() /** * Log (at DEBUG level) a string explaining the status of all work assigned to this worker. */ - private static void logKernelStatus(final Collection kernels) + private static void logKernelStatus(final Iterable kernels) { if (log.isDebugEnabled()) { log.debug( "Stages: %s", - kernels.stream() - .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber())) - .map(WorkerImpl::makeKernelStageStatusString) - .collect(Collectors.joining("; ")) + StreamSupport.stream(kernels.spliterator(), false) + .sorted(Comparator.comparing(k -> k.getStageDefinition().getStageNumber())) + .map(WorkerImpl::makeKernelStageStatusString) + .collect(Collectors.joining("; ")) ); } } @@ -978,936 +1067,205 @@ private static String makeKernelStageStatusString(final WorkerStageKernel kernel } /** - * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other - * workers. Used when durable shuffle storage is off. + * Holds {@link WorkerStageKernel} and {@link Closer}, one per {@link WorkOrder}. Also holds {@link ControllerClient}. + * Only manipulated by the main loop. Other threads that need to manipulate kernels must do so through + * {@link #kernelManipulationQueue}. */ - private class WorkerOrLocalInputChannelFactory implements InputChannelFactory + private static class KernelHolders implements Closeable { - private final Supplier> taskList; - private final WorkerInputChannelFactory workerInputChannelFactory; - - public WorkerOrLocalInputChannelFactory(final Supplier> taskList) - { - this.workerInputChannelFactory = new WorkerInputChannelFactory(workerClient, taskList); - this.taskList = taskList; - } + private final WorkerContext workerContext; + private final ControllerClient controllerClient; - @Override - public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) - { - final String taskId = taskList.get().get(workerNumber); - if (taskId.equals(id())) { - final ConcurrentMap partitionOutputsForStage = stageOutputs.get(stageId); - if (partitionOutputsForStage == null) { - throw new ISE("Unable to find outputs for stage [%s]", stageId); - } - - final ReadableFrameChannel myChannel = partitionOutputsForStage.get(partitionNumber); + /** + * Stage number -> kernel holder. + */ + private final Int2ObjectMap holderMap = new Int2ObjectOpenHashMap<>(); - if (myChannel instanceof ReadableFileFrameChannel) { - // Must duplicate the channel to avoid double-closure upon task cleanup. - final FrameFile frameFile = ((ReadableFileFrameChannel) myChannel).newFrameFileReference(); - return new ReadableFileFrameChannel(frameFile); - } else if (myChannel instanceof ReadableNilFrameChannel) { - return myChannel; - } else { - throw new ISE("Output for stage [%s] are stored in an instance of %s which is not " - + "supported", stageId, myChannel.getClass()); - } - } else { - return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber); - } - } - } + private boolean done = false; - /** - * Main worker logic for executing a {@link WorkOrder}. - */ - private class RunWorkOrder - { - private final WorkerStageKernel kernel; - private final InputChannelFactory inputChannelFactory; - private final CounterTracker counterTracker; - private final FrameProcessorExecutor exec; - private final String cancellationId; - private final int parallelism; - private final FrameContext frameContext; - private final MSQWarningReportPublisher warningPublisher; - - private InputSliceReader inputSliceReader; - private OutputChannelFactory workOutputChannelFactory; - private OutputChannelFactory shuffleOutputChannelFactory; - private ResultAndChannels workResultAndOutputChannels; - private SettableFuture stagePartitionBoundariesFuture; - private ListenableFuture shuffleOutputChannelsFuture; - - public RunWorkOrder( - final WorkerStageKernel kernel, - final InputChannelFactory inputChannelFactory, - final CounterTracker counterTracker, - final FrameProcessorExecutor exec, - final String cancellationId, - final int parallelism, - final FrameContext frameContext, - final MSQWarningReportPublisher warningPublisher - ) + private KernelHolders(final WorkerContext workerContext, final ControllerClient controllerClient) { - this.kernel = kernel; - this.inputChannelFactory = inputChannelFactory; - this.counterTracker = counterTracker; - this.exec = exec; - this.cancellationId = cancellationId; - this.parallelism = parallelism; - this.frameContext = frameContext; - this.warningPublisher = warningPublisher; + this.workerContext = workerContext; + this.controllerClient = controllerClient; } - private void start() throws IOException + public static KernelHolders create(final WorkerContext workerContext, final Closer closer) { - final WorkOrder workOrder = kernel.getWorkOrder(); - final StageDefinition stageDef = workOrder.getStageDefinition(); - - final boolean isFinalStage = stageDef.getStageNumber() == workOrder.getQueryDefinition() - .getFinalStageDefinition() - .getStageNumber(); - - makeInputSliceReader(); - makeWorkOutputChannelFactory(isFinalStage); - makeShuffleOutputChannelFactory(isFinalStage); - makeAndRunWorkProcessors(); - - if (stageDef.doesShuffle()) { - makeAndRunShuffleProcessors(); - } else { - // No shuffling: work output _is_ shuffle output. Retain read-only versions to reduce memory footprint. - shuffleOutputChannelsFuture = - Futures.immediateFuture(workResultAndOutputChannels.getOutputChannels().readOnly()); - } - - setUpCompletionCallbacks(isFinalStage); + return closer.register(new KernelHolders(workerContext, closer.register(workerContext.makeControllerClient()))); } /** - * Settable {@link ClusterByPartitions} future for global sort. Necessary because we don't know ahead of time - * what the boundaries will be. The controller decides based on statistics from all workers. Once the controller - * decides, its decision is written to this future, which allows sorting on workers to proceed. + * Add a {@link WorkerStageKernel} to this holder. Also creates a {@link ControllerClient} for the query ID + * if one does not yet exist. Does nothing if a kernel with the same {@link StageId} is already being tracked. */ - @Nullable - public SettableFuture getStagePartitionBoundariesFuture() + public void addKernel(final WorkerStageKernel kernel) { - return stagePartitionBoundariesFuture; - } + final StageId stageId = verifyQueryId(kernel.getWorkOrder().getStageDefinition().getId()); - private void makeInputSliceReader() - { - if (inputSliceReader != null) { - throw new ISE("inputSliceReader already created"); + if (holderMap.putIfAbsent(stageId.getStageNumber(), new KernelHolder(kernel)) != null) { + // Already added. Do nothing. } - - final WorkOrder workOrder = kernel.getWorkOrder(); - final String queryId = workOrder.getQueryDefinition().getQueryId(); - - final InputChannels inputChannels = - new InputChannelsImpl( - workOrder.getQueryDefinition(), - InputSlices.allReadablePartitions(workOrder.getInputs()), - inputChannelFactory, - () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), - exec, - cancellationId, - MultiStageQueryContext.removeNullBytes(QueryContext.of(task.getContext())) - ); - - inputSliceReader = new MapInputSliceReader( - ImmutableMap., InputSliceReader>builder() - .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) - .put(StageInputSlice.class, new StageInputSliceReader(queryId, inputChannels)) - .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir())) - .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) - .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) - .put( - SegmentsInputSlice.class, - new SegmentsInputSliceReader( - frameContext, - MultiStageQueryContext.isReindex(QueryContext.of(task().getContext())) - ) - ) - .build() - ); - } - - private void makeWorkOutputChannelFactory(boolean isFinalStage) - { - if (workOutputChannelFactory != null) { - throw new ISE("processorOutputChannelFactory already created"); - } - - final OutputChannelFactory baseOutputChannelFactory; - - if (kernel.getStageDefinition().doesShuffle()) { - // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame - // size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames. - // Otherwise, use the standard frame size. - final int frameSize; - - if (kernel.getStageDefinition().getShuffleSpec().kind().isSort()) { - frameSize = frameContext.memoryParameters().getLargeFrameSize(); - } else { - frameSize = frameContext.memoryParameters().getStandardFrameSize(); - } - - baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize); - } else { - // Writing stage output. - baseOutputChannelFactory = - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage); - } - - workOutputChannelFactory = new CountingOutputChannelFactory( - baseOutputChannelFactory, - counterTracker.channel(CounterNames.outputChannel()) - ); - } - - private void makeShuffleOutputChannelFactory(boolean isFinalStage) - { - shuffleOutputChannelFactory = - new CountingOutputChannelFactory( - makeStageOutputChannelFactory(frameContext, kernel.getStageDefinition().getStageNumber(), isFinalStage), - counterTracker.channel(CounterNames.shuffleChannel()) - ); } /** - * Use {@link FrameProcessorFactory#makeProcessors} to create {@link ProcessorsAndChannels}. Executes the - * processors using {@link #exec} and sets the output channels in {@link #workResultAndOutputChannels}. + * Called when processing for a stage is complete. Releases processing resources associated with the stage, i.e., + * those that are part of {@link KernelHolder#processorCloser}. * - * @param type of {@link StageDefinition#getProcessorFactory()} - * @param return type of {@link FrameProcessor} created by the manager - * @param result type of {@link ProcessorManager#result()} - * @param type of {@link WorkOrder#getExtraInfo()} + * Does not release results-fetching resources, i.e., does not release {@link KernelHolder#resultsCloser}. Those + * resources are released on {@link #removeKernel(StageId)} only. */ - private , ProcessorReturnType, ManagerReturnType, ExtraInfoType> void makeAndRunWorkProcessors() - throws IOException - { - if (workResultAndOutputChannels != null) { - throw new ISE("workResultAndOutputChannels already set"); - } - - @SuppressWarnings("unchecked") - final FactoryType processorFactory = (FactoryType) kernel.getStageDefinition().getProcessorFactory(); - - @SuppressWarnings("unchecked") - final ProcessorsAndChannels processors = - processorFactory.makeProcessors( - kernel.getStageDefinition(), - kernel.getWorkOrder().getWorkerNumber(), - kernel.getWorkOrder().getInputs(), - inputSliceReader, - (ExtraInfoType) kernel.getWorkOrder().getExtraInfo(), - workOutputChannelFactory, - frameContext, - parallelism, - counterTracker, - e -> warningPublisher.publishException(kernel.getStageDefinition().getStageNumber(), e), - isRemoveNullBytes - ); - - final ProcessorManager processorManager = processors.getProcessorManager(); - - final int maxOutstandingProcessors; - - if (processors.getOutputChannels().getAllChannels().isEmpty()) { - // No output channels: run up to "parallelism" processors at once. - maxOutstandingProcessors = Math.max(1, parallelism); - } else { - // If there are output channels, that acts as a ceiling on the number of processors that can run at once. - maxOutstandingProcessors = - Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); - } - - final ListenableFuture workResultFuture = exec.runAllFully( - processorManager, - maxOutstandingProcessors, - processorBouncer, - cancellationId - ); - - workResultAndOutputChannels = new ResultAndChannels<>(workResultFuture, processors.getOutputChannels()); - } - - private void makeAndRunShuffleProcessors() + public void finishProcessing(final StageId stageId) { - if (shuffleOutputChannelsFuture != null) { - throw new ISE("shuffleOutputChannelsFuture already set"); - } + final KernelHolder kernel = holderMap.get(verifyQueryId(stageId).getStageNumber()); - final ShuffleSpec shuffleSpec = kernel.getWorkOrder().getStageDefinition().getShuffleSpec(); - - final ShufflePipelineBuilder shufflePipeline = new ShufflePipelineBuilder( - kernel, - counterTracker, - exec, - cancellationId, - frameContext - ); - - shufflePipeline.initialize(workResultAndOutputChannels); - - switch (shuffleSpec.kind()) { - case MIX: - shufflePipeline.mix(shuffleOutputChannelFactory); - break; - - case HASH: - shufflePipeline.hashPartition(shuffleOutputChannelFactory); - break; - - case HASH_LOCAL_SORT: - final OutputChannelFactory hashOutputChannelFactory; - - if (shuffleSpec.partitionCount() == 1) { - // Single partition; no need to write temporary files. - hashOutputChannelFactory = - new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize()); - } else { - // Multi-partition; write temporary files and then sort each one file-by-file. - hashOutputChannelFactory = - new FileOutputChannelFactory( - context.tempDir(kernel.getStageDefinition().getStageNumber(), "hash-parts"), - frameContext.memoryParameters().getStandardFrameSize(), - null - ); - } - - shufflePipeline.hashPartition(hashOutputChannelFactory); - shufflePipeline.localSort(shuffleOutputChannelFactory); - break; - - case GLOBAL_SORT: - shufflePipeline.gatherResultKeyStatisticsIfNeeded(); - shufflePipeline.globalSort(shuffleOutputChannelFactory, makeGlobalSortPartitionBoundariesFuture()); - break; - - default: - throw new UOE("Cannot handle shuffle kind [%s]", shuffleSpec.kind()); - } - - shuffleOutputChannelsFuture = shufflePipeline.build(); - } - - private ListenableFuture makeGlobalSortPartitionBoundariesFuture() - { - if (kernel.getStageDefinition().mustGatherResultKeyStatistics()) { - if (stagePartitionBoundariesFuture != null) { - throw new ISE("Cannot call 'makeGlobalSortPartitionBoundariesFuture' twice"); + if (kernel != null) { + try { + kernel.processorCloser.close(); + } + catch (IOException e) { + throw new RuntimeException(e); } - - return (stagePartitionBoundariesFuture = SettableFuture.create()); - } else { - return Futures.immediateFuture(kernel.getResultPartitionBoundaries()); } } - private void setUpCompletionCallbacks(boolean isFinalStage) - { - final StageDefinition stageDef = kernel.getStageDefinition(); - - Futures.addCallback( - Futures.allAsList( - Arrays.asList( - workResultAndOutputChannels.getResultFuture(), - shuffleOutputChannelsFuture - ) - ), - new FutureCallback>() - { - @Override - public void onSuccess(final List workerResultAndOutputChannelsResolved) - { - final Object resultObject = workerResultAndOutputChannelsResolved.get(0); - final OutputChannels outputChannels = (OutputChannels) workerResultAndOutputChannelsResolved.get(1); - - for (OutputChannel channel : outputChannels.getAllChannels()) { - try { - stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>()) - .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel()); - } - catch (Exception e) { - kernelManipulationQueue.add(holder -> { - throw new RE(e, "Worker completion callback error for stage [%s]", stageDef.getId()); - }); - - // Don't make the "setResultsComplete" call below. - return; - } - } - - // Once the outputs channels have been resolved and are ready for reading, write success file, if - // using durable storage. - writeDurableStorageSuccessFileIfNeeded(stageDef.getStageNumber(), isFinalStage); - - kernelManipulationQueue.add(holder -> holder.getStageKernelMap() - .get(stageDef.getId()) - .setResultsComplete(resultObject)); - } - - @Override - public void onFailure(final Throwable t) - { - kernelManipulationQueue.add( - kernelHolder -> - kernelHolder.getStageKernelMap().get(stageDef.getId()).fail(t) - ); - } - }, - MoreExecutors.directExecutor() - ); - } - /** - * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. + * Remove the {@link WorkerStageKernel} for a given {@link StageId} from this holder. Closes all the associated + * {@link Closeable}. Removes and closes the {@link ControllerClient} for this query ID, if there are no longer + * any active work orders for that query ID + * + * @throws IllegalStateException if there is no active kernel for this stage */ - private void writeDurableStorageSuccessFileIfNeeded(final int stageNumber, boolean isFinalStage) + public void removeKernel(final StageId stageId) { - final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory; - if (durableStageStorageEnabled || (isFinalStage - && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination))) { - durableStorageOutputChannelFactory = DurableStorageOutputChannelFactory.createStandardImplementation( - task.getControllerTaskId(), - task().getWorkerNumber(), - stageNumber, - task().getId(), - frameContext.memoryParameters().getStandardFrameSize(), - MSQTasks.makeStorageConnector(context.injector()), - context.tempDir(), - (isFinalStage && MSQSelectDestination.DURABLESTORAGE.equals(selectDestination)) - ); - } else { - return; + final KernelHolder removed = holderMap.remove(verifyQueryId(stageId).getStageNumber()); + + if (removed == null) { + throw new ISE("No kernel for stage[%s]", stageId); } + try { - durableStorageOutputChannelFactory.createSuccessFile(task.getId()); + removed.processorCloser.close(); + removed.resultsCloser.close(); } catch (IOException e) { - throw new ISE( - e, - "Unable to create the success file [%s] at the location [%s]", - DurableStorageUtils.SUCCESS_MARKER_FILENAME, - durableStorageOutputChannelFactory.getSuccessFilePath() - ); + throw new RuntimeException(e); } } - } - - /** - * Helper for {@link RunWorkOrder#makeAndRunShuffleProcessors()}. Builds a {@link FrameProcessor} pipeline to - * handle the shuffle. - */ - private class ShufflePipelineBuilder - { - private final WorkerStageKernel kernel; - private final CounterTracker counterTracker; - private final FrameProcessorExecutor exec; - private final String cancellationId; - private final FrameContext frameContext; - - // Current state of the pipeline. It's a future to allow pipeline construction to be deferred if necessary. - private ListenableFuture> pipelineFuture; - - public ShufflePipelineBuilder( - final WorkerStageKernel kernel, - final CounterTracker counterTracker, - final FrameProcessorExecutor exec, - final String cancellationId, - final FrameContext frameContext - ) - { - this.kernel = kernel; - this.counterTracker = counterTracker; - this.exec = exec; - this.cancellationId = cancellationId; - this.frameContext = frameContext; - } /** - * Start the pipeline with the outputs of the main processor. + * Returns all currently-active kernel holders. */ - public void initialize(final ResultAndChannels resultAndChannels) + public Iterable getAllKernelHolders() { - if (pipelineFuture != null) { - throw new ISE("already initialized"); - } - - pipelineFuture = Futures.immediateFuture(resultAndChannels); + return holderMap.values(); } /** - * Add {@link FrameChannelMixer}, which mixes all current outputs into a single channel from the provided factory. + * Returns all currently-active kernels. */ - public void mix(final OutputChannelFactory outputChannelFactory) + public Iterable getAllKernels() { - // No sorting or statistics gathering, just combining all outputs into one big partition. Use a mixer to get - // everything into one file. Note: even if there is only one output channel, we'll run it through the mixer - // anyway, to ensure the data gets written to a file. (httpGetChannelData requires files.) - - push( - resultAndChannels -> { - final OutputChannel outputChannel = outputChannelFactory.openChannel(0); - - final FrameChannelMixer mixer = - new FrameChannelMixer( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - outputChannel.getWritableChannel() - ); - - return new ResultAndChannels<>( - exec.runFully(mixer, cancellationId), - OutputChannels.wrap(Collections.singletonList(outputChannel.readOnly())) - ); - } - ); + return Iterables.transform(holderMap.values(), holder -> holder.kernel); } /** - * Add {@link KeyStatisticsCollectionProcessor} if {@link StageDefinition#mustGatherResultKeyStatistics()}. + * Returns the number of kernels that are in running states, where {@link WorkerStagePhase#isRunning()}. */ - public void gatherResultKeyStatisticsIfNeeded() + public int runningKernelCount() { - push( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final OutputChannels channels = resultAndChannels.getOutputChannels(); - - if (channels.getAllChannels().isEmpty()) { - // No data coming out of this processor. Report empty statistics, if the kernel is expecting statistics. - if (stageDefinition.mustGatherResultKeyStatistics()) { - kernelManipulationQueue.add( - holder -> - holder.getStageKernelMap().get(stageDefinition.getId()) - .setResultKeyStatisticsSnapshot(ClusterByStatisticsSnapshot.empty()) - ); - } - - // Generate one empty channel so the SuperSorter has something to do. - final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); - channel.writable().close(); - - final OutputChannel outputChannel = OutputChannel.readOnly( - channel.readable(), - FrameWithPartition.NO_PARTITION - ); - - return new ResultAndChannels<>( - Futures.immediateFuture(null), - OutputChannels.wrap(Collections.singletonList(outputChannel)) - ); - } else if (stageDefinition.mustGatherResultKeyStatistics()) { - return gatherResultKeyStatistics(channels); - } else { - return resultAndChannels; - } - } - ); - } - - /** - * Add a {@link SuperSorter} using {@link StageDefinition#getSortKey()} and partition boundaries - * from {@code partitionBoundariesFuture}. - */ - public void globalSort( - final OutputChannelFactory outputChannelFactory, - final ListenableFuture partitionBoundariesFuture - ) - { - pushAsync( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - - final File sorterTmpDir = context.tempDir(stageDefinition.getStageNumber(), "super-sort"); - FileUtils.mkdirp(sorterTmpDir); - if (!sorterTmpDir.isDirectory()) { - throw new IOException("Cannot create directory: " + sorterTmpDir); - } + int retVal = 0; + for (final KernelHolder holder : holderMap.values()) { + if (holder.kernel.getPhase().isRunning()) { + retVal++; + } + } - final WorkerMemoryParameters memoryParameters = frameContext.memoryParameters(); - final SuperSorter sorter = new SuperSorter( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - stageDefinition.getFrameReader(), - stageDefinition.getSortKey(), - partitionBoundariesFuture, - exec, - outputChannelFactory, - makeSuperSorterIntermediateOutputChannelFactory( - frameContext, - stageDefinition.getStageNumber(), - sorterTmpDir - ), - memoryParameters.getSuperSorterMaxActiveProcessors(), - memoryParameters.getSuperSorterMaxChannelsPerProcessor(), - -1, - cancellationId, - counterTracker.sortProgress(), - isRemoveNullBytes - ); - - return FutureUtils.transform( - sorter.run(), - sortedChannels -> new ResultAndChannels<>(Futures.immediateFuture(null), sortedChannels) - ); - } - ); + return retVal; } /** - * Add a {@link FrameChannelHashPartitioner} using {@link StageDefinition#getSortKey()}. + * Return the kernel for a particular {@link StageId}. + * + * @return kernel, or null if there is no active kernel for this stage */ - public void hashPartition(final OutputChannelFactory outputChannelFactory) + @Nullable + public WorkerStageKernel getKernelFor(final StageId stageId) { - pushAsync( - resultAndChannels -> { - final ShuffleSpec shuffleSpec = kernel.getStageDefinition().getShuffleSpec(); - final int partitions = shuffleSpec.partitionCount(); - - final List outputChannels = new ArrayList<>(); - - for (int i = 0; i < partitions; i++) { - outputChannels.add(outputChannelFactory.openChannel(i)); - } - - final FrameChannelHashPartitioner partitioner = new FrameChannelHashPartitioner( - resultAndChannels.getOutputChannels().getAllReadableChannels(), - outputChannels.stream().map(OutputChannel::getWritableChannel).collect(Collectors.toList()), - kernel.getStageDefinition().getFrameReader(), - kernel.getStageDefinition().getClusterBy().getColumns().size(), - FrameWriters.makeRowBasedFrameWriterFactory( - new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()), - kernel.getStageDefinition().getSignature(), - kernel.getStageDefinition().getSortKey(), - isRemoveNullBytes - ) - ); - - final ListenableFuture partitionerFuture = exec.runFully(partitioner, cancellationId); - - final ResultAndChannels retVal = - new ResultAndChannels<>(partitionerFuture, OutputChannels.wrap(outputChannels)); - - if (retVal.getOutputChannels().areReadableChannelsReady()) { - return Futures.immediateFuture(retVal); - } else { - return FutureUtils.transform(partitionerFuture, ignored -> retVal); - } - } - ); + final KernelHolder holder = holderMap.get(verifyQueryId(stageId).getStageNumber()); + if (holder != null) { + return holder.kernel; + } else { + return null; + } } /** - * Add a sequence of {@link SuperSorter}, operating on each current output channel in order, one at a time. + * Retrieves the {@link ControllerClient}, which is shared across all {@link WorkOrder} for this worker. */ - public void localSort(final OutputChannelFactory outputChannelFactory) + public ControllerClient getControllerClient() { - pushAsync( - resultAndChannels -> { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final OutputChannels channels = resultAndChannels.getOutputChannels(); - final List> sortedChannelFutures = new ArrayList<>(); - - ListenableFuture nextFuture = Futures.immediateFuture(null); - - for (final OutputChannel channel : channels.getAllChannels()) { - final File sorterTmpDir = context.tempDir( - stageDefinition.getStageNumber(), - StringUtils.format("hash-parts-super-sort-%06d", channel.getPartitionNumber()) - ); - - FileUtils.mkdirp(sorterTmpDir); - - // SuperSorter will try to write to output partition zero; we remap it to the correct partition number. - final OutputChannelFactory partitionOverrideOutputChannelFactory = new OutputChannelFactory() - { - @Override - public OutputChannel openChannel(int expectedZero) throws IOException - { - if (expectedZero != 0) { - throw new ISE("Unexpected part [%s]", expectedZero); - } - - return outputChannelFactory.openChannel(channel.getPartitionNumber()); - } - - @Override - public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) - { - throw new UnsupportedOperationException(); - } - - @Override - public OutputChannel openNilChannel(int expectedZero) - { - if (expectedZero != 0) { - throw new ISE("Unexpected part [%s]", expectedZero); - } - - return outputChannelFactory.openNilChannel(channel.getPartitionNumber()); - } - }; - - // Chain futures so we only sort one partition at a time. - nextFuture = Futures.transformAsync( - nextFuture, - (AsyncFunction) ignored -> { - final SuperSorter sorter = new SuperSorter( - Collections.singletonList(channel.getReadableChannel()), - stageDefinition.getFrameReader(), - stageDefinition.getSortKey(), - Futures.immediateFuture(ClusterByPartitions.oneUniversalPartition()), - exec, - partitionOverrideOutputChannelFactory, - makeSuperSorterIntermediateOutputChannelFactory( - frameContext, - stageDefinition.getStageNumber(), - sorterTmpDir - ), - 1, - 2, - -1, - cancellationId, - - // Tracker is not actually tracked, since it doesn't quite fit into the way we report counters. - // There's a single SuperSorterProgressTrackerCounter per worker, but workers that do local - // sorting have a SuperSorter per partition. - new SuperSorterProgressTracker(), - isRemoveNullBytes - ); - - return FutureUtils.transform(sorter.run(), r -> Iterables.getOnlyElement(r.getAllChannels())); - }, - MoreExecutors.directExecutor() - ); - - sortedChannelFutures.add(nextFuture); - } - - return FutureUtils.transform( - Futures.allAsList(sortedChannelFutures), - sortedChannels -> new ResultAndChannels<>( - Futures.immediateFuture(null), - OutputChannels.wrap(sortedChannels) - ) - ); - } - ); + return controllerClient; } /** - * Return the (future) output channels for this pipeline. + * Remove all {@link WorkerStageKernel} and close all {@link ControllerClient}. */ - public ListenableFuture build() + @Override + public void close() { - if (pipelineFuture == null) { - throw new ISE("Not initialized"); - } + for (final int stageNumber : ImmutableList.copyOf(holderMap.keySet())) { + final StageId stageId = new StageId(workerContext.queryId(), stageNumber); - return Futures.transformAsync( - pipelineFuture, - (AsyncFunction, OutputChannels>) resultAndChannels -> - Futures.transform( - resultAndChannels.getResultFuture(), - (Function) input -> { - sanityCheckOutputChannels(resultAndChannels.getOutputChannels()); - return resultAndChannels.getOutputChannels(); - }, - MoreExecutors.directExecutor() - ), - MoreExecutors.directExecutor() - ); - } - - /** - * Adds {@link KeyStatisticsCollectionProcessor}. Called by {@link #gatherResultKeyStatisticsIfNeeded()}. - */ - private ResultAndChannels gatherResultKeyStatistics(final OutputChannels channels) - { - final StageDefinition stageDefinition = kernel.getStageDefinition(); - final List retVal = new ArrayList<>(); - final List processors = new ArrayList<>(); - - for (final OutputChannel outputChannel : channels.getAllChannels()) { - final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); - retVal.add(OutputChannel.readOnly(channel.readable(), outputChannel.getPartitionNumber())); - - processors.add( - new KeyStatisticsCollectionProcessor( - outputChannel.getReadableChannel(), - channel.writable(), - stageDefinition.getFrameReader(), - stageDefinition.getClusterBy(), - stageDefinition.createResultKeyStatisticsCollector( - frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() - ) - ) - ); + try { + removeKernel(stageId); + } + catch (Exception e) { + log.warn(e, "Failed to remove kernel for stage[%s].", stageId); + } } - - final ListenableFuture clusterByStatisticsCollectorFuture = - exec.runAllFully( - ProcessorManagers.of(processors) - .withAccumulation( - stageDefinition.createResultKeyStatisticsCollector( - frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() - ), - ClusterByStatisticsCollector::addAll - ), - // Run all processors simultaneously. They are lightweight and this keeps things moving. - processors.size(), - Bouncer.unlimited(), - cancellationId - ); - - Futures.addCallback( - clusterByStatisticsCollectorFuture, - new FutureCallback() - { - @Override - public void onSuccess(final ClusterByStatisticsCollector result) - { - result.logSketches(); - kernelManipulationQueue.add( - holder -> - holder.getStageKernelMap().get(stageDefinition.getId()) - .setResultKeyStatisticsSnapshot(result.snapshot()) - ); - } - - @Override - public void onFailure(Throwable t) - { - kernelManipulationQueue.add( - holder -> { - log.noStackTrace() - .warn(t, "Failed to gather clusterBy statistics for stage [%s]", stageDefinition.getId()); - holder.getStageKernelMap().get(stageDefinition.getId()).fail(t); - } - ); - } - }, - MoreExecutors.directExecutor() - ); - - return new ResultAndChannels<>( - clusterByStatisticsCollectorFuture, - OutputChannels.wrap(retVal) - ); } /** - * Update the {@link #pipelineFuture}. + * Check whether {@link #setDone()} has been called. */ - private void push(final ExceptionalFunction, ResultAndChannels> fn) + public boolean isDone() { - pushAsync( - channels -> - Futures.immediateFuture(fn.apply(channels)) - ); + return done; } /** - * Update the {@link #pipelineFuture} asynchronously. + * Mark the holder as "done", signaling to the main loop that it should clean up and exit as soon as possible. */ - private void pushAsync(final ExceptionalFunction, ListenableFuture>> fn) + public void setDone() { - if (pipelineFuture == null) { - throw new ISE("Not initialized"); - } - - pipelineFuture = FutureUtils.transform( - Futures.transformAsync( - pipelineFuture, - new AsyncFunction, ResultAndChannels>() - { - @Override - public ListenableFuture> apply(ResultAndChannels t) throws Exception - { - return fn.apply(t); - } - }, - MoreExecutors.directExecutor() - ), - resultAndChannels -> new ResultAndChannels<>( - resultAndChannels.getResultFuture(), - resultAndChannels.getOutputChannels().readOnly() - ) - ); + this.done = true; } - /** - * Verifies there is exactly one channel per partition. - */ - private void sanityCheckOutputChannels(final OutputChannels outputChannels) + private StageId verifyQueryId(final StageId stageId) { - for (int partitionNumber : outputChannels.getPartitionNumbers()) { - final List outputChannelsForPartition = - outputChannels.getChannelsForPartition(partitionNumber); - - Preconditions.checkState(partitionNumber >= 0, "Expected partitionNumber >= 0, but got [%s]", partitionNumber); - Preconditions.checkState( - outputChannelsForPartition.size() == 1, - "Expected one channel for partition [%s], but got [%s]", - partitionNumber, - outputChannelsForPartition.size() - ); + if (!stageId.getQueryId().equals(workerContext.queryId())) { + throw new ISE("Unexpected queryId[%s], expected queryId[%s]", stageId.getQueryId(), workerContext.queryId()); } - } - } - - private class KernelHolder - { - private boolean done = false; - - public Map getStageKernelMap() - { - return stageKernelMap; - } - - public boolean isDone() - { - return done; - } - public void setDone() - { - this.done = true; + return stageId; } } - private static class ResultAndChannels + /** + * Holder for a single {@link WorkerStageKernel} and associated items, contained within {@link KernelHolders}. + */ + private static class KernelHolder { - private final ListenableFuture resultFuture; - private final OutputChannels outputChannels; - - public ResultAndChannels( - ListenableFuture resultFuture, - OutputChannels outputChannels - ) - { - this.resultFuture = resultFuture; - this.outputChannels = outputChannels; - } - - public ListenableFuture getResultFuture() - { - return resultFuture; - } + private final WorkerStageKernel kernel; + private final Closer processorCloser; + private final Closer resultsCloser; + private SettableFuture partitionBoundariesFuture; - public OutputChannels getOutputChannels() + public KernelHolder(WorkerStageKernel kernel) { - return outputChannels; + this.kernel = kernel; + this.processorCloser = Closer.create(); + this.resultsCloser = Closer.create(); } } - - private interface ExceptionalFunction - { - R apply(T t) throws Exception; - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java index b36b1b4155a8..aeaae030e613 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java @@ -168,29 +168,14 @@ public class WorkerMemoryParameters this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; } - /** - * Create a production instance for {@link org.apache.druid.msq.indexing.MSQControllerTask}. - */ - public static WorkerMemoryParameters createProductionInstanceForController(final Injector injector) - { - long totalLookupFootprint = computeTotalLookupFootprint(injector); - return createInstance( - Runtime.getRuntime().maxMemory(), - computeNumWorkersInJvm(injector), - computeNumProcessorsInJvm(injector), - 0, - 0, - totalLookupFootprint - ); - } - /** * Create a production instance for {@link org.apache.druid.msq.indexing.MSQWorkerTask}. */ public static WorkerMemoryParameters createProductionInstanceForWorker( final Injector injector, final QueryDefinition queryDef, - final int stageNumber + final int stageNumber, + final int maxConcurrentStages ) { final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber); @@ -212,6 +197,7 @@ public static WorkerMemoryParameters createProductionInstanceForWorker( Runtime.getRuntime().maxMemory(), computeNumWorkersInJvm(injector), computeNumProcessorsInJvm(injector), + maxConcurrentStages, numInputWorkers, numHashOutputPartitions, totalLookupFootprint @@ -228,6 +214,7 @@ public static WorkerMemoryParameters createProductionInstanceForWorker( * @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to * the task capacity. * @param numProcessingThreadsInJvm size of the processing thread pool in the JVM. + * @param maxConcurrentStages maximum number of concurrent stages per worker. * @param numInputWorkers total number of workers across all input stages. * @param numHashOutputPartitions total number of output partitions, if using hash partitioning; zero if not using * hash partitioning. @@ -237,6 +224,7 @@ public static WorkerMemoryParameters createInstance( final long maxMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numInputWorkers, final int numHashOutputPartitions, final long totalLookupFootprint @@ -257,7 +245,8 @@ public static WorkerMemoryParameters createInstance( ); final long usableMemoryInJvm = computeUsableMemoryInJvm(maxMemoryInJvm, totalLookupFootprint); final long workerMemory = memoryPerWorker(usableMemoryInJvm, numWorkersInJvm); - final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); + final long bundleMemory = + memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm) / maxConcurrentStages; final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers); final long bundleMemoryForHashPartitioning = memoryNeededForHashPartitioning(numHashOutputPartitions); final long bundleMemoryForProcessing = @@ -268,6 +257,7 @@ public static WorkerMemoryParameters createInstance( usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm, + maxConcurrentStages, numHashOutputPartitions ); @@ -281,12 +271,14 @@ public static WorkerMemoryParameters createInstance( estimateUsableMemory( numWorkersInJvm, numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels + PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, + maxConcurrentStages ), totalLookupFootprint), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -301,14 +293,16 @@ public static WorkerMemoryParameters createInstance( calculateSuggestedMinMemoryFromUsableMemory( estimateUsableMemory( numWorkersInJvm, - (MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE + (MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE, + maxConcurrentStages ), totalLookupFootprint ), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -338,12 +332,14 @@ public static WorkerMemoryParameters createInstance( estimateUsableMemory( numWorkersInJvm, numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels + PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, + maxConcurrentStages ), totalLookupFootprint), maxMemoryInJvm, usableMemoryInJvm, numWorkersInJvm, - numProcessingThreadsInJvm + numProcessingThreadsInJvm, + maxConcurrentStages ) ); } @@ -352,7 +348,9 @@ public static WorkerMemoryParameters createInstance( bundleMemoryForProcessing, superSorterMaxActiveProcessors, superSorterMaxChannelsPerProcessor, - Ints.checkedCast(workerMemory) // 100% of worker memory is devoted to partition statistics + + // 100% of worker memory is devoted to partition statistics + Ints.checkedCast(workerMemory / maxConcurrentStages) ); } @@ -459,18 +457,19 @@ static int computeMaxWorkers( final long usableMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numHashOutputPartitions ) { final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); - // Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle, while accounting for - // memoryNeededForInputChannels + memoryNeededForHashPartitioning. + // Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle per concurrent stage, while + // accounting for memoryNeededForInputChannels + memoryNeededForHashPartitioning. final int isHashing = numHashOutputPartitions > 0 ? 1 : 0; - return Math.max( - 0, - Ints.checkedCast((bundleMemory - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1) - ); + final long bundleMemoryPerStage = bundleMemory / maxConcurrentStages; + final long maxWorkers = + (bundleMemoryPerStage - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1; + return Math.max(0, Ints.checkedCast(maxWorkers)); } /** @@ -528,7 +527,8 @@ private static long memoryPerWorker( } /** - * Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory(int, int, long)} + * Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be + * done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory} */ private static long memoryPerBundle( final long usableMemoryInJvm, @@ -536,6 +536,8 @@ private static long memoryPerBundle( final int numProcessingThreadsInJvm ) { + // One bundle per worker + one per processor. The worker bundles are used for sorting (SuperSorter) and the + // processing bundles are used for reading input and doing per-partition processing. final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; // Need to subtract memoryForWorkers off the top of usableMemoryInJvm, since this is reserved for @@ -553,24 +555,28 @@ private static long memoryPerBundle( private static long estimateUsableMemory( final int numWorkersInJvm, final int numProcessingThreadsInJvm, - final long estimatedEachBundleMemory + final long estimatedEachBundleMemory, + final int maxConcurrentStages ) { final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; - return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount); - + return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount, maxConcurrentStages); } /** * Add overheads to the estimated bundle memoery for all the workers. Checkout {@link WorkerMemoryParameters#memoryPerWorker(long, int)} * for the overhead calculation outside the processing bundles. */ - private static long estimateUsableMemory(final int numWorkersInJvm, final long estimatedTotalBundleMemory) + private static long estimateUsableMemory( + final int numWorkersInJvm, + final long estimatedTotalBundleMemory, + final int maxConcurrentStages + ) { - // Currently, we only add the partition stats overhead since it will be the single largest overhead per worker. final long estimateStatOverHeadPerWorker = PARTITION_STATS_MEMORY_MAX_BYTES; - return estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm); + final long requiredUsableMemory = estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm); + return requiredUsableMemory * maxConcurrentStages; } private static long memoryNeededForHashPartitioning(final int numOutputPartitions) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java index 59576ec90bfb..53e12dd2ab4d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerStorageParameters.java @@ -70,11 +70,13 @@ private WorkerStorageParameters(final long intermediateSuperSorterStorageMaxLoca public static WorkerStorageParameters createProductionInstance( final Injector injector, - final boolean isIntermediateSuperSorterStorageEnabled + final OutputChannelMode outputChannelMode ) { long tmpStorageBytesPerTask = injector.getInstance(TaskConfig.class).getTmpStorageBytesPerTask(); - return createInstance(tmpStorageBytesPerTask, isIntermediateSuperSorterStorageEnabled); + + // If durable storage is enabled, then super sorter intermediate storage should be enabled as well. + return createInstance(tmpStorageBytesPerTask, outputChannelMode.isDurable()); } @VisibleForTesting diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index e0de5bdc27e2..fb6e4a0079f1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,9 +20,13 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; @@ -35,25 +39,31 @@ public class IndexerFrameContext implements FrameContext { + private final StageId stageId; private final IndexerWorkerContext context; private final IndexIO indexIO; private final DataSegmentProvider dataSegmentProvider; private final WorkerMemoryParameters memoryParameters; + private final WorkerStorageParameters storageParameters; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; public IndexerFrameContext( + StageId stageId, IndexerWorkerContext context, IndexIO indexIO, DataSegmentProvider dataSegmentProvider, DataServerQueryHandlerFactory dataServerQueryHandlerFactory, - WorkerMemoryParameters memoryParameters + WorkerMemoryParameters memoryParameters, + WorkerStorageParameters storageParameters ) { + this.stageId = stageId; this.context = context; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.memoryParameters = memoryParameters; + this.storageParameters = storageParameters; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; } @Override @@ -90,7 +100,8 @@ public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() @Override public File tempDir() { - return context.tempDir(); + // No need to include query ID; each task handles a single query, so there is no ambiguity. + return new File(context.tempDir(), StringUtils.format("stage_%06d", stageId.getStageNumber())); } @Override @@ -128,4 +139,22 @@ public WorkerMemoryParameters memoryParameters() { return memoryParameters; } + + @Override + public Bouncer processorBouncer() + { + return context.injector().getInstance(Bouncer.class); + } + + @Override + public WorkerStorageParameters storageParameters() + { + return storageParameters; + } + + @Override + public void close() + { + // Nothing to close. + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java index 30bc75282fa4..2dedaf204ec7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerResourcePermissionMapper.java @@ -52,4 +52,10 @@ public List getAdminPermissions() ) ); } + + @Override + public List getQueryPermissions(String queryId) + { + return getAdminPermissions(); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 1bd789df7690..63358467489b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -24,9 +24,7 @@ import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Injector; import com.google.inject.Key; -import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.annotations.EscalatedGlobal; -import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; @@ -35,16 +33,21 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.OutputChannelMode; import org.apache.druid.msq.exec.TaskDataSegmentProvider; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.indexing.client.IndexerControllerClient; import org.apache.druid.msq.indexing.client.IndexerWorkerClient; import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocations; @@ -67,37 +70,49 @@ public class IndexerWorkerContext implements WorkerContext private static final long FREQUENCY_CHECK_MILLIS = 1000; private static final long FREQUENCY_CHECK_JITTER = 30; + private final MSQWorkerTask task; private final TaskToolbox toolbox; private final Injector injector; + private final OverlordClient overlordClient; private final IndexIO indexIO; private final TaskDataSegmentProvider dataSegmentProvider; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final ServiceClientFactory clientFactory; - - @GuardedBy("this") - private OverlordClient overlordClient; + private final MemoryIntrospector memoryIntrospector; + private final int maxConcurrentStages; @GuardedBy("this") private ServiceLocator controllerLocator; public IndexerWorkerContext( + final MSQWorkerTask task, final TaskToolbox toolbox, final Injector injector, + final OverlordClient overlordClient, final IndexIO indexIO, final TaskDataSegmentProvider dataSegmentProvider, - final DataServerQueryHandlerFactory dataServerQueryHandlerFactory, - final ServiceClientFactory clientFactory + final ServiceClientFactory clientFactory, + final MemoryIntrospector memoryIntrospector, + final DataServerQueryHandlerFactory dataServerQueryHandlerFactory ) { + this.task = task; this.toolbox = toolbox; this.injector = injector; + this.overlordClient = overlordClient; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.clientFactory = clientFactory; + this.memoryIntrospector = memoryIntrospector; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; + this.maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStages(QueryContext.of(task.getContext())); } - public static IndexerWorkerContext createProductionInstance(final TaskToolbox toolbox, final Injector injector) + public static IndexerWorkerContext createProductionInstance( + final MSQWorkerTask task, + final TaskToolbox toolbox, + final Injector injector + ) { final IndexIO indexIO = injector.getInstance(IndexIO.class); final SegmentCacheManager segmentCacheManager = @@ -105,28 +120,42 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch")); final ServiceClientFactory serviceClientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); + final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class); + final OverlordClient overlordClient = + injector.getInstance(OverlordClient.class).withRetryPolicy(StandardRetryPolicy.unlimited()); final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class)); final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class); return new IndexerWorkerContext( + task, toolbox, injector, + overlordClient, indexIO, - new TaskDataSegmentProvider( - toolbox.getCoordinatorClient(), - segmentCacheManager, - indexIO - ), + new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO), + serviceClientFactory, + memoryIntrospector, new DataServerQueryHandlerFactory( toolbox.getCoordinatorClient(), serviceClientFactory, smileMapper, warehouse - ), - serviceClientFactory + ) ); } + @Override + public String queryId() + { + return task.getControllerTaskId(); + } + + @Override + public String workerId() + { + return task.getId(); + } + public TaskToolbox toolbox() { return toolbox; @@ -147,7 +176,8 @@ public Injector injector() @Override public void registerWorker(Worker worker, Closer closer) { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + final WorkerChatHandler chatHandler = + new WorkerChatHandler(worker, toolbox.getAuthorizerMapper(), task.getDataSource()); toolbox.getChatHandlerProvider().register(worker.id(), chatHandler, false); closer.register(() -> toolbox.getChatHandlerProvider().unregister(worker.id())); closer.register(() -> { @@ -161,7 +191,7 @@ public void registerWorker(Worker worker, Closer closer) // Register the periodic controller checker final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s"); closer.register(periodicControllerCheckerExec::shutdownNow); - final ServiceLocator controllerLocator = makeControllerLocator(worker.task().getControllerTaskId()); + final ServiceLocator controllerLocator = makeControllerLocator(task.getControllerTaskId()); periodicControllerCheckerExec.submit(() -> controllerCheckerRunnable(controllerLocator, worker)); } @@ -218,15 +248,21 @@ public File tempDir() } @Override - public ControllerClient makeControllerClient(String controllerId) + public int maxConcurrentStages() + { + return maxConcurrentStages; + } + + @Override + public ControllerClient makeControllerClient() { - final ServiceLocator locator = makeControllerLocator(controllerId); + final ServiceLocator locator = makeControllerLocator(task.getControllerTaskId()); return new IndexerControllerClient( clientFactory.makeClient( - controllerId, + task.getControllerTaskId(), locator, - new SpecificTaskRetryPolicy(controllerId, StandardRetryPolicy.unlimited()) + new SpecificTaskRetryPolicy(task.getControllerTaskId(), StandardRetryPolicy.unlimited()) ), jsonMapper(), locator @@ -237,37 +273,33 @@ public ControllerClient makeControllerClient(String controllerId) public WorkerClient makeWorkerClient() { // Ignore workerId parameter. The workerId is passed into each method of WorkerClient individually. - return new IndexerWorkerClient(clientFactory, makeOverlordClient(), jsonMapper()); + return new IndexerWorkerClient(clientFactory, overlordClient, jsonMapper()); } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) { return new IndexerFrameContext( + queryDef.getStageDefinition(stageNumber).getId(), this, indexIO, dataSegmentProvider, dataServerQueryHandlerFactory, - WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber) + WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber, maxConcurrentStages), + WorkerStorageParameters.createProductionInstance(injector, outputChannelMode) ); } @Override public int threadCount() { - return processorBouncer().getMaxCount(); + return memoryIntrospector.numProcessorsInJvm(); } @Override public DruidNode selfNode() { - return injector.getInstance(Key.get(DruidNode.class, Self.class)); - } - - @Override - public Bouncer processorBouncer() - { - return injector.getInstance(Bouncer.class); + return toolbox.getDruidNode(); } @Override @@ -276,21 +308,13 @@ public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() return dataServerQueryHandlerFactory; } - private synchronized OverlordClient makeOverlordClient() - { - if (overlordClient == null) { - overlordClient = injector.getInstance(OverlordClient.class) - .withRetryPolicy(StandardRetryPolicy.unlimited()); - } - return overlordClient; - } - private synchronized ServiceLocator makeControllerLocator(final String controllerId) { if (controllerLocator == null) { - controllerLocator = new SpecificTaskServiceLocator(controllerId, makeOverlordClient()); + controllerLocator = new SpecificTaskServiceLocator(controllerId, overlordClient); } return controllerLocator; } + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index fb25097e9800..7b4d3235dc0c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -49,7 +49,9 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.expression.TimestampFloorExprMacro; @@ -58,11 +60,13 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -123,7 +127,8 @@ public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInjec */ @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask + CompactionTask compactionTask, + Map intervalToDataSchemaMap ) { List validationResults = new ArrayList<>(); @@ -139,11 +144,55 @@ public CompactionConfigValidationResult validateCompactionTask( )); } validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext())); - validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec())); + validationResults.add(validateRolledUpSegments(intervalToDataSchemaMap)); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() - .orElse(new CompactionConfigValidationResult(true, null)); + .orElse(CompactionConfigValidationResult.success()); + } + + /** + * Valides that there are no rolled-up segments where either: + *
    + *
  • aggregator factory differs from its combining factory
  • + *
  • input col name is different from the output name (non-idempotent)
  • + *
+ */ + private CompactionConfigValidationResult validateRolledUpSegments(Map intervalToDataSchemaMap) + { + for (Map.Entry intervalDataSchema : intervalToDataSchemaMap.entrySet()) { + if (intervalDataSchema.getValue() instanceof CombinedDataSchema) { + CombinedDataSchema combinedDataSchema = (CombinedDataSchema) intervalDataSchema.getValue(); + if (combinedDataSchema.hasRolledUpSegments()) { + for (AggregatorFactory aggregatorFactory : combinedDataSchema.getAggregators()) { + // This is a conservative check as existing rollup may have been idempotent but the aggregator provided in + // compaction spec isn't. This would get properly compacted yet fails in the below pre-check. + if ( + !( + aggregatorFactory.getClass().equals(aggregatorFactory.getCombiningFactory().getClass()) && + ( + aggregatorFactory.requiredFields().isEmpty() || + (aggregatorFactory.requiredFields().size() == 1 && + aggregatorFactory.requiredFields() + .get(0) + .equals(aggregatorFactory.getName())) + ) + ) + ) { + // MSQ doesn't support rolling up already rolled-up segments when aggregate column name is different from + // the aggregated column name. This is because the aggregated values would then get overwritten by new + // values and the existing values would be lost. Note that if no rollup is specified in an index spec, + // the default value is true. + return CompactionConfigValidationResult.failure( + "MSQ: Rolled-up segments in compaction interval[%s].", + intervalDataSchema.getKey() + ); + } + } + } + } + } + return CompactionConfigValidationResult.success(); } @Override @@ -291,6 +340,10 @@ private static RowSignature getRowSignature(DataSchema dataSchema) for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) { rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName())); } + // There can be columns that are part of metricsSpec for a datasource. + for (AggregatorFactory aggregatorFactory : dataSchema.getAggregators()) { + rowSignatureBuilder.add(aggregatorFactory.getName(), aggregatorFactory.getIntermediateType()); + } return rowSignatureBuilder.build(); } @@ -354,14 +407,30 @@ private static List getOrderBySpec(PartitionsSpec partitionSp private static Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { RowSignature rowSignature = getRowSignature(dataSchema); - return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) - .columns(rowSignature.getColumnNames()) - .virtualColumns(getVirtualColumns(dataSchema, interval)) - .columnTypes(rowSignature.getColumnTypes()) - .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) - .filters(dataSchema.getTransformSpec().getFilter()) - .context(compactionTask.getContext()) - .build(); + Druids.ScanQueryBuilder scanQueryBuilder = new Druids.ScanQueryBuilder() + .dataSource(dataSchema.getDataSource()) + .columns(rowSignature.getColumnNames()) + .virtualColumns(getVirtualColumns(dataSchema, interval)) + .columnTypes(rowSignature.getColumnTypes()) + .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) + .filters(dataSchema.getTransformSpec().getFilter()) + .context(compactionTask.getContext()); + + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + List orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); + + scanQueryBuilder.orderBy( + orderByColumnSpecs + .stream() + .map(orderByColumnSpec -> + new ScanQuery.OrderBy( + orderByColumnSpec.getDimension(), + ScanQuery.Order.fromString(orderByColumnSpec.getDirection().toString()) + )) + .collect(Collectors.toList()) + ); + } + return scanQueryBuilder.build(); } private static boolean isGroupBy(DataSchema dataSchema) @@ -468,7 +537,10 @@ private Map createMSQTaskContext(CompactionTask compactionTask, ); } // Similar to compaction using the native engine, don't finalize aggregations. + // Used for writing the data schema during segment generation phase. context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); + // Add appropriate finalization to native query context i.e. for the GroupBy query + context.put(QueryContexts.FINALIZE_KEY, false); // Only scalar or array-type dimensions are allowed as grouping keys. context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); return context; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java index b4d18ea390e9..31b03d63ba6e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTask.java @@ -33,10 +33,13 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.MSQTasks; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerContext; import org.apache.druid.msq.exec.WorkerImpl; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.MSQFaultUtils; import org.apache.druid.server.security.ResourceAction; import javax.annotation.Nonnull; @@ -48,6 +51,7 @@ public class MSQWorkerTask extends AbstractTask { public static final String TYPE = "query_worker"; + private static final Logger log = new Logger(MSQWorkerTask.class); private final String controllerTaskId; private final int workerNumber; @@ -132,18 +136,25 @@ public boolean isReady(final TaskActionClient taskActionClient) } @Override - public TaskStatus runTask(final TaskToolbox toolbox) throws Exception + public TaskStatus runTask(final TaskToolbox toolbox) { - final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector); + final WorkerContext context = IndexerWorkerContext.createProductionInstance(this, toolbox, injector); worker = new WorkerImpl(this, context); - return worker.run(); + + try { + worker.run(); + return TaskStatus.success(context.workerId()); + } + catch (MSQException e) { + return TaskStatus.failure(context.workerId(), MSQFaultUtils.generateMessageWithErrorCode(e.getFault())); + } } @Override public void stopGracefully(TaskConfig taskConfig) { if (worker != null) { - worker.stopGracefully(); + worker.stop(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java index 81303eb43848..1e31de71a8ac 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java @@ -152,7 +152,7 @@ public void postWorkerWarning(List MSQErrorReports) throws IOExc } @Override - public List getTaskList() throws IOException + public List getWorkerIds() throws IOException { final BytesFullResponseHolder retVal = doRequest( new RequestBuilder(HttpMethod.GET, "/taskList"), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java index 70d1ab11d380..7c8b86bb9d64 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java @@ -19,310 +19,25 @@ package org.apache.druid.msq.indexing.client; -import com.google.common.collect.ImmutableMap; -import it.unimi.dsi.fastutil.bytes.ByteArrays; -import org.apache.commons.lang.mutable.MutableLong; -import org.apache.druid.frame.file.FrameFileHttpResponseHandler; -import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.Worker; -import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.WorkOrder; -import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde; +import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper; +import org.apache.druid.msq.rpc.WorkerResource; import org.apache.druid.segment.realtime.ChatHandler; -import org.apache.druid.segment.realtime.ChatHandlers; -import org.apache.druid.server.security.Action; -import org.apache.druid.utils.CloseableUtils; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.server.security.AuthorizerMapper; -import javax.annotation.Nullable; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.StreamingOutput; -import java.io.IOException; -import java.io.InputStream; - -public class WorkerChatHandler implements ChatHandler +/** + * Subclass of {@link WorkerResource} that implements {@link ChatHandler}, suitable for registration + * with a {@link ChatHandlerProvider}. + */ +public class WorkerChatHandler extends WorkerResource implements ChatHandler { - private static final Logger log = new Logger(WorkerChatHandler.class); - - /** - * Callers must be able to store an entire chunk in memory. It can't be too large. - */ - private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000; - - private final Worker worker; - private final MSQWorkerTask task; - private final TaskToolbox toolbox; - - public WorkerChatHandler(TaskToolbox toolbox, Worker worker) - { - this.worker = worker; - this.task = worker.task(); - this.toolbox = toolbox; - } - - /** - * Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data. - *

- * See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API. - */ - @GET - @Path("/channels/{queryId}/{stageNumber}/{partitionNumber}") - @Produces(MediaType.APPLICATION_OCTET_STREAM) - public Response httpGetChannelData( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @PathParam("partitionNumber") final int partitionNumber, - @QueryParam("offset") final long offset, - @Context final HttpServletRequest req + public WorkerChatHandler( + final Worker worker, + final AuthorizerMapper authorizerMapper, + final String dataSource ) { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - try { - final InputStream inputStream = worker.readChannel(queryId, stageNumber, partitionNumber, offset); - if (inputStream == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - - final Response.ResponseBuilder responseBuilder = Response.ok(); - - final byte[] readBuf = new byte[8192]; - final MutableLong bytesReadTotal = new MutableLong(0L); - final int firstRead = inputStream.read(readBuf); - - if (firstRead == -1) { - // Empty read means we're at the end of the channel. Set the last fetch header so the client knows this. - inputStream.close(); - return responseBuilder - .header( - FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME, - FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE - ) - .entity(ByteArrays.EMPTY_ARRAY) - .build(); - } - - return Response.ok((StreamingOutput) output -> { - try { - int bytesReadThisCall = firstRead; - do { - final int bytesToWrite = - (int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal.longValue(), bytesReadThisCall); - output.write(readBuf, 0, bytesToWrite); - bytesReadTotal.add(bytesReadThisCall); - } while (bytesReadTotal.longValue() < CHANNEL_DATA_CHUNK_SIZE - && (bytesReadThisCall = inputStream.read(readBuf)) != -1); - } - catch (Throwable e) { - // Suppress the exception to ensure nothing gets written over the wire once we've sent a 200. The client - // will resume from where it left off. - log.noStackTrace().warn( - e, - "Error writing channel for query [%s] stage [%s] partition [%s] offset [%,d] to [%s]", - queryId, - stageNumber, - partitionNumber, - offset, - req.getRemoteAddr() - ); - } - finally { - CloseableUtils.closeAll(inputStream, output); - } - }).build(); - } - catch (IOException e) { - return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); - } - } - - /** - * See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API. - */ - @POST - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - @Path("/workOrder") - public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req) - { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - worker.postWorkOrder(workOrder); - return Response.status(Response.Status.ACCEPTED).build(); - } - - /** - * See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API. - */ - @POST - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - @Path("/resultPartitionBoundaries/{queryId}/{stageNumber}") - public Response httpPostResultPartitionBoundaries( - final ClusterByPartitions stagePartitionBoundaries, - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - if (worker.postResultPartitionBoundaries(stagePartitionBoundaries, queryId, stageNumber)) { - return Response.status(Response.Status.ACCEPTED).build(); - } else { - return Response.status(Response.Status.BAD_REQUEST).build(); - } - } - - @POST - @Path("/keyStatistics/{queryId}/{stageNumber}") - @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) - @Consumes(MediaType.APPLICATION_JSON) - public Response httpFetchKeyStatistics( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); - ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; - StageId stageId = new StageId(queryId, stageNumber); - try { - clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); - if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_OCTET_STREAM) - .entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot)) - .build(); - } else { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_JSON) - .entity(clusterByStatisticsSnapshot) - .build(); - } - } - catch (Exception e) { - String errorMessage = StringUtils.format( - "Invalid request for key statistics for query[%s] and stage[%d]", - queryId, - stageNumber - ); - log.error(e, errorMessage); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", errorMessage)) - .build(); - } - } - - @POST - @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") - @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) - @Consumes(MediaType.APPLICATION_JSON) - public Response httpFetchKeyStatisticsWithSnapshot( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @PathParam("timeChunk") final long timeChunk, - @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper()); - ClusterByStatisticsSnapshot snapshotForTimeChunk; - StageId stageId = new StageId(queryId, stageNumber); - try { - snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); - if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_OCTET_STREAM) - .entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk)) - .build(); - } else { - return Response.status(Response.Status.ACCEPTED) - .type(MediaType.APPLICATION_JSON) - .entity(snapshotForTimeChunk) - .build(); - } - } - catch (Exception e) { - String errorMessage = StringUtils.format( - "Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]", - queryId, - stageNumber, - timeChunk - ); - log.error(e, errorMessage); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", errorMessage)) - .build(); - } - } - - /** - * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. - */ - @POST - @Path("/cleanupStage/{queryId}/{stageNumber}") - public Response httpPostCleanupStage( - @PathParam("queryId") final String queryId, - @PathParam("stageNumber") final int stageNumber, - @Context final HttpServletRequest req - ) - { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - worker.postCleanupStage(new StageId(queryId, stageNumber)); - return Response.status(Response.Status.ACCEPTED).build(); - } - - /** - * See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API. - */ - @POST - @Path("/finish") - public Response httpPostFinish(@Context final HttpServletRequest req) - { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - worker.postFinish(); - return Response.status(Response.Status.ACCEPTED).build(); - } - - - /** - * See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API. - */ - @GET - @Produces(MediaType.APPLICATION_JSON) - @Path("/counters") - public Response httpGetCounters(@Context final HttpServletRequest req) - { - ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper()); - return Response.status(Response.Status.OK).entity(worker.getCounters()).build(); - } - - /** - * Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and - * {@link #httpFetchKeyStatisticsWithSnapshot}. - */ - public enum SketchEncoding - { - /** - * The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}. - */ - OCTET_STREAM, - /** - * The key collector is encoded as json - */ - JSON + super(worker, new IndexerResourcePermissionMapper(dataSource), authorizerMapper); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java index 5c80f065eef3..6f4b36da1eec 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing.error; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; @@ -35,6 +36,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault private final long usableMemory; private final int serverWorkers; private final int serverThreads; + private final int maxConcurrentStages; @JsonCreator public NotEnoughMemoryFault( @@ -42,19 +44,23 @@ public NotEnoughMemoryFault( @JsonProperty("serverMemory") final long serverMemory, @JsonProperty("usableMemory") final long usableMemory, @JsonProperty("serverWorkers") final int serverWorkers, - @JsonProperty("serverThreads") final int serverThreads + @JsonProperty("serverThreads") final int serverThreads, + @JsonProperty("maxConcurrentStages") final int maxConcurrentStages ) { super( CODE, "Not enough memory. Required at least %,d bytes. (total = %,d bytes; usable = %,d bytes; " - + "worker capacity = %,d; processing threads = %,d). Increase JVM memory with the -Xmx option" - + (serverWorkers > 1 ? " or reduce worker capacity on this server" : ""), + + "worker capacity = %,d; processing threads = %,d; concurrent stages = %,d). " + + "Increase JVM memory with the -Xmx option" + + (serverWorkers > 1 ? ", or reduce worker capacity on this server" : "") + + (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : ""), suggestedServerMemory, serverMemory, usableMemory, serverWorkers, - serverThreads + serverThreads, + maxConcurrentStages ); this.suggestedServerMemory = suggestedServerMemory; @@ -62,6 +68,7 @@ public NotEnoughMemoryFault( this.usableMemory = usableMemory; this.serverWorkers = serverWorkers; this.serverThreads = serverThreads; + this.maxConcurrentStages = maxConcurrentStages; } @JsonProperty @@ -94,6 +101,13 @@ public int getServerThreads() return serverThreads; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getMaxConcurrentStages() + { + return maxConcurrentStages; + } + @Override public boolean equals(Object o) { @@ -107,12 +121,12 @@ public boolean equals(Object o) return false; } NotEnoughMemoryFault that = (NotEnoughMemoryFault) o; - return - suggestedServerMemory == that.suggestedServerMemory - && serverMemory == that.serverMemory - && usableMemory == that.usableMemory - && serverWorkers == that.serverWorkers - && serverThreads == that.serverThreads; + return suggestedServerMemory == that.suggestedServerMemory + && serverMemory == that.serverMemory + && usableMemory == that.usableMemory + && serverWorkers == that.serverWorkers + && serverThreads == that.serverThreads + && maxConcurrentStages == that.maxConcurrentStages; } @Override @@ -124,7 +138,8 @@ public int hashCode() serverMemory, usableMemory, serverWorkers, - serverThreads + serverThreads, + maxConcurrentStages ); } @@ -137,6 +152,7 @@ public String toString() " bytes, usableMemory=" + usableMemory + " bytes, serverWorkers=" + serverWorkers + ", serverThreads=" + serverThreads + + ", maxConcurrentStages=" + maxConcurrentStages + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java index 028f1b5bd48a..de01235447aa 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java @@ -41,6 +41,22 @@ private InputSlices() // No instantiation. } + /** + * Returns all {@link StageInputSlice} from the provided list of input slices. Ignores other types of input slices. + */ + public static List allStageSlices(final List slices) + { + final List retVal = new ArrayList<>(); + + for (final InputSlice slice : slices) { + if (slice instanceof StageInputSlice) { + retVal.add((StageInputSlice) slice); + } + } + + return retVal; + } + /** * Combines all {@link StageInputSlice#getPartitions()} from the input slices that are {@link StageInputSlice}. * Ignores other types of input slices. @@ -49,10 +65,8 @@ public static ReadablePartitions allReadablePartitions(final List sl { final List partitionsList = new ArrayList<>(); - for (final InputSlice slice : slices) { - if (slice instanceof StageInputSlice) { - partitionsList.add(((StageInputSlice) slice).getPartitions()); - } + for (final StageInputSlice slice : allStageSlices(slices)) { + partitionsList.add(slice.getPartitions()); } return ReadablePartitions.combine(partitionsList); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 03aa7cd0fe4f..4b68a3bf1b01 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -31,7 +31,7 @@ import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; @@ -53,6 +53,7 @@ import org.apache.druid.timeline.SegmentId; import java.io.File; +import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.function.Consumer; @@ -94,7 +95,7 @@ public ReadableInputs attach( externalInputSlice.getInputSources(), externalInputSlice.getInputFormat(), externalInputSlice.getSignature(), - temporaryDirectory, + new File(temporaryDirectory, String.valueOf(inputNumber)), counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()), counters.warnings(), warningPublisher @@ -128,9 +129,13 @@ private static Iterator inputSourceSegmentIterator( ColumnsFilter.all() ); - if (!temporaryDirectory.exists() && !temporaryDirectory.mkdir()) { - throw new ISE("Cannot create temporary directory at [%s]", temporaryDirectory); + try { + FileUtils.mkdirp(temporaryDirectory); } + catch (IOException e) { + throw new RuntimeException(e); + } + return Iterators.transform( inputSources.iterator(), inputSource -> { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index 7db2fa1a9dd9..da962a9d3931 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -20,8 +20,11 @@ package org.apache.druid.msq.kernel; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.OutputChannelMode; import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; @@ -30,12 +33,16 @@ import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import java.io.Closeable; import java.io.File; /** - * Provides services and objects for the functioning of the frame processors + * Provides services and objects for the functioning of the frame processors. Scoped to a specific stage of a + * specific query, i.e., one {@link WorkOrder}. + * + * Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(QueryDefinition, int, OutputChannelMode)}. */ -public interface FrameContext +public interface FrameContext extends Closeable { SegmentWrangler segmentWrangler(); @@ -59,5 +66,14 @@ public interface FrameContext IndexMergerV9 indexMerger(); + Bouncer processorBouncer(); + WorkerMemoryParameters memoryParameters(); + + WorkerStorageParameters storageParameters(); + + default File tempDir(String name) + { + return new File(tempDir(), name); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java index 201a1783c05f..0c8578702103 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java @@ -109,7 +109,7 @@ ExtraInfoHolder getExtraInfoHolder() /** * Worker IDs for this query, if known in advance (at the time the work order is created). May be null, in which - * case workers use {@link ControllerClient#getTaskList()} to find worker IDs. + * case workers use {@link ControllerClient#getWorkerIds()} to find worker IDs. */ @Nullable @JsonProperty("workers") diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java index 632b8a8106dd..b838092ca714 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java @@ -42,6 +42,8 @@ * This separation of decision-making from the "real world" allows the decision-making to live in one, * easy-to-follow place. * + * Not thread-safe. + * * @see org.apache.druid.msq.kernel.controller.ControllerQueryKernel state machine on the controller side */ public class WorkerStageKernel @@ -51,9 +53,10 @@ public class WorkerStageKernel private WorkerStagePhase phase = WorkerStagePhase.NEW; - // We read this variable in the main thread and the netty threads @Nullable - private volatile ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot; + private ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot; + + private boolean doneReadingInput; @Nullable private ClusterByPartitions resultPartitionBoundaries; @@ -107,25 +110,25 @@ public void startReading() public void startPreshuffleWaitingForResultPartitionBoundaries() { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(true); transitionTo(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES); } public void startPreshuffleWritingOutput() { - assertPreshuffleStatisticsNeeded(); transitionTo(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT); } - public void setResultKeyStatisticsSnapshot(final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot) + public void setResultKeyStatisticsSnapshot(@Nullable final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot) { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(resultKeyStatisticsSnapshot != null); this.resultKeyStatisticsSnapshot = resultKeyStatisticsSnapshot; + this.doneReadingInput = true; } public void setResultPartitionBoundaries(final ClusterByPartitions resultPartitionBoundaries) { - assertPreshuffleStatisticsNeeded(); + assertPreshuffleStatisticsNeeded(true); this.resultPartitionBoundaries = resultPartitionBoundaries; } @@ -134,6 +137,11 @@ public boolean hasResultKeyStatisticsSnapshot() return resultKeyStatisticsSnapshot != null; } + public boolean isDoneReadingInput() + { + return doneReadingInput; + } + public boolean hasResultPartitionBoundaries() { return resultPartitionBoundaries != null; @@ -152,10 +160,10 @@ public ClusterByPartitions getResultPartitionBoundaries() @Nullable public Object getResultObject() { - if (phase == WorkerStagePhase.RESULTS_READY || phase == WorkerStagePhase.FINISHED) { + if (phase == WorkerStagePhase.RESULTS_COMPLETE) { return resultObject; } else { - throw new ISE("Results are not ready yet"); + throw new ISE("Results are not ready in phase[%s]", phase); } } @@ -174,7 +182,7 @@ public void setResultsComplete(Object resultObject) throw new NullPointerException("resultObject must not be null"); } - transitionTo(WorkerStagePhase.RESULTS_READY); + transitionTo(WorkerStagePhase.RESULTS_COMPLETE); this.resultObject = resultObject; } @@ -196,16 +204,18 @@ public void fail(Throwable t) } } - public boolean addPostedResultsComplete(Pair stageIdAndWorkerNumber) + public boolean addPostedResultsComplete(StageId stageId, int workerNumber) { - return postedResultsComplete.add(stageIdAndWorkerNumber); + return postedResultsComplete.add(Pair.of(stageId, workerNumber)); } - private void assertPreshuffleStatisticsNeeded() + private void assertPreshuffleStatisticsNeeded(final boolean delivered) { - if (!workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { + if (delivered != workOrder.getStageDefinition().mustGatherResultKeyStatistics()) { throw new ISE( - "Result partitioning is not necessary for stage [%s]", + "Result key statistics %s, but %s, for stage[%s]", + delivered ? "delivered" : "not delivered", + workOrder.getStageDefinition().mustGatherResultKeyStatistics() ? "expected" : "not expected", workOrder.getStageDefinition().getId() ); } @@ -222,7 +232,12 @@ private void transitionTo(final WorkerStagePhase newPhase) ); phase = newPhase; } else { - throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase); + throw new IAE( + "Cannot transition stage[%s] from[%s] to[%s]", + workOrder.getStageDefinition().getId(), + phase, + newPhase + ); } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java index f54aa52349ea..4e59e7d17a89 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java @@ -54,11 +54,12 @@ public boolean canTransitionFrom(final WorkerStagePhase priorPhase) @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { - return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES; + return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES /* if globally sorting */ + || priorPhase == READING_INPUT /* if locally sorting */; } }, - RESULTS_READY { + RESULTS_COMPLETE { @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { @@ -70,7 +71,9 @@ public boolean canTransitionFrom(final WorkerStagePhase priorPhase) @Override public boolean canTransitionFrom(final WorkerStagePhase priorPhase) { - return priorPhase == RESULTS_READY; + // Stages can transition to FINISHED even if they haven't generated all output yet. For example, this is + // possible if the downstream stage is applying a limit. + return priorPhase.compareTo(FINISHED) < 0; } }, @@ -84,4 +87,24 @@ public boolean canTransitionFrom(final WorkerStagePhase priorPhase) }; public abstract boolean canTransitionFrom(WorkerStagePhase priorPhase); + + /** + * Whether this phase indicates that the stage is no longer running. + */ + public boolean isTerminal() + { + return this == FINISHED || this == FAILED; + } + + /** + * Whether this phase indicates a stage is running and consuming its full complement of resources. + * + * There are still some resources that can be consumed by stages that are not running. For example, in the + * {@link #FINISHED} state, stages can still have data on disk that has not been cleaned-up yet, some pointers + * to that data that still reside in memory, and some counters in memory available for collection by the controller. + */ + public boolean isRunning() + { + return this != NEW && this != RESULTS_COMPLETE && this != FINISHED && this != FAILED; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java index 8d0fba72a216..fd1a0323d0fb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java @@ -97,7 +97,7 @@ public ListenableFuture fetchClusterByStatisticsSna "/keyStatistics/%s/%d?sketchEncoding=%s", StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), - WorkerChatHandler.SketchEncoding.OCTET_STREAM + WorkerResource.SketchEncoding.OCTET_STREAM ); return getClient(workerId).asyncRequest( @@ -118,7 +118,7 @@ public ListenableFuture fetchClusterByStatisticsSna StringUtils.urlEncode(stageId.getQueryId()), stageId.getStageNumber(), timeChunk, - WorkerChatHandler.SketchEncoding.OCTET_STREAM + WorkerResource.SketchEncoding.OCTET_STREAM ); return getClient(workerId).asyncRequest( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java index d3e9eefa86d2..cc570ec992ad 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ControllerResource.java @@ -82,6 +82,27 @@ public Response httpPostPartialKeyStatistics( return Response.status(Response.Status.ACCEPTED).build(); } + /** + * Used by subtasks to inform the controller that they are done reading their input, in cases where they would + * not be calling {@link #httpPostPartialKeyStatistics(Object, String, int, int, HttpServletRequest)}. + * + * See {@link ControllerClient#postDoneReadingInput(StageId, int)} for the client-side code that calls this API. + */ + @POST + @Path("/doneReadingInput/{queryId}/{stageNumber}/{workerNumber}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response httpPostDoneReadingInput( + @PathParam("stageNumber") final int stageNumber, + @PathParam("workerNumber") final int workerNumber, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + controller.doneReadingInput(stageNumber, workerNumber); + return Response.status(Response.Status.ACCEPTED).build(); + } + /** * Used by subtasks to post system errors. Note that the errors are organized by taskId, not by query/stage/worker, * because system errors are associated with a task rather than a specific query/stage/worker execution context. @@ -166,7 +187,7 @@ public Response httpPostResultsComplete( } /** - * See {@link ControllerClient#getTaskList()} for the client-side code that calls this API. + * See {@link ControllerClient#getWorkerIds} for the client-side code that calls this API. */ @GET @Path("/taskList") @@ -174,7 +195,7 @@ public Response httpPostResultsComplete( public Response httpGetTaskList(@Context final HttpServletRequest req) { MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); - return Response.ok(new MSQTaskList(controller.getTaskIds())).build(); + return Response.ok(new MSQTaskList(controller.getWorkerIds())).build(); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java index 30a8179fe0f0..8820b4ead5a0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java @@ -47,4 +47,20 @@ public static void authorizeAdminRequest( throw new ForbiddenException(access.toString()); } } + + public static void authorizeQueryRequest( + final ResourcePermissionMapper permissionMapper, + final AuthorizerMapper authorizerMapper, + final HttpServletRequest request, + final String queryId + ) + { + final List resourceActions = permissionMapper.getQueryPermissions(queryId); + + Access access = AuthorizationUtils.authorizeAllResourceActions(request, resourceActions, authorizerMapper); + + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java index 8c79f4fa0e05..0a7fb874f6d1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/ResourcePermissionMapper.java @@ -23,11 +23,9 @@ import java.util.List; -/** - * Provides HTTP resources such as {@link ControllerResource} with information about which permissions are needed - * for requests. - */ public interface ResourcePermissionMapper { List getAdminPermissions(); + + List getQueryPermissions(String queryId); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java new file mode 100644 index 000000000000..a0bfecff5427 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java @@ -0,0 +1,391 @@ +/* + * 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.msq.rpc; + +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.file.FrameFileHttpResponseHandler; +import org.apache.druid.frame.key.ClusterByPartitions; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; +import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; +import java.io.InputStream; +import java.io.OutputStream; + +public class WorkerResource +{ + private static final Logger log = new Logger(WorkerResource.class); + + /** + * Callers must be able to store an entire chunk in memory. It can't be too large. + */ + private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000; + private static final long GET_CHANNEL_DATA_TIMEOUT = 30_000L; + + protected final Worker worker; + protected final ResourcePermissionMapper permissionMapper; + protected final AuthorizerMapper authorizerMapper; + + public WorkerResource( + final Worker worker, + final ResourcePermissionMapper permissionMapper, + final AuthorizerMapper authorizerMapper + ) + { + this.worker = worker; + this.permissionMapper = permissionMapper; + this.authorizerMapper = authorizerMapper; + } + + /** + * Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data. + *

+ * See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API. + */ + @GET + @Path("/channels/{queryId}/{stageNumber}/{partitionNumber}") + @Produces(MediaType.APPLICATION_OCTET_STREAM) + public Response httpGetChannelData( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("partitionNumber") final int partitionNumber, + @QueryParam("offset") final long offset, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + + final ListenableFuture dataFuture = + worker.readStageOutput(new StageId(queryId, stageNumber), partitionNumber, offset); + + final AsyncContext asyncContext = req.startAsync(); + asyncContext.setTimeout(GET_CHANNEL_DATA_TIMEOUT); + asyncContext.addListener( + new AsyncListener() + { + @Override + public void onComplete(AsyncEvent event) + { + } + + @Override + public void onTimeout(AsyncEvent event) + { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + event.getAsyncContext().complete(); + } + + @Override + public void onError(AsyncEvent event) + { + } + + @Override + public void onStartAsync(AsyncEvent event) + { + } + } + ); + + // Save these items, since "req" becomes inaccessible in future exception handlers. + final String remoteAddr = req.getRemoteAddr(); + final String requestURI = req.getRequestURI(); + + Futures.addCallback( + dataFuture, + new FutureCallback() + { + @Override + public void onSuccess(final InputStream inputStream) + { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + + try (final OutputStream outputStream = response.getOutputStream()) { + if (inputStream == null) { + response.setStatus(HttpServletResponse.SC_NOT_FOUND); + } else { + response.setStatus(HttpServletResponse.SC_OK); + response.setContentType(MediaType.APPLICATION_OCTET_STREAM); + + final byte[] readBuf = new byte[8192]; + final int firstRead = inputStream.read(readBuf); + + if (firstRead == -1) { + // Empty read means we're at the end of the channel. + // Set the last fetch header so the client knows this. + response.setHeader( + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME, + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE + ); + } else { + long bytesReadTotal = 0; + int bytesReadThisCall = firstRead; + do { + final int bytesToWrite = + (int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal, bytesReadThisCall); + outputStream.write(readBuf, 0, bytesToWrite); + bytesReadTotal += bytesReadThisCall; + } while (bytesReadTotal < CHANNEL_DATA_CHUNK_SIZE + && (bytesReadThisCall = inputStream.read(readBuf)) != -1); + } + } + } + catch (Exception e) { + log.noStackTrace().warn(e, "Could not respond to request from[%s] to[%s]", remoteAddr, requestURI); + } + finally { + CloseableUtils.closeAndSuppressExceptions(inputStream, e -> log.warn("Failed to close output channel")); + asyncContext.complete(); + } + } + + @Override + public void onFailure(Throwable e) + { + if (!dataFuture.isCancelled()) { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + asyncContext.complete(); + } + catch (Exception e2) { + e.addSuppressed(e2); + } + + log.noStackTrace().warn(e, "Request failed from[%s] to[%s]", remoteAddr, requestURI); + } + } + }, + Execs.directExecutor() + ); + + return null; + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Path("/workOrder") + public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req) + { + final String queryId = workOrder.getQueryDefinition().getQueryId(); + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + worker.postWorkOrder(workOrder); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Path("/resultPartitionBoundaries/{queryId}/{stageNumber}") + public Response httpPostResultPartitionBoundaries( + final ClusterByPartitions stagePartitionBoundaries, + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + if (worker.postResultPartitionBoundaries(new StageId(queryId, stageNumber), stagePartitionBoundaries)) { + return Response.status(Response.Status.ACCEPTED).build(); + } else { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + } + + @POST + @Path("/keyStatistics/{queryId}/{stageNumber}") + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) + public Response httpFetchKeyStatistics( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + ClusterByStatisticsSnapshot clusterByStatisticsSnapshot; + StageId stageId = new StageId(queryId, stageNumber); + try { + clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId); + if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_OCTET_STREAM) + .entity( + (StreamingOutput) output -> + ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot) + ) + .build(); + } else { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_JSON) + .entity(clusterByStatisticsSnapshot) + .build(); + } + } + catch (Exception e) { + String errorMessage = StringUtils.format( + "Invalid request for key statistics for query[%s] and stage[%d]", + queryId, + stageNumber + ); + log.error(e, errorMessage); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", errorMessage)) + .build(); + } + } + + @POST + @Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}") + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM}) + public Response httpFetchKeyStatisticsWithSnapshot( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @PathParam("timeChunk") final long timeChunk, + @QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + ClusterByStatisticsSnapshot snapshotForTimeChunk; + StageId stageId = new StageId(queryId, stageNumber); + try { + snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk); + if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_OCTET_STREAM) + .entity( + (StreamingOutput) output -> + ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk) + ) + .build(); + } else { + return Response.status(Response.Status.ACCEPTED) + .type(MediaType.APPLICATION_JSON) + .entity(snapshotForTimeChunk) + .build(); + } + } + catch (Exception e) { + String errorMessage = StringUtils.format( + "Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]", + queryId, + stageNumber, + timeChunk + ); + log.error(e, errorMessage); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", errorMessage)) + .build(); + } + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API. + */ + @POST + @Path("/cleanupStage/{queryId}/{stageNumber}") + public Response httpPostCleanupStage( + @PathParam("queryId") final String queryId, + @PathParam("stageNumber") final int stageNumber, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId); + worker.postCleanupStage(new StageId(queryId, stageNumber)); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API. + */ + @POST + @Path("/finish") + public Response httpPostFinish(@Context final HttpServletRequest req) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + worker.postFinish(); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API. + */ + @GET + @Produces({MediaType.APPLICATION_JSON + "; qs=0.9", SmileMediaTypes.APPLICATION_JACKSON_SMILE + "; qs=0.1"}) + @Path("/counters") + public Response httpGetCounters(@Context final HttpServletRequest req) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + return Response.status(Response.Status.OK).entity(worker.getCounters()).build(); + } + + /** + * Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and + * {@link #httpFetchKeyStatisticsWithSnapshot}. + */ + public enum SketchEncoding + { + /** + * The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}. + */ + OCTET_STREAM, + /** + * The key collector is encoded as json + */ + JSON + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java new file mode 100644 index 000000000000..37595050c819 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.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.msq.shuffle.input; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** + * Meta-factory that wraps {@link #inputChannelFactoryProvider}, and can create various other kinds of factories. + */ +public class MetaInputChannelFactory implements InputChannelFactory +{ + private final Int2ObjectMap stageOutputModeMap; + private final Function inputChannelFactoryProvider; + private final Map inputChannelFactoryMap = new HashMap<>(); + + public MetaInputChannelFactory( + final Int2ObjectMap stageOutputModeMap, + final Function inputChannelFactoryProvider + ) + { + this.stageOutputModeMap = stageOutputModeMap; + this.inputChannelFactoryProvider = inputChannelFactoryProvider; + } + + /** + * Create a meta-factory. + * + * @param slices stage slices from {@link WorkOrder#getInputs()} + * @param defaultOutputChannelMode mode to use when {@link StageInputSlice#getOutputChannelMode()} is null; i.e., + * when running with an older controller + * @param inputChannelFactoryProvider provider of {@link InputChannelFactory} for various {@link OutputChannelMode} + */ + public static MetaInputChannelFactory create( + final List slices, + final OutputChannelMode defaultOutputChannelMode, + final Function inputChannelFactoryProvider + ) + { + final Int2ObjectMap stageOutputModeMap = new Int2ObjectOpenHashMap<>(); + + for (final StageInputSlice slice : slices) { + final OutputChannelMode newMode; + + if (slice.getOutputChannelMode() != null) { + newMode = slice.getOutputChannelMode(); + } else { + newMode = defaultOutputChannelMode; + } + + final OutputChannelMode prevMode = stageOutputModeMap.putIfAbsent( + slice.getStageNumber(), + newMode + ); + + if (prevMode != null && prevMode != newMode) { + throw new ISE( + "Inconsistent output modes for stage[%s], got[%s] and[%s]", + slice.getStageNumber(), + prevMode, + newMode + ); + } + } + + return new MetaInputChannelFactory(stageOutputModeMap, inputChannelFactoryProvider); + } + + @Override + public ReadableFrameChannel openChannel( + final StageId stageId, + final int workerNumber, + final int partitionNumber + ) throws IOException + { + final OutputChannelMode outputChannelMode = stageOutputModeMap.get(stageId.getStageNumber()); + + if (outputChannelMode == null) { + throw new ISE("No output mode for stageNumber[%s]", stageId.getStageNumber()); + } + + return inputChannelFactoryMap.computeIfAbsent(outputChannelMode, inputChannelFactoryProvider) + .openChannel(stageId, workerNumber, partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java new file mode 100644 index 000000000000..08c7176b7c2b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java @@ -0,0 +1,70 @@ +/* + * 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.msq.shuffle.input; + +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.shuffle.output.StageOutputHolder; + +import java.io.IOException; +import java.util.List; +import java.util.function.Supplier; + +/** + * An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other + * workers. Used when durable shuffle storage is off. + */ +public class WorkerOrLocalInputChannelFactory implements InputChannelFactory +{ + private final String myId; + private final Supplier> workerIdsSupplier; + private final InputChannelFactory workerInputChannelFactory; + private final StageOutputHolderProvider stageOutputHolderProvider; + + public WorkerOrLocalInputChannelFactory( + final String myId, + final Supplier> workerIdsSupplier, + final InputChannelFactory workerInputChannelFactory, + final StageOutputHolderProvider stageOutputHolderProvider + ) + { + this.myId = myId; + this.workerIdsSupplier = workerIdsSupplier; + this.workerInputChannelFactory = workerInputChannelFactory; + this.stageOutputHolderProvider = stageOutputHolderProvider; + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException + { + final String taskId = workerIdsSupplier.get().get(workerNumber); + if (taskId.equals(myId)) { + return stageOutputHolderProvider.getHolder(stageId, partitionNumber).readLocally(); + } else { + return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber); + } + } + + public interface StageOutputHolderProvider + { + StageOutputHolder getHolder(StageId stageId, int partitionNumber); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.java new file mode 100644 index 000000000000..f623e58f65b3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStream.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.msq.shuffle.output; + +import org.apache.druid.error.DruidException; + +import java.io.InputStream; +import java.util.List; + +/** + * Input stream based on a list of byte arrays. + */ +public class ByteChunksInputStream extends InputStream +{ + private final List chunks; + private int chunkNum; + private int positionWithinChunk; + + /** + * Create a new stream wrapping a list of chunks. + * + * @param chunks byte arrays + * @param positionWithinFirstChunk starting position within the first byte array + */ + public ByteChunksInputStream(final List chunks, final int positionWithinFirstChunk) + { + this.chunks = chunks; + this.positionWithinChunk = positionWithinFirstChunk; + this.chunkNum = -1; + advanceChunk(); + } + + @Override + public int read() + { + if (chunkNum >= chunks.size()) { + return -1; + } else { + final byte[] currentChunk = chunks.get(chunkNum); + final byte b = currentChunk[positionWithinChunk++]; + + if (positionWithinChunk == currentChunk.length) { + chunkNum++; + positionWithinChunk = 0; + } + + return b & 0xFF; + } + } + + @Override + public int read(byte[] b) + { + return read(b, 0, b.length); + } + + @Override + public int read(byte[] b, int off, int len) + { + if (len == 0) { + return 0; + } else if (chunkNum >= chunks.size()) { + return -1; + } else { + int r = 0; + + while (r < len && chunkNum < chunks.size()) { + final byte[] currentChunk = chunks.get(chunkNum); + int toReadFromCurrentChunk = Math.min(len - r, currentChunk.length - positionWithinChunk); + System.arraycopy(currentChunk, positionWithinChunk, b, off + r, toReadFromCurrentChunk); + r += toReadFromCurrentChunk; + positionWithinChunk += toReadFromCurrentChunk; + if (positionWithinChunk == currentChunk.length) { + chunkNum++; + positionWithinChunk = 0; + } + } + + return r; + } + } + + @Override + public void close() + { + chunkNum = chunks.size(); + positionWithinChunk = 0; + } + + private void advanceChunk() + { + chunkNum++; + + // Verify nonempty + if (chunkNum < chunks.size() && chunks.get(chunkNum).length == 0) { + throw DruidException.defensive("Empty chunk not allowed"); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java new file mode 100644 index 000000000000..ec95ca7af6a7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReader.java @@ -0,0 +1,274 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFileWriter; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayDeque; +import java.util.Deque; + +/** + * Reader for the case where stage output is a generic {@link ReadableFrameChannel}. + * + * Because this reader returns an underlying channel directly, it must only be used when it is certain that + * only a single consumer exists, i.e., when using output mode {@link OutputChannelMode#MEMORY}. See + * {@link ControllerQueryKernelUtils#canUseMemoryOutput} for the code that ensures that there is only a single + * consumer in the in-memory case. + */ +public class ChannelStageOutputReader implements StageOutputReader +{ + enum State + { + INIT, + LOCAL, + REMOTE, + CLOSED + } + + private final ReadableFrameChannel channel; + private final FrameFileWriter writer; + + /** + * Pair of chunk size + chunk InputStream. + */ + private final Deque chunks = new ArrayDeque<>(); + + /** + * State of this reader. + */ + @GuardedBy("this") + private State state = State.INIT; + + /** + * Position of {@link #positionWithinFirstChunk} in the first chunk of {@link #chunks}, within the overall stream. + */ + @GuardedBy("this") + private long cursor; + + /** + * Offset of the first chunk in {@link #chunks} which corresponds to {@link #cursor}. + */ + @GuardedBy("this") + private int positionWithinFirstChunk; + + /** + * Whether {@link FrameFileWriter#close()} is called on {@link #writer}. + */ + @GuardedBy("this") + private boolean didCloseWriter; + + public ChannelStageOutputReader(final ReadableFrameChannel channel) + { + this.channel = channel; + this.writer = FrameFileWriter.open(new ChunkAcceptor(), null, ByteTracker.unboundedTracker()); + } + + /** + * Returns an input stream starting at the provided offset. + * + * The returned {@link InputStream} is non-blocking, and is slightly buffered (up to one frame). It does not + * necessarily contain the complete remaining dataset; this means that multiple calls to this method are necessary + * to fetch the complete dataset. + * + * The provided offset must be greater than, or equal to, the offset provided to the prior call. + * + * This class supports either remote or local reads, but not both. Calling both this method and {@link #readLocally()} + * on the same instance of this class is an error. + * + * @param offset offset into the stage output stream + */ + @Override + public synchronized ListenableFuture readRemotelyFrom(final long offset) + { + if (state == State.INIT) { + state = State.REMOTE; + } else if (state == State.LOCAL) { + throw new ISE("Cannot read both remotely and locally"); + } else if (state == State.CLOSED) { + throw new ISE("Closed"); + } + + if (offset < cursor) { + return Futures.immediateFailedFuture( + new ISE("Offset[%,d] no longer available, current cursor is[%,d]", offset, cursor)); + } + + while (chunks.isEmpty() || offset > cursor) { + // Fetch additional chunks if needed. + if (chunks.isEmpty()) { + if (didCloseWriter) { + if (offset == cursor) { + return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY)); + } else { + throw DruidException.defensive( + "Channel finished but cursor[%,d] does not match requested offset[%,d]", + cursor, + offset + ); + } + } else if (channel.isFinished()) { + try { + writer.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + didCloseWriter = true; + continue; + } else if (channel.canRead()) { + try { + writer.writeFrame(channel.read(), FrameFileWriter.NO_PARTITION); + } + catch (Exception e) { + try { + writer.abort(); + } + catch (IOException e2) { + e.addSuppressed(e2); + } + + throw new RuntimeException(e); + } + } else { + return FutureUtils.transformAsync(channel.readabilityFuture(), ignored -> readRemotelyFrom(offset)); + } + } + + // Advance cursor to the provided offset, or the end of the current chunk, whichever is earlier. + final byte[] chunk = chunks.peek(); + final long amountToAdvance = Math.min(offset - cursor, chunk.length - positionWithinFirstChunk); + cursor += amountToAdvance; + positionWithinFirstChunk += Ints.checkedCast(amountToAdvance); + + // Remove first chunk if it is no longer needed. (i.e., if the cursor is at the end of it.) + if (positionWithinFirstChunk == chunk.length) { + chunks.poll(); + positionWithinFirstChunk = 0; + } + } + + if (chunks.isEmpty() || offset != cursor) { + throw DruidException.defensive( + "Expected cursor[%,d] to be caught up to offset[%,d] by this point, and to have nonzero chunks", + cursor, + offset + ); + } + + return Futures.immediateFuture(new ByteChunksInputStream(ImmutableList.copyOf(chunks), positionWithinFirstChunk)); + } + + /** + * Returns the {@link ReadableFrameChannel} that backs this reader. + * + * Callers are responsible for closing the returned channel. Once this method is called, the caller becomes the + * owner of the channel, and this class's {@link #close()} method will no longer close the channel. + * + * Only a single reader is supported. Once this method is called, it cannot be called again. + * + * This class supports either remote or local reads, but not both. Calling both this method and + * {@link #readRemotelyFrom(long)} on the same instance of this class is an error. + */ + @Override + public synchronized ReadableFrameChannel readLocally() + { + if (state == State.INIT) { + state = State.LOCAL; + return channel; + } else if (state == State.REMOTE) { + throw new ISE("Cannot read both remotely and locally"); + } else if (state == State.LOCAL) { + throw new ISE("Cannot read channel multiple times"); + } else { + assert state == State.CLOSED; + throw new ISE("Closed"); + } + } + + /** + * Closes the {@link ReadableFrameChannel} backing this reader, unless {@link #readLocally()} has been called. + * In that case, the caller of {@link #readLocally()} is responsible for closing the channel. + */ + @Override + public synchronized void close() + { + // Call channel.close() unless readLocally() has been called. In that case, we expect the caller to close it. + if (state != State.LOCAL) { + state = State.CLOSED; + channel.close(); + } + } + + /** + * Input stream that can have bytes appended to it, and that can have bytes acknowledged. + */ + private class ChunkAcceptor implements WritableByteChannel + { + private boolean open = true; + + @Override + public int write(final ByteBuffer src) throws IOException + { + if (!open) { + throw new IOException("Closed"); + } + + final int len = src.remaining(); + if (len > 0) { + final byte[] bytes = new byte[len]; + src.get(bytes); + chunks.add(bytes); + } + + return len; + } + + @Override + public boolean isOpen() + { + return open; + } + + @Override + public void close() + { + open = false; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java new file mode 100644 index 000000000000..29fb7b17ee78 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FileStageOutputReader.java @@ -0,0 +1,99 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.channels.Channels; + +/** + * Reader for the case where stage output is stored in a {@link FrameFile} on disk. + */ +public class FileStageOutputReader implements StageOutputReader +{ + private final FrameFile frameFile; + + public FileStageOutputReader(FrameFile frameFile) + { + this.frameFile = frameFile; + } + + /** + * Returns an input stream starting at the provided offset. The file is opened and seeked in-line with this method + * call, so the returned future is always immediately resolved. Callers are responsible for closing the returned + * input stream. + * + * This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when + * broadcasting the output of a stage. + * + * @param offset offset into the stage output file + */ + @Override + public ListenableFuture readRemotelyFrom(long offset) + { + try { + final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r"); + + if (offset >= randomAccessFile.length()) { + randomAccessFile.close(); + return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY)); + } else { + randomAccessFile.seek(offset); + return Futures.immediateFuture(Channels.newInputStream(randomAccessFile.getChannel())); + } + } + catch (Exception e) { + return Futures.immediateFailedFuture(e); + } + } + + /** + * Returns a channel pointing to a fresh {@link FrameFile#newReference()} of the underlying frame file. Callers are + * responsible for closing the returned channel. + * + * This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when + * broadcasting the output of a stage. + */ + @Override + public ReadableFrameChannel readLocally() + { + return new ReadableFileFrameChannel(frameFile.newReference()); + } + + /** + * Closes the initial reference to the underlying {@link FrameFile}. Does not close additional references created by + * calls to {@link #readLocally()}; those references are closed when the channel(s) returned by {@link #readLocally()} + * are closed. + */ + @Override + public void close() throws IOException + { + frameFile.close(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.java new file mode 100644 index 000000000000..8dcb8786713b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/FutureReadableFrameChannel.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.msq.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.NoSuchElementException; + +/** + * Channel that wraps a {@link ListenableFuture} of a {@link ReadableFrameChannel}, but acts like a regular (non-future) + * {@link ReadableFrameChannel}. + */ +public class FutureReadableFrameChannel implements ReadableFrameChannel +{ + private static final Logger log = new Logger(FutureReadableFrameChannel.class); + + private final ListenableFuture channelFuture; + private ReadableFrameChannel channel; + + public FutureReadableFrameChannel(final ListenableFuture channelFuture) + { + this.channelFuture = channelFuture; + } + + @Override + public boolean isFinished() + { + if (populateChannel()) { + return channel.isFinished(); + } else { + return false; + } + } + + @Override + public boolean canRead() + { + if (populateChannel()) { + return channel.canRead(); + } else { + return false; + } + } + + @Override + public Frame read() + { + if (populateChannel()) { + return channel.read(); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public ListenableFuture readabilityFuture() + { + if (populateChannel()) { + return channel.readabilityFuture(); + } else { + return FutureUtils.transformAsync(channelFuture, ignored -> readabilityFuture()); + } + } + + @Override + public void close() + { + if (populateChannel()) { + channel.close(); + } else { + channelFuture.cancel(true); + + // In case of a race where channelFuture resolved between populateChannel() and here, the cancel call above would + // have no effect. Guard against this case by checking if the channelFuture has resolved, and if so, close the + // channel here. + try { + final ReadableFrameChannel theChannel = FutureUtils.getUncheckedImmediately(channelFuture); + + try { + theChannel.close(); + } + catch (Throwable t) { + log.noStackTrace().warn(t, "Failed to close channel"); + } + } + catch (Throwable ignored) { + // Suppress. + } + } + } + + private boolean populateChannel() + { + if (channel != null) { + return true; + } else if (channelFuture.isDone()) { + channel = FutureUtils.getUncheckedImmediately(channelFuture); + return true; + } else { + return false; + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java new file mode 100644 index 000000000000..86530dad1d01 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/NilStageOutputReader.java @@ -0,0 +1,77 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.file.FrameFileWriter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.Channels; + +/** + * Reader for the case where stage output is known to be empty. + */ +public class NilStageOutputReader implements StageOutputReader +{ + public static final NilStageOutputReader INSTANCE = new NilStageOutputReader(); + + private static final byte[] EMPTY_FRAME_FILE; + + static { + try { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close(); + EMPTY_FRAME_FILE = baos.toByteArray(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public ListenableFuture readRemotelyFrom(final long offset) + { + final ByteArrayInputStream in = new ByteArrayInputStream(EMPTY_FRAME_FILE); + + //noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream. + in.skip(offset); + + return Futures.immediateFuture(in); + } + + @Override + public ReadableFrameChannel readLocally() + { + return ReadableNilFrameChannel.INSTANCE; + } + + @Override + public void close() + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java new file mode 100644 index 000000000000..c19519dfb7bb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java @@ -0,0 +1,141 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.rpc.WorkerResource; +import org.apache.druid.utils.CloseableUtils; + +import javax.servlet.http.HttpServletRequest; +import java.io.Closeable; +import java.io.InputStream; + +/** + * Container for a {@link StageOutputReader}, which is used to read the output of a stage. + */ +public class StageOutputHolder implements Closeable +{ + private final SettableFuture channelFuture; + private final ListenableFuture readerFuture; + + public StageOutputHolder() + { + this.channelFuture = SettableFuture.create(); + this.readerFuture = FutureUtils.transform(channelFuture, StageOutputHolder::createReader); + } + + /** + * Method for remote reads. + * + * Provides the implementation for {@link Worker#readStageOutput(StageId, int, long)}, which is in turn used by + * {@link WorkerResource#httpGetChannelData(String, int, int, long, HttpServletRequest)}. + * + * @see StageOutputReader#readRemotelyFrom(long) for details on behavior + */ + public ListenableFuture readRemotelyFrom(final long offset) + { + return FutureUtils.transformAsync(readerFuture, reader -> reader.readRemotelyFrom(offset)); + } + + /** + * Method for local reads. + * + * Used instead of {@link #readRemotelyFrom(long)} when a worker is reading a channel from itself, to avoid needless + * HTTP calls to itself. + * + * @see StageOutputReader#readLocally() for details on behavior + */ + public ReadableFrameChannel readLocally() + { + return new FutureReadableFrameChannel(FutureUtils.transform(readerFuture, StageOutputReader::readLocally)); + } + + /** + * Sets the channel that backs {@link #readLocally()} and {@link #readRemotelyFrom(long)}. + */ + public void setChannel(final ReadableFrameChannel channel) + { + if (!channelFuture.set(channel)) { + if (FutureUtils.getUncheckedImmediately(channelFuture) == null) { + throw new ISE("Closed"); + } else { + throw new ISE("Channel already set"); + } + } + } + + @Override + public void close() + { + channelFuture.set(null); + + final StageOutputReader reader; + + try { + reader = FutureUtils.getUnchecked(readerFuture, true); + } + catch (Throwable e) { + // Error creating the reader, nothing to close. Suppress. + return; + } + + if (reader != null) { + CloseableUtils.closeAndWrapExceptions(reader); + } + } + + private static StageOutputReader createReader(final ReadableFrameChannel channel) + { + if (channel == null) { + // Happens if close() was called before the channel resolved. + throw new ISE("Closed"); + } + + if (channel instanceof ReadableNilFrameChannel) { + return NilStageOutputReader.INSTANCE; + } + + if (channel instanceof ReadableFileFrameChannel) { + // Optimized implementation when reading an entire file. + final ReadableFileFrameChannel fileChannel = (ReadableFileFrameChannel) channel; + + if (fileChannel.isEntireFile()) { + final FrameFile frameFile = fileChannel.newFrameFileReference(); + + // Close original channel, so we don't leak a frame file reference. + channel.close(); + + return new FileStageOutputReader(frameFile); + } + } + + // Generic implementation for any other type of channel. + return new ChannelStageOutputReader(channel); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java new file mode 100644 index 000000000000..36b993611ca4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputReader.java @@ -0,0 +1,77 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.StageId; + +import java.io.Closeable; +import java.io.InputStream; + +/** + * Interface for reading output channels for a particular stage. Each instance of this interface represents a + * stream from a single {@link org.apache.druid.msq.kernel.StagePartition} in {@link FrameFile} format. + * + * @see FileStageOutputReader implementation backed by {@link FrameFile} + * @see ChannelStageOutputReader implementation backed by {@link ReadableFrameChannel} + * @see NilStageOutputReader implementation for an empty channel + */ +public interface StageOutputReader extends Closeable +{ + /** + * Method for remote reads. + * + * This method ultimately backs {@link Worker#readStageOutput(StageId, int, long)}. Refer to that method's + * documentation for details about behavior of the returned future. + * + * Callers are responsible for closing the returned {@link InputStream}. This input stream may encapsulate + * resources that are not closed by this class's {@link #close()} method. + * + * It is implementation-dependent whether calls to this method must have monotonically increasing offsets. + * In particular, {@link ChannelStageOutputReader} requires monotonically increasing offsets, but + * {@link FileStageOutputReader} and {@link NilStageOutputReader} do not. + * + * @param offset offset into the stage output file + * + * @see StageOutputHolder#readRemotelyFrom(long) which uses this method + * @see Worker#readStageOutput(StageId, int, long) for documentation on behavior of the returned future + */ + ListenableFuture readRemotelyFrom(long offset); + + /** + * Method for local reads. + * + * Depending on implementation, this method may or may not be able to be called multiple times, and may or may not + * be able to be mixed with {@link #readRemotelyFrom(long)}. Refer to the specific implementation for more details. + * + * Callers are responsible for closing the returned channel. The returned channel may encapsulate resources that + * are not closed by this class's {@link #close()} method. + * + * It is implementation-dependent whether this method can be called multiple times. In particular, + * {@link ChannelStageOutputReader#readLocally()} can only be called one time, but the implementations in + * {@link FileStageOutputReader} and {@link NilStageOutputReader} can be called multiple times. + * + * @see StageOutputHolder#readLocally() which uses this method + */ + ReadableFrameChannel readLocally(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index de66550a5875..bd33d76adb1e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -23,12 +23,14 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.error.ErrorResponse; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.report.MSQStagesReport; import org.apache.druid.msq.sql.SqlStatementState; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; -import java.util.Objects; public class SqlStatementResult { @@ -51,6 +53,27 @@ public class SqlStatementResult @Nullable private final ErrorResponse errorResponse; + @Nullable + private final MSQStagesReport stages; + + @Nullable + private final CounterSnapshotsTree counters; + + @Nullable + private final List warnings; + + public SqlStatementResult( + String queryId, + SqlStatementState state, + DateTime createdAt, + List sqlRowSignature, + Long durationMs, + ResultSetInformation resultSetInformation, + ErrorResponse errorResponse + ) + { + this(queryId, state, createdAt, sqlRowSignature, durationMs, resultSetInformation, errorResponse, null, null, null); + } @JsonCreator public SqlStatementResult( @@ -67,8 +90,13 @@ public SqlStatementResult( @Nullable @JsonProperty("result") ResultSetInformation resultSetInformation, @Nullable @JsonProperty("errorDetails") - ErrorResponse errorResponse - + ErrorResponse errorResponse, + @Nullable @JsonProperty("stages") + MSQStagesReport stages, + @Nullable @JsonProperty("counters") + CounterSnapshotsTree counters, + @Nullable @JsonProperty("warnings") + List warnings ) { this.queryId = queryId; @@ -78,6 +106,9 @@ public SqlStatementResult( this.durationMs = durationMs; this.resultSetInformation = resultSetInformation; this.errorResponse = errorResponse; + this.stages = stages; + this.counters = counters; + this.warnings = warnings; } @JsonProperty @@ -130,41 +161,28 @@ public ErrorResponse getErrorResponse() return errorResponse; } + @JsonProperty("stages") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public MSQStagesReport getStages() + { + return stages; + } - @Override - public boolean equals(Object o) + @JsonProperty("counters") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public CounterSnapshotsTree getCounters() { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SqlStatementResult that = (SqlStatementResult) o; - return Objects.equals(queryId, that.queryId) && state == that.state && Objects.equals( - createdAt, - that.createdAt - ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( - durationMs, - that.durationMs - ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( - errorResponse == null ? null : errorResponse.getAsMap(), - that.errorResponse == null ? null : that.errorResponse.getAsMap() - ); + return counters; } - @Override - public int hashCode() + @JsonProperty("warnings") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getWarnings() { - return Objects.hash( - queryId, - state, - createdAt, - sqlRowSignature, - durationMs, - resultSetInformation, - errorResponse == null ? null : errorResponse.getAsMap() - ); + return warnings; } @Override @@ -180,6 +198,9 @@ public String toString() ", errorResponse=" + (errorResponse == null ? "{}" : errorResponse.getAsMap().toString()) + + ", stages=" + stages + + ", counters=" + counters + + ", warnings=" + warnings + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 4beb2a869ef0..f56622804786 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -231,7 +231,9 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques @Path("/{id}") @Produces(MediaType.APPLICATION_JSON) public Response doGetStatus( - @PathParam("id") final String queryId, @Context final HttpServletRequest req + @PathParam("id") final String queryId, + @QueryParam("detail") boolean detail, + @Context final HttpServletRequest req ) { try { @@ -242,7 +244,8 @@ public Response doGetStatus( queryId, authenticationResult, true, - Action.READ + Action.READ, + detail ); if (sqlStatementResult.isPresent()) { @@ -369,7 +372,8 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina queryId, authenticationResult, false, - Action.WRITE + Action.WRITE, + false ); if (sqlStatementResult.isPresent()) { switch (sqlStatementResult.get().getState()) { @@ -479,7 +483,7 @@ private Response buildTaskResponse(Sequence sequence, AuthenticationRe } String taskId = String.valueOf(firstRow[0]); - Optional statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ); + Optional statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ, false); if (statementResult.isPresent()) { return Response.status(Response.Status.OK).entity(statementResult.get()).build(); @@ -565,7 +569,8 @@ private Optional getStatementStatus( String queryId, AuthenticationResult authenticationResult, boolean withResults, - Action forAction + Action forAction, + boolean detail ) throws DruidException { TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId), queryId); @@ -582,14 +587,29 @@ private Optional getStatementStatus( MSQControllerTask msqControllerTask = getMSQControllerTaskAndCheckPermission(queryId, authenticationResult, forAction); SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + MSQTaskReportPayload taskReportPayload = null; + if (detail || SqlStatementState.FAILED == sqlStatementState) { + try { + taskReportPayload = SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId), queryId) + ); + } + catch (DruidException e) { + if (!e.getErrorCode().equals("notFound") && !e.getMessage().contains("Unable to contact overlord")) { + throw e; + } + } + } + if (SqlStatementState.FAILED == sqlStatementState) { return SqlStatementResourceHelper.getExceptionPayload( queryId, taskResponse, statusPlus, sqlStatementState, - contactOverlord(overlordClient.taskReportAsMap(queryId), queryId), - jsonMapper + taskReportPayload, + jsonMapper, + detail ); } else { Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); @@ -605,7 +625,10 @@ private Optional getStatementStatus( sqlStatementState, msqControllerTask.getQuerySpec().getDestination() ).orElse(null) : null, - null + null, + SqlStatementResourceHelper.getQueryStagesReport(taskReportPayload), + SqlStatementResourceHelper.getQueryCounters(taskReportPayload), + SqlStatementResourceHelper.getQueryWarningDetails(taskReportPayload) )); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index 4f07dcb2cc02..0820342ba727 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -250,11 +250,12 @@ public static Optional getExceptionPayload( TaskStatusResponse taskResponse, TaskStatusPlus statusPlus, SqlStatementState sqlStatementState, - TaskReport.ReportMap msqPayload, - ObjectMapper jsonMapper + MSQTaskReportPayload msqTaskReportPayload, + ObjectMapper jsonMapper, + boolean detail ) { - final MSQErrorReport exceptionDetails = getQueryExceptionDetails(getPayload(msqPayload)); + final MSQErrorReport exceptionDetails = getQueryExceptionDetails(msqTaskReportPayload); final MSQFault fault = exceptionDetails == null ? null : exceptionDetails.getFault(); if (exceptionDetails == null || fault == null) { return Optional.of(new SqlStatementResult( @@ -267,7 +268,10 @@ public static Optional getExceptionPayload( DruidException.forPersona(DruidException.Persona.DEVELOPER) .ofCategory(DruidException.Category.UNCATEGORIZED) .build("%s", taskResponse.getStatus().getErrorMsg()) - .toErrorResponse() + .toErrorResponse(), + detail ? getQueryStagesReport(msqTaskReportPayload) : null, + detail ? getQueryCounters(msqTaskReportPayload) : null, + detail ? getQueryWarningDetails(msqTaskReportPayload) : null )); } @@ -293,7 +297,10 @@ protected DruidException makeException(DruidException.DruidExceptionBuilder bob) ex.withContext(exceptionContext); return ex; } - }).toErrorResponse() + }).toErrorResponse(), + detail ? getQueryStagesReport(msqTaskReportPayload) : null, + detail ? getQueryCounters(msqTaskReportPayload) : null, + detail ? getQueryWarningDetails(msqTaskReportPayload) : null )); } @@ -353,7 +360,7 @@ public Object[] next() } @Nullable - public static MSQStagesReport.Stage getFinalStage(MSQTaskReportPayload msqTaskReportPayload) + public static MSQStagesReport.Stage getFinalStage(@Nullable MSQTaskReportPayload msqTaskReportPayload) { if (msqTaskReportPayload == null || msqTaskReportPayload.getStages().getStages() == null) { return null; @@ -369,11 +376,29 @@ public static MSQStagesReport.Stage getFinalStage(MSQTaskReportPayload msqTaskRe } @Nullable - private static MSQErrorReport getQueryExceptionDetails(MSQTaskReportPayload payload) + private static MSQErrorReport getQueryExceptionDetails(@Nullable MSQTaskReportPayload payload) { return payload == null ? null : payload.getStatus().getErrorReport(); } + @Nullable + public static List getQueryWarningDetails(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : new ArrayList<>(payload.getStatus().getWarningReports()); + } + + @Nullable + public static MSQStagesReport getQueryStagesReport(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : payload.getStages(); + } + + @Nullable + public static CounterSnapshotsTree getQueryCounters(@Nullable MSQTaskReportPayload payload) + { + return payload == null ? null : payload.getCounters(); + } + @Nullable public static MSQTaskReportPayload getPayload(TaskReport.ReportMap reportMap) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 3cf4bf12ed1e..20a871b953cb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -230,7 +230,7 @@ public void testReplaceMvdWithStringArraySkipValidation(String contextName, Map< .setQueryContext(adjustedContext) .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( NullHandling.sqlCompatible() ? ImmutableList.of( @@ -278,7 +278,7 @@ public void testReplaceMvdWithMvd(String contextName, Map contex .setQueryContext(adjustedContext) .setExpectedDataSource("foo") .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows( ImmutableList.of( new Object[]{0L, null}, @@ -325,7 +325,7 @@ public void testInsertOnFoo1WithMultiValueToArrayGroupByWithDefaultContext(Strin .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) .setExpectedResultRows(expectedRows) .verifyResults(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java index efddab577e77..f937a988d8e0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java @@ -22,11 +22,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.systemfield.SystemFields; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; @@ -34,8 +36,20 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.DataSource; import org.apache.druid.query.NestedDataTestUtils; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -44,8 +58,10 @@ import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CompressionUtils; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -64,12 +80,13 @@ public class MSQComplexGroupByTest extends MSQTestBase { static { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private String dataFileNameJsonString; private String dataFileSignatureJsonString; private DataSource dataFileExternalDataSource; + private File dataFile; public static Collection data() { @@ -85,9 +102,9 @@ public static Collection data() @BeforeEach public void setup() throws IOException { - File dataFile = newTempFile("dataFile"); + dataFile = newTempFile("dataFile"); final InputStream resourceStream = this.getClass().getClassLoader() - .getResourceAsStream(NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE); + .getResourceAsStream(NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE); final InputStream decompressing = CompressionUtils.decompress( resourceStream, "nested-all-types-test-data.json" @@ -109,7 +126,7 @@ public void setup() throws IOException dataFileSignature ); - objectMapper.registerModules(NestedDataModule.getJacksonModulesList()); + objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); } @MethodSource("data") @@ -129,7 +146,7 @@ public void testInsertWithoutRollupOnNestedData(String contextName, Map cont )) .verifyResults(); } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testExactCountDistinctOnNestedData(String contextName, Map context) + { + Assumptions.assumeTrue(NullHandling.sqlCompatible()); + RowSignature rowSignature = RowSignature.builder() + .add("distinct_obj", ColumnType.LONG) + .build(); + + Map modifiedContext = ImmutableMap.builder() + .putAll(context) + .put(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, false) + .build(); + + DimFilter innerFilter = NullHandling.replaceWithDefault() + ? new SelectorDimFilter("d0", null, null) + : new NullFilter("d0", null); + + testSelectQuery().setSql("SELECT\n" + + " COUNT(DISTINCT obj) AS distinct_obj\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + dataFileNameJsonString + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"obj\", \"type\": \"COMPLEX\"}]'\n" + + " )\n" + + " )\n" + + " ORDER BY 1") + .setQueryContext(ImmutableMap.of(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, false)) + .setExpectedMSQSpec( + MSQSpec + .builder() + .query( + GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(dataFileExternalDataSource) + .setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY)) + .setDimensions( + new DefaultDimensionSpec("obj", "d0", ColumnType.NESTED_DATA) + ) + .setGranularity(Granularities.ALL) + .setContext(modifiedContext) + .build() + ) + ) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + new NotDimFilter(innerFilter), + "a0" + ) + ) + .setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .setLimitSpec(new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.NUMERIC + ) + ), + Integer.MAX_VALUE + )) + .setContext(modifiedContext) + .build() + ) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("a0", "distinct_obj") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setQueryContext(modifiedContext) + .setExpectedResultRows(ImmutableList.of( + new Object[]{7L} + )) + .verifyResults(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testExactCountDistinctOnNestedData2(String contextName, Map context) + { + Assumptions.assumeTrue(NullHandling.sqlCompatible()); + RowSignature dataFileSignature = RowSignature.builder() + .add("timestamp", ColumnType.STRING) + .add("cObj", ColumnType.NESTED_DATA) + .build(); + DataSource dataFileExternalDataSource2 = new ExternalDataSource( + new LocalInputSource(null, null, ImmutableList.of(dataFile), SystemFields.none()), + new JsonInputFormat(null, null, null, null, null), + dataFileSignature + ); + RowSignature rowSignature = RowSignature.builder() + .add("distinct_obj", ColumnType.LONG) + .build(); + + Map modifiedContext = ImmutableMap.builder() + .putAll(context) + .put(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, false) + .build(); + + DimFilter innerFilter = NullHandling.replaceWithDefault() + ? new SelectorDimFilter("d0", null, null) + : new NullFilter("d0", null); + + testSelectQuery().setSql("SELECT\n" + + " COUNT(DISTINCT cObj) AS distinct_obj\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + dataFileNameJsonString + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"cObj\", \"type\": \"COMPLEX\"}]'\n" + + " )\n" + + " )\n" + + " ORDER BY 1") + .setQueryContext(ImmutableMap.of(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, false)) + .setExpectedMSQSpec( + MSQSpec + .builder() + .query( + GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(dataFileExternalDataSource2) + .setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY)) + .setDimensions( + new DefaultDimensionSpec("cObj", "d0", ColumnType.NESTED_DATA) + ) + .setGranularity(Granularities.ALL) + .setContext(modifiedContext) + .build() + ) + ) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + new NotDimFilter(innerFilter), + "a0" + ) + ) + .setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .setLimitSpec(new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.NUMERIC + ) + ), + Integer.MAX_VALUE + )) + .setContext(modifiedContext) + .build() + ) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("a0", "distinct_obj") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setExpectedRowSignature(rowSignature) + .setQueryContext(modifiedContext) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L} + )) + .verifyResults(); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java index d3a67fdd659c..1b2eebe7742e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.test.CalciteMSQTestsHelper; import org.apache.druid.msq.test.ExtractResultsFactory; +import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestOverlordServiceClient; import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.msq.test.VerifyMSQSupportedNativeQueriesPredicate; @@ -63,15 +64,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java index 538cd4714200..dde587c0c049 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java @@ -57,7 +57,7 @@ public void testExport() throws IOException .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -88,7 +88,7 @@ public void testExport2() throws IOException .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -127,7 +127,7 @@ public void testNumberOfRowsPerFile() .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -168,7 +168,7 @@ void testExportComplexColumns() throws IOException .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -219,7 +219,7 @@ void testExportSketchColumns() throws IOException .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -258,7 +258,7 @@ void testEmptyExport() throws IOException .setExpectedDataSource("foo1") .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); @@ -335,7 +335,7 @@ public void testExportWithLimit() throws IOException .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of()) + .setExpectedSegments(ImmutableSet.of()) .setExpectedResultRows(ImmutableList.of()) .verifyResults(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 098b143b2772..4449090fb69f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -113,7 +113,7 @@ public void testInsertOnFoo1(String contextName, Map context) .setExpectedDataSource("foo1") .setQueryContext(context) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -310,7 +310,7 @@ public void testInsertOnExternalDataSource(String contextName, Map con .setExpectedRollUp(true) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedRows) .setExpectedCountersForStageWorkerChannel( CounterSnapshotMatcher @@ -969,7 +969,7 @@ public void testRollUpOnFoo1WithTimeFunction(String contextName, Map c .setExpectedRollUp(true) .addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true)) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn()) .verifyResults(); @@ -1102,7 +1102,7 @@ public void testRollUpOnExternalDataSource(String contextName, Map co .setExpectedDataSource("foo1") .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -1538,7 +1545,7 @@ public void testCorrectNumberOfWorkersUsedAutoModeWithoutBytesLimit(String conte .setExpectedDataSource("foo1") .setQueryContext(localContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", @@ -1590,7 +1597,7 @@ public void testCorrectNumberOfWorkersUsedAutoModeWithBytesLimit(String contextN .setExpectedDataSource("foo1") .setQueryContext(localContext) .setExpectedRowSignature(rowSignature) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo1", Intervals.of("2016-06-27/P1D"), "test", diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index f6eb80b32826..cbdddf4a46ef 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -49,6 +49,7 @@ import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -141,7 +142,7 @@ public void testReplaceOnFooWithAll(String contextName, Map cont .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo", Intervals.of("2000-01-01T/P1D"), "test", 0), SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0), @@ -213,7 +214,7 @@ public void testReplaceOnFooWithWhere(String contextName, Map co "2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-02T/P1D"), "test", @@ -283,7 +284,7 @@ public void testReplaceOnFoo1WithAllExtern(String contextName, Map c .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -639,7 +640,7 @@ public void testReplaceWithWhereClause(String contextName, Map c new Object[]{946771200000L, 2.0f} ) ) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -719,7 +720,7 @@ public void testReplaceWhereClauseLargerThanData(String contextName, Map contex .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T"))) - .setExpectedSegment(ImmutableSet.of(SegmentId.of( + .setExpectedSegments(ImmutableSet.of(SegmentId.of( "foo", Intervals.of("2000-01-01T/P1M"), "test", @@ -949,7 +955,7 @@ public void testReplaceOnFoo1WithLimit(String contextName, Map c .setQueryContext(queryContext) .setExpectedRowSignature(rowSignature) .setExpectedShardSpec(DimensionRangeShardSpec.class) - .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) + .setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1))) .setExpectedResultRows(expectedRows) .setExpectedMSQSegmentReport( new MSQSegmentReport( @@ -957,6 +963,13 @@ public void testReplaceOnFoo1WithLimit(String contextName, Map c "Using RangeShardSpec to generate segments." ) ) + .setExpectedLastCompactionState(expectedCompactionState( + queryContext, + Collections.singletonList("dim1"), + Collections.singletonList(new StringDimensionSchema("dim1")), + GranularityType.ALL, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1001,7 +1014,7 @@ public void testReplaceTimeChunksLargerThanData(String contextName, Map conte ) .setExpectedRowSignature(rowSignature) .setQueryContext(context) - .setExpectedSegment(expectedFooSegments()) + .setExpectedSegments(expectedFooSegments()) .setExpectedResultRows(expectedFooRows()) .setExpectedLastCompactionState( expectedCompactionState( @@ -1150,11 +1163,10 @@ public void testReplaceOnFoo1RangeClusteredBySubset(String contextName, Map co new Object[]{978480000000L, 6.0f, 6.0, 6.0} ) ) - .setExpectedSegment( + .setExpectedSegments( ImmutableSet.of( SegmentId.of("foo1", Intervals.of("2000-01-01T/P1D"), "test", 0), SegmentId.of("foo1", Intervals.of("2000-01-02T/P1D"), "test", 0), @@ -2045,7 +2045,7 @@ public void testReplaceGroupByOnWikipedia(String contextName, Map(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE)); - Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshot(new StageId("xx", 1))); - } - - @Test - public void testFetchStatsWithTimeChunkThrows() - { - WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE)); - Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshotForTimeChunk(new StageId("xx", 1), 1L)); - } - -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java index 29614fc07347..1ead2a181fd9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java @@ -32,34 +32,54 @@ public class WorkerMemoryParametersTest @Test public void test_oneWorkerInJvm_alone() { - Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 0, 0)); - Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 0, 0)); - Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 0, 0)); - Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 0, 0)); + Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 1, 0, 0)); + Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 1, 0, 0)); + Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 1, 0, 0)); + Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 1, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 0, 0) + () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0)) + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) .getFault(); - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault); + Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); + } + + @Test + public void test_oneWorkerInJvm_alone_twoConcurrentStages() + { + Assert.assertEquals(params(166_750_000, 1, 20, 37_500_000), create(1_000_000_000, 1, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(110_500_000, 2, 6, 37_500_000), create(1_000_000_000, 1, 2, 2, 1, 0, 0)); + Assert.assertEquals(params(65_500_000, 2, 3, 37_500_000), create(1_000_000_000, 1, 4, 2, 1, 0, 0)); + Assert.assertEquals(params(35_500_000, 1, 3, 37_500_000), create(1_000_000_000, 1, 8, 2, 1, 0, 0)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> create(1_000_000_000, 1, 12, 2, 1, 0, 0) + ); + + Assert.assertEquals(new NotEnoughMemoryFault(1_736_034_666, 1_000_000_000, 750_000_000, 1, 12, 2), e.getFault()); + + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 2, 1, 0, 0)) + .getFault(); + Assert.assertEquals(new NotEnoughMemoryFault(4_048_090_666L, 1_000_000_000, 750_000_000, 2, 32, 2), fault); } @Test public void test_oneWorkerInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 200, 0, 0)); - Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 200, 0, 0)); + Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 1, 200, 0, 0)); + Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 1, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 4, 200, 0, 0) + () -> create(1_000_000_000, 1, 4, 1, 200, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); @@ -68,76 +88,102 @@ public void test_oneWorkerInJvm_twoHundredWorkersInCluster() @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 0, 0)); - Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 0, 0)); - Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 0, 0)); - Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 0, 0)); - Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 200, 0, 0)); + Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 0, 0)); + Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 0, 0)); + Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 0, 0)); + Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 0, 0)); + Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(8_000_000_000L, 4, 32, 200, 0, 0) + () -> create(8_000_000_000L, 4, 32, 1, 200, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault()); // Make sure 124 actually works, and 125 doesn't. (Verify the error message above.) - Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 124, 0, 0)); + Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 1, 124, 0, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(8_000_000_000L, 4, 32, 125, 0, 0) + () -> create(8_000_000_000L, 4, 32, 1, 125, 0, 0) ); Assert.assertEquals(new TooManyWorkersFault(125, 124), e2.getFault()); } + @Test + public void test_fourWorkersInJvm_twoHundredWorkersInCluster_twoConcurrentStages() + { + Assert.assertEquals(params(406_500_000, 1, 74, 84_375_000), create(9_000_000_000L, 4, 1, 2, 200, 0, 0)); + Assert.assertEquals(params(305_250_000, 2, 30, 84_375_000), create(9_000_000_000L, 4, 2, 2, 200, 0, 0)); + Assert.assertEquals(params(178_687_500, 4, 10, 84_375_000), create(9_000_000_000L, 4, 4, 2, 200, 0, 0)); + Assert.assertEquals(params(52_125_000, 4, 6, 84_375_000), create(9_000_000_000L, 4, 8, 2, 200, 0, 0)); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> create(8_000_000_000L, 4, 16, 2, 200, 0, 0) + ); + + Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); + + // Make sure 109 actually works, and 110 doesn't. (Verify the error message above.) + Assert.assertEquals(params(25_000_000, 4, 3, 75_000_000), create(8_000_000_000L, 4, 16, 2, 109, 0, 0)); + + final MSQException e2 = Assert.assertThrows( + MSQException.class, + () -> create(8_000_000_000L, 4, 16, 2, 110, 0, 0) + ); + + Assert.assertEquals(new TooManyWorkersFault(110, 109), e2.getFault()); + } + @Test public void test_oneWorkerInJvm_smallWorkerCapacity() { // Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels - Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 1, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 0, 0) + () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(128_000_000, 1, 4, 1, 0, 0) + () -> create(128_000_000, 1, 4, 1, 1, 0, 0) ); - Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4), e2.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4, 1), e2.getFault()); - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0)) + final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) .getFault(); - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault); + Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); } @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions() { - Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 200, 0)); - Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 200, 0)); - Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 200, 0)); - Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 200, 0)); + Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 200, 0)); + Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 200, 0)); + Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 200, 0)); + Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 200, 0)); final MSQException e = Assert.assertThrows( MSQException.class, - () -> create(9_000_000_000L, 4, 16, 200, 200, 0) + () -> create(9_000_000_000L, 4, 16, 1, 200, 200, 0) ); Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault()); // Make sure 138 actually works, and 139 doesn't. (Verify the error message above.) - Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 138, 138, 0)); + Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 138, 138, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, - () -> create(9_000_000_000L, 4, 16, 139, 139, 0) + () -> create(9_000_000_000L, 4, 16, 1, 139, 139, 0) ); Assert.assertEquals(new TooManyWorkersFault(139, 138), e2.getFault()); @@ -148,10 +194,10 @@ public void test_oneWorkerInJvm_oneByteUsableMemory() { final MSQException e = Assert.assertThrows( MSQException.class, - () -> WorkerMemoryParameters.createInstance(1, 1, 1, 32, 1, 1) + () -> WorkerMemoryParameters.createInstance(1, 1, 1, 1, 32, 1, 1) ); - Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1), e.getFault()); + Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1, 1), e.getFault()); } @Test @@ -179,6 +225,7 @@ private static WorkerMemoryParameters create( final long maxMemoryInJvm, final int numWorkersInJvm, final int numProcessingThreadsInJvm, + final int maxConcurrentStages, final int numInputWorkers, final int numHashOutputPartitions, final int totalLookUpFootprint @@ -188,6 +235,7 @@ private static WorkerMemoryParameters create( maxMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm, + maxConcurrentStages, numInputWorkers, numHashOutputPartitions, totalLookUpFootprint diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java index 583c21d3407c..dfb88d17b216 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.indexing; +import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; import com.google.inject.Injector; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -30,6 +31,7 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; +import org.mockito.quality.Strictness; import java.util.Collections; @@ -44,12 +46,19 @@ public void setup() Mockito.when(injectorMock.getInstance(SegmentCacheManagerFactory.class)) .thenReturn(Mockito.mock(SegmentCacheManagerFactory.class)); + final MSQWorkerTask task = + Mockito.mock(MSQWorkerTask.class, Mockito.withSettings().strictness(Strictness.STRICT_STUBS)); + Mockito.when(task.getContext()).thenReturn(ImmutableMap.of()); + indexerWorkerContext = new IndexerWorkerContext( + task, Mockito.mock(TaskToolbox.class), injectorMock, null, null, null, + null, + null, null ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index b95243f7783f..6c5d19572652 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.math.expr.ExprMacroTable; @@ -54,10 +55,12 @@ import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; @@ -73,6 +76,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class MSQCompactionRunnerTest { @@ -127,7 +131,7 @@ public void testHashedPartitionsSpecIsInvalid() null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -140,7 +144,7 @@ public void testDimensionRangePartitionsSpecIsValid() null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -153,7 +157,7 @@ public void testMaxTotalRowsIsInvalid() null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -166,7 +170,7 @@ public void testDynamicPartitionsSpecIsValid() null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -179,7 +183,7 @@ public void testQueryGranularityAllIsValid() new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -192,28 +196,7 @@ public void testRollupFalseWithMetricsSpecIsInValid() new ClientCompactionTaskGranularitySpec(null, null, false), AGGREGATORS.toArray(new AggregatorFactory[0]) ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); - } - - @Test - public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() - { - // Aggregators having different input and ouput column names are unsupported. - final String inputColName = "added"; - final String outputColName = "sum_added"; - CompactionTask compactionTask = createCompactionTask( - new DynamicPartitionsSpec(3, null), - null, - Collections.emptyMap(), - new ClientCompactionTaskGranularitySpec(null, null, null), - new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} - ); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask); - Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals( - "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", - validationResult.getReason() - ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -288,6 +271,10 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce ); Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); + Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new ScanQuery.OrderBy( + col, + ScanQuery.Order.ASCENDING + )).collect(Collectors.toList()), ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys()); } @Test @@ -358,6 +345,48 @@ public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcess Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); } + @Test + public void testIntervalsWithRolledUpSegmentsAndNonIdempotentAggregatorFails() + { + final String inputColName = "added"; + final String outputColName = "sum_added"; + CompactionTask compactionTask = createCompactionTask( + null, + null, + Collections.emptyMap(), + null, + new AggregatorFactory[]{ + new LongSumAggregatorFactory( + outputColName, + inputColName + ) + } + ); + CombinedDataSchema dataSchema = new CombinedDataSchema( + DATA_SOURCE, + new TimestampSpec(TIMESTAMP_COLUMN, null, null), + new DimensionsSpec(DIMENSIONS), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}, + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ), + null, + true + ); + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals(validationResult.getReason(), StringUtils.format( + "MSQ: Rolled-up segments in compaction interval[%s].", + COMPACTION_INTERVAL + )); + } + private CompactionTask createCompactionTask( @Nullable PartitionsSpec partitionsSpec, @Nullable DimFilter dimFilter, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java index 5d86abd129ce..ccf91acb6667 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java @@ -19,12 +19,8 @@ package org.apache.druid.msq.indexing; -import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.key.ClusterByPartitions; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.exec.Worker; @@ -32,12 +28,9 @@ import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.After; import org.junit.Assert; @@ -51,15 +44,16 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import java.io.InputStream; -import java.util.HashMap; public class WorkerChatHandlerTest { private static final StageId TEST_STAGE = new StageId("123", 0); + private static final String DATASOURCE = "foo"; + @Mock private HttpServletRequest req; - private TaskToolbox toolbox; + private AuthorizerMapper authorizerMapper; private AutoCloseable mocks; private final TestWorker worker = new TestWorker(); @@ -67,29 +61,16 @@ public class WorkerChatHandlerTest @Before public void setUp() { - ObjectMapper mapper = new DefaultObjectMapper(); - IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - mapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - + authorizerMapper = CalciteTests.TEST_AUTHORIZER_MAPPER; mocks = MockitoAnnotations.openMocks(this); Mockito.when(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .thenReturn(new AuthenticationResult("druid", "druid", null, null)); - TaskToolbox.Builder builder = new TaskToolbox.Builder(); - toolbox = builder.authorizerMapper(CalciteTests.TEST_AUTHORIZER_MAPPER) - .indexIO(indexIO) - .indexMergerV9(indexMerger) - .taskReportFileWriter(new NoopTestTaskReportFileWriter()) - .build(); } @Test public void testFetchSnapshot() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( ClusterByStatisticsSnapshot.empty(), chatHandler.httpFetchKeyStatistics(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), null, req) @@ -100,7 +81,7 @@ public void testFetchSnapshot() @Test public void testFetchSnapshot404() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), chatHandler.httpFetchKeyStatistics("123", 2, null, req) @@ -111,7 +92,7 @@ public void testFetchSnapshot404() @Test public void testFetchSnapshotWithTimeChunk() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( ClusterByStatisticsSnapshot.empty(), chatHandler.httpFetchKeyStatisticsWithSnapshot(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), 1, null, req) @@ -122,7 +103,7 @@ public void testFetchSnapshotWithTimeChunk() @Test public void testFetchSnapshotWithTimeChunk404() { - WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker); + WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), chatHandler.httpFetchKeyStatisticsWithSnapshot("123", 2, 1, null, req) @@ -133,7 +114,6 @@ public void testFetchSnapshotWithTimeChunk404() private static class TestWorker implements Worker { - @Override public String id() { @@ -141,25 +121,25 @@ public String id() } @Override - public MSQWorkerTask task() + public void run() { - return new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0); + } @Override - public TaskStatus run() + public void stop() { - return null; + } @Override - public void stopGracefully() + public void controllerFailed() { } @Override - public void controllerFailed() + public void awaitStop() { } @@ -192,9 +172,8 @@ public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId s @Override public boolean postResultPartitionBoundaries( - ClusterByPartitions stagePartitionBoundaries, - String queryId, - int stageNumber + StageId stageId, + ClusterByPartitions stagePartitionBoundaries ) { return false; @@ -202,7 +181,7 @@ public boolean postResultPartitionBoundaries( @Nullable @Override - public InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) + public ListenableFuture readStageOutput(StageId stageId, int partitionNumber, long offset) { return null; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index c33faa40c14e..cffc0f78a497 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,7 +74,7 @@ public void testFaultSerde() throws IOException )); assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2)); assertFaultSerde(new InvalidFieldFault("the source", "the column", 1, "the error", "the log msg")); - assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2)); + assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2, 2)); assertFaultSerde(QueryNotSupportedFault.INSTANCE); assertFaultSerde(new QueryRuntimeFault("new error", "base error")); assertFaultSerde(new QueryRuntimeFault("new error", null)); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java new file mode 100644 index 000000000000..bc349d56c8fd --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ByteChunksInputStreamTest.java @@ -0,0 +1,149 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.collect.ImmutableList; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; + +public class ByteChunksInputStreamTest +{ + private final List chunks = ImmutableList.of( + new byte[]{-128, -127, -1, 0, 1, 126, 127}, + new byte[]{0}, + new byte[]{3, 4, 5} + ); + + @Test + public void test_read_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + int c; + while ((c = in.read()) != -1) { + MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0)); + baos.write(c); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + int c; + while ((c = in.read()) != -1) { + MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0)); + baos.write(c); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + @Test + public void test_read_array1_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[2]; + + int r; + while ((r = in.read(buf, 1, 1)) != -1) { + Assert.assertEquals("InputStream#read bytes read", 1, r); + baos.write(buf, 1, 1); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_array1_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[2]; + + int r; + while ((r = in.read(buf, 1, 1)) != -1) { + Assert.assertEquals("InputStream#read bytes read", 1, r); + baos.write(buf, 1, 1); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + @Test + public void test_read_array3_fromStart() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 0)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[5]; + + int r; + while ((r = in.read(buf, 2, 3)) != -1) { + baos.write(buf, 2, r); + } + + Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray()); + } + } + + @Test + public void test_read_array3_fromSecondByte() throws IOException + { + try (final InputStream in = new ByteChunksInputStream(chunks, 1)) { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final byte[] buf = new byte[6]; + + int r; + while ((r = in.read(buf, 2, 3)) != -1) { + baos.write(buf, 2, r); + } + + Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray()); + } + } + + private byte[] chunksSubset(final int positionInFirstChunk) + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + for (int chunk = 0, p = positionInFirstChunk; chunk < chunks.size(); chunk++, p = 0) { + baos.write(chunks.get(chunk), p, chunks.get(chunk).length - p); + } + + return baos.toByteArray(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java new file mode 100644 index 000000000000..927372a3a6ae --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -0,0 +1,255 @@ +/* + * 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.msq.shuffle.output; + +import com.google.common.io.ByteStreams; +import com.google.common.math.IntMath; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.math.RoundingMode; +import java.util.List; + +public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest +{ + private static final int MAX_FRAMES = 10; + private static final int EXPECTED_NUM_ROWS = 1209; + + private final BlockingQueueFrameChannel channel = new BlockingQueueFrameChannel(MAX_FRAMES); + private final ChannelStageOutputReader reader = new ChannelStageOutputReader(channel.readable()); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private FrameReader frameReader; + private List frameList; + + @Before + public void setUp() + { + final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + frameReader = FrameReader.create(adapter.getRowSignature()); + frameList = FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) + .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) + .frames() + .toList(); + } + + @After + public void tearDown() + { + reader.close(); + } + + @Test + public void test_readLocally() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + reader.close(); // Won't close the channel, because it's already been returned by readLocally + + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readLocally_closePriorToRead() throws IOException + { + writeAllFramesToChannel(); + + reader.close(); + + // Can't read the channel after closing the reader + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readLocally_thenReadRemotely() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + + // Can't read remotely after reading locally + Assert.assertThrows( + IllegalStateException.class, + () -> reader.readRemotelyFrom(0) + ); + + // Can still read locally after this error + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideBasedOnReturnedChunk() throws IOException + { + // Test that reads entire chunks from readRemotelyFrom. This is a typical usage pattern. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final long bytesWritten = ByteStreams.copy(in, tmpOut); + offset += bytesWritten; + + if (bytesWritten == 0) { + break; + } + } + } + + MatcherAssert.assertThat(numReads, Matchers.greaterThan(1)); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideOneByte() throws IOException + { + // Test that reads one byte at a time from readRemotelyFrom. This helps ensure that there are no edge cases + // in the chunk-reading logic. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final int nextByte = in.read(); + + if (nextByte < 0) { + break; + } + + tmpOut.write(nextByte); + offset++; + } + } + + Assert.assertEquals(numReads, offset + 1); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_thenLocally() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely + FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true); + + // Then read locally + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readRemotely_cannotReverse() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely from offset = 1. + final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(1), true); + final int offset = ByteStreams.toByteArray(in).length; + MatcherAssert.assertThat(offset, Matchers.greaterThan(0)); + + // Then read again from offset = 0; should get an error. + final RuntimeException e = Assert.assertThrows( + RuntimeException.class, + () -> FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true) + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause( + Matchers.allOf( + CoreMatchers.instanceOf(IllegalStateException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Offset[0] no longer available")) + ) + ) + ); + } + + private void writeAllFramesToChannel() throws IOException + { + for (Frame frame : frameList) { + channel.writable().write(frame); + } + channel.writable().close(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java index 03c017b7442d..96ef0ac6b1f1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java @@ -71,14 +71,6 @@ public void sanityTest() throws JsonProcessingException { Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(SQL_STATEMENT_RESULT)); - Assert.assertEquals( - SQL_STATEMENT_RESULT, - MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class) - ); - Assert.assertEquals( - SQL_STATEMENT_RESULT.hashCode(), - MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class).hashCode() - ); Assert.assertEquals( "SqlStatementResult{" + "queryId='q1'," @@ -87,7 +79,10 @@ public void sanityTest() throws JsonProcessingException + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + " durationInMs=100," + " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=null, dataSource='ds', pages=[PageInformation{id=0, numRows=null, sizeInBytes=1, worker=null, partition=null}]}," - + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", + + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}," + + " stages=null," + + " counters=null," + + " warnings=null}", SQL_STATEMENT_RESULT.toString() ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java index 1b0483d0b5a3..cef3e00daa2d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java @@ -206,7 +206,7 @@ public void emptyInsert() new ResultSetInformation(0L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -236,7 +236,7 @@ public void emptyReplace() new ResultSetInformation(0L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -282,7 +282,7 @@ protected DruidException makeException(DruidException.DruidExceptionBuilder bob) } }).toErrorResponse() ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); } @Test @@ -687,11 +687,11 @@ public void testInsert() new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); - Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest()); + Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus()); - Assert.assertEquals(expected, getResponse.getEntity()); + assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity()); Response resultsResponse = resource.doGetResults( actual.getQueryId(), @@ -730,11 +730,11 @@ public void testReplaceAll() new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null), null ); - Assert.assertEquals(expected, actual); + assertSqlStatementResult(expected, actual); - Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest()); + Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus()); - Assert.assertEquals(expected, getResponse.getEntity()); + assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity()); Response resultsResponse = resource.doGetResults( actual.getQueryId(), @@ -754,4 +754,27 @@ private static Map defaultAsyncContext() return context; } + private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual) + { + Assert.assertEquals(expected.getQueryId(), actual.getQueryId()); + Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt()); + Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature()); + Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs()); + Assert.assertEquals(expected.getStages(), actual.getStages()); + Assert.assertEquals(expected.getState(), actual.getState()); + Assert.assertEquals(expected.getWarnings(), actual.getWarnings()); + Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation()); + + if (actual.getCounters() == null || expected.getCounters() == null) { + Assert.assertEquals(expected.getCounters(), actual.getCounters()); + } else { + Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString()); + } + + if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) { + Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse()); + } else { + Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap()); + } + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index a97ee01297fc..4ea2993050ed 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -97,6 +97,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -449,7 +450,11 @@ private void setupMocks(OverlordClient indexingServiceClient) ))); - Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) + .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) .thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get()))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) @@ -584,6 +589,10 @@ private void setupMocks(OverlordClient indexingServiceClient) Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT))); Mockito.when(indexingServiceClient.taskPayload(FINISHED_INSERT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -690,9 +699,9 @@ public void init() throws Exception @Test public void testMSQSelectAcceptedQuery() { - Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( ACCEPTED_SELECT_MSQ_QUERY, SqlStatementState.ACCEPTED, @@ -702,7 +711,7 @@ public void testMSQSelectAcceptedQuery() null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -724,9 +733,9 @@ public void testMSQSelectAcceptedQuery() public void testMSQSelectRunningQuery() { - Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( RUNNING_SELECT_MSQ_QUERY, SqlStatementState.RUNNING, @@ -736,7 +745,7 @@ public void testMSQSelectRunningQuery() null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -754,10 +763,40 @@ public void testMSQSelectRunningQuery() ); } + @Test + public void testMSQSelectRunningQueryWithDetail() + { + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, true, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + SqlStatementResult expectedSqlStatementResult = new SqlStatementResult( + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null, + selectTaskReport.get().getPayload().getStages(), + selectTaskReport.get().getPayload().getCounters(), + new ArrayList<>(selectTaskReport.get().getPayload().getStatus().getWarningReports()) + ); + + assertSqlStatementResult( + expectedSqlStatementResult, + (SqlStatementResult) response.getEntity() + ); + + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + @Test public void testFinishedSelectMSQQuery() throws Exception { - Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); Assert.assertEquals(objectMapper.writeValueAsString(new SqlStatementResult( FINISHED_SELECT_MSQ_QUERY, @@ -825,7 +864,7 @@ private void assertExpectedResults(String expectedResult, Response resultsRespon public void testFailedMSQQuery() { for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { - assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); + assertExceptionMessage(resource.doGetStatus(queryID, false, makeOkRequest()), FAILURE_MSG, Response.Status.OK); assertExceptionMessage( resource.doGetResults(queryID, 0L, null, makeOkRequest()), StringUtils.format( @@ -845,9 +884,9 @@ public void testFailedMSQQuery() @Test public void testFinishedInsertMSQQuery() { - Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( + assertSqlStatementResult(new SqlStatementResult( FINISHED_INSERT_MSQ_QUERY, SqlStatementState.SUCCESS, CREATED_TIME, @@ -855,7 +894,7 @@ public void testFinishedInsertMSQQuery() 100L, new ResultSetInformation(null, null, null, "test", null, null), null - ), response.getEntity()); + ), (SqlStatementResult) response.getEntity()); Assert.assertEquals( Response.Status.OK.getStatusCode(), @@ -876,7 +915,7 @@ public void testFinishedInsertMSQQuery() public void testNonMSQTasks() { for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { - assertNotFound(resource.doGetStatus(queryID, makeOkRequest()), queryID); + assertNotFound(resource.doGetStatus(queryID, false, makeOkRequest()), queryID); assertNotFound(resource.doGetResults(queryID, 0L, null, makeOkRequest()), queryID); assertNotFound(resource.deleteQuery(queryID, makeOkRequest()), queryID); } @@ -885,9 +924,9 @@ public void testNonMSQTasks() @Test public void testMSQInsertAcceptedQuery() { - Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()); + Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( ACCEPTED_INSERT_MSQ_TASK, SqlStatementState.ACCEPTED, @@ -897,7 +936,7 @@ public void testMSQInsertAcceptedQuery() null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -918,9 +957,9 @@ public void testMSQInsertAcceptedQuery() @Test public void testMSQInsertRunningQuery() { - Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()); + Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, false, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals( + assertSqlStatementResult( new SqlStatementResult( RUNNING_INSERT_MSQ_QUERY, SqlStatementState.RUNNING, @@ -930,7 +969,7 @@ public void testMSQInsertRunningQuery() null, null ), - response.getEntity() + (SqlStatementResult) response.getEntity() ); assertExceptionMessage( @@ -955,6 +994,7 @@ public void testAPIBehaviourWithSuperUsers() Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(makeAuthResultForUser(SUPERUSER)) ).getStatus() ); @@ -984,6 +1024,7 @@ public void testAPIBehaviourWithDifferentUserAndNoStatePermission() Response.Status.FORBIDDEN.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1013,6 +1054,7 @@ public void testAPIBehaviourWithDifferentUserAndStateRPermission() Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1042,6 +1084,7 @@ public void testAPIBehaviourWithDifferentUserAndStateWPermission() Response.Status.FORBIDDEN.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1071,6 +1114,7 @@ public void testAPIBehaviourWithDifferentUserAndStateRWPermission() Response.Status.OK.getStatusCode(), resource.doGetStatus( RUNNING_SELECT_MSQ_QUERY, + false, makeExpectedReq(differentUserAuthResult) ).getStatus() ); @@ -1107,7 +1151,7 @@ public void testTaskIdNotFound() Assert.assertEquals( Response.Status.NOT_FOUND.getStatusCode(), - resource.doGetStatus(taskIdNotFound, makeOkRequest()).getStatus() + resource.doGetStatus(taskIdNotFound, false, makeOkRequest()).getStatus() ); Assert.assertEquals( Response.Status.NOT_FOUND.getStatusCode(), @@ -1124,4 +1168,28 @@ public void testIsEnabled() { Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.isEnabled(makeOkRequest()).getStatus()); } + + private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual) + { + Assert.assertEquals(expected.getQueryId(), actual.getQueryId()); + Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt()); + Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature()); + Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs()); + Assert.assertEquals(expected.getStages(), actual.getStages()); + Assert.assertEquals(expected.getState(), actual.getState()); + Assert.assertEquals(expected.getWarnings(), actual.getWarnings()); + Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation()); + + if (actual.getCounters() == null || expected.getCounters() == null) { + Assert.assertEquals(expected.getCounters(), actual.getCounters()); + } else { + Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString()); + } + + if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) { + Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse()); + } else { + Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap()); + } + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java index b60c6c71d2e2..124b4fce2588 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java @@ -64,15 +64,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java index 2d8067e900e9..5d4c0994ea06 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java @@ -67,15 +67,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java index 317fe30a646d..6bbf9c6da5e4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java @@ -136,15 +136,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java index 3008f9d43b47..2de9229b4adc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java @@ -73,15 +73,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java index b5d8368b068f..e4b678402a8b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java @@ -79,15 +79,7 @@ public SqlEngine createEngine( Injector injector ) { - final WorkerMemoryParameters workerMemoryParameters = - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 0, - 0 - ); + final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters(); final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient( queryJsonMapper, injector, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 5f0bd545b7c6..5076e9f3d441 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -48,6 +48,7 @@ import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.ExpressionModule; @@ -55,7 +56,6 @@ import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.JoinableFactoryModule; import org.apache.druid.guice.JsonConfigProvider; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.guice.annotations.EscalatedGlobal; @@ -186,6 +186,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.calcite.view.InProcessViewManager; import org.apache.druid.sql.guice.SqlBindings; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.storage.StorageConfig; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; @@ -333,16 +334,7 @@ public class MSQTestBase extends BaseCalciteQueryTest private SegmentCacheManager segmentCacheManager; private TestGroupByBuffers groupByBuffers; - protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy( - WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, - 2, - 1, - 0 - ) - ); + protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(makeTestWorkerMemoryParameters()); protected static class MSQBaseComponentSupplier extends StandardComponentSupplier { @@ -366,8 +358,8 @@ public void configure(Binder binder) { // We want this module to bring InputSourceModule along for the ride. binder.install(new InputSourceModule()); - binder.install(new NestedDataModule()); - NestedDataModule.registerHandlersAndSerde(); + binder.install(new BuiltInTypesModule()); + BuiltInTypesModule.registerHandlersAndSerde(); SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class); SqlBindings.addOperatorConversion(binder, HttpOperatorConversion.class); SqlBindings.addOperatorConversion(binder, InlineOperatorConversion.class); @@ -530,7 +522,7 @@ public String getFormatString() objectMapper = setupObjectMapper(injector); objectMapper.registerModules(new StorageConnectorModule().getJacksonModules()); objectMapper.registerModules(sqlModule.getJacksonModules()); - objectMapper.registerModules(NestedDataModule.getJacksonModulesList()); + objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); @@ -571,7 +563,8 @@ public String getFormatString() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), catalogResolver, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory); @@ -753,6 +746,19 @@ public static ObjectMapper setupObjectMapper(Injector injector) return mapper; } + public static WorkerMemoryParameters makeTestWorkerMemoryParameters() + { + return WorkerMemoryParameters.createInstance( + WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, + 2, + 10, + 1, + 2, + 1, + 0 + ); + } + private String runMultiStageQuery(String query, Map context) { final DirectStatement stmt = sqlStatementFactory.directStatement( @@ -900,7 +906,7 @@ public Builder setExpectedRowSignature(RowSignature expectedRowSignature) return asBuilder(); } - public Builder setExpectedSegment(Set expectedSegments) + public Builder setExpectedSegments(Set expectedSegments) { Preconditions.checkArgument(expectedSegments != null, "Segments cannot be null"); this.expectedSegments = expectedSegments; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java index 96e26cba77e1..4c7ca61be023 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java @@ -87,9 +87,9 @@ public void postWorkerWarning(List MSQErrorReports) } @Override - public List getTaskList() + public List getWorkerIds() { - return controller.getTaskIds(); + return controller.getWorkerIds(); } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 20d31fbd4cfe..e65104302032 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -156,32 +156,33 @@ public ListenableFuture runTask(String taskId, Object taskObject) Worker worker = new WorkerImpl( task, new MSQTestWorkerContext( + task.getId(), inMemoryWorkers, controller, mapper, injector, - workerMemoryParameters - ), - workerStorageParameters + workerMemoryParameters, + workerStorageParameters + ) ); inMemoryWorkers.put(task.getId(), worker); statusMap.put(task.getId(), TaskStatus.running(task.getId())); - ListenableFuture future = executor.submit(() -> { + ListenableFuture future = executor.submit(() -> { try { - return worker.run(); + worker.run(); } catch (Exception e) { throw new RuntimeException(e); } }); - Futures.addCallback(future, new FutureCallback() + Futures.addCallback(future, new FutureCallback() { @Override - public void onSuccess(@Nullable TaskStatus result) + public void onSuccess(@Nullable Object result) { - statusMap.put(task.getId(), result); + statusMap.put(task.getId(), TaskStatus.success(task.getId())); } @Override @@ -261,7 +262,7 @@ public ListenableFuture cancelTask(String workerId) { final Worker worker = inMemoryWorkers.remove(workerId); if (worker != null) { - worker.stopGracefully(); + worker.stop(); } return Futures.immediateFuture(null); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java index 72cb246a43e1..65145b5f5c01 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerClient.java @@ -80,11 +80,7 @@ public ListenableFuture postResultPartitionBoundaries( ) { try { - inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries( - partitionBoundaries, - stageId.getQueryId(), - stageId.getStageNumber() - ); + inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries(stageId, partitionBoundaries); return Futures.immediateFuture(null); } catch (Exception e) { @@ -122,8 +118,7 @@ public ListenableFuture fetchChannelData( ) { try (InputStream inputStream = - inMemoryWorkers.get(workerTaskId) - .readChannel(stageId.getQueryId(), stageId.getStageNumber(), partitionNumber, offset)) { + inMemoryWorkers.get(workerTaskId).readStageOutput(stageId, partitionNumber, offset).get()) { byte[] buffer = new byte[8 * 1024]; boolean didRead = false; int bytesRead; @@ -138,12 +133,11 @@ public ListenableFuture fetchChannelData( catch (Exception e) { throw new ISE(e, "Error reading frame file channel"); } - } @Override public void close() { - inMemoryWorkers.forEach((k, v) -> v.stopGracefully()); + inMemoryWorkers.forEach((k, v) -> v.stop()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 14f6f73b24ab..082429a9d7b1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -22,59 +22,69 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.indexer.report.TaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.OutputChannelMode; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; import org.apache.druid.msq.exec.WorkerMemoryParameters; -import org.apache.druid.msq.indexing.IndexerFrameContext; -import org.apache.druid.msq.indexing.IndexerWorkerContext; +import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.NoopRowIngestionMeters; +import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.security.AuthTestUtils; import java.io.File; import java.util.Map; public class MSQTestWorkerContext implements WorkerContext { + private final String workerId; private final Controller controller; private final ObjectMapper mapper; private final Injector injector; private final Map inMemoryWorkers; private final File file = FileUtils.createTempDir(); + private final Bouncer bouncer = new Bouncer(1); private final WorkerMemoryParameters workerMemoryParameters; + private final WorkerStorageParameters workerStorageParameters; public MSQTestWorkerContext( + String workerId, Map inMemoryWorkers, Controller controller, ObjectMapper mapper, Injector injector, - WorkerMemoryParameters workerMemoryParameters + WorkerMemoryParameters workerMemoryParameters, + WorkerStorageParameters workerStorageParameters ) { + this.workerId = workerId; this.inMemoryWorkers = inMemoryWorkers; this.controller = controller; this.mapper = mapper; this.injector = injector; this.workerMemoryParameters = workerMemoryParameters; + this.workerStorageParameters = workerStorageParameters; + } + + @Override + public String queryId() + { + return controller.queryId(); } @Override @@ -96,7 +106,13 @@ public void registerWorker(Worker worker, Closer closer) } @Override - public ControllerClient makeControllerClient(String controllerId) + public String workerId() + { + return workerId; + } + + @Override + public ControllerClient makeControllerClient() { return new MSQTestControllerClient(controller); } @@ -114,42 +130,9 @@ public File tempDir() } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) + public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) { - IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - mapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true - ); - final TaskReportFileWriter reportFileWriter = new NoopTestTaskReportFileWriter(); - - return new IndexerFrameContext( - new IndexerWorkerContext( - new TaskToolbox.Builder() - .segmentPusher(injector.getInstance(DataSegmentPusher.class)) - .segmentAnnouncer(injector.getInstance(DataSegmentAnnouncer.class)) - .jsonMapper(mapper) - .taskWorkDir(tempDir()) - .indexIO(indexIO) - .indexMergerV9(indexMerger) - .taskReportFileWriter(reportFileWriter) - .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) - .chatHandlerProvider(new NoopChatHandlerProvider()) - .rowIngestionMetersFactory(NoopRowIngestionMeters::new) - .build(), - injector, - indexIO, - null, - null, - null - ), - indexIO, - injector.getInstance(DataSegmentProvider.class), - injector.getInstance(DataServerQueryHandlerFactory.class), - workerMemoryParameters - ); + return new FrameContextImpl(new File(tempDir(), queryDef.getStageDefinition(stageNumber).getId().toString())); } @Override @@ -165,9 +148,9 @@ public DruidNode selfNode() } @Override - public Bouncer processorBouncer() + public int maxConcurrentStages() { - return injector.getInstance(Bouncer.class); + return 1; } @Override @@ -175,4 +158,109 @@ public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() { return injector.getInstance(DataServerQueryHandlerFactory.class); } + + class FrameContextImpl implements FrameContext + { + private final File tempDir; + + public FrameContextImpl(File tempDir) + { + this.tempDir = tempDir; + } + + @Override + public SegmentWrangler segmentWrangler() + { + return injector.getInstance(SegmentWrangler.class); + } + + @Override + public GroupingEngine groupingEngine() + { + return injector.getInstance(GroupingEngine.class); + } + + @Override + public RowIngestionMeters rowIngestionMeters() + { + return new NoopRowIngestionMeters(); + } + + @Override + public DataSegmentProvider dataSegmentProvider() + { + return injector.getInstance(DataSegmentProvider.class); + } + + @Override + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() + { + return injector.getInstance(DataServerQueryHandlerFactory.class); + } + + @Override + public File tempDir() + { + return new File(tempDir, "tmp"); + } + + @Override + public ObjectMapper jsonMapper() + { + return mapper; + } + + @Override + public IndexIO indexIO() + { + return new IndexIO(mapper, ColumnConfig.DEFAULT); + } + + @Override + public File persistDir() + { + return new File(tempDir, "persist"); + } + + @Override + public DataSegmentPusher segmentPusher() + { + return injector.getInstance(DataSegmentPusher.class); + } + + @Override + public IndexMergerV9 indexMerger() + { + return new IndexMergerV9( + mapper, + indexIO(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + true + ); + } + + @Override + public Bouncer processorBouncer() + { + return bouncer; + } + + @Override + public WorkerMemoryParameters memoryParameters() + { + return workerMemoryParameters; + } + + @Override + public WorkerStorageParameters storageParameters() + { + return workerStorageParameters; + } + + @Override + public void close() + { + + } + } } diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index 1b3a531dc421..048ad964bf8e 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -123,7 +123,11 @@ junit-vintage-engine test - + + org.reflections + reflections + test + org.apache.druid druid-processing diff --git a/extensions-core/testing-tools/pom.xml b/extensions-core/testing-tools/pom.xml index 086d32559d35..fd0eed880c28 100644 --- a/extensions-core/testing-tools/pom.xml +++ b/extensions-core/testing-tools/pom.xml @@ -145,6 +145,11 @@ junit-jupiter-params test + + org.reflections + reflections + test + org.junit.vintage junit-vintage-engine diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 8d30a60d04e6..0abaeed8eb27 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -57,6 +57,9 @@ TaskStatus runCompactionTasks( * Checks if the provided compaction config is supported by the runner. * The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask} */ - CompactionConfigValidationResult validateCompactionTask(CompactionTask compactionTask); + CompactionConfigValidationResult validateCompactionTask( + CompactionTask compactionTask, + Map intervalToDataSchemaMap + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 8659eb0f397e..683203878451 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -77,6 +77,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -459,11 +460,13 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception transformSpec, metricsSpec, granularitySpec, - getMetricBuilder() + getMetricBuilder(), + compactionRunner ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); + CompactionConfigValidationResult supportsCompactionConfig = + compactionRunner.validateCompactionTask(this, intervalDataSchemas); if (!supportsCompactionConfig.isValid()) { throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } @@ -485,7 +488,8 @@ static Map createDataSchemasForIntervals( @Nullable final ClientCompactionTaskTransformSpec transformSpec, @Nullable final AggregatorFactory[] metricsSpec, @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, - final ServiceMetricEvent.Builder metricBuilder + final ServiceMetricEvent.Builder metricBuilder, + CompactionRunner compactionRunner ) throws IOException { final Iterable timelineSegments = retrieveRelevantTimelineHolders( @@ -549,7 +553,8 @@ static Map createDataSchemasForIntervals( metricsSpec, granularitySpec == null ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) - : granularitySpec.withSegmentGranularity(segmentGranularityToUse) + : granularitySpec.withSegmentGranularity(segmentGranularityToUse), + compactionRunner ); intervalDataSchemaMap.put(interval, dataSchema); } @@ -574,7 +579,8 @@ static Map createDataSchemasForIntervals( dimensionsSpec, transformSpec, metricsSpec, - granularitySpec + granularitySpec, + compactionRunner ); return Collections.singletonMap(segmentProvider.interval, dataSchema); } @@ -604,13 +610,17 @@ private static DataSchema createDataSchema( @Nullable DimensionsSpec dimensionsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nonnull ClientCompactionTaskGranularitySpec granularitySpec + @Nonnull ClientCompactionTaskGranularitySpec granularitySpec, + @Nullable CompactionRunner compactionRunner ) { // Check index metadata & decide which values to propagate (i.e. carry over) for rollup & queryGranularity final ExistingSegmentAnalyzer existingSegmentAnalyzer = new ExistingSegmentAnalyzer( segments, - granularitySpec.isRollup() == null, + // For MSQ, always need rollup to check if there are some rollup segments already present. + compactionRunner instanceof NativeCompactionRunner + ? (granularitySpec.isRollup() == null) + : true, granularitySpec.getQueryGranularity() == null, dimensionsSpec == null, metricsSpec == null @@ -665,13 +675,14 @@ private static DataSchema createDataSchema( finalMetricsSpec = metricsSpec; } - return new DataSchema( + return new CombinedDataSchema( dataSource, new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), finalDimensionsSpec, finalMetricsSpec, uniformGranularitySpec, - transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null) + transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null), + existingSegmentAnalyzer.hasRolledUpSegments() ); } @@ -748,6 +759,7 @@ static class ExistingSegmentAnalyzer // For processRollup: private boolean rollup = true; + private boolean hasRolledUpSegments = false; // For processQueryGranularity: private Granularity queryGranularity; @@ -815,6 +827,11 @@ public Boolean getRollup() return rollup; } + public boolean hasRolledUpSegments() + { + return hasRolledUpSegments; + } + public Granularity getQueryGranularity() { if (!needQueryGranularity) { @@ -904,6 +921,7 @@ private void processRollup(final QueryableIndex index) // Pick rollup value if all segments being compacted have the same, non-null, value otherwise set it to false final Boolean isIndexRollup = index.getMetadata().isRollup(); rollup = rollup && Boolean.valueOf(true).equals(isIndexRollup); + hasRolledUpSegments = hasRolledUpSegments || Boolean.valueOf(true).equals(isIndexRollup); } private void processQueryGranularity(final QueryableIndex index) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 30761b674e54..5aa7af714512 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -85,10 +85,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask + CompactionTask compactionTask, + Map intervalToDataSchemaMap ) { - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 2155ac2c2655..bebb52157d6f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; 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.UOE; @@ -49,6 +50,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -992,50 +994,76 @@ public Map> getLockedIntervals(List loc } /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * Here, Segment Locks are being treated the same as Time Chunk Locks i.e. - * a Task with a Segment Lock is assumed to lock a whole Interval and not just - * the corresponding Segment. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the - * Intervals that are locked by Tasks with equal or - * higher priority than this are returned. Locked intervals - * for datasources that are not present in this Map are - * not returned. - * @return Map from Datasource to List of Intervals locked by Tasks that have - * priority greater than or equal to the {@code minTaskPriority} for that datasource. + * @param lockFilterPolicies Lock filters for the given datasources + * @return Map from datasource to list of non-revoked locks with at least as much priority and an overlapping interval */ - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getActiveLocks(List lockFilterPolicies) { - final Map> datasourceToIntervals = new HashMap<>(); + final Map> datasourceToLocks = new HashMap<>(); // Take a lock and populate the maps giant.lock(); + try { - running.forEach( - (datasource, datasourceLocks) -> { - // If this datasource is not requested, do not proceed - if (!minTaskPriority.containsKey(datasource)) { + lockFilterPolicies.forEach( + lockFilter -> { + final String datasource = lockFilter.getDatasource(); + if (!running.containsKey(datasource)) { return; } - datasourceLocks.forEach( + final int priority = lockFilter.getPriority(); + final List intervals; + if (lockFilter.getIntervals() != null) { + intervals = lockFilter.getIntervals(); + } else { + intervals = Collections.singletonList(Intervals.ETERNITY); + } + + final Map context = lockFilter.getContext(); + final boolean ignoreAppendLocks; + final Boolean useConcurrentLocks = QueryContexts.getAsBoolean( + Tasks.USE_CONCURRENT_LOCKS, + context.get(Tasks.USE_CONCURRENT_LOCKS) + ); + if (useConcurrentLocks == null) { + TaskLockType taskLockType = QueryContexts.getAsEnum( + Tasks.TASK_LOCK_TYPE, + context.get(Tasks.TASK_LOCK_TYPE), + TaskLockType.class + ); + if (taskLockType == null) { + ignoreAppendLocks = Tasks.DEFAULT_USE_CONCURRENT_LOCKS; + } else { + ignoreAppendLocks = taskLockType == TaskLockType.APPEND; + } + } else { + ignoreAppendLocks = useConcurrentLocks; + } + + running.get(datasource).forEach( (startTime, startTimeLocks) -> startTimeLocks.forEach( (interval, taskLockPosses) -> taskLockPosses.forEach( taskLockPosse -> { if (taskLockPosse.getTaskLock().isRevoked()) { - // Do not proceed if the lock is revoked - return; + // do nothing } else if (taskLockPosse.getTaskLock().getPriority() == null - || taskLockPosse.getTaskLock().getPriority() < minTaskPriority.get(datasource)) { - // Do not proceed if the lock has a priority strictly less than the minimum - return; + || taskLockPosse.getTaskLock().getPriority() < priority) { + // do nothing + } else if (ignoreAppendLocks + && taskLockPosse.getTaskLock().getType() == TaskLockType.APPEND) { + // do nothing + } else { + for (Interval filterInterval : intervals) { + if (interval.overlaps(filterInterval)) { + datasourceToLocks.computeIfAbsent(datasource, ds -> new ArrayList<>()) + .add(taskLockPosse.getTaskLock()); + break; + } + } } - - datasourceToIntervals - .computeIfAbsent(datasource, k -> new HashSet<>()) - .add(interval); - }) + } + ) ) ); } @@ -1045,11 +1073,7 @@ public Map> getLockedIntervals(Map minTa giant.unlock(); } - return datasourceToIntervals.entrySet().stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - entry -> new ArrayList<>(entry.getValue()) - )); + return datasourceToLocks; } public void unlock(final Task task, final Interval interval) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index f5351d7c6e51..b25bde067c7f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -27,6 +27,7 @@ import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.autoscaling.ProvisioningStrategy; import org.apache.druid.indexing.overlord.http.TaskStateLookup; @@ -94,19 +95,12 @@ public Map> getLockedIntervals(List loc } /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the - * Intervals that are locked by Tasks with equal or - * higher priority than this are returned. Locked intervals - * for datasources that are not present in this Map are - * not returned. - * @return Map from Datasource to List of Intervals locked by Tasks that have - * priority greater than or equal to the {@code minTaskPriority} for that datasource. + * @param lockFilterPolicies Requests for active locks for various datasources + * @return Map from datasource to conflicting lock infos */ - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getActiveLocks(List lockFilterPolicies) { - return taskLockbox.getLockedIntervals(minTaskPriority); + return taskLockbox.getActiveLocks(lockFilterPolicies); } public List> getActiveTaskInfo(@Nullable String dataSource) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 54ada7cb2b43..b62e1de055f2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -241,33 +241,32 @@ public Response isLeader() } } - @Deprecated @POST - @Path("/lockedIntervals") + @Path("/lockedIntervals/v2") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response getDatasourceLockedIntervals(Map minTaskPriority) + public Response getDatasourceLockedIntervals(List lockFilterPolicies) { - if (minTaskPriority == null || minTaskPriority.isEmpty()) { - return Response.status(Status.BAD_REQUEST).entity("No Datasource provided").build(); + if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { + return Response.status(Status.BAD_REQUEST).entity("No filter provided").build(); } // Build the response - return Response.ok(taskQueryTool.getLockedIntervals(minTaskPriority)).build(); + return Response.ok(taskQueryTool.getLockedIntervals(lockFilterPolicies)).build(); } @POST - @Path("/lockedIntervals/v2") + @Path("/activeLocks") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response getDatasourceLockedIntervalsV2(List lockFilterPolicies) + public Response getActiveLocks(List lockFilterPolicies) { if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { return Response.status(Status.BAD_REQUEST).entity("No filter provided").build(); } // Build the response - return Response.ok(taskQueryTool.getLockedIntervals(lockFilterPolicies)).build(); + return Response.ok(new TaskLockResponse(taskQueryTool.getActiveLocks(lockFilterPolicies))).build(); } @GET diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java new file mode 100644 index 000000000000..df4a5d8b03c8 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.TaskLock; + +import java.util.List; +import java.util.Map; + +public class TaskLockResponse +{ + private final Map> datasourceToLocks; + + @JsonCreator + public TaskLockResponse( + @JsonProperty("datasourceToLocks") final Map> datasourceToLocks + ) + { + this.datasourceToLocks = datasourceToLocks; + } + + @JsonProperty + public Map> getDatasourceToLocks() + { + return datasourceToLocks; + } + + @Override + public String toString() + { + return "TaskLockResponse{" + + "datasourceToLocks='" + datasourceToLocks + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java index 729ac1d1617e..a1c131ad0f49 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java @@ -640,6 +640,22 @@ public Map getWorkerCompletedTasks() return getNumTasksPerDatasource(this.getCompletedTasks().values(), TaskAnnouncement::getTaskDataSource); } + @Override + public Map getWorkerFailedTasks() + { + return getNumTasksPerDatasource(completedTasks.entrySet().stream() + .filter(entry -> entry.getValue().getTaskStatus().isFailure()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)).values(), TaskAnnouncement::getTaskDataSource); + } + + @Override + public Map getWorkerSuccessfulTasks() + { + return getNumTasksPerDatasource(completedTasks.entrySet().stream() + .filter(entry -> entry.getValue().getTaskStatus().isSuccess()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)).values(), TaskAnnouncement::getTaskDataSource); + } + private static class TaskDetails { private final Task task; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index f9849b1483df..3a386bc4aa79 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -749,7 +749,8 @@ public void testCreateIngestionSchema() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -810,7 +811,8 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -872,7 +874,8 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -935,7 +938,8 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1005,7 +1009,8 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1055,7 +1060,8 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException null, customMetricsSpec, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1098,7 +1104,8 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1148,7 +1155,8 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); NativeCompactionRunner.createIngestionSpecs( @@ -1178,7 +1186,8 @@ public void testMissingMetadata() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); NativeCompactionRunner.createIngestionSpecs( @@ -1219,7 +1228,8 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException null, null, new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1263,7 +1273,8 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException null, null, new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, @@ -1308,7 +1319,8 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio new PeriodGranularity(Period.months(3), null, null), null ), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1355,7 +1367,8 @@ public void testNullGranularitySpec() throws IOException null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1400,7 +1413,8 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1445,7 +1459,8 @@ public void testGranularitySpecWithNotNullRollup() null, null, new ClientCompactionTaskGranularitySpec(null, null, true), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1475,7 +1490,8 @@ public void testGranularitySpecWithNullRollup() null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index a02b51087675..a8c4b5117b1b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -75,7 +75,6 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -1174,99 +1173,6 @@ public void testGetTimeChunkAndSegmentLockForDifferentGroup() ); } - @Test - public void testGetLockedIntervals() - { - // Acquire locks for task1 - final Task task1 = NoopTask.forDatasource("ds1"); - lockbox.add(task1); - - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task1, - Intervals.of("2017-01-01/2017-02-01") - ); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task1, - Intervals.of("2017-04-01/2017-05-01") - ); - - // Acquire locks for task2 - final Task task2 = NoopTask.forDatasource("ds2"); - lockbox.add(task2); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task2, - Intervals.of("2017-03-01/2017-04-01") - ); - - // Verify the locked intervals - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(task1.getDataSource(), 10); - minTaskPriority.put(task2.getDataSource(), 10); - final Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(2, lockedIntervals.size()); - - Assert.assertEquals( - Arrays.asList( - Intervals.of("2017-01-01/2017-02-01"), - Intervals.of("2017-04-01/2017-05-01") - ), - lockedIntervals.get(task1.getDataSource()) - ); - - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017-03-01/2017-04-01")), - lockedIntervals.get(task2.getDataSource()) - ); - } - - @Test - public void testGetLockedIntervalsForLowPriorityTask() - { - // Acquire lock for a low priority task - final Task lowPriorityTask = NoopTask.ofPriority(5); - lockbox.add(lowPriorityTask); - taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - lowPriorityTask, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(lowPriorityTask.getDataSource(), 10); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertTrue(lockedIntervals.isEmpty()); - } - - @Test - public void testGetLockedIntervalsForEqualPriorityTask() - { - // Acquire lock for a low priority task - final Task task = NoopTask.ofPriority(5); - lockbox.add(task); - taskStorage.insert(task, TaskStatus.running(task.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(task.getDataSource(), 5); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList(Intervals.of("2017/2018")), - lockedIntervals.get(task.getDataSource()) - ); - } - @Test public void testGetLockedIntervalsForHigherPriorityExclusiveLock() { @@ -1282,6 +1188,7 @@ public void testGetLockedIntervalsForHigherPriorityExclusiveLock() LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 75, + null, null ); @@ -1305,6 +1212,7 @@ public void testGetLockedIntervalsForLowerPriorityExclusiveLock() LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, null ); @@ -1332,6 +1240,7 @@ public void testGetLockedIntervalsForLowerPriorityReplaceLock() LockFilterPolicy requestForReplaceLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.REPLACE.name()) ); @@ -1355,6 +1264,7 @@ public void testGetLockedIntervalsForLowerPriorityUseConcurrentLocks() LockFilterPolicy requestForReplaceLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, ImmutableMap.of( Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name(), @@ -1369,6 +1279,171 @@ public void testGetLockedIntervalsForLowerPriorityUseConcurrentLocks() } + @Test + public void testGetActiveLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + final TaskLock overlappingAppendLock = + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + expectedLocks.add(overlappingAppendLock); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + null + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithAppendLockIgnoresAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithConcurrentLocksIgnoresAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true, Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithoutConcurrentLocksConsidersAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + final TaskLock overlappingAppendLock = + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + expectedLocks.add(overlappingAppendLock); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, false, Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + @Test public void testExclusiveLockCompatibility() { @@ -1770,50 +1845,6 @@ public void testTimechunkLockTypeTransitionForSameTaskGroup() validator.expectLockNotGranted(TaskLockType.APPEND, otherGroupTask, Intervals.of("2024/2025")); } - @Test - public void testGetLockedIntervalsForRevokedLocks() - { - // Acquire lock for a low priority task - final Task lowPriorityTask = NoopTask.ofPriority(5); - lockbox.add(lowPriorityTask); - taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - lowPriorityTask, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(lowPriorityTask.getDataSource(), 1); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017/2018")), - lockedIntervals.get(lowPriorityTask.getDataSource()) - ); - - // Revoke the lowPriorityTask - final Task highPriorityTask = NoopTask.ofPriority(10); - lockbox.add(highPriorityTask); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - highPriorityTask, - Intervals.of("2017-05-01/2017-06-01") - ); - - // Verify the locked intervals - minTaskPriority.put(highPriorityTask.getDataSource(), 1); - lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017-05-01/2017-06-01")), - lockedIntervals.get(highPriorityTask.getDataSource()) - ); - } - @Test public void testFailedToReacquireTaskLock() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index e6dee0c7e403..93f80ac97096 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -36,6 +36,9 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; @@ -61,6 +64,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; @@ -1057,31 +1061,33 @@ public void testGetTaskStatus() throws Exception @Test public void testGetLockedIntervals() throws Exception { - final Map minTaskPriority = Collections.singletonMap("ds1", 0); - final Map> expectedLockedIntervals = Collections.singletonMap( + final List lockFilterPolicies = ImmutableList.of( + new LockFilterPolicy("ds1", 25, null, null) + ); + final Map> expectedIntervals = Collections.singletonMap( "ds1", Arrays.asList( Intervals.of("2012-01-01/2012-01-02"), - Intervals.of("2012-01-02/2012-01-03") + Intervals.of("2012-01-01/2012-01-02") ) ); - EasyMock.expect(taskLockbox.getLockedIntervals(minTaskPriority)) - .andReturn(expectedLockedIntervals); + EasyMock.expect(taskLockbox.getLockedIntervals(lockFilterPolicies)) + .andReturn(expectedIntervals); replayAll(); - final Response response = overlordResource.getDatasourceLockedIntervals(minTaskPriority); + final Response response = overlordResource.getDatasourceLockedIntervals(lockFilterPolicies); Assert.assertEquals(200, response.getStatus()); final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - Map> observedLockedIntervals = jsonMapper.readValue( + Map> observedIntervals = jsonMapper.readValue( jsonMapper.writeValueAsString(response.getEntity()), new TypeReference>>() { } ); - Assert.assertEquals(expectedLockedIntervals, observedLockedIntervals); + Assert.assertEquals(expectedIntervals, observedIntervals); } @Test @@ -1092,7 +1098,65 @@ public void testGetLockedIntervalsWithEmptyBody() Response response = overlordResource.getDatasourceLockedIntervals(null); Assert.assertEquals(400, response.getStatus()); - response = overlordResource.getDatasourceLockedIntervals(Collections.emptyMap()); + response = overlordResource.getDatasourceLockedIntervals(Collections.emptyList()); + Assert.assertEquals(400, response.getStatus()); + } + + @Test + public void testGetActiveLocks() throws Exception + { + final List lockFilterPolicies = ImmutableList.of( + new LockFilterPolicy("ds1", 25, null, null) + ); + final Map> expectedLocks = Collections.singletonMap( + "ds1", + Arrays.asList( + new TimeChunkLock( + TaskLockType.REPLACE, + "groupId", + "datasource", + Intervals.of("2012-01-01/2012-01-02"), + "version", + 25 + ), + new TimeChunkLock( + TaskLockType.EXCLUSIVE, + "groupId", + "datasource", + Intervals.of("2012-01-02/2012-01-03"), + "version", + 75 + ) + ) + ); + + EasyMock.expect(taskLockbox.getActiveLocks(lockFilterPolicies)) + .andReturn(expectedLocks); + replayAll(); + + final Response response = overlordResource.getActiveLocks(lockFilterPolicies); + Assert.assertEquals(200, response.getStatus()); + + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + Map> observedLocks = jsonMapper.readValue( + jsonMapper.writeValueAsString(response.getEntity()), + new TypeReference() + { + } + ).getDatasourceToLocks(); + + Assert.assertEquals(expectedLocks, observedLocks); + } + + @Test + public void testGetActiveLocksWithEmptyBody() + { + replayAll(); + + Response response = overlordResource.getActiveLocks(null); + Assert.assertEquals(400, response.getStatus()); + + response = overlordResource.getActiveLocks(Collections.emptyList()); Assert.assertEquals(400, response.getStatus()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 6b08be3a3c65..37839f8e0777 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -455,6 +455,19 @@ private NoopTask createNoopTask(String id, String dataSource) return new NoopTask(id, null, dataSource, 100, 0, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)); } + private NoopTask createNoopFailingTask(String id, String dataSource) + { + return new NoopTask(id, null, dataSource, 100, 0, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)) + { + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + Thread.sleep(getRunTime()); + return TaskStatus.failure(getId(), "Failed to complete the task"); + } + }; + } + /** * Start the {@link #workerTaskManager}, submit a {@link NoopTask}, wait for it to be complete. Common preamble * for various tests of {@link WorkerTaskManager#doCompletedTasksCleanup()}. @@ -494,7 +507,7 @@ public void getWorkerTaskStatsTest() throws Exception Task task1 = createNoopTask("task1", "wikipedia"); Task task2 = createNoopTask("task2", "wikipedia"); - Task task3 = createNoopTask("task3", "animals"); + Task task3 = createNoopFailingTask("task3", "animals"); workerTaskManager.start(); // befor assigning tasks we should get no running tasks @@ -517,11 +530,19 @@ public void getWorkerTaskStatsTest() throws Exception Thread.sleep(10); } while (!runningTasks.isEmpty()); - // When running tasks are empty all task should be reported as completed + // When running tasks are empty all task should be reported as completed and + // one of the task for animals datasource should fail and other 2 tasks in + // the wikipedia datasource should succeed Assert.assertEquals(workerTaskManager.getWorkerCompletedTasks(), ImmutableMap.of( "wikipedia", 2L, "animals", 1L )); + Assert.assertEquals(workerTaskManager.getWorkerFailedTasks(), ImmutableMap.of( + "animals", 1L + )); + Assert.assertEquals(workerTaskManager.getWorkerSuccessfulTasks(), ImmutableMap.of( + "wikipedia", 2L + )); Assert.assertEquals(workerTaskManager.getWorkerAssignedTasks().size(), 0L); } } diff --git a/integration-tests-ex/cases/cluster/BackwardCompatibilityMain/docker-compose.yaml b/integration-tests-ex/cases/cluster/BackwardCompatibilityMain/docker-compose.yaml new file mode 100644 index 000000000000..4fbf0f711975 --- /dev/null +++ b/integration-tests-ex/cases/cluster/BackwardCompatibilityMain/docker-compose.yaml @@ -0,0 +1,111 @@ +# 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 + image: ${DRUID_PREVIOUS_IT_IMAGE_NAME} + 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 + image: ${DRUID_PREVIOUS_IT_IMAGE_NAME} + 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 + + middlemanager: + extends: + file: ../Common/druid.yaml + service: middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_msq_intermediate_storage_enable=true + - druid_msq_intermediate_storage_type=local + - druid_msq_intermediate_storage_basePath=/shared/durablestorage/ + - druid_export_storage_baseDir=/ + volumes: + # Test data + - ../../resources:/resources + depends_on: + - zookeeper + + kafka: + extends: + file: ../Common/dependencies.yaml + service: kafka + depends_on: + - zookeeper diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 40461dd8ef17..433608bbe240 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -459,6 +459,15 @@ GcsDeepStorage + + IT-BackwardCompatibilityMain + + false + + + BackwardCompatibilityMain + + docker-tests diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainIndexerTest.java new file mode 100644 index 000000000000..01be3f3dbeaa --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainIndexerTest.java @@ -0,0 +1,32 @@ +/* + * 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.BackwardCompatibilityMain; + +import org.apache.druid.testsEx.categories.BackwardCompatibilityMain; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.indexer.IndexerTest; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(BackwardCompatibilityMain.class) +public class ITBCMainIndexerTest extends IndexerTest +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainMultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainMultiStageQuery.java new file mode 100644 index 000000000000..5c975f64f563 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainMultiStageQuery.java @@ -0,0 +1,32 @@ +/* + * 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.BackwardCompatibilityMain; + +import org.apache.druid.testsEx.categories.BackwardCompatibilityMain; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.msq.MultiStageQuery; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(BackwardCompatibilityMain.class) +public class ITBCMainMultiStageQuery extends MultiStageQuery +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainUnionQueryTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainUnionQueryTest.java new file mode 100644 index 000000000000..94979d869809 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibilityMain/ITBCMainUnionQueryTest.java @@ -0,0 +1,32 @@ +/* + * 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.BackwardCompatibilityMain; + +import org.apache.druid.testsEx.categories.BackwardCompatibilityMain; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.query.UnionQueryTest; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(BackwardCompatibilityMain.class) +public class ITBCMainUnionQueryTest extends UnionQueryTest +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibilityMain.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibilityMain.java new file mode 100644 index 000000000000..f71b07243a1e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibilityMain.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class BackwardCompatibilityMain +{ +} 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 index 65b8dc0b1ac0..d173180db3b2 100644 --- 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 @@ -19,368 +19,13 @@ 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 +public class ITIndexerTest extends IndexerTest { - 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/IndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/IndexerTest.java new file mode 100644 index 000000000000..e1e7109969c5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/IndexerTest.java @@ -0,0 +1,381 @@ +/* + * 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.metadata.LockFilterPolicy; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +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; + +public class IndexerTest 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 List lockFilterPolicies = Collections.singletonList(new LockFilterPolicy(datasourceName, 0, null, null)); + final Map> lockedIntervals = new HashMap<>(); + ITRetryUtil.retryUntilFalse( + () -> { + lockedIntervals.clear(); + lockedIntervals.putAll(indexer.getLockedIntervals(lockFilterPolicies)); + 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/msq/ITMultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java index 7ede24cd8f9a..ce6462e225a5 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java @@ -19,254 +19,12 @@ package org.apache.druid.testsEx.msq; -import com.google.api.client.util.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.inject.Inject; -import org.apache.druid.indexer.report.TaskContextReport; -import org.apache.druid.indexer.report.TaskReport; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.msq.indexing.report.MSQResultsReport; -import org.apache.druid.msq.indexing.report.MSQTaskReport; -import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; -import org.apache.druid.msq.sql.SqlTaskStatus; -import org.apache.druid.storage.local.LocalFileExportStorageProvider; -import org.apache.druid.testing.clients.CoordinatorResourceTestClient; -import org.apache.druid.testing.utils.DataLoaderHelper; -import org.apache.druid.testing.utils.MsqTestQueryHelper; -import org.apache.druid.testsEx.categories.MultiStageQuery; import org.apache.druid.testsEx.config.DruidTestRunner; -import org.junit.Assert; -import org.junit.Ignore; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - @RunWith(DruidTestRunner.class) @Category(MultiStageQuery.class) -public class ITMultiStageQuery +public class ITMultiStageQuery extends MultiStageQuery { - @Inject - private MsqTestQueryHelper msqHelper; - - @Inject - private DataLoaderHelper dataLoaderHelper; - - @Inject - private CoordinatorResourceTestClient coordinatorClient; - - private static final String QUERY_FILE = "/multi-stage-query/wikipedia_msq_select_query1.json"; - - @Test - public void testMsqIngestionAndQuerying() throws Exception - { - String datasource = "dst"; - - // Clear up the datasource from the previous runs - coordinatorClient.unloadSegmentsForDataSource(datasource); - - String queryLocal = - StringUtils.format( - "INSERT INTO %s\n" - + "SELECT\n" - + " TIME_PARSE(\"timestamp\") AS __time,\n" - + " isRobot,\n" - + " diffUrl,\n" - + " added,\n" - + " countryIsoCode,\n" - + " regionName,\n" - + " channel,\n" - + " flags,\n" - + " delta,\n" - + " isUnpatrolled,\n" - + " isNew,\n" - + " deltaBucket,\n" - + " isMinor,\n" - + " isAnonymous,\n" - + " deleted,\n" - + " cityName,\n" - + " metroCode,\n" - + " namespace,\n" - + " comment,\n" - + " page,\n" - + " commentLength,\n" - + " countryName,\n" - + " user,\n" - + " regionIsoCode\n" - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" - + " '{\"type\":\"json\"}',\n" - + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" - + " )\n" - + ")\n" - + "PARTITIONED BY DAY\n", - datasource - ); - - // Submit the task and wait for the datasource to get loaded - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); - - if (sqlTaskStatus.getState().isFailure()) { - Assert.fail(StringUtils.format( - "Unable to start the task successfully.\nPossible exception: %s", - sqlTaskStatus.getError() - )); - } - - msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); - dataLoaderHelper.waitUntilDatasourceIsReady(datasource); - - msqHelper.testQueriesFromFile(QUERY_FILE, datasource); - } - - @Test - @Ignore("localfiles() is disabled") - public void testMsqIngestionAndQueryingWithLocalFn() throws Exception - { - String datasource = "dst"; - - // Clear up the datasource from the previous runs - coordinatorClient.unloadSegmentsForDataSource(datasource); - - String queryLocal = - StringUtils.format( - "INSERT INTO %s\n" - + "SELECT\n" - + " TIME_PARSE(\"timestamp\") AS __time,\n" - + " isRobot,\n" - + " diffUrl,\n" - + " added,\n" - + " countryIsoCode,\n" - + " regionName,\n" - + " channel,\n" - + " flags,\n" - + " delta,\n" - + " isUnpatrolled,\n" - + " isNew,\n" - + " deltaBucket,\n" - + " isMinor,\n" - + " isAnonymous,\n" - + " deleted,\n" - + " cityName,\n" - + " metroCode,\n" - + " namespace,\n" - + " comment,\n" - + " page,\n" - + " commentLength,\n" - + " countryName,\n" - + " user,\n" - + " regionIsoCode\n" - + "FROM TABLE(\n" - + " LOCALFILES(\n" - + " files => ARRAY['/resources/data/batch_index/json/wikipedia_index_data1.json'],\n" - + " format => 'json'\n" - + " ))\n" - + " (\"timestamp\" VARCHAR, isRobot VARCHAR, diffUrl VARCHAR, added BIGINT, countryIsoCode VARCHAR, regionName VARCHAR,\n" - + " channel VARCHAR, flags VARCHAR, delta BIGINT, isUnpatrolled VARCHAR, isNew VARCHAR, deltaBucket DOUBLE,\n" - + " isMinor VARCHAR, isAnonymous VARCHAR, deleted BIGINT, cityName VARCHAR, metroCode BIGINT, namespace VARCHAR,\n" - + " comment VARCHAR, page VARCHAR, commentLength BIGINT, countryName VARCHAR, \"user\" VARCHAR, regionIsoCode VARCHAR)\n" - + "PARTITIONED BY DAY\n", - datasource - ); - - // Submit the task and wait for the datasource to get loaded - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); - - if (sqlTaskStatus.getState().isFailure()) { - Assert.fail(StringUtils.format( - "Unable to start the task successfully.\nPossible exception: %s", - sqlTaskStatus.getError() - )); - } - - msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); - dataLoaderHelper.waitUntilDatasourceIsReady(datasource); - - msqHelper.testQueriesFromFile(QUERY_FILE, datasource); - } - - @Test - public void testExport() throws Exception - { - String exportQuery = - StringUtils.format( - "INSERT INTO extern(%s(exportPath => '%s'))\n" - + "AS CSV\n" - + "SELECT page, added, delta\n" - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" - + " '{\"type\":\"json\"}',\n" - + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" - + " )\n" - + ")\n", - LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" - ); - - SqlTaskStatus exportTask = msqHelper.submitMsqTaskSuccesfully(exportQuery); - - msqHelper.pollTaskIdForSuccess(exportTask.getTaskId()); - - if (exportTask.getState().isFailure()) { - Assert.fail(StringUtils.format( - "Unable to start the task successfully.\nPossible exception: %s", - exportTask.getError() - )); - } - - String resultQuery = StringUtils.format( - "SELECT page, delta, added\n" - + " FROM TABLE(\n" - + " EXTERN(\n" - + " '{\"type\":\"local\",\"baseDir\":\"/shared/export/\",\"filter\":\"*.csv\"}',\n" - + " '{\"type\":\"csv\",\"findColumnsFromHeader\":true}'\n" - + " )\n" - + " ) EXTEND (\"added\" BIGINT, \"delta\" BIGINT, \"page\" VARCHAR)\n" - + " WHERE delta != 0\n" - + " ORDER BY page"); - - SqlTaskStatus resultTaskStatus = msqHelper.submitMsqTaskSuccesfully(resultQuery); - - msqHelper.pollTaskIdForSuccess(resultTaskStatus.getTaskId()); - - TaskReport.ReportMap statusReport = msqHelper.fetchStatusReports(resultTaskStatus.getTaskId()); - - MSQTaskReport taskReport = (MSQTaskReport) statusReport.get(MSQTaskReport.REPORT_KEY); - if (taskReport == null) { - throw new ISE("Unable to fetch the status report for the task [%]", resultTaskStatus.getTaskId()); - } - TaskContextReport taskContextReport = (TaskContextReport) statusReport.get(TaskContextReport.REPORT_KEY); - Assert.assertFalse(taskContextReport.getPayload().isEmpty()); - - MSQTaskReportPayload taskReportPayload = Preconditions.checkNotNull( - taskReport.getPayload(), - "payload" - ); - MSQResultsReport resultsReport = Preconditions.checkNotNull( - taskReportPayload.getResults(), - "Results report for the task id is empty" - ); - - List> actualResults = new ArrayList<>(); - - for (final Object[] row : resultsReport.getResults()) { - actualResults.add(Arrays.asList(row)); - } - - ImmutableList> expectedResults = ImmutableList.of( - ImmutableList.of("Cherno Alpha", 111, 123), - ImmutableList.of("Gypsy Danger", -143, 57), - ImmutableList.of("Striker Eureka", 330, 459) - ); - - Assert.assertEquals( - expectedResults, - actualResults - ); - } } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/MultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/MultiStageQuery.java new file mode 100644 index 000000000000..bda1c2434536 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/MultiStageQuery.java @@ -0,0 +1,266 @@ +/* + * 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.msq; + +import com.google.api.client.util.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import org.apache.druid.indexer.report.TaskContextReport; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.DataLoaderHelper; +import org.apache.druid.testing.utils.MsqTestQueryHelper; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class MultiStageQuery +{ + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + @Inject + private CoordinatorResourceTestClient coordinatorClient; + + private static final String QUERY_FILE = "/multi-stage-query/wikipedia_msq_select_query1.json"; + + @Test + public void testMsqIngestionAndQuerying() throws Exception + { + String datasource = "dst"; + + // Clear up the datasource from the previous runs + coordinatorClient.unloadSegmentsForDataSource(datasource); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n", + datasource + ); + + // Submit the task and wait for the datasource to get loaded + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + @Test + @Ignore("localfiles() is disabled") + public void testMsqIngestionAndQueryingWithLocalFn() throws Exception + { + String datasource = "dst"; + + // Clear up the datasource from the previous runs + coordinatorClient.unloadSegmentsForDataSource(datasource); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " LOCALFILES(\n" + + " files => ARRAY['/resources/data/batch_index/json/wikipedia_index_data1.json'],\n" + + " format => 'json'\n" + + " ))\n" + + " (\"timestamp\" VARCHAR, isRobot VARCHAR, diffUrl VARCHAR, added BIGINT, countryIsoCode VARCHAR, regionName VARCHAR,\n" + + " channel VARCHAR, flags VARCHAR, delta BIGINT, isUnpatrolled VARCHAR, isNew VARCHAR, deltaBucket DOUBLE,\n" + + " isMinor VARCHAR, isAnonymous VARCHAR, deleted BIGINT, cityName VARCHAR, metroCode BIGINT, namespace VARCHAR,\n" + + " comment VARCHAR, page VARCHAR, commentLength BIGINT, countryName VARCHAR, \"user\" VARCHAR, regionIsoCode VARCHAR)\n" + + "PARTITIONED BY DAY\n", + datasource + ); + + // Submit the task and wait for the datasource to get loaded + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); + + if (sqlTaskStatus.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + sqlTaskStatus.getError() + )); + } + + msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId()); + dataLoaderHelper.waitUntilDatasourceIsReady(datasource); + + msqHelper.testQueriesFromFile(QUERY_FILE, datasource); + } + + @Test + public void testExport() throws Exception + { + String exportQuery = + StringUtils.format( + "INSERT INTO extern(%s(exportPath => '%s'))\n" + + "AS CSV\n" + + "SELECT page, added, delta\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n", + LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" + ); + + SqlTaskStatus exportTask = msqHelper.submitMsqTaskSuccesfully(exportQuery); + + msqHelper.pollTaskIdForSuccess(exportTask.getTaskId()); + + if (exportTask.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + exportTask.getError() + )); + } + + String resultQuery = StringUtils.format( + "SELECT page, delta, added\n" + + " FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"baseDir\":\"/shared/export/\",\"filter\":\"*.csv\"}',\n" + + " '{\"type\":\"csv\",\"findColumnsFromHeader\":true}'\n" + + " )\n" + + " ) EXTEND (\"added\" BIGINT, \"delta\" BIGINT, \"page\" VARCHAR)\n" + + " WHERE delta != 0\n" + + " ORDER BY page"); + + SqlTaskStatus resultTaskStatus = msqHelper.submitMsqTaskSuccesfully(resultQuery); + + msqHelper.pollTaskIdForSuccess(resultTaskStatus.getTaskId()); + + TaskReport.ReportMap statusReport = msqHelper.fetchStatusReports(resultTaskStatus.getTaskId()); + + MSQTaskReport taskReport = (MSQTaskReport) statusReport.get(MSQTaskReport.REPORT_KEY); + if (taskReport == null) { + throw new ISE("Unable to fetch the status report for the task [%]", resultTaskStatus.getTaskId()); + } + TaskContextReport taskContextReport = (TaskContextReport) statusReport.get(TaskContextReport.REPORT_KEY); + Assert.assertFalse(taskContextReport.getPayload().isEmpty()); + + MSQTaskReportPayload taskReportPayload = Preconditions.checkNotNull( + taskReport.getPayload(), + "payload" + ); + MSQResultsReport resultsReport = Preconditions.checkNotNull( + taskReportPayload.getResults(), + "Results report for the task id is empty" + ); + + List> actualResults = new ArrayList<>(); + + for (final Object[] row : resultsReport.getResults()) { + actualResults.add(Arrays.asList(row)); + } + + ImmutableList> expectedResults = ImmutableList.of( + ImmutableList.of("Cherno Alpha", 111, 123), + ImmutableList.of("Gypsy Danger", -143, 57), + ImmutableList.of("Striker Eureka", 330, 459) + ); + + Assert.assertEquals( + expectedResults, + actualResults + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java index 2d0d8e4f50e4..03022da32fa5 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java @@ -19,195 +19,13 @@ package org.apache.druid.testsEx.query; -import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; -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.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.testing.utils.KafkaAdminClient; -import org.apache.druid.testing.utils.KafkaEventWriter; -import org.apache.druid.testing.utils.KafkaUtil; -import org.apache.druid.testing.utils.StreamEventWriter; import org.apache.druid.testsEx.categories.Query; import org.apache.druid.testsEx.config.DruidTestRunner; -import org.apache.druid.testsEx.indexer.AbstractIndexerTest; -import org.joda.time.Interval; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; -import java.util.function.Function; - @RunWith(DruidTestRunner.class) @Category(Query.class) -public class ITUnionQueryTest extends AbstractIndexerTest +public class ITUnionQueryTest extends UnionQueryTest { - private static final Logger LOG = new Logger(ITUnionQueryTest.class); - private static final String UNION_SUPERVISOR_TEMPLATE = "/query/union_kafka_supervisor_template.json"; - private static final String UNION_DATA_FILE = "/query/union_data.json"; - private static final String UNION_QUERIES_RESOURCE = "/query/union_queries.json"; - private static final String UNION_DATASOURCE = "wikipedia_index_test"; - private String fullDatasourceName; - - @Test - public void testUnionQuery() throws Exception - { - fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix(); - final String baseName = fullDatasourceName + UUID.randomUUID(); - KafkaAdminClient streamAdminClient = new KafkaAdminClient(config); - List supervisors = new ArrayList<>(); - - final int numDatasources = 3; - for (int i = 0; i < numDatasources; i++) { - String datasource = baseName + "-" + i; - streamAdminClient.createStream(datasource, 1, Collections.emptyMap()); - ITRetryUtil.retryUntil( - () -> streamAdminClient.isStreamActive(datasource), - true, - 10000, - 30, - "Wait for stream active" - ); - String supervisorSpec = generateStreamIngestionPropsTransform( - datasource, - datasource, - config - ).apply(getResourceAsString(UNION_SUPERVISOR_TEMPLATE)); - LOG.info("supervisorSpec: [%s]\n", supervisorSpec); - // Start supervisor - String specResponse = indexer.submitSupervisor(supervisorSpec); - LOG.info("Submitted supervisor [%s]", specResponse); - supervisors.add(specResponse); - - int ctr = 0; - try ( - StreamEventWriter streamEventWriter = new KafkaEventWriter(config, false); - BufferedReader reader = new BufferedReader( - new InputStreamReader(getResourceAsStream(UNION_DATA_FILE), StandardCharsets.UTF_8) - ) - ) { - String line; - while ((line = reader.readLine()) != null) { - streamEventWriter.write(datasource, StringUtils.toUtf8(line)); - ctr++; - } - } - final int numWritten = ctr; - - LOG.info("Waiting for stream indexing tasks to consume events"); - - ITRetryUtil.retryUntilTrue( - () -> - numWritten == this.queryHelper.countRows( - datasource, - Intervals.ETERNITY, - name -> new LongSumAggregatorFactory(name, "count") - ), - StringUtils.format( - "dataSource[%s] consumed [%,d] events, expected [%,d]", - datasource, - this.queryHelper.countRows( - datasource, - Intervals.ETERNITY, - name -> new LongSumAggregatorFactory(name, "count") - ), - numWritten - ) - ); - } - - String queryResponseTemplate = StringUtils.replace( - getResourceAsString(UNION_QUERIES_RESOURCE), - "%%DATASOURCE%%", - baseName - ); - - queryHelper.testQueriesFromString(queryResponseTemplate); - - - for (int i = 0; i < numDatasources; i++) { - indexer.terminateSupervisor(supervisors.get(i)); - streamAdminClient.deleteStream(baseName + "-" + i); - } - - for (int i = 0; i < numDatasources; i++) { - final int datasourceNumber = i; - ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(baseName + "-" + datasourceNumber), - true, - 10000, - 10, - "Kafka segments loaded" - ); - } - - queryHelper.testQueriesFromString(queryResponseTemplate); - - for (int i = 0; i < numDatasources; i++) { - final String datasource = baseName + "-" + i; - List intervals = coordinator.getSegmentIntervals(datasource); - - Collections.sort(intervals); - String first = intervals.get(0).split("/")[0]; - String last = intervals.get(intervals.size() - 1).split("/")[1]; - Interval interval = Intervals.of(first + "/" + last); - coordinator.unloadSegmentsForDataSource(baseName + "-" + i); - ITRetryUtil.retryUntilFalse( - () -> coordinator.areSegmentsLoaded(datasource), - "Segment Unloading" - ); - coordinator.deleteSegmentsDataSource(baseName + "-" + i, interval); - } - } - - - /** - * sad version of - * {@link org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest#generateStreamIngestionPropsTransform} - */ - private Function generateStreamIngestionPropsTransform( - String streamName, - String fullDatasourceName, - IntegrationTestingConfig config - ) - { - final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); - final Properties consumerProperties = new Properties(); - consumerProperties.putAll(consumerConfigs); - consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); - KafkaUtil.addPropertiesFromTestConfig(config, consumerProperties); - return spec -> { - try { - spec = StringUtils.replace( - spec, - "%%DATASOURCE%%", - fullDatasourceName - ); - spec = StringUtils.replace( - spec, - "%%TOPIC_VALUE%%", - streamName - ); - return StringUtils.replace( - spec, - "%%STREAM_PROPERTIES_VALUE%%", - jsonMapper.writeValueAsString(consumerProperties) - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - } } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/UnionQueryTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/UnionQueryTest.java new file mode 100644 index 000000000000..a1d62fe55f5e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/UnionQueryTest.java @@ -0,0 +1,207 @@ +/* + * 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.query; + +import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; +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.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.KafkaAdminClient; +import org.apache.druid.testing.utils.KafkaEventWriter; +import org.apache.druid.testing.utils.KafkaUtil; +import org.apache.druid.testing.utils.StreamEventWriter; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; +import org.joda.time.Interval; +import org.junit.Test; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.function.Function; + +public class UnionQueryTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(UnionQueryTest.class); + private static final String UNION_SUPERVISOR_TEMPLATE = "/query/union_kafka_supervisor_template.json"; + private static final String UNION_DATA_FILE = "/query/union_data.json"; + private static final String UNION_QUERIES_RESOURCE = "/query/union_queries.json"; + private static final String UNION_DATASOURCE = "wikipedia_index_test"; + private String fullDatasourceName; + + @Test + public void testUnionQuery() throws Exception + { + fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix(); + final String baseName = fullDatasourceName + UUID.randomUUID(); + KafkaAdminClient streamAdminClient = new KafkaAdminClient(config); + List supervisors = new ArrayList<>(); + + final int numDatasources = 3; + for (int i = 0; i < numDatasources; i++) { + String datasource = baseName + "-" + i; + streamAdminClient.createStream(datasource, 1, Collections.emptyMap()); + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(datasource), + true, + 10000, + 30, + "Wait for stream active" + ); + String supervisorSpec = generateStreamIngestionPropsTransform( + datasource, + datasource, + config + ).apply(getResourceAsString(UNION_SUPERVISOR_TEMPLATE)); + LOG.info("supervisorSpec: [%s]\n", supervisorSpec); + // Start supervisor + String specResponse = indexer.submitSupervisor(supervisorSpec); + LOG.info("Submitted supervisor [%s]", specResponse); + supervisors.add(specResponse); + + int ctr = 0; + try ( + StreamEventWriter streamEventWriter = new KafkaEventWriter(config, false); + BufferedReader reader = new BufferedReader( + new InputStreamReader(getResourceAsStream(UNION_DATA_FILE), StandardCharsets.UTF_8) + ) + ) { + String line; + while ((line = reader.readLine()) != null) { + streamEventWriter.write(datasource, StringUtils.toUtf8(line)); + ctr++; + } + } + final int numWritten = ctr; + + LOG.info("Waiting for stream indexing tasks to consume events"); + + ITRetryUtil.retryUntilTrue( + () -> + numWritten == this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + StringUtils.format( + "dataSource[%s] consumed [%,d] events, expected [%,d]", + datasource, + this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + numWritten + ) + ); + } + + String queryResponseTemplate = StringUtils.replace( + getResourceAsString(UNION_QUERIES_RESOURCE), + "%%DATASOURCE%%", + baseName + ); + + queryHelper.testQueriesFromString(queryResponseTemplate); + + + for (int i = 0; i < numDatasources; i++) { + indexer.terminateSupervisor(supervisors.get(i)); + streamAdminClient.deleteStream(baseName + "-" + i); + } + + for (int i = 0; i < numDatasources; i++) { + final int datasourceNumber = i; + ITRetryUtil.retryUntil( + () -> coordinator.areSegmentsLoaded(baseName + "-" + datasourceNumber), + true, + 10000, + 10, + "Kafka segments loaded" + ); + } + + queryHelper.testQueriesFromString(queryResponseTemplate); + + for (int i = 0; i < numDatasources; i++) { + final String datasource = baseName + "-" + i; + List intervals = coordinator.getSegmentIntervals(datasource); + + Collections.sort(intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + Interval interval = Intervals.of(first + "/" + last); + coordinator.unloadSegmentsForDataSource(baseName + "-" + i); + ITRetryUtil.retryUntilFalse( + () -> coordinator.areSegmentsLoaded(datasource), + "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(baseName + "-" + i, interval); + } + } + + + /** + * sad version of + * {@link org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest#generateStreamIngestionPropsTransform} + */ + private Function generateStreamIngestionPropsTransform( + String streamName, + String fullDatasourceName, + IntegrationTestingConfig config + ) + { + final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); + final Properties consumerProperties = new Properties(); + consumerProperties.putAll(consumerConfigs); + consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); + KafkaUtil.addPropertiesFromTestConfig(config, consumerProperties); + return spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_VALUE%%", + streamName + ); + return StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_VALUE%%", + jsonMapper.writeValueAsString(consumerProperties) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + } +} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibilityMain/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibilityMain/docker.yaml new file mode 100644 index 000000000000..1c51532c6a9f --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibilityMain/docker.yaml @@ -0,0 +1,199 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# 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 + - /cluster/Common/kafka.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 + +# 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/docs/backward-compatibility-tests.md b/integration-tests-ex/docs/backward-compatibility-tests.md new file mode 100644 index 000000000000..a2d56217b85e --- /dev/null +++ b/integration-tests-ex/docs/backward-compatibility-tests.md @@ -0,0 +1,43 @@ + + +#### Problem +The idea of this test group is to simulate issues that can arise during rolling upgrade/downgrade. + +#### Implementation +In this test group, the docker compose cluster is launched with services on different versions. +The docker image for the previous version is built by downloading the previous version druid tar. +Currently, the case where Coordinator and Overlord is on the previous version is tested. + +#### Test coverage +Existing +- MSQ ingestion +- Native ingestion +- Stream ingestion +- Querying + +Pending +- Compaction + +#### Limitations +* `druid-testing-tools` jar is not published. The image for the previous version still uses the +extension from the current build. +This could break in case of incompatible changes in this extension. +In such a scenario the test should be disabled. However, this extension is primarily used to +test specific error scenarios and launch custom node role service (used in HighAvailability test group). diff --git a/integration-tests-ex/image/docker-build.sh b/integration-tests-ex/image/docker-build.sh index 6a945aa6129a..4ff5d7a74387 100755 --- a/integration-tests-ex/image/docker-build.sh +++ b/integration-tests-ex/image/docker-build.sh @@ -53,4 +53,23 @@ docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg CONFLUENT_VERSION=$CONFLUENT_VERSION \ --build-arg HADOOP_VERSION=$HADOOP_VERSION \ --build-arg MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME \ + --build-arg DRUID_TESTING_TOOLS_VERSION=$DRUID_VERSION \ + . + +if [[ -z "${BACKWARD_COMPATIBILITY_IT_ENABLED:-""}" || $BACKWARD_COMPATIBILITY_IT_ENABLED != "true" ]]; then + echo "Not building previous version image." + exit 0 +fi + +# Download the previous druid tar +curl -L $DRUID_PREVIOUS_VERSION_DOWNLOAD_URL --output apache-druid-$DRUID_PREVIOUS_VERSION-bin.tar.gz + +docker build -t $DRUID_PREVIOUS_IT_IMAGE_NAME \ + --build-arg DRUID_VERSION=$DRUID_PREVIOUS_VERSION \ + --build-arg MYSQL_VERSION=$MYSQL_VERSION \ + --build-arg MARIADB_VERSION=$MARIADB_VERSION \ + --build-arg CONFLUENT_VERSION=$CONFLUENT_VERSION \ + --build-arg HADOOP_VERSION=$HADOOP_VERSION \ + --build-arg MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME \ + --build-arg DRUID_TESTING_TOOLS_VERSION=$DRUID_VERSION \ . diff --git a/integration-tests-ex/image/docker/Dockerfile b/integration-tests-ex/image/docker/Dockerfile index a77a5c2d023e..90955eae3c11 100644 --- a/integration-tests-ex/image/docker/Dockerfile +++ b/integration-tests-ex/image/docker/Dockerfile @@ -46,13 +46,15 @@ ARG MARIADB_VERSION ENV MARIADB_VERSION=$MARIADB_VERSION ARG MYSQL_DRIVER_CLASSNAME=com.mysql.jdbc.Driver ENV MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME +ARG DRUID_TESTING_TOOLS_VERSION ENV DRUID_HOME=/usr/local/druid # Populate build artifacts COPY apache-druid-${DRUID_VERSION}-bin.tar.gz /usr/local/ -COPY druid-it-tools-${DRUID_VERSION}.jar /tmp/druid/extensions/druid-it-tools/ +COPY druid-it-tools-${DRUID_TESTING_TOOLS_VERSION}.jar /tmp/druid/extensions/druid-it-tools/ + COPY kafka-protobuf-provider-${CONFLUENT_VERSION}.jar /tmp/druid/lib/ COPY mysql-connector-j-${MYSQL_VERSION}.jar /tmp/druid/lib/ COPY mariadb-java-client-${MARIADB_VERSION}.jar /tmp/druid/lib/ @@ -60,6 +62,7 @@ COPY test-setup.sh / COPY druid.sh / COPY launch.sh / + # Do the setup tasks. The tasks are done within a script, rather than # here, so they are easier to describe and debug. Turn on the "-x" flag # within the script to trace the steps if needed for debugging. diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java index 8e5ee737f03c..74d56065ef45 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java @@ -28,8 +28,8 @@ 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.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -86,7 +86,7 @@ public void submitCompactionConfig(final DataSourceCompactionConfig dataSourceCo } } - public void deleteCompactionConfig(final String dataSource) throws Exception + public void deleteDataSourceCompactionConfig(final String dataSource) throws Exception { String url = StringUtils.format("%sconfig/compaction/%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); StatusResponseHolder response = httpClient.go(new Request(HttpMethod.DELETE, new URL(url)), responseHandler).get(); @@ -100,7 +100,7 @@ public void deleteCompactionConfig(final String dataSource) throws Exception } } - public CoordinatorCompactionConfig getCoordinatorCompactionConfigs() throws Exception + public DruidCompactionConfig getCompactionConfig() throws Exception { String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); StatusResponseHolder response = httpClient.go( @@ -113,7 +113,7 @@ public CoordinatorCompactionConfig getCoordinatorCompactionConfigs() throws Exce response.getContent() ); } - return jsonMapper.readValue(response.getContent(), new TypeReference() {}); + return jsonMapper.readValue(response.getContent(), new TypeReference() {}); } public DataSourceCompactionConfig getDataSourceCompactionConfig(String dataSource) throws Exception 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 8167b9b64e12..f75dc6043f9d 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 @@ -39,6 +39,7 @@ 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.metadata.LockFilterPolicy; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; @@ -334,13 +335,13 @@ public TaskReport.ReportMap getTaskReport(String taskId) } } - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getLockedIntervals(List lockFilterPolicies) { try { - String jsonBody = jsonMapper.writeValueAsString(minTaskPriority); + String jsonBody = jsonMapper.writeValueAsString(lockFilterPolicies); StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.POST, new URL(getIndexerURL() + "lockedIntervals")) + new Request(HttpMethod.POST, new URL(getIndexerURL() + "lockedIntervals/v2")) .setContent( "application/json", StringUtils.toUtf8(jsonBody) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java deleted file mode 100644 index 63316e76687d..000000000000 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.testing.utils; - -import org.apache.druid.data.input.MaxSizeSplitHintSpec; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.joda.time.Period; - -/** - * Contains utility methods for Compaction. - */ -public class CompactionUtil -{ - - private CompactionUtil() - { - // no instantiation - } - - public static DataSourceCompactionConfig createCompactionConfig( - String fullDatasourceName, - Integer maxRowsPerSegment, - Period skipOffsetFromLatest - ) - { - return new DataSourceCompactionConfig( - fullDatasourceName, - null, - null, - null, - skipOffsetFromLatest, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - new MaxSizeSplitHintSpec(null, 1), - new DynamicPartitionsSpec(maxRowsPerSegment, null), - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - 1, - null - ), - null, - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - null - ); - } - -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java index 8d980d76f12a..0a1071c38e71 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java @@ -19,18 +19,19 @@ package org.apache.druid.tests.coordinator.duty; +import com.google.common.collect.ImmutableList; import com.google.inject.Inject; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; 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.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.clients.CompactionResourceTestClient; import org.apache.druid.testing.clients.TaskResponseObject; import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.CompactionUtil; import org.apache.druid.testing.utils.EventSerializer; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.KafkaUtil; @@ -53,7 +54,6 @@ import java.io.Closeable; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -265,13 +265,13 @@ private void ensureSegmentsLoaded() */ private void ensureLockedIntervals(Interval... intervals) { - final Map minTaskPriority = Collections.singletonMap(fullDatasourceName, 0); + final LockFilterPolicy lockFilterPolicy = new LockFilterPolicy(fullDatasourceName, 0, null, null); final List lockedIntervals = new ArrayList<>(); ITRetryUtil.retryUntilTrue( () -> { lockedIntervals.clear(); - Map> allIntervals = indexer.getLockedIntervals(minTaskPriority); + Map> allIntervals = indexer.getLockedIntervals(ImmutableList.of(lockFilterPolicy)); if (allIntervals.containsKey(fullDatasourceName)) { lockedIntervals.addAll(allIntervals.get(fullDatasourceName)); } @@ -304,26 +304,26 @@ private boolean shouldSkipTest(boolean testEnableTransaction) */ private void submitAndVerifyCompactionConfig() throws Exception { - final DataSourceCompactionConfig compactionConfig = CompactionUtil - .createCompactionConfig(fullDatasourceName, Specs.MAX_ROWS_PER_SEGMENT, Period.ZERO); + final DataSourceCompactionConfig dataSourceCompactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(fullDatasourceName) + .withSkipOffsetFromLatest(Period.ZERO) + .withMaxRowsPerSegment(Specs.MAX_ROWS_PER_SEGMENT) + .build(); compactionResource.updateCompactionTaskSlot(0.5, 10, null); - compactionResource.submitCompactionConfig(compactionConfig); + compactionResource.submitCompactionConfig(dataSourceCompactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig observedCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - observedCompactionConfig = dataSourceCompactionConfig; - } - } - Assert.assertEquals(observedCompactionConfig, compactionConfig); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig observedCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); + Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); observedCompactionConfig = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); - Assert.assertEquals(observedCompactionConfig, compactionConfig); + Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); } /** diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 230a19236c16..d09bced3313c 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -28,10 +28,10 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.overlord.http.TaskPayloadResponse; @@ -54,8 +54,8 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; @@ -328,8 +328,8 @@ public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExis } } - @Test - public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception { // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); @@ -356,7 +356,8 @@ public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, - false + false, + engine ); // should now only have 1 row after compaction // added = null, count = 2, sum_added = 62 @@ -559,8 +560,8 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception } } - @Test - public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -571,9 +572,9 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); // Dummy compaction config which will be overwritten - submitCompactionConfig(10000, NO_SKIP_OFFSET); + submitCompactionConfig(10000, NO_SKIP_OFFSET, engine); // New compaction config should overwrites the existing compaction config - submitCompactionConfig(1, NO_SKIP_OFFSET); + submitCompactionConfig(1, NO_SKIP_OFFSET, engine); LOG.info("Auto compaction test with dynamic partitioning"); @@ -584,25 +585,28 @@ public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception verifySegmentsCompacted(10, 1); checkCompactionIntervals(intervalsBeforeCompaction); - LOG.info("Auto compaction test with hash partitioning"); - - final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); - submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); - // 2 segments published per day after compaction. - forceTriggerAutoCompaction(4); - verifyQuery(INDEX_QUERIES_RESOURCE); - verifySegmentsCompacted(hashedPartitionsSpec, 4); - checkCompactionIntervals(intervalsBeforeCompaction); + if (engine == CompactionEngine.NATIVE) { + // HashedPartitionsSpec not supported by MSQ. + LOG.info("Auto compaction test with hash partitioning"); + + final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); + submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); + // 2 segments published per day after compaction. + forceTriggerAutoCompaction(4); + verifyQuery(INDEX_QUERIES_RESOURCE); + verifySegmentsCompacted(hashedPartitionsSpec, 4); + checkCompactionIntervals(intervalsBeforeCompaction); + } LOG.info("Auto compaction test with range partitioning"); - final SingleDimensionPartitionsSpec rangePartitionsSpec = new SingleDimensionPartitionsSpec( + final DimensionRangePartitionsSpec rangePartitionsSpec = new DimensionRangePartitionsSpec( 5, null, - "city", + ImmutableList.of("city"), false ); - submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); + submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(rangePartitionsSpec, 2); @@ -695,8 +699,8 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, // "maxRowsPerSegment": 3 @@ -732,7 +736,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); List expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsBeforeCompaction) { @@ -750,7 +760,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", newGranularity = Granularities.MONTH; // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); // Since dropExisting is set to true... // Again data is only in two days @@ -778,7 +794,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( // compact only tombstones, so it should be a tombstone itself. newGranularity = new PeriodGranularity(new Period("P6M"), null, DateTimeZone.UTC); // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); // Since dropExisting is set to true... // The earlier 12 segments with MONTH granularity will be completely covered, overshadowed, by the @@ -804,8 +826,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue( } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueThenFalse() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueThenFalse(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, // "maxRowsPerSegment": 3 @@ -841,7 +863,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); List expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsBeforeCompaction) { @@ -859,7 +881,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", newGranularity = Granularities.MONTH; // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); // Since dropExisting is set to true... // Again data is only in two days @@ -885,9 +907,13 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueT LOG.info("Auto compaction test with SEMESTER segment granularity, dropExisting is false, over tombstones"); newGranularity = new PeriodGranularity(new Period("P6M"), null, DateTimeZone.UTC); // Set dropExisting to false - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, - null, null - ), false); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + engine + ); // Since dropExisting is set to false the first semester will be forced to dropExisting true // Hence, we will have two, one tombstone for the first semester and one data segment for the second. @@ -963,8 +989,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingFalse } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -974,14 +1000,14 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), engine); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -1001,8 +1027,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveSameSegmentGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveSameSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1013,7 +1039,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1023,7 +1049,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.DAY; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1033,8 +1059,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveDifferentSegmentGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveDifferentSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1045,7 +1071,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1055,7 +1081,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1066,8 +1092,8 @@ public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegm } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranularityCoveringMultipleSegmentsInTimelineAndDropExistingTrue() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranularityCoveringMultipleSegmentsInTimelineAndDropExistingTrue(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1079,7 +1105,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); List expectedIntervalAfterCompaction = new ArrayList<>(); // We will still have one visible segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) @@ -1109,7 +1135,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranula coordinator.getSegmentIntervals(fullDatasourceName); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); // verify: expectedIntervalAfterCompaction = new ArrayList<>(); // The previous segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) will be @@ -1245,8 +1271,8 @@ public void testAutoCompactionDutyWithSegmentGranularityFinerAndNotAlignWithSegm } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment(CompactionEngine engine) throws Exception { updateCompactionTaskSlot(1, 1, null); final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); @@ -1263,7 +1289,8 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null), - false + false, + engine ); // Before compaction, we have segments with the interval 2013-08-26T00:00:00.000Z/2013-09-02T00:00:00.000Z // We will compact the latest segment to MONTH. @@ -1292,8 +1319,8 @@ public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSe } } - @Test - public void testAutoCompactionDutyWithRollup() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithRollup(CompactionEngine engine) throws Exception { final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); @@ -1309,7 +1336,8 @@ public void testAutoCompactionDutyWithRollup() throws Exception MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(null, null, true), - false + false, + engine ); forceTriggerAutoCompaction(2); queryAndResultFields = ImmutableMap.of( @@ -1328,8 +1356,8 @@ public void testAutoCompactionDutyWithRollup() throws Exception } } - @Test - public void testAutoCompactionDutyWithQueryGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) throws Exception { final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, true, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); @@ -1345,7 +1373,8 @@ public void testAutoCompactionDutyWithQueryGranularity() throws Exception MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(null, Granularities.DAY, null), - false + false, + engine ); forceTriggerAutoCompaction(2); queryAndResultFields = ImmutableMap.of( @@ -1364,8 +1393,8 @@ public void testAutoCompactionDutyWithQueryGranularity() throws Exception } } - @Test - public void testAutoCompactionDutyWithDimensionsSpec() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) throws Exception { // Index data with dimensions "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", // "namespace", "continent", "country", "region", "city" @@ -1392,7 +1421,8 @@ public void testAutoCompactionDutyWithDimensionsSpec() throws Exception new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, null, - false + false, + engine ); forceTriggerAutoCompaction(2); @@ -1575,17 +1605,17 @@ public void testUpdateCompactionTaskSlotWithUseAutoScaleSlots() throws Exception { // First try update without useAutoScaleSlots updateCompactionTaskSlot(3, 5, null); - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); // Should be default value which is false - Assert.assertFalse(coordinatorCompactionConfig.isUseAutoScaleSlots()); + Assert.assertFalse(compactionConfig.isUseAutoScaleSlots()); // Now try update from default value to useAutoScaleSlots=true updateCompactionTaskSlot(3, 5, true); - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertTrue(coordinatorCompactionConfig.isUseAutoScaleSlots()); + compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertTrue(compactionConfig.isUseAutoScaleSlots()); // Now try update from useAutoScaleSlots=true to useAutoScaleSlots=false updateCompactionTaskSlot(3, 5, false); - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertFalse(coordinatorCompactionConfig.isUseAutoScaleSlots()); + compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertFalse(compactionConfig.isUseAutoScaleSlots()); } private void loadData(String indexTask) throws Exception @@ -1760,30 +1790,6 @@ private void submitCompactionConfig( ); } - private void submitCompactionConfig( - PartitionsSpec partitionsSpec, - Period skipOffsetFromLatest, - int maxNumConcurrentSubTasks, - UserCompactionTaskGranularityConfig granularitySpec, - UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, - AggregatorFactory[] metricsSpec, - boolean dropExisting - ) throws Exception - { - submitCompactionConfig( - partitionsSpec, - skipOffsetFromLatest, - maxNumConcurrentSubTasks, - granularitySpec, - dimensionsSpec, - transformSpec, - metricsSpec, - dropExisting, - null - ); - } - private void submitCompactionConfig( PartitionsSpec partitionsSpec, Period skipOffsetFromLatest, @@ -1796,7 +1802,7 @@ private void submitCompactionConfig( @Nullable CompactionEngine engine ) throws Exception { - DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( + DataSourceCompactionConfig dataSourceCompactionConfig = new DataSourceCompactionConfig( fullDatasourceName, null, null, @@ -1829,21 +1835,17 @@ private void submitCompactionConfig( transformSpec, !dropExisting ? null : new UserCompactionTaskIOConfig(true), engine, - null + ImmutableMap.of("maxNumTasks", 2) ); - compactionResource.submitCompactionConfig(compactionConfig); + compactionResource.submitCompactionConfig(dataSourceCompactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); @@ -1858,16 +1860,12 @@ private void submitCompactionConfig( private void deleteCompactionConfig() throws Exception { - compactionResource.deleteCompactionConfig(fullDatasourceName); + compactionResource.deleteDataSourceCompactionConfig(fullDatasourceName); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); Assert.assertNull(foundDataSourceCompactionConfig); } @@ -1949,11 +1947,11 @@ private void updateCompactionTaskSlot(double compactionTaskSlotRatio, int maxCom { compactionResource.updateCompactionTaskSlot(compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertEquals(coordinatorCompactionConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); - Assert.assertEquals(coordinatorCompactionConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertEquals(compactionConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); + Assert.assertEquals(compactionConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); if (useAutoScaleSlots != null) { - Assert.assertEquals(coordinatorCompactionConfig.isUseAutoScaleSlots(), useAutoScaleSlots.booleanValue()); + Assert.assertEquals(compactionConfig.isUseAutoScaleSlots(), useAutoScaleSlots.booleanValue()); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java index 9e1b54143bbd..b2ca693a4422 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java @@ -24,9 +24,8 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; @@ -44,7 +43,6 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITAutoCompactionUpgradeTest extends AbstractIndexerTest { - private static final Logger LOG = new Logger(ITAutoCompactionUpgradeTest.class); private static final String UPGRADE_DATASOURCE_NAME = "upgradeTest"; @Inject @@ -58,67 +56,56 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers { // Verify that compaction config already exist. This config was inserted manually into the database using SQL script. // This auto compaction configuration payload is from Druid 0.21.0 - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(UPGRADE_DATASOURCE_NAME)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig coordinatorCompactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); // Now submit a new auto compaction configuration PartitionsSpec newPartitionsSpec = new DynamicPartitionsSpec(4000, null); Period newSkipOffset = Period.seconds(0); - DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( - UPGRADE_DATASOURCE_NAME, - null, - null, - null, - newSkipOffset, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - new MaxSizeSplitHintSpec(null, 1), - newPartitionsSpec, - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - 1, - null - ), - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - null - ); + DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(UPGRADE_DATASOURCE_NAME) + .withSkipOffsetFromLatest(newSkipOffset) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + new MaxSizeSplitHintSpec(null, 1), + newPartitionsSpec, + null, + null, + null, + null, + null, + 1, + null, + null, + null, + null, + null, + 1, + null + ) + ) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) + ) + .withIoConfig(new UserCompactionTaskIOConfig(true)) + .build(); compactionResource.submitCompactionConfig(compactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that compaction was successfully updated - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(UPGRADE_DATASOURCE_NAME)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + coordinatorCompactionConfig = compactionResource.getCompactionConfig(); + foundDataSourceCompactionConfig + = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), newPartitionsSpec); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index f527135c80d3..dfe308e2c1b1 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -24,6 +24,7 @@ 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.metadata.LockFilterPolicy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.guice.DruidTestModuleFactory; @@ -342,12 +343,12 @@ public void testGetLockedIntervals() throws Exception submitIndexTask(INDEX_TASK, datasourceName); // Wait until it acquires a lock - final Map minTaskPriority = Collections.singletonMap(datasourceName, 0); + final List lockFilterPolicies = Collections.singletonList(new LockFilterPolicy(datasourceName, 0, null, null)); final Map> lockedIntervals = new HashMap<>(); ITRetryUtil.retryUntilFalse( () -> { lockedIntervals.clear(); - lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority)); + lockedIntervals.putAll(indexer.getLockedIntervals(lockFilterPolicies)); return lockedIntervals.isEmpty(); }, "Verify Intervals are Locked" diff --git a/pom.xml b/pom.xml index ea9c3dcfa179..11738830512e 100644 --- a/pom.xml +++ b/pom.xml @@ -242,6 +242,7 @@ integration-tests-ex/tools integration-tests-ex/image integration-tests-ex/cases + quidem-ut diff --git a/processing/src/main/java/org/apache/druid/collections/CircularList.java b/processing/src/main/java/org/apache/druid/collections/CircularList.java new file mode 100644 index 000000000000..7cf551d6cf88 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/collections/CircularList.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.collections; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A circular list that is backed by an ordered list of elements containing no duplicates. The list is ordered by the + * supplied comparator. The iterator keeps track of the current position, so iterating the list multiple times will + * resume from the last location and continue until a caller explicitly terminates it. + *

+ * This class is not thread-safe and must be used from a single thread. + */ +@NotThreadSafe +public class CircularList implements Iterable +{ + private final List elements = new ArrayList<>(); + private int currentPosition; + + public CircularList(final Set elements, final Comparator comparator) + { + this.elements.addAll(elements); + this.elements.sort(comparator); + this.currentPosition = -1; + } + + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return elements.size() > 0; + } + + @Override + public T next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + advanceCursor(); + return elements.get(currentPosition); + } + + private void advanceCursor() + { + if (++currentPosition >= elements.size()) { + currentPosition = 0; + } + } + }; + } + + /** + * @return true if the supplied set is equal to the set used to instantiate this circular list, otherwise false. + */ + public boolean equalsSet(final Set inputSet) + { + return new HashSet<>(elements).equals(inputSet); + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java index a65ddb7b9aec..b5933e722d8c 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Strings; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -110,10 +111,9 @@ public static MultiValueHandling fromString(String name) return name == null ? ofDefault() : valueOf(StringUtils.toUpperCase(name)); } - // this can be system configuration public static MultiValueHandling ofDefault() { - return SORTED_ARRAY; + return BuiltInTypesModule.getStringMultiValueHandlingMode(); } } diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java index 963a001ad6db..7da6550ccca7 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ReadableFileFrameChannel.java @@ -104,6 +104,14 @@ public void close() } } + /** + * Returns whether this channel represents the entire underlying {@link FrameFile}. + */ + public boolean isEntireFile() + { + return currentFrame == 0 && endFrame == frameFile.numFrames(); + } + /** * Returns a new reference to the {@link FrameFile} that this channel is reading from. Callers should close this * reference when done reading. diff --git a/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java index c0a0c872faec..29bf0945adbe 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java @@ -150,12 +150,16 @@ private static class Selector extends ObjectColumnSelector private final Memory memory; private final ReadableFieldPointer fieldPointer; private final ComplexMetricSerde serde; + @SuppressWarnings("rawtypes") + private final Class clazz; private Selector(Memory memory, ReadableFieldPointer fieldPointer, ComplexMetricSerde serde) { this.memory = memory; this.fieldPointer = fieldPointer; this.serde = serde; + //noinspection deprecation + this.clazz = serde.getObjectStrategy().getClazz(); } @Nullable @@ -169,7 +173,8 @@ public T getObject() @Override public Class classOfObject() { - return serde.getExtractor().extractedClass(); + //noinspection unchecked + return clazz; } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java index 7059d76f391d..23f3a7d7432d 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java @@ -124,6 +124,7 @@ private static class ComplexFrameColumn extends ObjectColumnAccessorBase impleme { private final Frame frame; private final ComplexMetricSerde serde; + private final Class clazz; private final Memory memory; private final long startOfOffsetSection; private final long startOfDataSection; @@ -138,6 +139,8 @@ private ComplexFrameColumn( { this.frame = frame; this.serde = serde; + //noinspection deprecation + this.clazz = serde.getObjectStrategy().getClazz(); this.memory = memory; this.startOfOffsetSection = startOfOffsetSection; this.startOfDataSection = startOfDataSection; @@ -158,7 +161,7 @@ public Object getObject() @Override public Class classOfObject() { - return serde.getExtractor().extractedClass(); + return clazz; } @Override diff --git a/processing/src/main/java/org/apache/druid/guice/NestedDataModule.java b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java similarity index 68% rename from processing/src/main/java/org/apache/druid/guice/NestedDataModule.java rename to processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java index daaf4ff2f65a..dc9701e7d8b3 100644 --- a/processing/src/main/java/org/apache/druid/guice/NestedDataModule.java +++ b/processing/src/main/java/org/apache/druid/guice/BuiltInTypesModule.java @@ -25,6 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.inject.Binder; import com.google.inject.Provides; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.initialization.DruidModule; import org.apache.druid.segment.DefaultColumnFormatConfig; import org.apache.druid.segment.DimensionHandler; @@ -38,11 +39,19 @@ import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; +import javax.annotation.Nullable; import java.util.Collections; import java.util.List; -public class NestedDataModule implements DruidModule +public class BuiltInTypesModule implements DruidModule { + /** + * Initialized with a default value so tests can just get it via {@link #getStringMultiValueHandlingMode} without any + * explicit initialization. In production, this default may be overridden if a value is configured via + * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}. + */ + private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY; + @Override public List getJacksonModules() { @@ -53,14 +62,15 @@ public List getJacksonModules() public void configure(Binder binder) { registerSerde(); - // binding our side effect class to the lifecycle causes registerHandler to be called on service start, allowing - // use of the config to get the system default format version - LifecycleModule.register(binder, SideEffectHandlerRegisterer.class); + // binding our side effect classes to the lifecycle causes the initDimensionHandlerAndMvHandlingMode to be + // called on service start, allowing use of the config to get the system default format version and string multi + // value handling mode. + LifecycleModule.register(binder, SideEffectRegisterer.class); } @Provides @LazySingleton - public SideEffectHandlerRegisterer registerHandler(DefaultColumnFormatConfig formatsConfig) + public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig) { if (formatsConfig.getNestedColumnFormatVersion() != null && formatsConfig.getNestedColumnFormatVersion() == 4) { DimensionHandlerUtils.registerDimensionHandlerProvider( @@ -73,7 +83,25 @@ public SideEffectHandlerRegisterer registerHandler(DefaultColumnFormatConfig for new NestedCommonFormatHandlerProvider() ); } - return new SideEffectHandlerRegisterer(); + + setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode()); + return new SideEffectRegisterer(); + } + + private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String stringMultiValueHandlingMode) + { + if (stringMultiValueHandlingMode != null) { + STRING_MV_MODE = DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); + } + } + + /** + * @return the configured string multi value handling mode from the system config if set; otherwise, returns + * the default. + */ + public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode() + { + return STRING_MV_MODE; } public static List getJacksonModulesList() @@ -126,13 +154,15 @@ public DimensionHandler get(Stri return new NestedDataColumnHandlerV4(dimensionName); } } + /** - * this is used as a vehicle to register the correct version of the system default nested column handler by side - * effect with the help of binding to {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that - * {@link #registerHandler(DefaultColumnFormatConfig)} can be called with the injected + * this is used as a vehicle to register the correct version of the system default nested column handler and multi + * value handling mode by side effect with the help of binding to + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that + * {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)} can be called with the injected * {@link DefaultColumnFormatConfig}. */ - public static class SideEffectHandlerRegisterer + public static class SideEffectRegisterer { // nothing to see here } diff --git a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java index f85dd0016005..1f478809333b 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java @@ -32,6 +32,7 @@ public class DruidMetrics public static final String TYPE = "type"; public static final String INTERVAL = "interval"; public static final String ID = "id"; + public static final String SUBQUERY_ID = "subQueryId"; public static final String TASK_ID = "taskId"; public static final String GROUP_ID = "groupId"; public static final String STATUS = "status"; diff --git a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java index 3f41fb88c032..4df37f85b354 100644 --- a/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/DefaultColumnFormatConfig.java @@ -21,9 +21,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Objects; public class DefaultColumnFormatConfig @@ -39,16 +41,44 @@ public static void validateNestedFormatVersion(@Nullable Integer formatVersion) } } + private static void validateMultiValueHandlingMode(@Nullable String stringMultiValueHandlingMode) + { + if (stringMultiValueHandlingMode != null) { + try { + DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode); + } + catch (IllegalArgumentException e) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Invalid value[%s] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'." + + " Supported values are [%s].", + stringMultiValueHandlingMode, + Arrays.toString(DimensionSchema.MultiValueHandling.values()) + ); + } + } + } + + @Nullable @JsonProperty("nestedColumnFormatVersion") private final Integer nestedColumnFormatVersion; + @Nullable + @JsonProperty("stringMultiValueHandlingMode") + private final String stringMultiValueHandlingMode; + @JsonCreator public DefaultColumnFormatConfig( - @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion + @JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion, + @JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode ) { + validateNestedFormatVersion(nestedColumnFormatVersion); + validateMultiValueHandlingMode(stringMultiValueHandlingMode); + this.nestedColumnFormatVersion = nestedColumnFormatVersion; - validateNestedFormatVersion(this.nestedColumnFormatVersion); + this.stringMultiValueHandlingMode = stringMultiValueHandlingMode; } @Nullable @@ -58,6 +88,13 @@ public Integer getNestedColumnFormatVersion() return nestedColumnFormatVersion; } + @Nullable + @JsonProperty("stringMultiValueHandlingMode") + public String getStringMultiValueHandlingMode() + { + return stringMultiValueHandlingMode; + } + @Override public boolean equals(Object o) { @@ -68,13 +105,14 @@ public boolean equals(Object o) return false; } DefaultColumnFormatConfig that = (DefaultColumnFormatConfig) o; - return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion); + return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion) + && Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode); } @Override public int hashCode() { - return Objects.hash(nestedColumnFormatVersion); + return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode); } @Override @@ -82,6 +120,7 @@ public String toString() { return "DefaultColumnFormatConfig{" + "nestedColumnFormatVersion=" + nestedColumnFormatVersion + + ", stringMultiValueHandlingMode=" + stringMultiValueHandlingMode + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 923b48ca4784..46d45024437f 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -66,7 +66,7 @@ private static String emptyToNullIfNeeded(@Nullable Object o) private volatile boolean hasMultipleValues = false; public StringDimensionIndexer( - MultiValueHandling multiValueHandling, + @Nullable MultiValueHandling multiValueHandling, boolean hasBitmapIndexes, boolean hasSpatialIndexes, boolean useMaxMemoryEstimates diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java index 4d1bb347f218..56a66c64c066 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java @@ -25,7 +25,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileGenerator; import it.unimi.dsi.fastutil.Hash; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Comparators; @@ -62,7 +62,7 @@ public class NestedDataComplexTypeSerde extends ComplexMetricSerde smileFactory.delegateToTextual(true); final ObjectMapper mapper = new DefaultObjectMapper(smileFactory, null); mapper.getFactory().setCodec(mapper); - mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); OBJECT_MAPPER = mapper; } diff --git a/processing/src/test/java/org/apache/druid/collections/CircularListTest.java b/processing/src/test/java/org/apache/druid/collections/CircularListTest.java new file mode 100644 index 000000000000..51518a254e34 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/collections/CircularListTest.java @@ -0,0 +1,135 @@ +/* + * 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.collections; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; + +public class CircularListTest +{ + @Test + public void testIterateInNaturalOrder() + { + final Set input = ImmutableSet.of("b", "a", "c"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + final List observedElements = new ArrayList<>(); + int cnt = 0; + for (String x : circularList) { + observedElements.add(x); + if (++cnt >= input.size()) { + break; + } + } + Assert.assertEquals(ImmutableList.of("a", "b", "c"), observedElements); + } + + @Test + public void testIterateInReverseOrder() + { + final Set input = ImmutableSet.of(-1, 100, 0, -4); + final CircularList circularList = new CircularList<>(input, Comparator.reverseOrder()); + final List observedElements = new ArrayList<>(); + int cnt = 0; + for (Integer x : circularList) { + observedElements.add(x); + if (++cnt >= 2 * input.size()) { + break; + } + } + + Assert.assertEquals(ImmutableList.of(100, 0, -1, -4, 100, 0, -1, -4), observedElements); + } + + @Test + public void testIteratorResumesFromLastPosition() + { + final Set input = ImmutableSet.of("a", "b", "c", "d", "e", "f"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + + List observedElements = new ArrayList<>(); + int cnt = 0; + for (String element : circularList) { + observedElements.add(element); + if (++cnt >= input.size() / 2) { + break; + } + } + + Assert.assertEquals(ImmutableList.of("a", "b", "c"), observedElements); + + observedElements = new ArrayList<>(); + for (String element : circularList) { + observedElements.add(element); + // Resume and go till the end, and add two more elements looping around + if (++cnt == input.size() + 2) { + break; + } + } + + Assert.assertEquals(ImmutableList.of("d", "e", "f", "a", "b"), observedElements); + } + + @Test + public void testEqualsSet() + { + final Set input = ImmutableSet.of("a", "b", "c"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + Assert.assertTrue(circularList.equalsSet(ImmutableSet.of("b", "a", "c"))); + Assert.assertFalse(circularList.equalsSet(ImmutableSet.of("c"))); + Assert.assertFalse(circularList.equalsSet(ImmutableSet.of("a", "c"))); + } + + @Test + public void testEmptyIterator() + { + final Set input = ImmutableSet.of(); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + final List observedElements = new ArrayList<>(); + + int cnt = 0; + for (String x : circularList) { + observedElements.add(x); + if (++cnt >= input.size()) { + break; + } + } + Assert.assertEquals(ImmutableList.of(), observedElements); + } + + @Test + public void testNextOnEmptyIteratorThrowsException() + { + final Set input = ImmutableSet.of(); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + + final Iterator iterator = circularList.iterator(); + Assert.assertFalse(iterator.hasNext()); + Assert.assertThrows(NoSuchElementException.class, iterator::next); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java new file mode 100644 index 000000000000..9025d1820864 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.processor; + +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +public class ReadableFileFrameChannelTest extends InitializedNullHandlingTest +{ + private static final int ROWS_PER_FRAME = 20; + + private List> allRows; + private FrameReader frameReader; + private FrameFile frameFile; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setUp() throws IOException + { + final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final File file = FrameTestUtil.writeFrameFile( + FrameSequenceBuilder.fromAdapter(adapter) + .frameType(FrameType.ROW_BASED) + .maxRowsPerFrame(ROWS_PER_FRAME) + .frames(), + temporaryFolder.newFile() + ); + allRows = FrameTestUtil.readRowsFromAdapter(adapter, adapter.getRowSignature(), false).toList(); + frameReader = FrameReader.create(adapter.getRowSignature()); + frameFile = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE); + } + + @After + public void tearDown() throws Exception + { + frameFile.close(); + } + + @Test + public void test_fullFile() + { + final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile); + Assert.assertTrue(channel.isEntireFile()); + + FrameTestUtil.assertRowsEqual( + Sequences.simple(allRows), + FrameTestUtil.readRowsFromFrameChannel(channel, frameReader) + ); + + Assert.assertFalse(channel.isEntireFile()); + } + + @Test + public void test_partialFile() + { + final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile, 1, 2); + Assert.assertFalse(channel.isEntireFile()); + + FrameTestUtil.assertRowsEqual( + Sequences.simple(allRows).skip(ROWS_PER_FRAME).limit(ROWS_PER_FRAME), + FrameTestUtil.readRowsFromFrameChannel(channel, frameReader) + ); + + Assert.assertFalse(channel.isEntireFile()); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index cde617a395cc..c45ffa697350 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -37,7 +37,7 @@ import org.apache.druid.frame.segment.FrameSegment; import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; @@ -92,7 +92,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest static { ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final int DEFAULT_ALLOCATOR_CAPACITY = 1_000_000; diff --git a/processing/src/test/java/org/apache/druid/guice/NestedDataModuleTest.java b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java similarity index 65% rename from processing/src/test/java/org/apache/druid/guice/NestedDataModuleTest.java rename to processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java index 59b5b2df4158..23a2d18f39e4 100644 --- a/processing/src/test/java/org/apache/druid/guice/NestedDataModuleTest.java +++ b/processing/src/test/java/org/apache/druid/guice/BuiltInTypesModuleTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Injector; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.segment.DefaultColumnFormatConfig; import org.apache.druid.segment.DimensionHandlerProvider; import org.apache.druid.segment.DimensionHandlerUtils; @@ -35,7 +36,7 @@ import javax.annotation.Nullable; import java.util.Properties; -public class NestedDataModuleTest +public class BuiltInTypesModuleTest { @Nullable private static DimensionHandlerProvider DEFAULT_HANDLER_PROVIDER; @@ -69,12 +70,17 @@ public void testDefaults() Injector gadget = makeInjector(props); // side effects - gadget.getInstance(NestedDataModule.SideEffectHandlerRegisterer.class); + gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class); DimensionHandlerProvider provider = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get( NestedDataComplexTypeSerde.TYPE_NAME ); Assert.assertTrue(provider.get("test") instanceof NestedCommonFormatColumnHandler); + + Assert.assertEquals( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + BuiltInTypesModule.getStringMultiValueHandlingMode() + ); } @Test @@ -82,16 +88,54 @@ public void testOverride() { DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.remove(NestedDataComplexTypeSerde.TYPE_NAME); Properties props = new Properties(); - props.put("druid.indexing.formats.nestedColumnFormatVersion", "4"); + props.setProperty("druid.indexing.formats.nestedColumnFormatVersion", "4"); + props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "sorted_array"); Injector gadget = makeInjector(props); // side effects - gadget.getInstance(NestedDataModule.SideEffectHandlerRegisterer.class); + gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class); DimensionHandlerProvider provider = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get( NestedDataComplexTypeSerde.TYPE_NAME ); Assert.assertTrue(provider.get("test") instanceof NestedDataColumnHandlerV4); + + Assert.assertEquals( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + BuiltInTypesModule.getStringMultiValueHandlingMode() + ); + } + + @Test + public void testOverrideMultiValueHandlingModeCaseInsensitive() + { + final Properties props = new Properties(); + props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "ARRAY"); + final Injector gadget = makeInjector(props); + + gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class); + + Assert.assertEquals( + DimensionSchema.MultiValueHandling.ARRAY, + BuiltInTypesModule.getStringMultiValueHandlingMode() + ); + } + + @Test + public void testInvalidMultiValueHandlingMode() + { + final Properties props = new Properties(); + props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "boo"); + final Injector gadget = makeInjector(props); + + final Exception exception = Assert.assertThrows( + Exception.class, + () -> gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class) + ); + Assert.assertTrue(exception.getMessage().contains( + "Invalid value[boo] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'." + + " Supported values are [[SORTED_ARRAY, SORTED_SET, ARRAY]]." + )); } private Injector makeInjector(Properties props) @@ -104,7 +148,7 @@ private Injector makeInjector(Properties props) binder -> { JsonConfigProvider.bind(binder, "druid.indexing.formats", DefaultColumnFormatConfig.class); }, - new NestedDataModule() + new BuiltInTypesModule() ) ); diff --git a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java index 56b3583031f0..521c00f2f171 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -66,7 +66,7 @@ public static void setupClass() TypeStrategiesTest.NULLABLE_TEST_PAIR_TYPE.getComplexTypeName(), new TypeStrategiesTest.NullableLongPairTypeStrategy() ); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index 1ff564106c58..23215b95d1e4 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -31,7 +31,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -166,7 +166,7 @@ public class NestedDataTestUtils static { JSON_MAPPER = TestHelper.makeJsonMapper(); - JSON_MAPPER.registerModules(NestedDataModule.getJacksonModulesList()); + JSON_MAPPER.registerModules(BuiltInTypesModule.getJacksonModulesList()); } public static List createSimpleSegmentsTsv( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index 77f86bdc4c50..6c0ff3b49511 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; @@ -82,10 +82,10 @@ public NestedDataGroupByQueryTest( String vectorize ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.vectorize = QueryContexts.Vectorize.fromString(vectorize); this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), config, tempFolder ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java index d305cb95444a..f0e581291d36 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; @@ -74,10 +74,10 @@ public NestedGroupByArrayQueryTest( String vectorize ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.vectorize = QueryContexts.Vectorize.fromString(vectorize); this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), config, tempFolder ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java index a35432cc80e3..bdce3d693e88 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/NestedColumnGroupByColumnSelectorStrategyTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByColumnSelectorStrategyFactory; @@ -44,7 +44,7 @@ public class NestedColumnGroupByColumnSelectorStrategyTest extends InitializedNullHandlingTest { static { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } private static final GroupByColumnSelectorStrategyFactory STRATEGY_FACTORY = new GroupByColumnSelectorStrategyFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 64c68c943f52..80fa1fa0832c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -78,8 +78,8 @@ public void teardown() throws IOException public NestedDataScanQueryTest() { - NestedDataModule.registerHandlersAndSerde(); - List mods = NestedDataModule.getJacksonModulesList(); + BuiltInTypesModule.registerHandlersAndSerde(); + List mods = BuiltInTypesModule.getJacksonModulesList(); this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(mods, tempFolder); this.closer = Closer.create(); } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index 815a6ed9951f..f555d81cefe5 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; @@ -98,7 +98,7 @@ public NestedDataTimeseriesQueryTest( ) { this.helper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), tempFolder ); this.segmentsGenerator = segmentsGenerator; diff --git a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java index 392df70e1d7c..cb3b5fbfc9a9 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -72,9 +72,9 @@ public NestedDataTopNQueryTest( BiFunction> segmentGenerator ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.helper = AggregationTestHelper.createTopNQueryAggregationTestHelper( - NestedDataModule.getJacksonModulesList(), + BuiltInTypesModule.getJacksonModulesList(), tempFolder ); this.segmentsGenerator = segmentGenerator; diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index e397267907e6..73a2b6db50a4 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -25,7 +25,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -62,7 +62,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest @BeforeClass public static void setup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java index f259fc1dc237..c4de7e41f2b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DefaultColumnFormatsConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -33,21 +34,23 @@ public class DefaultColumnFormatsConfigTest @Test public void testDefaultsSerde() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(null, null); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); Assert.assertNull(andBack.getNestedColumnFormatVersion()); + Assert.assertNull(andBack.getStringMultiValueHandlingMode()); } @Test public void testDefaultsSerdeOverride() throws JsonProcessingException { - DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(4); + DefaultColumnFormatConfig defaultColumnFormatConfig = new DefaultColumnFormatConfig(4, "ARRAY"); String there = MAPPER.writeValueAsString(defaultColumnFormatConfig); DefaultColumnFormatConfig andBack = MAPPER.readValue(there, DefaultColumnFormatConfig.class); Assert.assertEquals(defaultColumnFormatConfig, andBack); Assert.assertEquals(4, (int) andBack.getNestedColumnFormatVersion()); + Assert.assertEquals(DimensionSchema.MultiValueHandling.ARRAY.toString(), andBack.getStringMultiValueHandlingMode()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index e670ae10003d..359d2de58cb7 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -25,7 +25,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -61,7 +61,7 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest @BeforeClass public static void setup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java index 8bfcfdefa4f4..847f8a9d1965 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnSchemaTest.java @@ -29,8 +29,8 @@ public class NestedDataColumnSchemaTest { - private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null); - private static final DefaultColumnFormatConfig DEFAULT_CONFIG_V4 = new DefaultColumnFormatConfig(4); + private static final DefaultColumnFormatConfig DEFAULT_CONFIG = new DefaultColumnFormatConfig(null, null); + private static final DefaultColumnFormatConfig DEFAULT_CONFIG_V4 = new DefaultColumnFormatConfig(4, null); private static final ObjectMapper MAPPER; private static final ObjectMapper MAPPER_V4; diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 4a104510f8fd..3143d2f7be74 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -33,7 +33,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -104,7 +104,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest public IncrementalIndexTest(String indexType, String mode, boolean isPreserveExistingMetrics) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.isPreserveExistingMetrics = isPreserveExistingMetrics; indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setSimpleTestingIndexSchema("rollup".equals(mode), isPreserveExistingMetrics, (AggregatorFactory[]) args[0]) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java index 2418941bf73e..f45885790dc3 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java @@ -27,7 +27,7 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.ArrayContainsElementFilter; @@ -1071,7 +1071,7 @@ public void testGetCacheKey() Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); f1 = new ArrayContainsElementFilter( "x", ColumnType.NESTED_DATA, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index d9eb693ab76d..179e4e96d10c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -41,7 +41,7 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.segment.FrameSegment; import org.apache.druid.frame.segment.FrameStorageAdapter; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -445,7 +445,7 @@ public BaseFilterTest( @Before public void setUp() throws Exception { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); String className = getClass().getName(); Map> adaptersForClass = adapterCache.get().get(className); if (adaptersForClass == null) { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index fd87969a042d..97be448e61ab 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -30,7 +30,7 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.ExprEval; @@ -1760,7 +1760,7 @@ public void testGetCacheKey() Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); f1 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3)), null); f1_2 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3)), null); f2 = new EqualityFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3, 4)), null); diff --git a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java index 3c7ff2389c66..5afb9fa04123 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java @@ -26,7 +26,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -140,7 +140,7 @@ public QueryableIndex generate( { // In case we need to generate hyperUniques or json ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde()); - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); final String dataHash = Hashing.sha256() .newHasher() diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java index 4ef0a0c69d9c..77e0470c5486 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.LongMaxAggregator; @@ -50,7 +50,7 @@ public class IncrementalIndexIngestionTest extends InitializedNullHandlingTest public IncrementalIndexIngestionTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setIndexSchema((IncrementalIndexSchema) args[0]) .setMaxRowCount(MAX_ROWS) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index b980c0251b6a..80c8207ed605 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -27,7 +27,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.CloserRule; @@ -56,7 +56,7 @@ public class IncrementalIndexMultiValueSpecTest extends InitializedNullHandlingT public IncrementalIndexMultiValueSpecTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setIndexSchema((IncrementalIndexSchema) args[0]) .setMaxRowCount(10_000) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 0d0f979f715b..09f78e066fb2 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -27,7 +27,7 @@ import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -97,7 +97,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT public IncrementalIndexStorageAdapterTest(String indexType) throws JsonProcessingException { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1_000) diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index cd85768e5acb..1ebd1731c331 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -29,7 +29,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.WrappedRoaringBitmap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -149,7 +149,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index bcc40b333c42..c7b1facec56e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -28,7 +28,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -141,7 +141,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 86455f12c6e8..e46b7893a0de 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -81,8 +81,8 @@ public class NestedFieldColumnSelectorsTest extends InitializedNullHandlingTest public NestedFieldColumnSelectorsTest() { - NestedDataModule.registerHandlersAndSerde(); - List mods = NestedDataModule.getJacksonModulesList(); + BuiltInTypesModule.registerHandlersAndSerde(); + List mods = BuiltInTypesModule.getJacksonModulesList(); this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper( mods, tempFolder diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 5d76e18f94c1..3f237b9396f1 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 4661d72cdbb7..5fd8ddd299ce 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index 5f0c1d020c83..ee7ef05149ea 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -105,7 +105,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Before diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 6ea4509bef7a..4a9423b87bc1 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -24,7 +24,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; @@ -167,7 +167,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest @BeforeClass public static void staticSetup() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); } @Parameterized.Parameters(name = "data = {0}") diff --git a/quidem-ut/README.md b/quidem-ut/README.md new file mode 100644 index 000000000000..5b594a3c8f43 --- /dev/null +++ b/quidem-ut/README.md @@ -0,0 +1,74 @@ + + +# Quidem UT + +Enables to write sql level tests easily. +Can be used to write tests against existing test backends (ComponentSupplier) - by doing so the testcases can be moved closer to the excercised codes. + +## Usage + +### Install java&maven (if needed) + +If you don't have java&maven - one way to set that up is by using sdkman like this: +```bash +# install sdkman +curl -s "https://get.sdkman.io" | bash +# at the end of installation either open a new terminal; or follow the instructions at the end + +# install java&maven +sdk install java 11.0.23-zulu +sdk install maven + +# run mvn to see if it works +mvn --version + +# download druid sourcces (FIXME: change this to the main repo/branch before merging) +git clone --branch quidem-record https://github.com/kgyrtkirk/druid +``` + + + + +### Running these tests + +* CI execution happens by a standard JUnit test `QTest` in this module +* the `dev/quidem` script can be used to run these tests (after the project is built) + +### Launching a test generating broker + +* make sure to build the project first; one way to do that is: + ```bash + mvn install -pl quidem-ut/ -am -DskipTests -Pskip-static-checks + ``` +* launch the broker instance with: + ```bash + mvn exec:exec -pl quidem-ut -Dquidem.record.autostart=true + ``` + * the broker will be running at http://localhost:12345 + * the used test configuration backend can configured by supplying `quidem.uri` + ```bash + mvn exec:exec -pl quidem-ut -Dquidem.uri=druidtest:///?componentSupplier=ThetaSketchComponentSupplier + ``` + * new record files can be started by calling http://localhost:12345/quidem/start + * if `quidem.record.autostart` is omitted recording will not start +* after finished with the usage of the broker ; a new `record-123.iq` file will be in the project's worktree - it will contain all the executed statements + * running `dev/quidem -Dquidem.overwrite` updates the resultsets of all `iq` files around there + * rename the testfile to have a more descriptive name + diff --git a/quidem-ut/pom.xml b/quidem-ut/pom.xml new file mode 100644 index 000000000000..ac48af25c17e --- /dev/null +++ b/quidem-ut/pom.xml @@ -0,0 +1,546 @@ + + + + + 4.0.0 + + druid-quidem-ut + druid-quidem-ut + druid-quidem-ut + + + org.apache.druid + druid + 31.0.0-SNAPSHOT + + + + + confluent + https://packages.confluent.io/maven/ + + + + + druidtest:/// + false + + + + com.amazonaws + amazon-kinesis-producer + 0.13.1 + + + org.apache.avro + avro + + + com.opencsv + opencsv + + + org.apache.commons + commons-lang3 + + + org.apache.httpcomponents + httpcore + + + org.apache.httpcomponents + httpclient + + + com.github.rvesse + airline + + + com.amazonaws + aws-java-sdk-kinesis + ${aws.sdk.version} + + + + com.amazonaws + aws-java-sdk-core + + + + + com.amazonaws + aws-java-sdk-core + ${aws.sdk.version} + + + commons-codec + commons-codec + + + com.amazonaws + aws-java-sdk-s3 + ${aws.sdk.version} + + + org.apache.druid.extensions + druid-orc-extensions + ${project.parent.version} + + + javax.servlet + servlet-api + + + com.squareup.okhttp + okhttp + + + + + org.apache.druid.extensions + druid-parquet-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-avro-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-protobuf-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-s3-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-kinesis-indexing-service + ${project.parent.version} + + + org.apache.druid.extensions + druid-azure-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-google-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-hdfs-storage + ${project.parent.version} + + + com.amazonaws + aws-java-sdk-bundle + + + + + org.apache.druid.extensions + druid-datasketches + ${project.parent.version} + + + org.apache.druid.extensions + druid-histogram + ${project.parent.version} + + + org.apache.druid + druid-aws-common + ${project.parent.version} + + + org.apache.druid + druid-processing + ${project.parent.version} + + + org.apache.druid + druid-sql + ${project.parent.version} + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + + + org.apache.druid.extensions + mysql-metadata-storage + ${project.parent.version} + + + org.apache.druid.extensions + druid-kafka-indexing-service + ${project.parent.version} + + + org.apache.kafka + kafka-clients + + + + + org.apache.druid.extensions + druid-basic-security + ${project.parent.version} + + + org.apache.druid.extensions + druid-lookups-cached-global + ${project.parent.version} + + + org.apache.druid.extensions + druid-testing-tools + ${project.parent.version} + + + org.apache.druid.extensions + simple-client-sslcontext + ${project.parent.version} + + + org.apache.druid.extensions + druid-multi-stage-query + ${project.parent.version} + + + org.apache.druid + druid-services + ${project.parent.version} + + + org.apache.druid + druid-server + ${project.parent.version} + + + commons-io + commons-io + + + joda-time + joda-time + + + com.fasterxml.jackson.core + jackson-annotations + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.kafka + kafka-clients + ${apache.kafka.version} + + + javax.ws.rs + jsr311-api + + + io.netty + netty + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + + + com.google.guava + guava + + + jakarta.validation + jakarta.validation-api + + + jakarta.inject + jakarta.inject-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + com.google.code.findbugs + jsr305 + + + com.github.docker-java + docker-java + + + com.github.docker-java + docker-java-transport-jersey + + + + + com.github.docker-java + docker-java-transport-netty + + + com.github.docker-java + docker-java-api + + + com.github.docker-java + docker-java-core + + + net.hydromatic + quidem + test + + + io.netty + netty-transport-native-kqueue + osx-x86_64 + runtime + + + io.netty + netty-transport-native-kqueue + osx-aarch_64 + runtime + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + com.google.inject.extensions + guice-servlet + + + io.confluent + kafka-schema-registry-client + 6.2.12 + + + org.slf4j + slf4j-log4j12 + + + org.apache.avro + avro + + + com.fasterxml.jackson.core + jackson-databind + + + javax.ws.rs + javax.ws.rs-api + + + javax.ws.rs + javax.ws.rs-api + + + javax.ws.rs + jsr311-api + + + jakarta.ws.rs + jakarta.ws.rs-api + + + + + io.confluent + kafka-protobuf-provider + 6.2.12 + + + com.google.protobuf + protobuf-java + + + + org.apache.druid + druid-sql + test-jar + ${project.parent.version} + + + + org.apache.druid + druid-processing + test-jar + ${project.parent.version} + + + org.apache.druid + druid-server + test-jar + ${project.parent.version} + + + + org.apache.druid.extensions + druid-datasketches + test-jar + ${project.parent.version} + compile + + + + org.slf4j + slf4j-simple + true + + + + org.easymock + easymock + compile + + + org.reflections + reflections + compile + + + + + + org.junit.jupiter + junit-jupiter + + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.jupiter + junit-jupiter-api + + + junit + junit + + + org.apache.calcite.avatica + avatica + + + org.apache.calcite.avatica + avatica-core + + + org.apache.calcite + calcite-testkit + test + + + com.github.os72 + protobuf-dynamic + 0.9.3 + compile + + + com.amazonaws + aws-java-sdk-sts + ${aws.sdk.version} + runtime + + + org.apache.datasketches + datasketches-java + runtime + + + org.hamcrest + hamcrest-all + compile + + + org.hamcrest + hamcrest-core + compile + + + + + + + org.owasp + dependency-check-maven + + true + + + + org.apache.maven.plugins + maven-dependency-plugin + + true + + + + org.codehaus.mojo + exec-maven-plugin + + java + + -classpath + + -Dquidem.uri=${quidem.uri} + -Dquidem.record.autostart=${quidem.record.autostart} + org.apache.druid.quidem.Launcher + + + + + + diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java new file mode 100644 index 000000000000..f853dd317d19 --- /dev/null +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java @@ -0,0 +1,370 @@ +/* + * 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.quidem; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.inject.AbstractModule; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Names; +import org.apache.druid.cli.CliBroker; +import org.apache.druid.cli.QueryJettyServerInitializer; +import org.apache.druid.client.BrokerSegmentWatcherConfig; +import org.apache.druid.client.BrokerServerView; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.selector.CustomTierSelectorStrategyConfig; +import org.apache.druid.client.selector.ServerSelectorStrategy; +import org.apache.druid.client.selector.TierSelectorStrategy; +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.BrokerProcessingModule; +import org.apache.druid.guice.BrokerServiceModule; +import org.apache.druid.guice.BuiltInTypesModule; +import org.apache.druid.guice.CoordinatorDiscoveryModule; +import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.guice.ExpressionModule; +import org.apache.druid.guice.ExtensionsModule; +import org.apache.druid.guice.JacksonConfigManagerModule; +import org.apache.druid.guice.JavaScriptModule; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JoinableFactoryModule; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.LocalDataStorageDruidModule; +import org.apache.druid.guice.MetadataConfigModule; +import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.SegmentWranglerModule; +import org.apache.druid.guice.ServerModule; +import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.guice.ServerViewModule; +import org.apache.druid.guice.StartupLoggingModule; +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.Json; +import org.apache.druid.guice.http.HttpClientModule; +import org.apache.druid.guice.security.AuthenticatorModule; +import org.apache.druid.guice.security.AuthorizerModule; +import org.apache.druid.guice.security.DruidAuthModule; +import org.apache.druid.initialization.CoreInjectorBuilder; +import org.apache.druid.initialization.Log4jShutterDownerModule; +import org.apache.druid.initialization.ServerInjectorBuilder; +import org.apache.druid.initialization.TombstoneDataStorageModule; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.RetryQueryRunnerConfig; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.rpc.guice.ServiceClientModule; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumModule; +import org.apache.druid.server.BrokerQueryResource; +import org.apache.druid.server.ClientInfoResource; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.QuerySchedulerProvider; +import org.apache.druid.server.ResponseContextConfig; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.server.SubqueryGuardrailHelper; +import org.apache.druid.server.SubqueryGuardrailHelperProvider; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.emitter.EmitterModule; +import org.apache.druid.server.http.BrokerResource; +import org.apache.druid.server.http.SelfDiscoveryResource; +import org.apache.druid.server.initialization.AuthorizerMapperModule; +import org.apache.druid.server.initialization.ExternalStorageAccessSecurityModule; +import org.apache.druid.server.initialization.jetty.JettyServerInitializer; +import org.apache.druid.server.initialization.jetty.JettyServerModule; +import org.apache.druid.server.log.NoopRequestLogger; +import org.apache.druid.server.log.RequestLogger; +import org.apache.druid.server.metrics.QueryCountStatsProvider; +import org.apache.druid.server.metrics.SubqueryCountStatsProvider; +import org.apache.druid.server.router.TieredBrokerConfig; +import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.server.security.Escalator; +import org.apache.druid.server.security.TLSCertificateCheckerModule; +import org.apache.druid.sql.calcite.planner.CalciteRulesManager; +import org.apache.druid.sql.calcite.planner.CatalogResolver; +import org.apache.druid.sql.calcite.run.NativeSqlEngine; +import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCache; +import org.apache.druid.sql.calcite.schema.DruidSchemaName; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.apache.druid.sql.calcite.util.SqlTestFramework.Builder; +import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; +import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier; +import org.apache.druid.sql.guice.SqlModule; +import org.apache.druid.storage.StorageConnectorModule; +import org.apache.druid.timeline.PruneLoadSpec; +import org.eclipse.jetty.server.Server; + +import java.io.IOException; +import java.util.List; +import java.util.Properties; + +/** + * A wrapper class to expose a {@link QueryComponentSupplier} as a Broker service. + */ +public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryComponentSupplier +{ + private QueryComponentSupplier delegate; + + public ExposedAsBrokerQueryComponentSupplierWrapper(QueryComponentSupplier delegate) + { + this.delegate = delegate; + } + + @Override + public void gatherProperties(Properties properties) + { + delegate.gatherProperties(properties); + } + + @Override + public void configureGuice(DruidInjectorBuilder builder) + { + } + + @Override + public void configureGuice(CoreInjectorBuilder builder, List overrideModules) + { + delegate.configureGuice(builder); + + installForServerModules(builder); + + overrideModules.addAll(ExposedAsBrokerQueryComponentSupplierWrapper.brokerModules()); + overrideModules.add(new BrokerTestModule()); + builder.add(QuidemCaptureModule.class); + } + + @Override + public QueryRunnerFactoryConglomerate createCongolmerate(Builder builder, Closer closer, ObjectMapper om) + { + return delegate.createCongolmerate(builder, closer, om); + } + + @Override + public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(QueryRunnerFactoryConglomerate conglomerate, + JoinableFactoryWrapper joinableFactory, Injector injector) + { + return delegate.createQuerySegmentWalker(conglomerate, joinableFactory, injector); + } + + @Override + public SqlEngine createEngine(QueryLifecycleFactory qlf, ObjectMapper objectMapper, Injector injector) + { + return delegate.createEngine(qlf, objectMapper, injector); + } + + @Override + public void configureJsonMapper(ObjectMapper mapper) + { + delegate.configureJsonMapper(mapper); + } + + @Override + public JoinableFactoryWrapper createJoinableFactoryWrapper(LookupExtractorFactoryContainerProvider lookupProvider) + { + return delegate.createJoinableFactoryWrapper(lookupProvider); + } + + @Override + public void finalizeTestFramework(SqlTestFramework sqlTestFramework) + { + delegate.finalizeTestFramework(sqlTestFramework); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + + @Override + public PlannerComponentSupplier getPlannerComponentSupplier() + { + return delegate.getPlannerComponentSupplier(); + } + + public static class BrokerTestModule extends AbstractModule + { + @Override + protected void configure() + { + bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER); + bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR); + bind(RequestLogger.class).toInstance(new NoopRequestLogger()); + bind(String.class) + .annotatedWith(DruidSchemaName.class) + .toInstance(CalciteTests.DRUID_SCHEMA_NAME); + bind(QuerySchedulerProvider.class).in(LazySingleton.class); + bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); + bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); + } + + @Provides + @LazySingleton + public BrokerSegmentMetadataCache provideCache() + { + return null; + } + + @Provides + @LazySingleton + public Properties getProps() + { + Properties localProps = new Properties(); + localProps.put("druid.enableTlsPort", "false"); + localProps.put("druid.zk.service.enabled", "false"); + localProps.put("druid.plaintextPort", "12345"); + localProps.put("druid.host", "localhost"); + localProps.put("druid.broker.segment.awaitInitializationOnStart", "false"); + return localProps; + } + + @Provides + @LazySingleton + public SqlEngine createMockSqlEngine( + final QuerySegmentWalker walker, + final QueryRunnerFactoryConglomerate conglomerate, + @Json ObjectMapper jsonMapper) + { + return new NativeSqlEngine(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), jsonMapper); + } + + @Provides + @LazySingleton + DruidNodeDiscoveryProvider getDruidNodeDiscoveryProvider() + { + final DruidNode coordinatorNode = CalciteTests.mockCoordinatorNode(); + return CalciteTests.mockDruidNodeDiscoveryProvider(coordinatorNode); + } + } + + /** + * Closely related to {@link CoreInjectorBuilder#forServer()} + */ + private void installForServerModules(CoreInjectorBuilder builder) + { + + builder.add( + new Log4jShutterDownerModule(), + new LifecycleModule(), + ExtensionsModule.SecondaryModule.class, + new DruidAuthModule(), + TLSCertificateCheckerModule.class, + EmitterModule.class, + HttpClientModule.global(), + HttpClientModule.escalatedGlobal(), + new HttpClientModule("druid.broker.http", Client.class, true), + new HttpClientModule("druid.broker.http", EscalatedClient.class, true), + new CuratorModule(), + new AnnouncerModule(), + new SegmentWriteOutMediumModule(), + new ServerModule(), + new StorageNodeModule(), + new JettyServerModule(), + new ExpressionModule(), + new BuiltInTypesModule(), + new DiscoveryModule(), + new ServerViewModule(), + new MetadataConfigModule(), + new DerbyMetadataStorageDruidModule(), + new JacksonConfigManagerModule(), + new CoordinatorDiscoveryModule(), + new LocalDataStorageDruidModule(), + new TombstoneDataStorageModule(), + new JavaScriptModule(), + new AuthenticatorModule(), + new AuthorizerModule(), + new AuthorizerMapperModule(), + new StartupLoggingModule(), + new ExternalStorageAccessSecurityModule(), + new ServiceClientModule(), + new StorageConnectorModule(), + new SqlModule(), + ServerInjectorBuilder.registerNodeRoleModule(ImmutableSet.of()) + ); + } + + /** + * Closely related to {@link CliBroker#getModules}. + */ + static List brokerModules() + { + return ImmutableList.of( + new BrokerProcessingModule(), + new QueryRunnerFactoryModule(), + new SegmentWranglerModule(), + new JoinableFactoryModule(), + new BrokerServiceModule(), + binder -> { + + binder.bindConstant().annotatedWith(Names.named("serviceName")).to( + TieredBrokerConfig.DEFAULT_BROKER_SERVICE_NAME + ); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8082); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8282); + binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(true); + binder.bind(ResponseContextConfig.class).toInstance(ResponseContextConfig.newConfig(false)); + + binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.broker.select", TierSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.broker.select.tier.custom", CustomTierSelectorStrategyConfig.class); + JsonConfigProvider.bind(binder, "druid.broker.balancer", ServerSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.broker.retryPolicy", RetryQueryRunnerConfig.class); + JsonConfigProvider.bind(binder, "druid.broker.segment", BrokerSegmentWatcherConfig.class); + JsonConfigProvider.bind(binder, "druid.broker.internal.query.config", InternalQueryConfig.class); + binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); + + binder.bind(BrokerQueryResource.class).in(LazySingleton.class); + Jerseys.addResource(binder, BrokerQueryResource.class); + binder.bind(SubqueryGuardrailHelper.class).toProvider(SubqueryGuardrailHelperProvider.class); + binder.bind(QueryCountStatsProvider.class).to(BrokerQueryResource.class).in(LazySingleton.class); + binder.bind(SubqueryCountStatsProvider.class).toInstance(new SubqueryCountStatsProvider()); + Jerseys.addResource(binder, BrokerResource.class); + Jerseys.addResource(binder, ClientInfoResource.class); + + LifecycleModule.register(binder, BrokerQueryResource.class); + + LifecycleModule.register(binder, Server.class); + binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.BROKER)); + + binder.bind(String.class) + .annotatedWith(DruidSchemaName.class) + .toInstance(CalciteTests.DRUID_SCHEMA_NAME); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); + } + ); + } +} diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/Launcher.java b/quidem-ut/src/main/java/org/apache/druid/quidem/Launcher.java new file mode 100644 index 000000000000..b185084e0a75 --- /dev/null +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/Launcher.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.quidem; + +import com.google.common.base.Stopwatch; +import org.apache.druid.cli.GuiceRunnable; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.ConfigurationInstance; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.SqlTestFrameworkConfigStore; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; + +import java.util.Map.Entry; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +public class Launcher +{ + static final SqlTestFrameworkConfigStore CONFIG_STORE = new SqlTestFrameworkConfigStore( + x -> new ExposedAsBrokerQueryComponentSupplierWrapper(x) + ); + private static final String QUIDEM_URI = "quidem.uri"; + private static Logger log = new Logger(Launcher.class); + private final SqlTestFramework framework; + private final ConfigurationInstance configurationInstance; + private Lifecycle lifecycle; + + public Launcher(String uri) throws Exception + { + SqlTestFrameworkConfig config = SqlTestFrameworkConfig.fromURL(uri); + configurationInstance = CONFIG_STORE.getConfigurationInstance(config); + framework = configurationInstance.framework; + } + + public void start() throws Exception + { + lifecycle = GuiceRunnable.initLifecycle(framework.injector(), log); + if (withAutoStart()) { + callQuidemStart(); + } + } + + private void callQuidemStart() throws Exception + { + CloseableHttpClient client = HttpClients.createDefault(); + HttpGet request = new HttpGet("http://localhost:12345/quidem/start"); + request.addHeader("Content-Type", "application/json"); + client.execute(request); + } + + private boolean withAutoStart() + { + return Boolean.valueOf(System.getProperty("quidem.record.autostart", "false")); + } + + public void shutdown() + { + lifecycle.stop(); + } + + public static void main(String[] args) throws Exception + { + String quidemUri = System.getProperty(QUIDEM_URI, "druidtest:///"); + Properties p = System.getProperties(); + for (Entry entry : p.entrySet()) { + Object key = entry.getKey(); + if (key.toString().startsWith("quidem")) { + log.info("[%s] -> %s", key, entry.getValue()); + } + } + log.info("Starting Quidem with URI[%s]", quidemUri); + Stopwatch stopwatch = Stopwatch.createStarted(); + Launcher launcher = new Launcher(quidemUri); + log.info("Framework creation time: %d ms", stopwatch.elapsed(TimeUnit.MILLISECONDS)); + launcher.start(); + log.info("Total time to launch: %d ms", stopwatch.elapsed(TimeUnit.MILLISECONDS)); + launcher.lifecycle.join(); + } +} diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureModule.java b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureModule.java new file mode 100644 index 000000000000..17d9009994e7 --- /dev/null +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureModule.java @@ -0,0 +1,34 @@ +/* + * 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.quidem; + +import com.google.inject.Binder; +import com.google.inject.Module; +import org.apache.druid.guice.Jerseys; + +public class QuidemCaptureModule implements Module +{ + + @Override + public void configure(Binder binder) + { + Jerseys.addResource(binder, QuidemCaptureResource.class); + } +} diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureResource.java b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureResource.java new file mode 100644 index 000000000000..68d2f31ac81a --- /dev/null +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemCaptureResource.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.quidem; + +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.sql.hook.DruidHookDispatcher; + +import javax.inject.Named; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +import java.io.File; +import java.net.URI; + +@Path("/quidem") +@LazySingleton +public class QuidemCaptureResource +{ + public static final File RECORD_PATH = ProjectPathUtils + .getPathFromProjectRoot("quidem-ut/src/test/quidem/org.apache.druid.quidem.QTest"); + private URI quidemURI; + private QuidemRecorder recorder = null; + private DruidHookDispatcher hookDispatcher; + + @Inject + public QuidemCaptureResource(@Named("quidem") URI quidemURI, DruidHookDispatcher hookDispatcher) + { + this.quidemURI = quidemURI; + this.hookDispatcher = hookDispatcher; + } + + @GET + @Path("/start") + @Produces(MediaType.TEXT_PLAIN) + public synchronized String start() + { + stopIfRunning(); + recorder = new QuidemRecorder( + quidemURI, + hookDispatcher, + genRecordFilePath() + ); + return recorder.toString(); + } + + private File genRecordFilePath() + { + String fileName = StringUtils.format("record-%d.iq", System.currentTimeMillis()); + return new File(RECORD_PATH, fileName); + } + + private synchronized void stopIfRunning() + { + if (recorder != null) { + recorder.close(); + recorder = null; + } + + } +} diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemRecorder.java b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemRecorder.java new file mode 100644 index 000000000000..5f2d6a192ed1 --- /dev/null +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/QuidemRecorder.java @@ -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. + */ + +package org.apache.druid.quidem; + +import org.apache.druid.sql.hook.DruidHook; +import org.apache.druid.sql.hook.DruidHookDispatcher; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.Date; + +public class QuidemRecorder implements AutoCloseable, DruidHook +{ + private PrintStream printStream; + private File file; + private DruidHookDispatcher hookDispatcher; + + public QuidemRecorder(URI quidemURI, DruidHookDispatcher hookDispatcher, File file) + { + this.hookDispatcher = hookDispatcher; + this.file = file; + try { + this.printStream = new PrintStream(new FileOutputStream(file), true, StandardCharsets.UTF_8.name()); + } + catch (UnsupportedEncodingException | FileNotFoundException e) { + throw new RuntimeException(e); + } + printStream.println("#started " + new Date()); + printStream.println("!use " + quidemURI); + printStream.println("!set outputformat mysql"); + hookDispatcher.register(DruidHook.SQL, this); + } + + @Override + public synchronized void close() + { + if (printStream != null) { + printStream.close(); + printStream = null; + } + hookDispatcher.unregister(DruidHook.SQL, this); + } + + @Override + public synchronized void invoke(HookKey key, String query) + { + if (DruidHook.SQL.equals(key)) { + printStream.println("# " + new Date()); + printStream.print(query); + printStream.println(";"); + printStream.println("!ok"); + printStream.flush(); + return; + } + } + + @Override + public String toString() + { + return "QuidemRecorder [file=" + file + "]"; + } + +} diff --git a/quidem-ut/src/test/java/org/apache/druid/quidem/LauncherSmokeTest.java b/quidem-ut/src/test/java/org/apache/druid/quidem/LauncherSmokeTest.java new file mode 100644 index 000000000000..6402cd63e433 --- /dev/null +++ b/quidem-ut/src/test/java/org/apache/druid/quidem/LauncherSmokeTest.java @@ -0,0 +1,76 @@ +/* + * 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.quidem; + +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class LauncherSmokeTest +{ + private static Launcher launcher; + + @BeforeClass + public static void setUp() throws Exception + { + launcher = new Launcher("druidtest:///"); + launcher.start(); + } + + @AfterClass + public static void tearDown() + { + launcher.shutdown(); + } + + @Test + public void chkSelectFromFoo() throws Exception + { + CloseableHttpClient client = HttpClients.createDefault(); + HttpPost request = new HttpPost("http://localhost:12345/druid/v2/sql"); + request.addHeader("Content-Type", "application/json"); + request.setEntity(new StringEntity("{\"query\":\"Select * from foo\"}")); + CloseableHttpResponse response = client.execute(request); + assertEquals(200, response.getStatusLine().getStatusCode()); + } + + @Test + public void chkStatusWorks() throws Exception + { + CloseableHttpClient client = HttpClients.createDefault(); + HttpGet request = new HttpGet("http://localhost:12345/status"); + request.addHeader("Content-Type", "application/json"); + CloseableHttpResponse response = client.execute(request); + assertEquals(200, response.getStatusLine().getStatusCode()); + String responseStr = EntityUtils.toString(response.getEntity()); + MatcherAssert.assertThat(responseStr, Matchers.containsString("\"version\":\"")); + } +} diff --git a/quidem-ut/src/test/java/org/apache/druid/quidem/QTest.java b/quidem-ut/src/test/java/org/apache/druid/quidem/QTest.java new file mode 100644 index 000000000000..83167a18a168 --- /dev/null +++ b/quidem-ut/src/test/java/org/apache/druid/quidem/QTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.quidem; + +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +public class QTest extends DruidQuidemTestBase +{ + public QTest() + { + super(); + } + + @Override + protected File getTestRoot() + { + return ProjectPathUtils.getPathFromProjectRoot("quidem-ut/src/test/quidem/" + getClass().getName()); + } + + @Test + public void ensureNoRecordFilesPresent() throws IOException + { + // ensure that the captured ones are saved into this test's input path + assertEquals(QuidemCaptureResource.RECORD_PATH, getTestRoot()); + for (String name : getFileNames()) { + if (name.startsWith("record-")) { + fail("Record file found: " + name); + } + } + } +} diff --git a/quidem-ut/src/test/quidem/org.apache.druid.quidem.QTest/example.iq b/quidem-ut/src/test/quidem/org.apache.druid.quidem.QTest/example.iq new file mode 100644 index 000000000000..3a8105927eee --- /dev/null +++ b/quidem-ut/src/test/quidem/org.apache.druid.quidem.QTest/example.iq @@ -0,0 +1,46 @@ +#started +!use druidtest:/// +!set outputformat mysql + +select * from numfoo; ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +| __time | dim1 | dim2 | dim3 | dim4 | dim5 | dim6 | d1 | d2 | f1 | f2 | l1 | l2 | cnt | m1 | m2 | unique_dim1 | ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | | a | ["a","b"] | a | aa | 1 | 1.0 | | 1.0 | | 7 | | 1 | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2000-01-02 00:00:00.000 | 10.1 | | ["b","c"] | a | ab | 2 | 1.7 | 1.7 | 0.1 | 0.1 | 325323 | 325323 | 1 | 2.0 | 2.0 | "AQAAAQAAAAHNBA==" | +| 2000-01-03 00:00:00.000 | 2 | | d | a | ba | 3 | 0.0 | 0.0 | 0.0 | 0.0 | 0 | 0 | 1 | 3.0 | 3.0 | "AQAAAQAAAAOzAg==" | +| 2001-01-01 00:00:00.000 | 1 | a | | b | ad | 4 | | | | | | | 1 | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | +| 2001-01-02 00:00:00.000 | def | abc | | b | aa | 5 | | | | | | | 1 | 5.0 | 5.0 | "AQAAAQAAAACyEA==" | +| 2001-01-03 00:00:00.000 | abc | | | b | ab | 6 | | | | | | | 1 | 6.0 | 6.0 | "AQAAAQAAAAEkAQ==" | ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +(6 rows) + +!ok +select * from numfoo; ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +| __time | dim1 | dim2 | dim3 | dim4 | dim5 | dim6 | d1 | d2 | f1 | f2 | l1 | l2 | cnt | m1 | m2 | unique_dim1 | ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | | a | ["a","b"] | a | aa | 1 | 1.0 | | 1.0 | | 7 | | 1 | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2000-01-02 00:00:00.000 | 10.1 | | ["b","c"] | a | ab | 2 | 1.7 | 1.7 | 0.1 | 0.1 | 325323 | 325323 | 1 | 2.0 | 2.0 | "AQAAAQAAAAHNBA==" | +| 2000-01-03 00:00:00.000 | 2 | | d | a | ba | 3 | 0.0 | 0.0 | 0.0 | 0.0 | 0 | 0 | 1 | 3.0 | 3.0 | "AQAAAQAAAAOzAg==" | +| 2001-01-01 00:00:00.000 | 1 | a | | b | ad | 4 | | | | | | | 1 | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | +| 2001-01-02 00:00:00.000 | def | abc | | b | aa | 5 | | | | | | | 1 | 5.0 | 5.0 | "AQAAAQAAAACyEA==" | +| 2001-01-03 00:00:00.000 | abc | | | b | ab | 6 | | | | | | | 1 | 6.0 | 6.0 | "AQAAAQAAAAEkAQ==" | ++-------------------------+------+------+-----------+------+------+------+-----+-----+-----+-----+--------+--------+-----+-----+-----+--------------------+ +(6 rows) + +!ok +select length(dim1) from numfoo; ++--------+ +| EXPR$0 | ++--------+ +| 0 | +| 1 | +| 1 | +| 3 | +| 3 | +| 4 | ++--------+ +(6 rows) + +!ok diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index ed9e22dfaa29..0f92d99db106 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -32,7 +32,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -92,7 +91,7 @@ public static CompactionConfigValidationResult validateCompactionConfig( { CompactionEngine compactionEngine = newConfig.getEngine() == null ? defaultCompactionEngine : newConfig.getEngine(); if (compactionEngine == CompactionEngine.NATIVE) { - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } else { return compactionConfigSupportedByMSQEngine(newConfig); } @@ -121,11 +120,10 @@ private static CompactionConfigValidationResult compactionConfigSupportedByMSQEn )); } validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext())); - validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() - .orElse(new CompactionConfigValidationResult(true, null)); + .orElse(CompactionConfigValidationResult.success()); } /** @@ -135,22 +133,19 @@ public static CompactionConfigValidationResult validatePartitionsSpecForMSQ(Part { if (!(partitionsSpec instanceof DimensionRangePartitionsSpec || partitionsSpec instanceof DynamicPartitionsSpec)) { - return new CompactionConfigValidationResult( - false, - "Invalid partitionsSpec type[%s] for MSQ engine. Type must be either 'dynamic' or 'range'.", + return CompactionConfigValidationResult.failure( + "MSQ: Invalid partitioning type[%s]. Must be either 'dynamic' or 'range'", partitionsSpec.getClass().getSimpleName() ); } if (partitionsSpec instanceof DynamicPartitionsSpec && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - return new CompactionConfigValidationResult( - false, - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", - ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + return CompactionConfigValidationResult.failure( + "MSQ: 'maxTotalRows' not supported with 'dynamic' partitioning" ); } - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** @@ -162,12 +157,11 @@ public static CompactionConfigValidationResult validateRollupForMSQ( ) { if (metricsSpec != null && isRollup != null && !isRollup) { - return new CompactionConfigValidationResult( - false, - "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine." + return CompactionConfigValidationResult.failure( + "MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified" ); } - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** @@ -179,38 +173,12 @@ public static CompactionConfigValidationResult validateMaxNumTasksForMSQ(Map - !(aggregatorFactory.requiredFields().isEmpty() - || aggregatorFactory.requiredFields().size() == 1 - && aggregatorFactory.requiredFields() - .get(0) - .equals(aggregatorFactory.getName()))) - .findFirst() - .map(aggregatorFactory -> - new CompactionConfigValidationResult( - false, - "Different name[%s] and fieldName(s)[%s] for aggregator unsupported for MSQ engine.", - aggregatorFactory.getName(), - aggregatorFactory.requiredFields() - )).orElse(new CompactionConfigValidationResult(true, null)); + return CompactionConfigValidationResult.success(); } } diff --git a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java index e07ac5ed1155..902d2855a0a5 100644 --- a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java @@ -24,6 +24,7 @@ import org.apache.druid.curator.discovery.DiscoveryModule; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.AnnouncerModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.CoordinatorDiscoveryModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; @@ -34,7 +35,6 @@ import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.LocalDataStorageDruidModule; import org.apache.druid.guice.MetadataConfigModule; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.guice.ServerModule; import org.apache.druid.guice.ServerViewModule; import org.apache.druid.guice.StartupLoggingModule; @@ -113,7 +113,7 @@ public CoreInjectorBuilder forServer() new StorageNodeModule(), new JettyServerModule(), new ExpressionModule(), - new NestedDataModule(), + new BuiltInTypesModule(), new DiscoveryModule(), new ServerViewModule(), new MetadataConfigModule(), diff --git a/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java index 88ab4673aa8a..019fd22807c3 100644 --- a/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java +++ b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java @@ -21,10 +21,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collections; +import java.util.List; import java.util.Map; -import java.util.Objects; /** * Specifies a policy to filter active locks held by a datasource @@ -33,17 +35,20 @@ public class LockFilterPolicy { private final String datasource; private final int priority; + private final List intervals; private final Map context; @JsonCreator public LockFilterPolicy( @JsonProperty("datasource") String datasource, @JsonProperty("priority") int priority, - @JsonProperty("context") Map context + @JsonProperty("intervals") @Nullable List intervals, + @JsonProperty("context") @Nullable Map context ) { this.datasource = datasource; this.priority = priority; + this.intervals = intervals; this.context = context == null ? Collections.emptyMap() : context; } @@ -65,24 +70,10 @@ public Map getContext() return context; } - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LockFilterPolicy that = (LockFilterPolicy) o; - return Objects.equals(datasource, that.datasource) - && priority == that.priority - && Objects.equals(context, that.context); - } - - @Override - public int hashCode() + @Nullable + @JsonProperty + public List getIntervals() { - return Objects.hash(datasource, priority, context); + return intervals; } } diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java index b0aaa54d5bac..1bf942df9f80 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java @@ -201,7 +201,7 @@ Iterable iterateAllUnusedSegmentsForDatasource( */ List getUnusedSegmentIntervals( String dataSource, - DateTime minStartTime, + @Nullable DateTime minStartTime, DateTime maxEndTime, int limit, DateTime maxUsedStatusLastUpdatedTime diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java new file mode 100644 index 000000000000..b2cb90bc0cec --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java @@ -0,0 +1,64 @@ +/* + * 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.segment.indexing; + +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.transform.TransformSpec; + +import javax.annotation.Nullable; + +/** + * Class representing the combined DataSchema of a set of segments, currently used only by Compaction. + */ +public class CombinedDataSchema extends DataSchema +{ + private final boolean hasRolledUpSegments; + + public CombinedDataSchema( + String dataSource, + @Nullable TimestampSpec timestampSpec, + @Nullable DimensionsSpec dimensionsSpec, + AggregatorFactory[] aggregators, + GranularitySpec granularitySpec, + TransformSpec transformSpec, + @Nullable boolean hasRolledUpSegments + ) + { + super( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + null, + null + ); + this.hasRolledUpSegments = hasRolledUpSegments; + } + + public boolean hasRolledUpSegments() + { + return hasRolledUpSegments; + } +} diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index d49ce3909f71..990878eda6e3 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -37,7 +37,9 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DataSource; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.FluentQueryRunner; import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.FrameSignaturePair; @@ -94,6 +96,8 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private static final Logger log = new Logger(ClientQuerySegmentWalker.class); private static final int FRAME_SIZE = 8_000_000; + public static final String ROWS_COUNT_METRIC = "subquery/rows"; + public static final String BYTES_COUNT_METRIC = "subquery/bytes"; private final ServiceEmitter emitter; private final QuerySegmentWalker clusterClient; @@ -447,7 +451,9 @@ private DataSource inlineIfNecessary( maxSubqueryRows, maxSubqueryMemory, useNestedForUnknownTypeInSubquery, - subqueryStatsProvider + subqueryStatsProvider, + !dryRun, + emitter ); } else { // Cannot inline subquery. Attempt to inline one level deeper, and then try again. @@ -553,8 +559,8 @@ private QueryRunner decorateClusterRunner(Query query, QueryRunner * It also plumbs parent query's id and sql id in case the subqueries don't have it set by default * * @param rootDataSource Datasource whose subqueries need to be populated - * @param parentQueryId Parent Query's ID, can be null if do not need to update this in the subqueries - * @param parentSqlQueryId Parent Query's SQL Query ID, can be null if do not need to update this in the subqueries + * @param parentQueryId Parent Query's ID, can be null if it does not need to update this in the subqueries + * @param parentSqlQueryId Parent Query's SQL Query ID, can be null if it does not need to update this in the subqueries * @return DataSource populated with the subqueries */ private DataSource generateSubqueryIds( @@ -642,6 +648,9 @@ private DataSource insertSubqueryIds( } /** + */ + /** + * * Convert the results of a particular query into a materialized (List-based) InlineDataSource. * * @param query the query @@ -651,6 +660,13 @@ private DataSource insertSubqueryIds( * particular master query * @param limit user-configured limit. If negative, will be treated as {@link Integer#MAX_VALUE}. * If zero, this method will throw an error immediately. + * @param memoryLimit User configured byte limit. + * @param useNestedForUnknownTypeInSubquery Uses nested json for unknown types when materializing subquery results + * @param subqueryStatsProvider Statistics about the subquery materialization + * @param emitMetrics Flag to control if the metrics need to be emitted while materializing. The metrics are omitted + * when we are performing a dry run of the query to avoid double reporting the same metric incorrectly + * @param emitter Metrics emitter + * @return Inlined datasource represented by the provided results * @throws ResourceLimitExceededException if the limit is exceeded */ private static > DataSource toInlineDataSource( @@ -662,8 +678,10 @@ private static > DataSource toInlineDataSource( final AtomicBoolean cannotMaterializeToFrames, final int limit, long memoryLimit, - boolean useNestedForUnknownTypeInSubquery, - SubqueryCountStatsProvider subqueryStatsProvider + final boolean useNestedForUnknownTypeInSubquery, + final SubqueryCountStatsProvider subqueryStatsProvider, + final boolean emitMetrics, + final ServiceEmitter emitter ) { final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit; @@ -683,7 +701,9 @@ private static > DataSource toInlineDataSource( toolChest, limitAccumulator, limit, - subqueryStatsProvider + subqueryStatsProvider, + emitMetrics, + emitter ); break; case MEMORY_LIMIT: @@ -699,7 +719,9 @@ private static > DataSource toInlineDataSource( memoryLimitAccumulator, memoryLimit, useNestedForUnknownTypeInSubquery, - subqueryStatsProvider + subqueryStatsProvider, + emitMetrics, + emitter ); if (!maybeDataSource.isPresent()) { cannotMaterializeToFrames.set(true); @@ -716,7 +738,9 @@ private static > DataSource toInlineDataSource( toolChest, limitAccumulator, limit, - subqueryStatsProvider + subqueryStatsProvider, + emitMetrics, + emitter ); } else { subqueryStatsProvider.incrementSubqueriesWithByteLimit(); @@ -739,9 +763,11 @@ private static > Optional materializeR final QueryToolChest toolChest, final AtomicInteger limitAccumulator, final AtomicLong memoryLimitAccumulator, - long memoryLimit, - boolean useNestedForUnknownTypeInSubquery, - final SubqueryCountStatsProvider subqueryStatsProvider + final long memoryLimit, + final boolean useNestedForUnknownTypeInSubquery, + final SubqueryCountStatsProvider subqueryStatsProvider, + final boolean emitMetrics, + final ServiceEmitter emitter ) { Optional> framesOptional; @@ -764,6 +790,8 @@ private static > Optional materializeR startedAccumulating = true; + final int initialSubqueryRows = limitAccumulator.get(); + final long initialSubqueryBytes = memoryLimitAccumulator.get(); frames.forEach( frame -> { limitAccumulator.addAndGet(frame.getFrame().numRows()); @@ -775,6 +803,21 @@ private static > Optional materializeR frameSignaturePairs.add(frame); } ); + if (emitMetrics) { + emitter.emit( + ServiceMetricEvent.builder() + .setDimension(DruidMetrics.ID, query.getId()) + .setDimension(DruidMetrics.SUBQUERY_ID, query.getSubQueryId()) + .setMetric(ROWS_COUNT_METRIC, limitAccumulator.get() - initialSubqueryRows) + ); + + emitter.emit( + ServiceMetricEvent.builder() + .setDimension(DruidMetrics.ID, query.getId()) + .setDimension(DruidMetrics.SUBQUERY_ID, query.getSubQueryId()) + .setMetric(BYTES_COUNT_METRIC, memoryLimitAccumulator.get() - initialSubqueryBytes) + ); + } return Optional.of(new FrameBasedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query))); } catch (UnsupportedColumnTypeException e) { @@ -811,7 +854,9 @@ private static > DataSource materializeResultsAsAr final QueryToolChest toolChest, final AtomicInteger limitAccumulator, final int limit, - final SubqueryCountStatsProvider subqueryStatsProvider + final SubqueryCountStatsProvider subqueryStatsProvider, + boolean emitMetrics, + final ServiceEmitter emitter ) { final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit; @@ -819,6 +864,7 @@ private static > DataSource materializeResultsAsAr final ArrayList resultList = new ArrayList<>(); + final int initialSubqueryRows = limitAccumulator.get(); toolChest.resultsAsArrays(query, results).accumulate( resultList, (acc, in) -> { @@ -830,6 +876,14 @@ private static > DataSource materializeResultsAsAr return acc; } ); + if (emitMetrics) { + emitter.emit( + ServiceMetricEvent.builder() + .setDimension(DruidMetrics.ID, query.getId()) + .setDimension(DruidMetrics.SUBQUERY_ID, query.getSubQueryId()) + .setMetric(ROWS_COUNT_METRIC, limitAccumulator.get() - initialSubqueryRows) + ); + } return InlineDataSource.fromIterable(resultList, signature); } diff --git a/server/src/main/java/org/apache/druid/server/StatusResource.java b/server/src/main/java/org/apache/druid/server/StatusResource.java index e15ddaf43bd1..cf0ed113c47c 100644 --- a/server/src/main/java/org/apache/druid/server/StatusResource.java +++ b/server/src/main/java/org/apache/druid/server/StatusResource.java @@ -24,6 +24,7 @@ import com.google.common.collect.Maps; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.client.DruidServerConfig; +import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.guice.ExtensionsLoader; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.StringUtils; @@ -144,7 +145,7 @@ public Status(Collection modules) private String getDruidVersion() { - return Status.class.getPackage().getImplementationVersion(); + return GuavaUtils.firstNonNull(Status.class.getPackage().getImplementationVersion(), "unknown"); } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java new file mode 100644 index 000000000000..6009dc12cf4c --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.CompactionEngine; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Cluster-level compaction configs. + * All fields of this class are nullable. A non-null value denotes that the + * corresponding field has been explicitly specified. + */ +public class ClusterCompactionConfig +{ + private final Double compactionTaskSlotRatio; + private final Integer maxCompactionTaskSlots; + private final Boolean useAutoScaleSlots; + private final CompactionEngine engine; + + @JsonCreator + public ClusterCompactionConfig( + @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, + @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, + @JsonProperty("engine") @Nullable CompactionEngine engine + ) + { + this.compactionTaskSlotRatio = compactionTaskSlotRatio; + this.maxCompactionTaskSlots = maxCompactionTaskSlots; + this.useAutoScaleSlots = useAutoScaleSlots; + this.engine = engine; + } + + @Nullable + @JsonProperty + public Double getCompactionTaskSlotRatio() + { + return compactionTaskSlotRatio; + } + + @Nullable + @JsonProperty + public Integer getMaxCompactionTaskSlots() + { + return maxCompactionTaskSlots; + } + + @Nullable + @JsonProperty + public Boolean getUseAutoScaleSlots() + { + return useAutoScaleSlots; + } + + @Nullable + @JsonProperty + public CompactionEngine getEngine() + { + return engine; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterCompactionConfig that = (ClusterCompactionConfig) o; + return Objects.equals(compactionTaskSlotRatio, that.compactionTaskSlotRatio) + && Objects.equals(maxCompactionTaskSlots, that.maxCompactionTaskSlots) + && Objects.equals(useAutoScaleSlots, that.useAutoScaleSlots) + && engine == that.engine; + } + + @Override + public int hashCode() + { + return Objects.hash(compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, engine); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java index 88eaa3e923a3..d482903e0d6b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java @@ -23,10 +23,23 @@ public class CompactionConfigValidationResult { + private static final CompactionConfigValidationResult SUCCESS + = new CompactionConfigValidationResult(true, null); + private final boolean valid; private final String reason; - public CompactionConfigValidationResult(boolean valid, String format, Object... args) + public static CompactionConfigValidationResult success() + { + return SUCCESS; + } + + public static CompactionConfigValidationResult failure(String msgFormat, Object... args) + { + return new CompactionConfigValidationResult(false, msgFormat, args); + } + + private CompactionConfigValidationResult(boolean valid, String format, Object... args) { this.valid = valid; this.reason = format == null ? null : StringUtils.format(format, args); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java index c82adfef1e8d..e8265bcdd1af 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java @@ -31,7 +31,7 @@ /** * Manager to fetch and update dynamic configs {@link CoordinatorDynamicConfig} - * and {@link CoordinatorCompactionConfig}. + * and {@link DruidCompactionConfig}. */ public class CoordinatorConfigManager { @@ -71,12 +71,12 @@ public ConfigManager.SetResult setDynamicConfig(CoordinatorDynamicConfig config, ); } - public CoordinatorCompactionConfig getCurrentCompactionConfig() + public DruidCompactionConfig getCurrentCompactionConfig() { - CoordinatorCompactionConfig config = jacksonConfigManager.watch( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.class, - CoordinatorCompactionConfig.empty() + DruidCompactionConfig config = jacksonConfigManager.watch( + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.class, + DruidCompactionConfig.empty() ).get(); return Preconditions.checkNotNull(config, "Got null config from watcher?!"); @@ -91,7 +91,7 @@ public CoordinatorCompactionConfig getCurrentCompactionConfig() * or if the update was successful. */ public ConfigManager.SetResult getAndUpdateCompactionConfig( - UnaryOperator operator, + UnaryOperator operator, AuditInfo auditInfo ) { @@ -102,16 +102,16 @@ public ConfigManager.SetResult getAndUpdateCompactionConfig( tablesConfig.getConfigTable(), MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - CoordinatorCompactionConfig.CONFIG_KEY + DruidCompactionConfig.CONFIG_KEY ); - CoordinatorCompactionConfig current = convertBytesToCompactionConfig(currentBytes); - CoordinatorCompactionConfig updated = operator.apply(current); + DruidCompactionConfig current = convertBytesToCompactionConfig(currentBytes); + DruidCompactionConfig updated = operator.apply(current); if (current.equals(updated)) { return ConfigManager.SetResult.ok(); } else { return jacksonConfigManager.set( - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, currentBytes, updated, auditInfo @@ -119,12 +119,12 @@ public ConfigManager.SetResult getAndUpdateCompactionConfig( } } - public CoordinatorCompactionConfig convertBytesToCompactionConfig(byte[] bytes) + public DruidCompactionConfig convertBytesToCompactionConfig(byte[] bytes) { return jacksonConfigManager.convertByteToConfig( bytes, - CoordinatorCompactionConfig.class, - CoordinatorCompactionConfig.empty() + DruidCompactionConfig.class, + DruidCompactionConfig.empty() ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 767e8218f319..193b53a5d058 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -42,6 +42,12 @@ public class DataSourceCompactionConfig private final String dataSource; private final int taskPriority; private final long inputSegmentSizeBytes; + + public static Builder builder() + { + return new Builder(); + } + /** * The number of input segments is limited because the byte size of a serialized task spec is limited by * org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig.maxZnodeBytes. @@ -227,4 +233,127 @@ public int hashCode() result = 31 * result + Arrays.hashCode(metricsSpec); return result; } + + public static class Builder + { + private String dataSource; + private Integer taskPriority; + private Long inputSegmentSizeBytes; + private Integer maxRowsPerSegment; + private Period skipOffsetFromLatest; + private UserCompactionTaskQueryTuningConfig tuningConfig; + private UserCompactionTaskGranularityConfig granularitySpec; + private UserCompactionTaskDimensionsConfig dimensionsSpec; + private AggregatorFactory[] metricsSpec; + private UserCompactionTaskTransformConfig transformSpec; + private UserCompactionTaskIOConfig ioConfig; + private CompactionEngine engine; + private Map taskContext; + + public DataSourceCompactionConfig build() + { + return new DataSourceCompactionConfig( + dataSource, + taskPriority, + inputSegmentSizeBytes, + maxRowsPerSegment, + skipOffsetFromLatest, + tuningConfig, + granularitySpec, + dimensionsSpec, + metricsSpec, + transformSpec, + ioConfig, + engine, + taskContext + ); + } + + public Builder forDataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Builder withTaskPriority(Integer taskPriority) + { + this.taskPriority = taskPriority; + return this; + } + + public Builder withInputSegmentSizeBytes(Long inputSegmentSizeBytes) + { + this.inputSegmentSizeBytes = inputSegmentSizeBytes; + return this; + } + + @Deprecated + public Builder withMaxRowsPerSegment(Integer maxRowsPerSegment) + { + this.maxRowsPerSegment = maxRowsPerSegment; + return this; + } + + public Builder withSkipOffsetFromLatest(Period skipOffsetFromLatest) + { + this.skipOffsetFromLatest = skipOffsetFromLatest; + return this; + } + + public Builder withTuningConfig( + UserCompactionTaskQueryTuningConfig tuningConfig + ) + { + this.tuningConfig = tuningConfig; + return this; + } + + public Builder withGranularitySpec( + UserCompactionTaskGranularityConfig granularitySpec + ) + { + this.granularitySpec = granularitySpec; + return this; + } + + public Builder withDimensionsSpec( + UserCompactionTaskDimensionsConfig dimensionsSpec + ) + { + this.dimensionsSpec = dimensionsSpec; + return this; + } + + public Builder withMetricsSpec(AggregatorFactory[] metricsSpec) + { + this.metricsSpec = metricsSpec; + return this; + } + + public Builder withTransformSpec( + UserCompactionTaskTransformConfig transformSpec + ) + { + this.transformSpec = transformSpec; + return this; + } + + public Builder withIoConfig(UserCompactionTaskIOConfig ioConfig) + { + this.ioConfig = ioConfig; + return this; + } + + public Builder withEngine(CompactionEngine engine) + { + this.engine = engine; + return this; + } + + public Builder withTaskContext(Map taskContext) + { + this.taskContext = taskContext; + return this; + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java index 3424212446f2..fdbbb57e53ef 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java @@ -20,24 +20,25 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.audit.AuditInfo; import org.joda.time.DateTime; +import java.util.Objects; + /** * A DTO containing audit information for compaction config for a datasource. */ public class DataSourceCompactionConfigAuditEntry { - private final GlobalCompactionConfig globalConfig; + private final ClusterCompactionConfig globalConfig; private final DataSourceCompactionConfig compactionConfig; private final AuditInfo auditInfo; private final DateTime auditTime; @JsonCreator public DataSourceCompactionConfigAuditEntry( - @JsonProperty("globalConfig") GlobalCompactionConfig globalConfig, + @JsonProperty("globalConfig") ClusterCompactionConfig globalConfig, @JsonProperty("compactionConfig") DataSourceCompactionConfig compactionConfig, @JsonProperty("auditInfo") AuditInfo auditInfo, @JsonProperty("auditTime") DateTime auditTime @@ -50,7 +51,7 @@ public DataSourceCompactionConfigAuditEntry( } @JsonProperty - public GlobalCompactionConfig getGlobalConfig() + public ClusterCompactionConfig getGlobalConfig() { return globalConfig; } @@ -73,52 +74,9 @@ public DateTime getAuditTime() return auditTime; } - /** - * A DTO containing compaction config for that affects the entire cluster. - */ - public static class GlobalCompactionConfig + public boolean hasSameConfig(DataSourceCompactionConfigAuditEntry other) { - private final double compactionTaskSlotRatio; - private final int maxCompactionTaskSlots; - private final boolean useAutoScaleSlots; - - @JsonCreator - public GlobalCompactionConfig( - @JsonProperty("compactionTaskSlotRatio") - double compactionTaskSlotRatio, - @JsonProperty("maxCompactionTaskSlots") int maxCompactionTaskSlots, - @JsonProperty("useAutoScaleSlots") boolean useAutoScaleSlots - ) - { - this.compactionTaskSlotRatio = compactionTaskSlotRatio; - this.maxCompactionTaskSlots = maxCompactionTaskSlots; - this.useAutoScaleSlots = useAutoScaleSlots; - } - - @JsonProperty - public double getCompactionTaskSlotRatio() - { - return compactionTaskSlotRatio; - } - - @JsonProperty - public int getMaxCompactionTaskSlots() - { - return maxCompactionTaskSlots; - } - - @JsonProperty - public boolean isUseAutoScaleSlots() - { - return useAutoScaleSlots; - } - - @JsonIgnore - public boolean hasSameConfig(CoordinatorCompactionConfig coordinatorCompactionConfig) - { - return useAutoScaleSlots == coordinatorCompactionConfig.isUseAutoScaleSlots() && - compactionTaskSlotRatio == coordinatorCompactionConfig.getCompactionTaskSlotRatio() && - coordinatorCompactionConfig.getMaxCompactionTaskSlots() == maxCompactionTaskSlots; - } + return Objects.equals(this.compactionConfig, other.compactionConfig) + && Objects.equals(this.globalConfig, other.globalConfig); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java index ceb4be0d8a6c..4ff9d57465b0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java @@ -27,7 +27,7 @@ /** * A utility class to build the config history for a datasource from audit entries for - * {@link CoordinatorCompactionConfig}. The {@link CoordinatorCompactionConfig} contains the entire config for the + * {@link DruidCompactionConfig}. The {@link DruidCompactionConfig} contains the entire config for the * cluster, so this class creates adds audit entires to the history only when a setting for this datasource or a global * setting has changed. */ @@ -41,54 +41,29 @@ public DataSourceCompactionConfigHistory(String dataSource) this.dataSource = dataSource; } - public void add(CoordinatorCompactionConfig coordinatorCompactionConfig, AuditInfo auditInfo, DateTime auditTime) + public void add(DruidCompactionConfig compactionConfig, AuditInfo auditInfo, DateTime auditTime) { - DataSourceCompactionConfigAuditEntry current = auditEntries.isEmpty() ? null : auditEntries.peek(); - DataSourceCompactionConfigAuditEntry newEntry = null; - boolean hasDataSourceCompactionConfig = false; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSource.equals(dataSourceCompactionConfig.getDataSource())) { - hasDataSourceCompactionConfig = true; - if ( - current == null || - ( - !dataSourceCompactionConfig.equals(current.getCompactionConfig()) || - !current.getGlobalConfig().hasSameConfig(coordinatorCompactionConfig) - ) - ) { - current = new DataSourceCompactionConfigAuditEntry( - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - coordinatorCompactionConfig.getCompactionTaskSlotRatio(), - coordinatorCompactionConfig.getMaxCompactionTaskSlots(), - coordinatorCompactionConfig.isUseAutoScaleSlots() - ), - dataSourceCompactionConfig, - auditInfo, - auditTime - ); - newEntry = current; - } - break; - } + final DataSourceCompactionConfigAuditEntry previousEntry = auditEntries.isEmpty() ? null : auditEntries.peek(); + final DataSourceCompactionConfigAuditEntry newEntry = new DataSourceCompactionConfigAuditEntry( + compactionConfig.clusterConfig(), + compactionConfig.findConfigForDatasource(dataSource).orNull(), + auditInfo, + auditTime + ); + + final boolean shouldAddEntry; + if (previousEntry == null) { + shouldAddEntry = newEntry.getCompactionConfig() != null; + } else { + shouldAddEntry = !newEntry.hasSameConfig(previousEntry); } - if (newEntry != null) { - auditEntries.push(newEntry); - } else if (current != null && !hasDataSourceCompactionConfig) { - newEntry = new DataSourceCompactionConfigAuditEntry( - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - coordinatorCompactionConfig.getCompactionTaskSlotRatio(), - coordinatorCompactionConfig.getMaxCompactionTaskSlots(), - coordinatorCompactionConfig.isUseAutoScaleSlots() - ), - null, - auditInfo, - auditTime - ); + + if (shouldAddEntry) { auditEntries.push(newEntry); } } - public List getHistory() + public List getEntries() { return auditEntries; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java similarity index 55% rename from server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java rename to server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index 036c53121e91..7793b55c4b94 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -21,83 +21,84 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; +import com.google.common.base.Optional; import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; -public class CoordinatorCompactionConfig +public class DruidCompactionConfig { public static final String CONFIG_KEY = "coordinator.compaction.config"; - private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; - private static final int DEFAULT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; - private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; - private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; + private static final DruidCompactionConfig EMPTY_INSTANCE + = new DruidCompactionConfig(Collections.emptyList(), null, null, null, null); private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; - private final CompactionEngine compactionEngine; + private final CompactionEngine engine; - public static CoordinatorCompactionConfig from( - CoordinatorCompactionConfig baseConfig, + public DruidCompactionConfig withDatasourceConfigs( List compactionConfigs ) { - return new CoordinatorCompactionConfig( + return new DruidCompactionConfig( compactionConfigs, - baseConfig.compactionTaskSlotRatio, - baseConfig.maxCompactionTaskSlots, - baseConfig.useAutoScaleSlots, - null + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + engine ); } - public static CoordinatorCompactionConfig from( - CoordinatorCompactionConfig baseConfig, - @Nullable Double compactionTaskSlotRatio, - @Nullable Integer maxCompactionTaskSlots, - @Nullable Boolean useAutoScaleSlots + public DruidCompactionConfig withClusterConfig( + ClusterCompactionConfig update ) { - return new CoordinatorCompactionConfig( - baseConfig.compactionConfigs, - compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, - maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, - null + return new DruidCompactionConfig( + this.compactionConfigs, + Configs.valueOrDefault(update.getCompactionTaskSlotRatio(), compactionTaskSlotRatio), + Configs.valueOrDefault(update.getMaxCompactionTaskSlots(), maxCompactionTaskSlots), + Configs.valueOrDefault(update.getUseAutoScaleSlots(), useAutoScaleSlots), + Configs.valueOrDefault(update.getEngine(), engine) ); } - public static CoordinatorCompactionConfig from(List compactionConfigs) + public DruidCompactionConfig withDatasourceConfig(DataSourceCompactionConfig dataSourceConfig) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); + final Map configs = dataSourceToCompactionConfigMap(); + configs.put(dataSourceConfig.getDataSource(), dataSourceConfig); + return withDatasourceConfigs(new ArrayList<>(configs.values())); } - public static CoordinatorCompactionConfig empty() + public static DruidCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); + return EMPTY_INSTANCE; } @JsonCreator - public CoordinatorCompactionConfig( + public DruidCompactionConfig( @JsonProperty("compactionConfigs") List compactionConfigs, @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, - @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine + @JsonProperty("engine") @Nullable CompactionEngine compactionEngine ) { - this.compactionConfigs = compactionConfigs; - this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, DEFAULT_COMPACTION_TASK_RATIO); - this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, DEFAULT_MAX_COMPACTION_TASK_SLOTS); - this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, DEFAULT_USE_AUTO_SCALE_SLOTS); - this.compactionEngine = Configs.valueOrDefault(compactionEngine, DEFAULT_COMPACTION_ENGINE); + this.compactionConfigs = Configs.valueOrDefault(compactionConfigs, Collections.emptyList()); + this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, 0.1); + this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, Integer.MAX_VALUE); + this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, false); + this.engine = Configs.valueOrDefault(compactionEngine, CompactionEngine.NATIVE); } @JsonProperty @@ -127,7 +128,36 @@ public boolean isUseAutoScaleSlots() @JsonProperty public CompactionEngine getEngine() { - return compactionEngine; + return engine; + } + + + // Null-safe getters not used for serialization + public ClusterCompactionConfig clusterConfig() + { + return new ClusterCompactionConfig( + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + engine + ); + } + + public Map dataSourceToCompactionConfigMap() + { + return getCompactionConfigs().stream().collect( + Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity()) + ); + } + + public Optional findConfigForDatasource(String dataSource) + { + for (DataSourceCompactionConfig dataSourceConfig : getCompactionConfigs()) { + if (dataSource.equals(dataSourceConfig.getDataSource())) { + return Optional.of(dataSourceConfig); + } + } + return Optional.absent(); } @Override @@ -139,11 +169,11 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - CoordinatorCompactionConfig that = (CoordinatorCompactionConfig) o; + DruidCompactionConfig that = (DruidCompactionConfig) o; return Double.compare(that.compactionTaskSlotRatio, compactionTaskSlotRatio) == 0 && maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && - compactionEngine == that.compactionEngine && + engine == that.engine && Objects.equals(compactionConfigs, that.compactionConfigs); } @@ -155,7 +185,7 @@ public int hashCode() compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, - compactionEngine + engine ); } @@ -167,7 +197,7 @@ public String toString() ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + - ", compactionEngine=" + compactionEngine + + ", engine=" + engine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 9710bda79b44..5e468182fa71 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -706,7 +706,7 @@ public void run() = metadataManager.segments().getSnapshotOfDataSourcesWithAllUsedSegments(); final CoordinatorDynamicConfig dynamicConfig = metadataManager.configs().getCurrentDynamicConfig(); - final CoordinatorCompactionConfig compactionConfig = metadataManager.configs().getCurrentCompactionConfig(); + final DruidCompactionConfig compactionConfig = metadataManager.configs().getCurrentCompactionConfig(); DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder(coordinatorStartTime) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index ebdbd4f500e8..45f3993caa5e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -67,7 +67,7 @@ private static TreeSet createUsedSegmentsSet(Iterable private final @Nullable TreeSet usedSegments; private final @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final CoordinatorDynamicConfig coordinatorDynamicConfig; - private final CoordinatorCompactionConfig coordinatorCompactionConfig; + private final DruidCompactionConfig compactionConfig; private final SegmentLoadingConfig segmentLoadingConfig; private final CoordinatorRunStats stats; private final BalancerStrategy balancerStrategy; @@ -81,7 +81,7 @@ private DruidCoordinatorRuntimeParams( @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, CoordinatorDynamicConfig coordinatorDynamicConfig, - CoordinatorCompactionConfig coordinatorCompactionConfig, + DruidCompactionConfig compactionConfig, SegmentLoadingConfig segmentLoadingConfig, CoordinatorRunStats stats, BalancerStrategy balancerStrategy, @@ -95,7 +95,7 @@ private DruidCoordinatorRuntimeParams( this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; this.coordinatorDynamicConfig = coordinatorDynamicConfig; - this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.compactionConfig = compactionConfig; this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; @@ -151,9 +151,9 @@ public CoordinatorDynamicConfig getCoordinatorDynamicConfig() return coordinatorDynamicConfig; } - public CoordinatorCompactionConfig getCoordinatorCompactionConfig() + public DruidCompactionConfig getCompactionConfig() { - return coordinatorCompactionConfig; + return compactionConfig; } public SegmentLoadingConfig getSegmentLoadingConfig() @@ -197,7 +197,7 @@ public Builder buildFromExisting() usedSegments, dataSourcesSnapshot, coordinatorDynamicConfig, - coordinatorCompactionConfig, + compactionConfig, segmentLoadingConfig, stats, balancerStrategy, @@ -215,7 +215,7 @@ public static class Builder private @Nullable TreeSet usedSegments; private @Nullable DataSourcesSnapshot dataSourcesSnapshot; private CoordinatorDynamicConfig coordinatorDynamicConfig; - private CoordinatorCompactionConfig coordinatorCompactionConfig; + private DruidCompactionConfig compactionConfig; private SegmentLoadingConfig segmentLoadingConfig; private CoordinatorRunStats stats; private BalancerStrategy balancerStrategy; @@ -225,7 +225,7 @@ private Builder(DateTime coordinatorStartTime) { this.coordinatorStartTime = coordinatorStartTime; this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); - this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); + this.compactionConfig = DruidCompactionConfig.empty(); this.broadcastDatasources = Collections.emptySet(); } @@ -237,7 +237,7 @@ private Builder( @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, CoordinatorDynamicConfig coordinatorDynamicConfig, - CoordinatorCompactionConfig coordinatorCompactionConfig, + DruidCompactionConfig compactionConfig, SegmentLoadingConfig segmentLoadingConfig, CoordinatorRunStats stats, BalancerStrategy balancerStrategy, @@ -251,7 +251,7 @@ private Builder( this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; this.coordinatorDynamicConfig = coordinatorDynamicConfig; - this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.compactionConfig = compactionConfig; this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; @@ -271,7 +271,7 @@ public DruidCoordinatorRuntimeParams build() usedSegments, dataSourcesSnapshot, coordinatorDynamicConfig, - coordinatorCompactionConfig, + compactionConfig, segmentLoadingConfig, stats, balancerStrategy, @@ -367,9 +367,9 @@ public Builder withSegmentLoadingConfig(SegmentLoadingConfig config) return this; } - public Builder withCompactionConfig(CoordinatorCompactionConfig config) + public Builder withCompactionConfig(DruidCompactionConfig config) { - this.coordinatorCompactionConfig = config; + this.compactionConfig = config; return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java index da2f1e1a04a3..3056d1c2bd0e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java @@ -70,6 +70,7 @@ private void collectLoadQueueStats(CoordinatorRunStats stats) stats.add(Stats.SegmentQueue.BYTES_TO_LOAD, rowKey, queuePeon.getSizeOfSegmentsToLoad()); stats.add(Stats.SegmentQueue.NUM_TO_LOAD, rowKey, queuePeon.getSegmentsToLoad().size()); stats.add(Stats.SegmentQueue.NUM_TO_DROP, rowKey, queuePeon.getSegmentsToDrop().size()); + stats.updateMax(Stats.SegmentQueue.LOAD_RATE_KBPS, rowKey, queuePeon.getLoadRateKbps()); queuePeon.getAndResetStats().forEachStat( (stat, key, statValue) -> diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 01f3bc77e9ee..7b2392b8c66c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -49,8 +49,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; @@ -120,7 +120,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { LOG.info("Running CompactSegments duty"); - final CoordinatorCompactionConfig dynamicConfig = params.getCoordinatorCompactionConfig(); + final DruidCompactionConfig dynamicConfig = params.getCompactionConfig(); final int maxCompactionTaskSlots = dynamicConfig.getMaxCompactionTaskSlots(); if (maxCompactionTaskSlots <= 0) { LOG.info("Skipping compaction as maxCompactionTaskSlots is [%d].", maxCompactionTaskSlots); @@ -278,7 +278,8 @@ private Map> getLockedIntervals( { final List lockFilterPolicies = compactionConfigs .stream() - .map(config -> new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), config.getTaskContext())) + .map(config -> + new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) .collect(Collectors.toList()); final Map> datasourceToLockedIntervals = new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); @@ -343,7 +344,7 @@ private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; } - private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig) + private int getCompactionTaskCapacity(DruidCompactionConfig dynamicConfig) { int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java index c231c31c5436..7395be6d8434 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java @@ -26,9 +26,9 @@ import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.apache.druid.server.coordinator.stats.Stats; import org.joda.time.DateTime; @@ -82,12 +82,12 @@ protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) /** * Creates a new compaction config by deleting entries for inactive datasources. */ - private CoordinatorCompactionConfig deleteConfigsForInactiveDatasources( - CoordinatorCompactionConfig current + private DruidCompactionConfig deleteConfigsForInactiveDatasources( + DruidCompactionConfig current ) { // If current compaction config is empty then there is nothing to do - if (CoordinatorCompactionConfig.empty().equals(current)) { + if (DruidCompactionConfig.empty().equals(current)) { log.info("Nothing to do as compaction config is already empty."); return current; } @@ -102,7 +102,7 @@ private CoordinatorCompactionConfig deleteConfigsForInactiveDatasources( .filter(dataSourceCompactionConfig -> activeDatasources.contains(dataSourceCompactionConfig.getDataSource())) .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - return CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(updated.values())); + return current.withDatasourceConfigs(ImmutableList.copyOf(updated.values())); } /** @@ -116,7 +116,7 @@ private int tryDeleteCompactionConfigs() throws RetryableException ConfigManager.SetResult result = configManager.getAndUpdateCompactionConfig( current -> { - final CoordinatorCompactionConfig updated = deleteConfigsForInactiveDatasources(current); + final DruidCompactionConfig updated = deleteConfigsForInactiveDatasources(current); int numCurrentConfigs = current.getCompactionConfigs() == null ? 0 : current.getCompactionConfigs().size(); int numUpdatedConfigs = updated.getCompactionConfigs() == null ? 0 : updated.getCompactionConfigs().size(); compactionConfigRemoved.set(Math.max(0, numCurrentConfigs - numUpdatedConfigs)); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java index 575c320b9e0a..64b61df5e539 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java @@ -20,6 +20,8 @@ package org.apache.druid.server.coordinator.duty; import com.google.common.base.Predicate; +import com.google.common.collect.Sets; +import org.apache.druid.collections.CircularList; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; @@ -40,10 +42,11 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** @@ -58,6 +61,11 @@ * as there can be multiple unused segments with different {@code used_status_last_updated} time. *

*

+ * The datasources to be killed during each cycle are selected from {@link #datasourceCircularKillList}. This state is + * refreshed in a run if the set of datasources to be killed changes. Consecutive duplicate datasources are avoided + * across runs, provided there are other datasources to be killed. + *

+ *

* See {@link org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}. *

*/ @@ -75,18 +83,22 @@ public class KillUnusedSegments implements CoordinatorDuty private final Duration durationToRetain; private final boolean ignoreDurationToRetain; private final int maxSegmentsToKill; + private final Duration bufferPeriod; /** * Used to keep track of the last interval end time that was killed for each * datasource. */ private final Map datasourceToLastKillIntervalEnd; + private DateTime lastKillTime; - private final Duration bufferPeriod; private final SegmentsMetadataManager segmentsMetadataManager; private final OverlordClient overlordClient; + private String prevDatasourceKilled; + private CircularList datasourceCircularKillList; + public KillUnusedSegments( SegmentsMetadataManager segmentsMetadataManager, OverlordClient overlordClient, @@ -94,7 +106,6 @@ public KillUnusedSegments( ) { this.period = killConfig.getCleanupPeriod(); - this.maxSegmentsToKill = killConfig.getMaxSegments(); this.ignoreDurationToRetain = killConfig.isIgnoreDurationToRetain(); this.durationToRetain = killConfig.getDurationToRetain(); @@ -107,8 +118,6 @@ public KillUnusedSegments( } this.bufferPeriod = killConfig.getBufferPeriod(); - datasourceToLastKillIntervalEnd = new ConcurrentHashMap<>(); - log.info( "Kill task scheduling enabled with period[%s], durationToRetain[%s], bufferPeriod[%s], maxSegmentsToKill[%s]", this.period, @@ -119,6 +128,7 @@ public KillUnusedSegments( this.segmentsMetadataManager = segmentsMetadataManager; this.overlordClient = overlordClient; + this.datasourceToLastKillIntervalEnd = new ConcurrentHashMap<>(); } @Override @@ -141,18 +151,27 @@ private DruidCoordinatorRuntimeParams runInternal(final DruidCoordinatorRuntimeP final CoordinatorRunStats stats = params.getCoordinatorStats(); final int availableKillTaskSlots = getAvailableKillTaskSlots(dynamicConfig, stats); - Collection dataSourcesToKill = dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn(); + if (availableKillTaskSlots <= 0) { + log.debug("Skipping KillUnusedSegments because there are no available kill task slots."); + return params; + } - if (availableKillTaskSlots > 0) { - // If no datasource has been specified, all are eligible for killing unused segments - if (CollectionUtils.isNullOrEmpty(dataSourcesToKill)) { - dataSourcesToKill = segmentsMetadataManager.retrieveAllDataSourceNames(); - } + final Set dataSourcesToKill; + if (CollectionUtils.isNullOrEmpty(dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn())) { + dataSourcesToKill = segmentsMetadataManager.retrieveAllDataSourceNames(); + } else { + dataSourcesToKill = dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn(); + } - lastKillTime = DateTimes.nowUtc(); - killUnusedSegments(dataSourcesToKill, availableKillTaskSlots, stats); + if (datasourceCircularKillList == null || + !datasourceCircularKillList.equalsSet(dataSourcesToKill)) { + datasourceCircularKillList = new CircularList<>(dataSourcesToKill, Comparator.naturalOrder()); } + lastKillTime = DateTimes.nowUtc(); + + killUnusedSegments(dataSourcesToKill, availableKillTaskSlots, stats); + // any datasources that are no longer being considered for kill should have their // last kill interval removed from map. datasourceToLastKillIntervalEnd.keySet().retainAll(dataSourcesToKill); @@ -163,30 +182,37 @@ private DruidCoordinatorRuntimeParams runInternal(final DruidCoordinatorRuntimeP * Spawn kill tasks for each datasource in {@code dataSourcesToKill} upto {@code availableKillTaskSlots}. */ private void killUnusedSegments( - @Nullable final Collection dataSourcesToKill, + final Set dataSourcesToKill, final int availableKillTaskSlots, final CoordinatorRunStats stats ) { - if (CollectionUtils.isNullOrEmpty(dataSourcesToKill) || availableKillTaskSlots <= 0) { + if (CollectionUtils.isNullOrEmpty(dataSourcesToKill)) { + log.debug("Skipping KillUnusedSegments because there are no datasources to kill."); stats.add(Stats.Kill.SUBMITTED_TASKS, 0); return; } - final Collection remainingDatasourcesToKill = new ArrayList<>(dataSourcesToKill); + final Set remainingDatasourcesToKill = new HashSet<>(dataSourcesToKill); + int submittedTasks = 0; - for (String dataSource : dataSourcesToKill) { - if (submittedTasks >= availableKillTaskSlots) { - log.info( - "Submitted [%d] kill tasks and reached kill task slot limit [%d].", - submittedTasks, availableKillTaskSlots - ); - break; + for (String dataSource : datasourceCircularKillList) { + if (dataSource.equals(prevDatasourceKilled) && remainingDatasourcesToKill.size() > 1) { + // Skip this dataSource if it's the same as the previous one and there are remaining datasources to kill. + continue; + } else { + prevDatasourceKilled = dataSource; + remainingDatasourcesToKill.remove(dataSource); } + final DateTime maxUsedStatusLastUpdatedTime = DateTimes.nowUtc().minus(bufferPeriod); final Interval intervalToKill = findIntervalForKill(dataSource, maxUsedStatusLastUpdatedTime, stats); if (intervalToKill == null) { datasourceToLastKillIntervalEnd.remove(dataSource); + // If no interval is found for this datasource, either terminate or continue based on remaining datasources to kill. + if (remainingDatasourcesToKill.isEmpty()) { + break; + } continue; } @@ -204,7 +230,11 @@ private void killUnusedSegments( ); ++submittedTasks; datasourceToLastKillIntervalEnd.put(dataSource, intervalToKill.getEnd()); - remainingDatasourcesToKill.remove(dataSource); + + // Termination conditions. + if (remainingDatasourcesToKill.isEmpty() || submittedTasks >= availableKillTaskSlots) { + break; + } } catch (Exception ex) { log.error(ex, "Failed to submit kill task for dataSource[%s] in interval[%s]", dataSource, intervalToKill); @@ -216,8 +246,12 @@ private void killUnusedSegments( } log.info( - "Submitted [%d] kill tasks for [%d] datasources. Remaining datasources to kill: %s", - submittedTasks, dataSourcesToKill.size() - remainingDatasourcesToKill.size(), remainingDatasourcesToKill + "Submitted [%d] kill tasks for [%d] datasources: [%s]. Remaining [%d] datasources to kill: [%s].", + submittedTasks, + dataSourcesToKill.size() - remainingDatasourcesToKill.size(), + Sets.difference(dataSourcesToKill, remainingDatasourcesToKill), + remainingDatasourcesToKill.size(), + remainingDatasourcesToKill ); stats.add(Stats.Kill.SUBMITTED_TASKS, submittedTasks); @@ -230,13 +264,14 @@ private Interval findIntervalForKill( final CoordinatorRunStats stats ) { + final DateTime minStartTime = datasourceToLastKillIntervalEnd.get(dataSource); final DateTime maxEndTime = ignoreDurationToRetain ? DateTimes.COMPARE_DATE_AS_STRING_MAX : DateTimes.nowUtc().minus(durationToRetain); final List unusedSegmentIntervals = segmentsMetadataManager.getUnusedSegmentIntervals( dataSource, - datasourceToLastKillIntervalEnd.get(dataSource), + minStartTime, maxEndTime, maxSegmentsToKill, maxUsedStatusLastUpdatedTime diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java index 52b012a81d46..333d3b0e3056 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java @@ -170,6 +170,12 @@ public long getSizeOfSegmentsToLoad() return queuedSize.get(); } + @Override + public long getLoadRateKbps() + { + return 0; + } + @Override public CoordinatorRunStats getAndResetStats() { @@ -179,7 +185,7 @@ public CoordinatorRunStats getAndResetStats() @Override public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback) { - SegmentHolder segmentHolder = new SegmentHolder(segment, action, callback); + SegmentHolder segmentHolder = new SegmentHolder(segment, action, Duration.ZERO, callback); final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); @@ -193,7 +199,7 @@ public void loadSegment(final DataSegment segment, SegmentAction action, @Nullab @Override public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) { - SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, callback); + SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, Duration.ZERO, callback); final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index cb32f95516b4..a72cb22e1519 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -36,6 +36,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeHandler; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; +import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; @@ -92,6 +93,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final ConcurrentMap segmentsToLoad = new ConcurrentHashMap<>(); private final ConcurrentMap segmentsToDrop = new ConcurrentHashMap<>(); private final Set segmentsMarkedToDrop = ConcurrentHashMap.newKeySet(); + private final LoadingRateTracker loadingRateTracker = new LoadingRateTracker(); /** * Segments currently in queue ordered by priority and interval. This includes @@ -169,11 +171,10 @@ private void doSegmentManagement() synchronized (lock) { final Iterator queuedSegmentIterator = queuedSegments.iterator(); - final long currentTimeMillis = System.currentTimeMillis(); while (newRequests.size() < batchSize && queuedSegmentIterator.hasNext()) { final SegmentHolder holder = queuedSegmentIterator.next(); final DataSegment segment = holder.getSegment(); - if (hasRequestTimedOut(holder, currentTimeMillis)) { + if (holder.hasRequestTimedOut()) { onRequestFailed(holder, "timed out"); queuedSegmentIterator.remove(); if (holder.isLoad()) { @@ -188,9 +189,13 @@ private void doSegmentManagement() activeRequestSegments.add(segment); } } + + if (segmentsToLoad.isEmpty()) { + loadingRateTracker.markBatchLoadingFinished(); + } } - if (newRequests.size() == 0) { + if (newRequests.isEmpty()) { log.trace( "[%s]Found no load/drop requests. SegmentsToLoad[%d], SegmentsToDrop[%d], batchSize[%d].", serverId, segmentsToLoad.size(), segmentsToDrop.size(), config.getBatchSize() @@ -201,6 +206,11 @@ private void doSegmentManagement() try { log.trace("Sending [%d] load/drop requests to Server[%s].", newRequests.size(), serverId); + final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); + if (hasLoadRequests && !loadingRateTracker.isLoadingBatch()) { + loadingRateTracker.markBatchLoadingStarted(); + } + BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); ListenableFuture future = httpClient.go( new Request(HttpMethod.POST, changeRequestURL) @@ -234,9 +244,16 @@ public void onSuccess(InputStream result) return; } + int numSuccessfulLoads = 0; + long successfulLoadSize = 0; for (DataSegmentChangeResponse e : statuses) { switch (e.getStatus().getState()) { case SUCCESS: + if (e.getRequest() instanceof SegmentChangeRequestLoad) { + ++numSuccessfulLoads; + successfulLoadSize += + ((SegmentChangeRequestLoad) e.getRequest()).getSegment().getSize(); + } case FAILED: handleResponseStatus(e.getRequest(), e.getStatus()); break; @@ -248,6 +265,10 @@ public void onSuccess(InputStream result) log.error("Server[%s] returned unknown state in status[%s].", serverId, e.getStatus()); } } + + if (numSuccessfulLoads > 0) { + loadingRateTracker.incrementBytesLoadedInBatch(successfulLoadSize); + } } } catch (Exception ex) { @@ -284,9 +305,7 @@ private void logRequestFailure(Throwable t) log.error( t, "Request[%s] Failed with status[%s]. Reason[%s].", - changeRequestURL, - responseHandler.getStatus(), - responseHandler.getDescription() + changeRequestURL, responseHandler.getStatus(), responseHandler.getDescription() ); } }, @@ -367,7 +386,7 @@ public void stop() if (stopped) { return; } - log.info("Stopping load queue peon for server [%s].", serverId); + log.info("Stopping load queue peon for server[%s].", serverId); stopped = true; // Cancel all queued requests @@ -379,6 +398,7 @@ public void stop() queuedSegments.clear(); activeRequestSegments.clear(); queuedSize.set(0L); + loadingRateTracker.stop(); stats.get().clear(); } } @@ -387,7 +407,7 @@ public void stop() public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback) { if (!action.isLoad()) { - log.warn("Invalid load action [%s] for segment [%s] on server [%s].", action, segment.getId(), serverId); + log.warn("Invalid load action[%s] for segment[%s] on server[%s].", action, segment.getId(), serverId); return; } @@ -407,7 +427,7 @@ public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallb if (holder == null) { log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId()); queuedSize.addAndGet(segment.getSize()); - holder = new SegmentHolder(segment, action, callback); + holder = new SegmentHolder(segment, action, config.getLoadTimeout(), callback); segmentsToLoad.put(segment, holder); queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); @@ -436,7 +456,7 @@ public void dropSegment(DataSegment segment, LoadPeonCallback callback) if (holder == null) { log.trace("Server[%s] to drop segment[%s] queued.", serverId, segment.getId()); - holder = new SegmentHolder(segment, SegmentAction.DROP, callback); + holder = new SegmentHolder(segment, SegmentAction.DROP, config.getLoadTimeout(), callback); segmentsToDrop.put(segment, holder); queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); @@ -481,6 +501,12 @@ public long getSizeOfSegmentsToLoad() return queuedSize.get(); } + @Override + public long getLoadRateKbps() + { + return loadingRateTracker.getMovingAverageLoadRateKbps(); + } + @Override public CoordinatorRunStats getAndResetStats() { @@ -505,19 +531,6 @@ public Set getSegmentsMarkedToDrop() return Collections.unmodifiableSet(segmentsMarkedToDrop); } - /** - * A request is considered to have timed out if the time elapsed since it was - * first sent to the server is greater than the configured load timeout. - * - * @see HttpLoadQueuePeonConfig#getLoadTimeout() - */ - private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis) - { - return holder.isRequestSentToServer() - && currentTimeMillis - holder.getFirstRequestMillis() - > config.getLoadTimeout().getMillis(); - } - private void onRequestFailed(SegmentHolder holder, String failureCause) { log.error( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java index 2a2163563db7..320062514b04 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java @@ -20,15 +20,11 @@ package org.apache.druid.server.coordinator.loading; /** + * Callback executed when the load or drop of a segment completes on a server + * either with success or failure. */ +@FunctionalInterface public interface LoadPeonCallback { - /** - * Ideally, this method is called after the load/drop opertion is successfully done, i.e., the historical node - * removes the zookeeper node from loadQueue and announces/unannouces the segment. However, this method will - * also be called in failure scenarios so for implementations of LoadPeonCallback that care about success it - * is important to take extra measures to ensure that whatever side effects they expect to happen upon success - * have happened. Coordinator will have a complete and correct view of the cluster in the next run period. - */ void execute(boolean success); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java index 49e5f9a7c087..8e9989717ab9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java @@ -54,6 +54,8 @@ public interface LoadQueuePeon long getSizeOfSegmentsToLoad(); + long getLoadRateKbps(); + CoordinatorRunStats getAndResetStats(); /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java new file mode 100644 index 000000000000..218a41df5147 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadingRateTracker.java @@ -0,0 +1,220 @@ +/* + * 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.server.coordinator.loading; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.EvictingQueue; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Stopwatch; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Tracks the current segment loading rate for a single server. + *

+ * The loading rate is computed as a moving average of the last + * {@link #MOVING_AVERAGE_WINDOW_SIZE} segment batches (or more if any batch was + * smaller than {@link #MIN_ENTRY_SIZE_BYTES}). A batch is defined as a set of + * segments added to the load queue together. Usage: + *

    + *
  • Call {@link #markBatchLoadingStarted()} exactly once to indicate start of + * a batch.
  • + *
  • Call {@link #incrementBytesLoadedInBatch(long)} any number of times to + * increment successful loads done in the batch.
  • + *
  • Call {@link #markBatchLoadingFinished()} exactly once to complete the batch.
  • + *
+ * + *
+ *   batchDurationMillis
+ *   = t(load queue becomes empty) - t(first load request in batch is sent to server)
+ *
+ *   batchBytes = total bytes successfully loaded in batch
+ *
+ *   avg loading rate in batch (kbps) = (8 * batchBytes) / batchDurationMillis
+ *
+ *   overall avg loading rate (kbps)
+ *   = (8 * sumOverWindow(batchBytes)) / sumOverWindow(batchDurationMillis)
+ * 
+ *

+ * This class is currently not required to be thread-safe as the caller + * {@link HttpLoadQueuePeon} itself ensures that the write methods of this class + * are only accessed by one thread at a time. + */ +@NotThreadSafe +public class LoadingRateTracker +{ + public static final int MOVING_AVERAGE_WINDOW_SIZE = 10; + + /** + * Minimum size of a single entry in the moving average window = 1 GiB. + */ + public static final long MIN_ENTRY_SIZE_BYTES = 1 << 30; + + private final EvictingQueue window = EvictingQueue.create(MOVING_AVERAGE_WINDOW_SIZE); + + /** + * Total stats for the whole window. This includes the total from the current + * batch as well. + *

+ * Maintained as an atomic reference to ensure computational correctness in + * {@link #getMovingAverageLoadRateKbps()}. Otherwise, it is possible to have + * a state where bytes have been updated for the entry but not time taken + * (or vice versa). + */ + private final AtomicReference windowTotal = new AtomicReference<>(); + + private Entry currentBatchTotal; + private Entry currentTail; + + private final Stopwatch currentBatchDuration = Stopwatch.createUnstarted(); + + /** + * Marks the start of loading of a batch of segments. This should be called when + * the first request in a batch is sent to the server. + */ + public void markBatchLoadingStarted() + { + if (isLoadingBatch()) { + // Do nothing + return; + } + + currentBatchDuration.restart(); + currentBatchTotal = new Entry(); + + // Add a fresh entry at the tail for this batch + final Entry evictedHead = addNewEntryIfTailIsFull(); + if (evictedHead != null) { + final Entry delta = new Entry(); + delta.bytes -= evictedHead.bytes; + delta.millisElapsed -= evictedHead.millisElapsed; + + windowTotal.updateAndGet(delta::incrementBy); + } + } + + /** + * @return if a batch of segments is currently being loaded. + */ + public boolean isLoadingBatch() + { + return currentBatchDuration.isRunning(); + } + + /** + * Adds the given number of bytes to the total data successfully loaded in the + * current batch. This causes an update of the current load rate. + * + * @throws DruidException if called without making a prior call to + * {@link #markBatchLoadingStarted()}. + */ + public void incrementBytesLoadedInBatch(long loadedBytes) + { + incrementBytesLoadedInBatch(loadedBytes, currentBatchDuration.millisElapsed()); + } + + @VisibleForTesting + void incrementBytesLoadedInBatch(final long bytes, final long batchDurationMillis) + { + if (!isLoadingBatch()) { + throw DruidException.defensive("markBatchLoadingStarted() must be called before tracking load progress."); + } + + final Entry delta = new Entry(); + delta.bytes = bytes; + delta.millisElapsed = batchDurationMillis - currentBatchTotal.millisElapsed; + + currentTail.incrementBy(delta); + currentBatchTotal.incrementBy(delta); + windowTotal.updateAndGet(delta::incrementBy); + } + + /** + * Marks the end of loading of a batch of segments. This method should be called + * when all the requests in the batch have been processed by the server. + */ + public void markBatchLoadingFinished() + { + if (isLoadingBatch()) { + currentBatchDuration.reset(); + currentBatchTotal = null; + } + } + + /** + * Stops this rate tracker and resets its current state. + */ + public void stop() + { + window.clear(); + windowTotal.set(null); + currentTail = null; + currentBatchTotal = null; + currentBatchDuration.reset(); + } + + /** + * Moving average load rate in kbps (1000 bits per second). + */ + public long getMovingAverageLoadRateKbps() + { + final Entry overallTotal = windowTotal.get(); + if (overallTotal == null || overallTotal.millisElapsed <= 0) { + return 0; + } else { + return (8 * overallTotal.bytes) / overallTotal.millisElapsed; + } + } + + /** + * Adds a fresh entry to the queue if the current tail entry is already full. + * + * @return Old head of the queue if it was evicted, null otherwise. + */ + private Entry addNewEntryIfTailIsFull() + { + final Entry oldHead = window.peek(); + + if (currentTail == null || currentTail.bytes >= MIN_ENTRY_SIZE_BYTES) { + currentTail = new Entry(); + window.add(currentTail); + } + + // Compare if the oldHead and the newHead are the same object (not equals) + final Entry newHead = window.peek(); + return newHead == oldHead ? null : oldHead; + } + + private static class Entry + { + long bytes; + long millisElapsed; + + Entry incrementBy(Entry delta) + { + if (delta != null) { + this.bytes += delta.bytes; + this.millisElapsed += delta.millisElapsed; + } + return this; + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java index ce199d654303..9f4a181699dd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java @@ -21,18 +21,20 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.timeline.DataSegment; +import org.joda.time.Duration; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; /** * Represents a segment queued for a load or drop operation in a LoadQueuePeon. @@ -57,14 +59,17 @@ public class SegmentHolder implements Comparable private final DataSegmentChangeRequest changeRequest; private final SegmentAction action; + private final Duration requestTimeout; + // Guaranteed to store only non-null elements private final List callbacks = new ArrayList<>(); - private final AtomicLong firstRequestMillis = new AtomicLong(0); + private final Stopwatch sinceRequestSentToServer = Stopwatch.createUnstarted(); private int runsInQueue = 0; public SegmentHolder( DataSegment segment, SegmentAction action, + Duration requestTimeout, @Nullable LoadPeonCallback callback ) { @@ -76,6 +81,7 @@ public SegmentHolder( if (callback != null) { callbacks.add(callback); } + this.requestTimeout = requestTimeout; } public DataSegment getSegment() @@ -124,17 +130,20 @@ public List getCallbacks() public void markRequestSentToServer() { - firstRequestMillis.compareAndSet(0L, System.currentTimeMillis()); - } - - public boolean isRequestSentToServer() - { - return firstRequestMillis.get() > 0; + if (!sinceRequestSentToServer.isRunning()) { + sinceRequestSentToServer.start(); + } } - public long getFirstRequestMillis() + /** + * A request is considered to have timed out if the time elapsed since it was + * first sent to the server is greater than the configured load timeout. + * + * @see HttpLoadQueuePeonConfig#getLoadTimeout() + */ + public boolean hasRequestTimedOut() { - return firstRequestMillis.get(); + return sinceRequestSentToServer.millisElapsed() > requestTimeout.getMillis(); } public int incrementAndGetRunsInQueue() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java index 5412230c2cc0..92b2acc052bc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -87,7 +87,7 @@ public long getSegmentStat(CoordinatorStat stat, String tier, String datasource) public long get(CoordinatorStat stat) { - return get(stat, RowKey.EMPTY); + return get(stat, RowKey.empty()); } public long get(CoordinatorStat stat, RowKey rowKey) @@ -196,7 +196,7 @@ public void clear() public void add(CoordinatorStat stat, long value) { - add(stat, RowKey.EMPTY, value); + add(stat, RowKey.empty(), value); } public void add(CoordinatorStat stat, RowKey rowKey, long value) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java index b0ee0a2d1f7f..874ac79b1d1b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java @@ -29,7 +29,7 @@ */ public class RowKey { - public static final RowKey EMPTY = new RowKey(Collections.emptyMap()); + private static final RowKey EMPTY = new RowKey(Collections.emptyMap()); private final Map values; private final int hashCode; @@ -52,6 +52,11 @@ public static RowKey of(Dimension dimension, String value) return with(dimension, value).build(); } + public static RowKey empty() + { + return EMPTY; + } + public Map getValues() { return values; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index e7b901e3ed9d..10873d894261 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -71,6 +71,8 @@ public static class SegmentQueue = CoordinatorStat.toDebugAndEmit("bytesToLoad", "segment/loadQueue/size"); public static final CoordinatorStat NUM_TO_DROP = CoordinatorStat.toDebugAndEmit("numToDrop", "segment/dropQueue/count"); + public static final CoordinatorStat LOAD_RATE_KBPS + = CoordinatorStat.toDebugAndEmit("loadRateKbps", "segment/loading/rateKbps"); public static final CoordinatorStat ASSIGNED_ACTIONS = CoordinatorStat.toDebugAndEmit("assignedActions", "segment/loadQueue/assigned"); diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 0bba5cf63fa1..7bd4ee851244 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -19,6 +19,7 @@ package org.apache.druid.server.http; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; @@ -33,13 +34,15 @@ import org.apache.druid.error.NotFound; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.servlet.http.HttpServletRequest; @@ -59,9 +62,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; -import java.util.function.Function; import java.util.function.UnaryOperator; -import java.util.stream.Collectors; @Path("/druid/coordinator/v1/config/compaction") @ResourceFilters(ConfigResourceFilter.class) @@ -69,7 +70,7 @@ public class CoordinatorCompactionConfigsResource { private static final Logger LOG = new Logger(CoordinatorCompactionConfigsResource.class); private static final long UPDATE_RETRY_DELAY = 1000; - static final int UPDATE_NUM_RETRY = 5; + static final int MAX_UPDATE_RETRIES = 5; private final CoordinatorConfigManager configManager; private final AuditManager auditManager; @@ -92,6 +93,45 @@ public Response getCompactionConfig() } @POST + @Path("/cluster") + @Consumes(MediaType.APPLICATION_JSON) + public Response updateClusterCompactionConfig( + ClusterCompactionConfig updatePayload, + @Context HttpServletRequest req + ) + { + UnaryOperator operator = current -> { + final DruidCompactionConfig newConfig = current.withClusterConfig(updatePayload); + + final List datasourceConfigs = newConfig.getCompactionConfigs(); + if (CollectionUtils.isNullOrEmpty(datasourceConfigs) + || current.getEngine() == newConfig.getEngine()) { + return newConfig; + } + + // Validate all the datasource configs against the new engine + for (DataSourceCompactionConfig datasourceConfig : datasourceConfigs) { + CompactionConfigValidationResult validationResult = + ClientCompactionRunnerInfo.validateCompactionConfig(datasourceConfig, newConfig.getEngine()); + if (!validationResult.isValid()) { + throw InvalidInput.exception( + "Cannot update engine to [%s] as it does not support" + + " compaction config of DataSource[%s]. Reason[%s].", + newConfig.getEngine(), datasourceConfig.getDataSource(), validationResult.getReason() + ); + } + } + + return newConfig; + }; + return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + } + + /** + * @deprecated in favor of {@link #updateClusterCompactionConfig}. + */ + @POST + @Deprecated @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) public Response setCompactionTaskLimit( @@ -101,39 +141,32 @@ public Response setCompactionTaskLimit( @Context HttpServletRequest req ) { - UnaryOperator operator = - current -> CoordinatorCompactionConfig.from( - current, + return updateClusterCompactionConfig( + new ClusterCompactionConfig( compactionTaskSlotRatio, maxCompactionTaskSlots, - useAutoScaleSlots - ); - return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + useAutoScaleSlots, + null + ), + req + ); } @POST @Consumes(MediaType.APPLICATION_JSON) - public Response addOrUpdateCompactionConfig( + public Response addOrUpdateDatasourceCompactionConfig( final DataSourceCompactionConfig newConfig, @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { - final CoordinatorCompactionConfig newCompactionConfig; - final Map newConfigs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + UnaryOperator callable = current -> { CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig(newConfig, current.getEngine()); - if (!validationResult.isValid()) { + if (validationResult.isValid()) { + return current.withDatasourceConfig(newConfig); + } else { throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); } - // Don't persist config with the default engine if engine not specified, to enable update of the default. - newConfigs.put(newConfig.getDataSource(), newConfig); - newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); - - return newCompactionConfig; }; return updateConfigHelper( callable, @@ -144,20 +177,15 @@ public Response addOrUpdateCompactionConfig( @GET @Path("/{dataSource}") @Produces(MediaType.APPLICATION_JSON) - public Response getCompactionConfig(@PathParam("dataSource") String dataSource) + public Response getDatasourceCompactionConfig(@PathParam("dataSource") String dataSource) { - final CoordinatorCompactionConfig current = configManager.getCurrentCompactionConfig(); - final Map configs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - - final DataSourceCompactionConfig config = configs.get(dataSource); - if (config == null) { + final DruidCompactionConfig current = configManager.getCurrentCompactionConfig(); + final Optional config = current.findConfigForDatasource(dataSource); + if (config.isPresent()) { + return Response.ok().entity(config.get()).build(); + } else { return Response.status(Response.Status.NOT_FOUND).build(); } - - return Response.ok().entity(config).build(); } @GET @@ -174,25 +202,25 @@ public Response getCompactionConfigHistory( List auditEntries; if (theInterval == null && count != null) { auditEntries = auditManager.fetchAuditHistory( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, count ); } else { auditEntries = auditManager.fetchAuditHistory( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, theInterval ); } DataSourceCompactionConfigHistory history = new DataSourceCompactionConfigHistory(dataSource); for (AuditEntry audit : auditEntries) { - CoordinatorCompactionConfig coordinatorCompactionConfig = configManager.convertBytesToCompactionConfig( + DruidCompactionConfig compactionConfig = configManager.convertBytesToCompactionConfig( audit.getPayload().serialized().getBytes(StandardCharsets.UTF_8) ); - history.add(coordinatorCompactionConfig, audit.getAuditInfo(), audit.getAuditTime()); + history.add(compactionConfig, audit.getAuditInfo(), audit.getAuditTime()); } - return Response.ok(history.getHistory()).build(); + return Response.ok(history.getEntries()).build(); } catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) @@ -209,31 +237,27 @@ public Response deleteCompactionConfig( @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { - final Map configs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - + UnaryOperator callable = current -> { + final Map configs = current.dataSourceToCompactionConfigMap(); final DataSourceCompactionConfig config = configs.remove(dataSource); if (config == null) { throw NotFound.exception("datasource not found"); } - return CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(configs.values())); + return current.withDatasourceConfigs(ImmutableList.copyOf(configs.values())); }; return updateConfigHelper(callable, AuthorizationUtils.buildAuditInfo(req)); } private Response updateConfigHelper( - UnaryOperator configOperator, + UnaryOperator configOperator, AuditInfo auditInfo ) { int attemps = 0; SetResult setResult = null; try { - while (attemps < UPDATE_NUM_RETRY) { + while (attemps < MAX_UPDATE_RETRIES) { setResult = configManager.getAndUpdateCompactionConfig(configOperator, auditInfo); if (setResult.isOk() || !setResult.isRetryable()) { break; diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java index 7a8c3e904dca..bffe2fe99d29 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java @@ -111,14 +111,23 @@ public Response getLoadQueue( return Response.ok( Maps.transformValues( coordinator.getLoadManagementPeons(), - input -> { - long loadSize = input.getSizeOfSegmentsToLoad(); - long dropSize = input.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum(); + peon -> { + long loadSize = peon.getSizeOfSegmentsToLoad(); + long dropSize = peon.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum(); + + // 1 kbps = 1/8 kB/s = 1/8 B/ms + long loadRateKbps = peon.getLoadRateKbps(); + long expectedLoadTimeMillis + = loadRateKbps > 0 && loadSize > 0 + ? (8 * loadSize) / loadRateKbps + : 0; + return new ImmutableMap.Builder<>() - .put("segmentsToLoad", input.getSegmentsToLoad().size()) - .put("segmentsToDrop", input.getSegmentsToDrop().size()) + .put("segmentsToLoad", peon.getSegmentsToLoad().size()) + .put("segmentsToDrop", peon.getSegmentsToDrop().size()) .put("segmentsToLoadSize", loadSize) .put("segmentsToDropSize", dropSize) + .put("expectedLoadTimeMillis", expectedLoadTimeMillis) .build(); } ) diff --git a/server/src/main/java/org/apache/druid/server/metrics/IndexerTaskCountStatsProvider.java b/server/src/main/java/org/apache/druid/server/metrics/IndexerTaskCountStatsProvider.java index 735bc27abb3a..b38b461eb362 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/IndexerTaskCountStatsProvider.java +++ b/server/src/main/java/org/apache/druid/server/metrics/IndexerTaskCountStatsProvider.java @@ -41,4 +41,8 @@ public interface IndexerTaskCountStatsProvider * Map from datasource name to the number of completed tasks by the Indexer. */ Map getWorkerCompletedTasks(); + + Map getWorkerFailedTasks(); + + Map getWorkerSuccessfulTasks(); } diff --git a/server/src/main/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitor.java b/server/src/main/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitor.java index d07311c1a462..bc09e95b5ce9 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitor.java +++ b/server/src/main/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitor.java @@ -72,6 +72,8 @@ public boolean doMonitor(ServiceEmitter emitter) emit(emitter, "worker/task/running/count", indexerStatsProvider.getWorkerRunningTasks()); emit(emitter, "worker/task/assigned/count", indexerStatsProvider.getWorkerAssignedTasks()); emit(emitter, "worker/task/completed/count", indexerStatsProvider.getWorkerCompletedTasks()); + emit(emitter, "worker/task/failed/count", indexerStatsProvider.getWorkerFailedTasks()); + emit(emitter, "worker/task/success/count", indexerStatsProvider.getWorkerSuccessfulTasks()); } return true; } diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index f6d4a2b6e581..7742eaaf1383 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -64,8 +64,7 @@ public void testMSQEngineWithHashedPartitionsSpecIsInvalid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "Invalid partitionsSpec type[HashedPartitionsSpec] for MSQ engine." - + " Type must be either 'dynamic' or 'range'.", + "MSQ: Invalid partitioning type[HashedPartitionsSpec]. Must be either 'dynamic' or 'range'", validationResult.getReason() ); } @@ -85,7 +84,7 @@ public void testMSQEngineWithMaxTotalRowsIsInvalid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "maxTotalRows[100] in DynamicPartitionsSpec not supported for MSQ engine.", + "MSQ: 'maxTotalRows' not supported with 'dynamic' partitioning", validationResult.getReason() ); } @@ -130,7 +129,7 @@ public void testMSQEngineWithQueryGranularityAllIsValid() } @Test - public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() + public void testMSQEngineWithRollupFalseWithMetricsSpecIsInvalid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), @@ -144,30 +143,7 @@ public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine.", - validationResult.getReason() - ); - } - - @Test - public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() - { - // Aggregators having different input and ouput column names are unsupported. - final String inputColName = "added"; - final String outputColName = "sum_added"; - DataSourceCompactionConfig compactionConfig = createCompactionConfig( - new DynamicPartitionsSpec(3, null), - Collections.emptyMap(), - new UserCompactionTaskGranularityConfig(null, null, null), - new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} - ); - CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( - compactionConfig, - CompactionEngine.NATIVE - ); - Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals( - "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", + "MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified", validationResult.getReason() ); } diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 8c3b867e3681..5f5837462667 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -225,7 +225,7 @@ public void test_findLockedIntervals() throws Exception final Map> lockMap = ImmutableMap.of("foo", Collections.singletonList(Intervals.of("2000/2001"))); final List requests = ImmutableList.of( - new LockFilterPolicy("foo", 3, null) + new LockFilterPolicy("foo", 3, null, null) ); serviceClient.expectAndRespond( @@ -246,7 +246,7 @@ public void test_findLockedIntervals() throws Exception public void test_findLockedIntervals_nullReturn() throws Exception { final List requests = ImmutableList.of( - new LockFilterPolicy("foo", 3, null) + new LockFilterPolicy("foo", 3, null, null) ); serviceClient.expectAndRespond( diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 78294fca0c4b..87ddac50f01a 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -684,4 +684,24 @@ public void testWithDimensionSpec() Assert.assertSame(oldSchema.getParserMap(), newSchema.getParserMap()); } + + @Test + public void testCombinedDataSchemaSetsHasRolledUpSegments() + { + CombinedDataSchema schema = new CombinedDataSchema( + IdUtilsTest.VALID_ID_CHARS, + new TimestampSpec("time", "auto", null), + DimensionsSpec.builder() + .setDimensions( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimA", "dimB", "metric1")) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build(), + null, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + true + ); + Assert.assertTrue(schema.hasRolledUpSegments()); + } } diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index 038fbce7d455..467f375f9f7b 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -32,8 +32,12 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.DataSource; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.Druids; import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.GlobalTableDataSource; @@ -214,6 +218,7 @@ public class ClientQuerySegmentWalkerTest private Closer closer; private QueryRunnerFactoryConglomerate conglomerate; + private final StubServiceEmitter emitter = new StubServiceEmitter(); // Queries that are issued; checked by "testQuery" against its "expectedQueries" parameter. private final List issuedQueries = new ArrayList<>(); @@ -228,6 +233,7 @@ public class ClientQuerySegmentWalkerTest public void setUp() { closer = Closer.create(); + emitter.flush(); conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer); scheduler = new ObservableQueryScheduler( 8, @@ -941,6 +947,113 @@ public void testTimeseriesOnGroupByOnTableErrorTooLarge() testQuery(query, ImmutableList.of(), ImmutableList.of()); } + @Test + public void testMetricsWithMaxSubqueryRowsEnabled() + { + final GroupByQuery subquery = + GroupByQuery.builder() + .setDataSource(FOO) + .setGranularity(Granularities.ALL) + .setInterval(Collections.singletonList(INTERVAL)) + .setDimensions(DefaultDimensionSpec.of("s")) + .build(); + + final TimeseriesQuery query = + (TimeseriesQuery) Druids.newTimeseriesQueryBuilder() + .dataSource(new QueryDataSource(subquery)) + .granularity(Granularities.ALL) + .intervals(Intervals.ONLY_ETERNITY) + .aggregators(new CountAggregatorFactory("cnt")) + .build() + .withId(DUMMY_QUERY_ID); + + testQuery( + query, + new ArrayList<>(ImmutableList.of( + ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")), + ExpectedQuery.local( + query.withDataSource( + InlineDataSource.fromIterable( + ImmutableList.of(new Object[]{"x"}, new Object[]{"y"}, new Object[]{"z"}), + RowSignature.builder().add("s", ColumnType.STRING).build() + ) + ) + ) + )), + ImmutableList.of(new Object[]{Intervals.ETERNITY.getStartMillis(), 3L}) + ); + + List events = emitter.getEvents(); + + for (Event event : events) { + EventMap map = event.toMap(); + if (ClientQuerySegmentWalker.ROWS_COUNT_METRIC.equals(map.get("metric"))) { + Assert.assertTrue(map.containsKey("host")); + Assert.assertTrue(map.containsKey("service")); + Assert.assertEquals(DUMMY_QUERY_ID, map.get(DruidMetrics.ID)); + Assert.assertEquals("1.1", map.get(DruidMetrics.SUBQUERY_ID)); + Assert.assertEquals(3, map.get("value")); + } + } + } + + @Test + public void testMetricsWithMaxSubqueryBytesEnabled() + { + final GroupByQuery subquery = + GroupByQuery.builder() + .setDataSource(FOO) + .setGranularity(Granularities.ALL) + .setInterval(Collections.singletonList(INTERVAL)) + .setDimensions(DefaultDimensionSpec.of("s")) + .build(); + + final TimeseriesQuery query = + (TimeseriesQuery) Druids.newTimeseriesQueryBuilder() + .dataSource(new QueryDataSource(subquery)) + .granularity(Granularities.ALL) + .intervals(Intervals.ONLY_ETERNITY) + .aggregators(new CountAggregatorFactory("cnt")) + .context(ImmutableMap.of(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "10000")) + .build() + .withId(DUMMY_QUERY_ID); + + testQuery( + query, + new ArrayList<>(ImmutableList.of( + ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")), + ExpectedQuery.local( + query.withDataSource( + InlineDataSource.fromIterable( + ImmutableList.of(new Object[]{"x"}, new Object[]{"y"}, new Object[]{"z"}), + RowSignature.builder().add("s", ColumnType.STRING).build() + ) + ) + ) + )), + ImmutableList.of(new Object[]{Intervals.ETERNITY.getStartMillis(), 3L}) + ); + + List events = emitter.getEvents(); + + for (Event event : events) { + EventMap map = event.toMap(); + if (ClientQuerySegmentWalker.ROWS_COUNT_METRIC.equals(map.get("metric"))) { + Assert.assertTrue(map.containsKey("host")); + Assert.assertTrue(map.containsKey("service")); + Assert.assertEquals(DUMMY_QUERY_ID, map.get(DruidMetrics.ID)); + Assert.assertEquals("1.1", map.get(DruidMetrics.SUBQUERY_ID)); + Assert.assertEquals(3, map.get("value")); + } else if (ClientQuerySegmentWalker.BYTES_COUNT_METRIC.equals(map.get("metric"))) { + Assert.assertTrue(map.containsKey("host")); + Assert.assertTrue(map.containsKey("service")); + Assert.assertEquals(DUMMY_QUERY_ID, map.get(DruidMetrics.ID)); + Assert.assertEquals("1.1", map.get(DruidMetrics.SUBQUERY_ID)); + Assert.assertEquals(43L, map.get("value")); + } + } + } + @Test public void testGroupByOnArraysDoubles() { @@ -1545,13 +1658,15 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + Assert.assertEquals(wikiCompactionConfig, firstEntry.getCompactionConfig()); + Assert.assertEquals(auditInfo, firstEntry.getAuditInfo()); + Assert.assertEquals(auditTime, firstEntry.getAuditTime()); + + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertNull(secondEntry.getCompactionConfig()); + Assert.assertEquals(firstEntry.getGlobalConfig(), secondEntry.getGlobalConfig()); + Assert.assertEquals(auditInfo, secondEntry.getAuditInfo()); + Assert.assertEquals(auditTime.plusHours(2), secondEntry.getAuditTime()); } @Test - public void testAddAndDeleteAnotherCompactionConfigShouldNotAddToHistory() + public void testAddDeleteAnotherDatasourceConfigShouldNotAddToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSource)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(1, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); + final DataSourceCompactionConfig koalaCompactionConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.KOALA).build(); + + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(koalaCompactionConfig), + auditInfo, + DateTimes.nowUtc() + ); + wikiAuditHistory.add(DruidCompactionConfig.empty(), auditInfo, DateTimes.nowUtc()); + + Assert.assertTrue(wikiAuditHistory.getEntries().isEmpty()); } @Test - public void testAddDeletedAddCompactionConfigShouldAddAllToHistory() + public void testAddDeleteAddDatasourceConfigShouldAddAllToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSource2)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Mockito.when(compactionConfig.getCompactionConfigs()) - .thenReturn(ImmutableList.of(configForDataSourceWithChange, configForDataSource2)); - target.add(compactionConfig, auditInfo3, AUDIT_TIME_3); - Assert.assertEquals(3, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - auditEntry = target.getHistory().get(2); - Assert.assertEquals(configForDataSourceWithChange, auditEntry.getCompactionConfig()); - Assert.assertEquals(auditInfo3, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_3, auditEntry.getAuditTime()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + wikiAuditHistory.add( + DruidCompactionConfig.empty(), + auditInfo, + auditTime.plusHours(2) + ); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime.plusHours(3) + ); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(3, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry thirdEntry = entries.get(2); + Assert.assertTrue(firstEntry.hasSameConfig(thirdEntry)); } @Test - public void testAddAndChangeCompactionConfigShouldAddBothToHistory() + public void testAddModifyDatasourceConfigShouldAddBothToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSourceWithChange)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(2, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - auditEntry = target.getHistory().get(1); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo2, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_2, auditEntry.getAuditTime()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + + + final DataSourceCompactionConfig updatedWikiConfig + = DataSourceCompactionConfig.builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.hours(5)) + .build(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(updatedWikiConfig), + auditInfo, + auditTime.plusHours(3) + ); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertEquals(firstEntry.getGlobalConfig(), secondEntry.getGlobalConfig()); + + Assert.assertEquals(wikiCompactionConfig, firstEntry.getCompactionConfig()); + Assert.assertEquals(updatedWikiConfig, secondEntry.getCompactionConfig()); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); } @Test - public void testAddAndChangeGlobalSettingsShouldAddTwice() + public void testAddAndModifyClusterConfigShouldAddTwice() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - int newMaxTaskSlots = MAX_COMPACTION_TASK_SLOTS - 1; - Mockito.when(compactionConfig.getMaxCompactionTaskSlots()).thenReturn(newMaxTaskSlots); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(2, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - Assert.assertEquals(MAX_COMPACTION_TASK_SLOTS, auditEntry.getGlobalConfig().getMaxCompactionTaskSlots()); - auditEntry = target.getHistory().get(1); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo2, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_2, auditEntry.getAuditTime()); - Assert.assertEquals(newMaxTaskSlots, auditEntry.getGlobalConfig().getMaxCompactionTaskSlots()); + final DruidCompactionConfig originalConfig + = DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig); + + wikiAuditHistory.add(originalConfig, auditInfo, DateTimes.nowUtc()); + + final DruidCompactionConfig updatedConfig = originalConfig.withClusterConfig( + new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ) + ); + wikiAuditHistory.add(updatedConfig, auditInfo, DateTimes.nowUtc()); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertEquals(secondEntry.getCompactionConfig(), firstEntry.getCompactionConfig()); + + Assert.assertEquals(originalConfig.clusterConfig(), firstEntry.getGlobalConfig()); + Assert.assertEquals(updatedConfig.clusterConfig(), secondEntry.getGlobalConfig()); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); } - @Test - public void testAddCompactionConfigDoesNotHaveDataSourceWithNoHistoryShouldNotAdd() + private static class DS { - target = new DataSourceCompactionConfigHistory(DATASOURCE_NOT_EXISTS); - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Assert.assertTrue(target.getHistory().isEmpty()); + static final String KOALA = "koala"; + static final String WIKI = "wiki"; } - } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index a9334f077a47..67f276f76512 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -42,9 +42,7 @@ import org.joda.time.Duration; import org.joda.time.Period; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; @@ -52,27 +50,15 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest { private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - @Test public void testSerdeBasic() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - null, - null, - new Period(3600), - null, - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -90,21 +76,15 @@ public void testSerdeBasic() throws IOException @Test public void testSerdeWithMaxRowsPerSegment() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - 30, - new Period(3600), - null, - null, - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withMaxRowsPerSegment(30) + .withSkipOffsetFromLatest(new Period(3600)) + .withEngine(CompactionEngine.MSQ) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -121,41 +101,14 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException @Test public void testSerdeWithMaxTotalRows() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 10000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - CompactionEngine.NATIVE, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withEngine(CompactionEngine.NATIVE) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -172,42 +125,14 @@ public void testSerdeWithMaxTotalRows() throws IOException @Test public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - 10000, - new Period(3600), - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 10000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withMaxRowsPerSegment(10000) + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -301,21 +226,14 @@ public void testSerdeUserCompactionTuningConfigWithAppendableIndexSpec() throws @Test public void testSerdeGranularitySpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -332,21 +250,14 @@ public void testSerdeGranularitySpec() throws IOException @Test public void testSerdeGranularitySpecWithQueryGranularity() throws Exception { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, Granularities.YEAR, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, Granularities.YEAR, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -366,21 +277,13 @@ public void testSerdeGranularitySpecWithQueryGranularity() throws Exception @Test public void testSerdeWithNullGranularitySpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -397,21 +300,14 @@ public void testSerdeWithNullGranularitySpec() throws IOException @Test public void testSerdeGranularitySpecWithNullValues() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, null, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -428,21 +324,14 @@ public void testSerdeGranularitySpecWithNullValues() throws IOException @Test public void testSerdeGranularitySpecWithRollup() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, null, true), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, null, true)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -462,21 +351,15 @@ public void testSerdeGranularitySpecWithRollup() throws IOException @Test public void testSerdeIOConfigWithNonNullDropExisting() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withIoConfig(new UserCompactionTaskIOConfig(true)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -494,21 +377,15 @@ public void testSerdeIOConfigWithNonNullDropExisting() throws IOException @Test public void testSerdeIOConfigWithNullDropExisting() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(null), - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withIoConfig(new UserCompactionTaskIOConfig(null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -526,21 +403,18 @@ public void testSerdeIOConfigWithNullDropExisting() throws IOException @Test public void testSerdeDimensionsSpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withDimensionsSpec( + new UserCompactionTaskDimensionsConfig( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo")) + ) + ) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -558,21 +432,14 @@ public void testSerdeDimensionsSpec() throws IOException public void testSerdeTransformSpec() throws IOException { NullHandling.initializeForTests(); - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - null, - new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withTransformSpec(new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null))) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -589,21 +456,14 @@ public void testSerdeTransformSpec() throws IOException @Test public void testSerdeMetricsSpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withMetricsSpec(new AggregatorFactory[]{new CountAggregatorFactory("cnt")}) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java new file mode 100644 index 000000000000..bf13a94e0080 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class DruidCompactionConfigTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerdeDefaultConfig() throws Exception + { + final DruidCompactionConfig defaultConfig = DruidCompactionConfig.empty(); + final String json = MAPPER.writeValueAsString(defaultConfig); + + DruidCompactionConfig deserialized = MAPPER.readValue(json, DruidCompactionConfig.class); + Assert.assertEquals(defaultConfig, deserialized); + } + + @Test + public void testSerdeWithDatasourceConfigs() throws Exception + { + final DruidCompactionConfig config = new DruidCompactionConfig( + Arrays.asList( + DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.hours(1)) + .build(), + DataSourceCompactionConfig + .builder() + .forDataSource(DS.KOALA) + .withSkipOffsetFromLatest(Period.hours(2)) + .build() + ), + null, + null, + null, + CompactionEngine.MSQ + ); + + final String json = MAPPER.writeValueAsString(config); + DruidCompactionConfig deserialized = MAPPER.readValue(json, DruidCompactionConfig.class); + Assert.assertEquals(config, deserialized); + } + + @Test + public void testCopyWithClusterConfig() + { + final DruidCompactionConfig config = DruidCompactionConfig.empty(); + + final ClusterCompactionConfig clusterConfig = new ClusterCompactionConfig(0.5, 10, false, CompactionEngine.MSQ); + final DruidCompactionConfig copy = config.withClusterConfig(clusterConfig); + + Assert.assertEquals(clusterConfig, copy.clusterConfig()); + Assert.assertNotEquals(clusterConfig, config.clusterConfig()); + } + + @Test + public void testCopyWithDatasourceConfigs() + { + final DruidCompactionConfig config = DruidCompactionConfig.empty(); + Assert.assertTrue(config.getCompactionConfigs().isEmpty()); + + final DataSourceCompactionConfig dataSourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).withEngine(CompactionEngine.NATIVE).build(); + final DruidCompactionConfig copy = config.withDatasourceConfigs(Collections.singletonList(dataSourceConfig)); + + Assert.assertEquals(1, copy.getCompactionConfigs().size()); + Assert.assertEquals(dataSourceConfig, copy.findConfigForDatasource(DS.WIKI).orNull()); + } + + private static class DS + { + static final String WIKI = "wiki"; + static final String KOALA = "koala"; + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 8c7e0ae14e5f..801016eb819a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -136,11 +136,11 @@ public void setUp() throws Exception ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); setupServerAndCurator(); curator.start(); @@ -734,11 +734,11 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); DruidDataSource dataSource = new DruidDataSource("dataSource1", Collections.emptyMap()); DataSegment dataSegment = new DataSegment( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java index fcbeeebc726b..bb6d0406c699 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java @@ -62,6 +62,7 @@ public void testCollectedSegmentStats() CoordinatorRunStats stats = params.getCoordinatorStats(); Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.NUM_TO_LOAD)); Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.NUM_TO_DROP)); + Assert.assertTrue(stats.hasStat(Stats.SegmentQueue.LOAD_RATE_KBPS)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 236cfaf7da54..eb03acc48bfa 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -69,8 +69,8 @@ import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -1834,7 +1834,7 @@ private CoordinatorRunStats doCompactSegments( .newBuilder(DateTimes.nowUtc()) .withDataSourcesSnapshot(dataSources) .withCompactionConfig( - new CoordinatorCompactionConfig( + new DruidCompactionConfig( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java index 3d441d9b06d9..3056a75c5649 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java @@ -28,9 +28,9 @@ import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; @@ -107,13 +107,13 @@ public void testRunDoNothingIfCurrentConfigIsEmpty() ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(null); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) - ).thenReturn(CoordinatorCompactionConfig.empty()); + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) + ).thenReturn(DruidCompactionConfig.empty()); final MetadataCleanupConfig config = new MetadataCleanupConfig(true, new Duration("PT6S"), null); @@ -129,14 +129,14 @@ public void testRunDoNothingIfCurrentConfigIsEmpty() Mockito.verify(mockJacksonConfigManager).convertByteToConfig( ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); } @@ -177,24 +177,26 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() null, ImmutableMap.of("key", "val") ); - CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig)); + DruidCompactionConfig originalCurrentConfig = DruidCompactionConfig.empty().withDatasourceConfigs( + ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig) + ); byte[] originalCurrentConfigBytes = {1, 2, 3}; Mockito.when(mockConnector.lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(originalCurrentConfigBytes); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) ).thenReturn(originalCurrentConfig); Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of(activeDatasourceName)); final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass(CoordinatorCompactionConfig.class); + final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass(DruidCompactionConfig.class); Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), oldConfigCaptor.capture(), newConfigCaptor.capture(), ArgumentMatchers.any()) @@ -221,19 +223,19 @@ public void testRunRemoveInactiveDatasourceCompactionConfig() Mockito.verify(mockJacksonConfigManager).convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); Mockito.verify(mockJacksonConfigManager).set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any() ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); @@ -261,24 +263,26 @@ public void testRunRetryForRetryableException() ImmutableMap.of("key", "val") ); - CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig)); + DruidCompactionConfig originalCurrentConfig = DruidCompactionConfig.empty().withDatasourceConfigs( + ImmutableList.of(inactiveDatasourceConfig) + ); byte[] originalCurrentConfigBytes = {1, 2, 3}; Mockito.when(mockConnector.lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(originalCurrentConfigBytes); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) ).thenReturn(originalCurrentConfig); Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of()); Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any()) ).thenReturn( // Return fail result with RetryableException the first three calls to updated set @@ -304,21 +308,21 @@ public void testRunRetryForRetryableException() // Should call convertByteToConfig and lookup (to refresh current compaction config) four times due to RetryableException when failed Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector, Mockito.times(4)).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); // Should call set (to try set new updated compaction config) four times due to RetryableException when failed Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any() ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 9d0f752869e2..f4e8cdb3cd4b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -225,6 +225,151 @@ public void testKillWithMultipleDatasources() validateLastKillStateAndReset(DS2, null); } + /** + * Set up multiple datasources {@link #DS1}, {@link #DS2} and {@link #DS3} with unused segments with 2 kill task + * slots. Running the kill duty each time should pick at least one unique datasource in a round-robin manner. + */ + @Test + public void testRoundRobinKillMultipleDatasources() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(2); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, NEXT_DAY, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, NEXT_MONTH, VERSION, NOW.minusDays(1)); + + createAndAddUnusedSegment(DS2, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, NEXT_DAY, VERSION, NOW.minusDays(1)); + + createAndAddUnusedSegment(DS3, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS3, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS3, NEXT_DAY, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(4, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS3_STAT_KEY)); + Assert.assertEquals(4, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(6, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(6, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(6, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS3_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(8, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(7, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(8, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(5, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + } + + /** + * The set of datasources to kill change in consecutive runs. The kill duty should avoid selecting two + * consecutive datasources across runs as long as there are other datasources to kill. + */ + @Test + public void testRoundRobinKillWhenDatasourcesChange() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(1); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(1, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(1, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + validateLastKillStateAndReset(DS1, new Interval(YEAR_OLD.getStart(), MONTH_OLD.getEnd())); + + createAndAddUnusedSegment(DS2, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, NEXT_DAY, VERSION, NOW.minusDays(1)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(3, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(3, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(4, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + } + + @Test + public void testKillSingleDatasourceMultipleRuns() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(2); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(6, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(6, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + } + /** * The {@code DAY_OLD} and {@code HOUR_OLD} segments are "more recent" in terms of last updated time. * Even though they fall within the umbrella kill interval computed by the duty, the kill task will narrow down to @@ -407,6 +552,36 @@ public void testLowerMaxSegmentsToKill() validateLastKillStateAndReset(DS1, YEAR_OLD); } + @Test + public void testKillDatasourceWithNoUnusedSegmentsInInitialRun() + { + configBuilder.withMaxSegmentsToKill(1); + + // create a datasource but no unused segments yet. + createAndAddUsedSegment(DS1, YEAR_OLD, VERSION); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(10, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(0, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(10, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(0, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(10)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(10)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(2)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(20, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(20, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + validateLastKillStateAndReset(DS1, YEAR_OLD); + } + /** * The kill period is honored after the first indexing run. */ @@ -723,12 +898,7 @@ private void validateLastKillStateAndReset(final String dataSource, @Nullable fi overlordClient.deleteLastKillInterval(dataSource); } - private void createAndAddUnusedSegment( - final String dataSource, - final Interval interval, - final String version, - final DateTime lastUpdatedTime - ) + private DataSegment createAndAddUsedSegment(final String dataSource, final Interval interval, final String version) { final DataSegment segment = createSegment(dataSource, interval, version); try { @@ -737,6 +907,17 @@ private void createAndAddUnusedSegment( catch (IOException e) { throw new RuntimeException(e); } + return segment; + } + + private void createAndAddUnusedSegment( + final String dataSource, + final Interval interval, + final String version, + final DateTime lastUpdatedTime + ) + { + final DataSegment segment = createAndAddUsedSegment(dataSource, interval, version); sqlSegmentsMetadataManager.markSegmentsAsUnused(ImmutableSet.of(segment.getId())); derbyConnectorRule.segments().updateUsedStatusLastUpdated(segment.getId().toString(), lastUpdatedTime); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 16251130e77e..5928f47e0bb2 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -60,9 +60,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -/** - * - */ public class HttpLoadQueuePeonTest { private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -75,26 +72,22 @@ public class HttpLoadQueuePeonTest private TestHttpClient httpClient; private HttpLoadQueuePeon httpLoadQueuePeon; - private BlockingExecutorService processingExecutor; - private BlockingExecutorService callbackExecutor; - - private final List processedSegments = new ArrayList<>(); @Before public void setUp() { httpClient = new TestHttpClient(); - processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); - callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); - processedSegments.clear(); - httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, 10), - new WrappingScheduledExecutorService("HttpLoadQueuePeonTest-%s", processingExecutor, true), - callbackExecutor + new WrappingScheduledExecutorService( + "HttpLoadQueuePeonTest-%s", + httpClient.processingExecutor, + true + ), + httpClient.callbackExecutor ); httpLoadQueuePeon.start(); } @@ -117,13 +110,12 @@ public void testSimple() httpLoadQueuePeon .loadSegment(segments.get(3), SegmentAction.MOVE_TO, markSegmentProcessed(segments.get(3))); - // Send requests to server - processingExecutor.finishAllPendingTasks(); + httpClient.sendRequestToServerAndHandleResponse(); Assert.assertEquals(segments, httpClient.segmentsSentToServer); // Verify that all callbacks are executed - callbackExecutor.finishAllPendingTasks(); - Assert.assertEquals(segments, processedSegments); + httpClient.executeCallbacks(); + Assert.assertEquals(segments, httpClient.processedSegments); } @Test @@ -170,8 +162,7 @@ public void testPriorityOfSegmentAction() Collections.shuffle(actions); actions.forEach(QueueAction::invoke); - // Send one batch of requests to the server - processingExecutor.finishAllPendingTasks(); + httpClient.sendRequestToServerAndHandleResponse(); // Verify that all segments are sent to the server in the expected order Assert.assertEquals(segmentsDay1, httpClient.segmentsSentToServer); @@ -194,7 +185,7 @@ public void testPriorityOfSegmentInterval() Collections.shuffle(segmentsDay2); // Assign segments to the actions in their order of priority - // Priority order: action (drop, priorityLoad, etc), then interval (new then old) + // Order: action (drop, priorityLoad, etc.), then interval (new then old) List actions = Arrays.asList( QueueAction.of(segmentsDay2.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), @@ -212,8 +203,7 @@ public void testPriorityOfSegmentInterval() Collections.shuffle(actions); actions.forEach(QueueAction::invoke); - // Send one batch of requests to the server - processingExecutor.finishNextPendingTask(); + httpClient.sendRequestToServerAndHandleResponse(); // Verify that all segments are sent to the server in the expected order Assert.assertEquals(expectedSegmentOrder, httpClient.segmentsSentToServer); @@ -230,7 +220,7 @@ public void testCancelLoad() Assert.assertTrue(cancelled); Assert.assertEquals(0, httpLoadQueuePeon.getSegmentsToLoad().size()); - Assert.assertTrue(processedSegments.isEmpty()); + Assert.assertTrue(httpClient.processedSegments.isEmpty()); } @Test @@ -244,7 +234,7 @@ public void testCancelDrop() Assert.assertTrue(cancelled); Assert.assertTrue(httpLoadQueuePeon.getSegmentsToDrop().isEmpty()); - Assert.assertTrue(processedSegments.isEmpty()); + Assert.assertTrue(httpClient.processedSegments.isEmpty()); } @Test @@ -254,8 +244,7 @@ public void testCannotCancelRequestSentToServer() httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); - // Send the request to the server - processingExecutor.finishNextPendingTask(); + httpClient.sendRequestToServer(); Assert.assertTrue(httpClient.segmentsSentToServer.contains(segment)); // Segment is still in queue but operation cannot be cancelled @@ -263,8 +252,7 @@ public void testCannotCancelRequestSentToServer() boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); Assert.assertFalse(cancelled); - // Handle response from server - processingExecutor.finishNextPendingTask(); + httpClient.handleResponseFromServer(); // Segment has been removed from queue Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().isEmpty()); @@ -272,8 +260,8 @@ public void testCannotCancelRequestSentToServer() Assert.assertFalse(cancelled); // Execute callbacks and verify segment is fully processed - callbackExecutor.finishAllPendingTasks(); - Assert.assertTrue(processedSegments.contains(segment)); + httpClient.executeCallbacks(); + Assert.assertTrue(httpClient.processedSegments.contains(segment)); } @Test @@ -287,14 +275,59 @@ public void testCannotCancelOperationMultipleTimes() Assert.assertFalse(httpLoadQueuePeon.cancelOperation(segment)); } + @Test + public void testLoadRateIsZeroWhenNoLoadHasFinishedYet() + { + httpLoadQueuePeon.loadSegment(segments.get(0), SegmentAction.LOAD, null); + httpClient.sendRequestToServer(); + Assert.assertEquals(1, httpLoadQueuePeon.getSegmentsToLoad().size()); + Assert.assertEquals(0, httpLoadQueuePeon.getLoadRateKbps()); + } + + @Test + public void testLoadRateIsUnchangedByDrops() throws InterruptedException + { + // Drop a segment after a small delay + final long millisTakenToDropSegment = 10; + httpLoadQueuePeon.dropSegment(segments.get(0), null); + httpClient.sendRequestToServer(); + Thread.sleep(millisTakenToDropSegment); + httpClient.handleResponseFromServer(); + + // Verify that load rate is still zero + Assert.assertEquals(0, httpLoadQueuePeon.getLoadRateKbps()); + } + + @Test + public void testLoadRateIsChangedWhenLoadSucceeds() throws InterruptedException + { + // Load a segment after a small delay + final long millisTakenToLoadSegment = 10; + httpLoadQueuePeon.loadSegment(segments.get(0), SegmentAction.LOAD, null); + httpClient.sendRequestToServer(); + Thread.sleep(millisTakenToLoadSegment); + httpClient.handleResponseFromServer(); + + // Verify that load rate has been updated + long expectedRateKbps = (8 * segments.get(0).getSize()) / millisTakenToLoadSegment; + long observedRateKbps = httpLoadQueuePeon.getLoadRateKbps(); + Assert.assertTrue( + observedRateKbps > expectedRateKbps / 2 + && observedRateKbps <= expectedRateKbps + ); + } + private LoadPeonCallback markSegmentProcessed(DataSegment segment) { - return success -> processedSegments.add(segment); + return success -> httpClient.processedSegments.add(segment); } private static class TestHttpClient implements HttpClient, DataSegmentChangeHandler { - private final List segmentsSentToServer = new ArrayList<>(); + final BlockingExecutorService processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); + final BlockingExecutorService callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); + final List processedSegments = new ArrayList<>(); + final List segmentsSentToServer = new ArrayList<>(); @Override public ListenableFuture go( @@ -353,6 +386,27 @@ public void removeSegment(DataSegment segment, DataSegmentChangeCallback callbac { segmentsSentToServer.add(segment); } + + void sendRequestToServerAndHandleResponse() + { + sendRequestToServer(); + handleResponseFromServer(); + } + + void sendRequestToServer() + { + processingExecutor.finishNextPendingTask(); + } + + void handleResponseFromServer() + { + processingExecutor.finishAllPendingTasks(); + } + + void executeCallbacks() + { + callbackExecutor.finishAllPendingTasks(); + } } /** diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java new file mode 100644 index 000000000000..fafa1f3e0e31 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadingRateTrackerTest.java @@ -0,0 +1,191 @@ +/* + * 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.server.coordinator.loading; + +import org.apache.druid.error.DruidException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Random; + +public class LoadingRateTrackerTest +{ + private LoadingRateTracker tracker; + + @Before + public void setup() + { + tracker = new LoadingRateTracker(); + } + + @Test + public void testUpdateThrowsExceptionIfBatchNotStarted() + { + DruidException e = Assert.assertThrows( + DruidException.class, + () -> tracker.incrementBytesLoadedInBatch(1000, 10) + ); + Assert.assertEquals( + "markBatchLoadingStarted() must be called before tracking load progress.", + e.getMessage() + ); + } + + @Test + public void testRateIsZeroWhenEmpty() + { + Assert.assertEquals(0, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateIsZeroAfterStop() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + + tracker.stop(); + Assert.assertEquals(0, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateAfter2UpdatesInBatch() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + + tracker.incrementBytesLoadedInBatch(1000, 15); + Assert.assertEquals(8 * 2000 / 15, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateAfter2Batches() + { + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 10); + Assert.assertEquals(8 * 1000 / 10, tracker.getMovingAverageLoadRateKbps()); + tracker.markBatchLoadingFinished(); + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(1000, 5); + Assert.assertEquals(8 * 2000 / 15, tracker.getMovingAverageLoadRateKbps()); + tracker.markBatchLoadingFinished(); + } + + @Test + public void test100UpdatesInABatch() + { + final Random random = new Random(1001); + + tracker.markBatchLoadingStarted(); + + long totalUpdateBytes = 0; + long monoticBatchDuration = 0; + for (int i = 0; i < 100; ++i) { + long updateBytes = 1 + random.nextInt(1000); + monoticBatchDuration = 1 + random.nextInt(10); + + tracker.incrementBytesLoadedInBatch(updateBytes, monoticBatchDuration); + + totalUpdateBytes += updateBytes; + Assert.assertEquals(8 * totalUpdateBytes / monoticBatchDuration, tracker.getMovingAverageLoadRateKbps()); + } + + tracker.markBatchLoadingFinished(); + Assert.assertEquals(8 * totalUpdateBytes / monoticBatchDuration, tracker.getMovingAverageLoadRateKbps()); + } + + @Test + public void testRateIsMovingAverage() + { + final Random random = new Random(1001); + final int windowSize = LoadingRateTracker.MOVING_AVERAGE_WINDOW_SIZE; + final long minEntrySizeBytes = LoadingRateTracker.MIN_ENTRY_SIZE_BYTES; + + // Add batch updates to fill up the window size + long[] updateBytes = new long[windowSize]; + long[] updateMillis = new long[windowSize]; + + long totalBytes = 0; + long totalMillis = 0; + for (int i = 0; i < windowSize; ++i) { + updateBytes[i] = minEntrySizeBytes + random.nextInt((int) minEntrySizeBytes); + updateMillis[i] = 1 + random.nextInt(1000); + + totalBytes += updateBytes[i]; + totalMillis += updateMillis[i]; + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(updateBytes[i], updateMillis[i]); + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + tracker.markBatchLoadingFinished(); + } + + // Add another batch update + long latestUpdateBytes = 1; + long latestUpdateMillis = 1 + random.nextInt(1000); + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(latestUpdateBytes, latestUpdateMillis); + tracker.markBatchLoadingFinished(); + + // Verify that the average window has moved + totalBytes = totalBytes - updateBytes[0] + latestUpdateBytes; + totalMillis = totalMillis - updateMillis[0] + latestUpdateMillis; + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + } + + @Test + public void testWindowMovesOnlyAfterMinSizeUpdates() + { + final Random random = new Random(1001); + + long totalBytes = 0; + long totalMillis = 0; + + final int windowSize = LoadingRateTracker.MOVING_AVERAGE_WINDOW_SIZE; + final long minEntrySizeBytes = LoadingRateTracker.MIN_ENTRY_SIZE_BYTES; + + for (int i = 0; i < windowSize * 10; ++i) { + long updateBytes = 1 + random.nextInt((int) minEntrySizeBytes / 100); + long updateMillis = 1 + random.nextInt(1000); + + totalBytes += updateBytes; + totalMillis += updateMillis; + + tracker.markBatchLoadingStarted(); + tracker.incrementBytesLoadedInBatch(updateBytes, updateMillis); + tracker.markBatchLoadingFinished(); + + // Verify that the average window doesn't move + Assert.assertEquals( + 8 * totalBytes / totalMillis, + tracker.getMovingAverageLoadRateKbps() + ); + } + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java index c496c37e3ecf..71ebc6fd8cf0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/TestLoadQueuePeon.java @@ -56,6 +56,12 @@ public long getSizeOfSegmentsToLoad() return 0; } + @Override + public long getLoadRateKbps() + { + return 0; + } + @Override public CoordinatorRunStats getAndResetStats() { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 7b6eb280a398..b32f8c055ca2 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -37,9 +37,9 @@ import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; @@ -512,11 +512,11 @@ private JacksonConfigManager mockConfigManager() EasyMock.expect( jacksonConfigManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), - EasyMock.eq(CoordinatorCompactionConfig.class), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); return jacksonConfigManager; } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 17db22854779..c44b9b2f358a 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -19,528 +19,529 @@ package org.apache.druid.server.http; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Suppliers; +import org.apache.druid.audit.AuditEntry; +import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.JacksonConfigManager; -import org.apache.druid.error.DruidException; +import org.apache.druid.common.config.TestConfigManagerConfig; import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataCASUpdate; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.metadata.TestMetadataStorageConnector; +import org.apache.druid.metadata.TestMetadataStorageTablesConfig; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.joda.time.Interval; import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import org.skife.jdbi.v2.Handle; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; -import java.util.Collection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.UnaryOperator; @RunWith(MockitoJUnitRunner.class) public class CoordinatorCompactionConfigsResourceTest { - private static final DataSourceCompactionConfig OLD_CONFIG = new DataSourceCompactionConfig( - "oldDataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - private static final DataSourceCompactionConfig NEW_CONFIG = new DataSourceCompactionConfig( - "newDataSource", - null, - 500L, - null, - new Period(1800), - null, - new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - private static final byte[] OLD_CONFIG_IN_BYTES = {1, 2, 3}; - - private static final CoordinatorCompactionConfig ORIGINAL_CONFIG - = CoordinatorCompactionConfig.from(ImmutableList.of(OLD_CONFIG)); - - private static final String DATASOURCE_NOT_EXISTS = "notExists"; - - @Mock - private JacksonConfigManager mockJacksonConfigManager; + private static final double DELTA = 1e-9; + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); @Mock private HttpServletRequest mockHttpServletRequest; - @Mock - private MetadataStorageConnector mockConnector; - - @Mock - private MetadataStorageTablesConfig mockConnectorConfig; - - @Mock - private AuditManager mockAuditManager; - - private CoordinatorCompactionConfigsResource coordinatorCompactionConfigsResource; + private TestCoordinatorConfigManager configManager; + private CoordinatorCompactionConfigsResource resource; @Before public void setup() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(OLD_CONFIG_IN_BYTES); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(OLD_CONFIG_IN_BYTES), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(ORIGINAL_CONFIG); - Mockito.when(mockConnectorConfig.getConfigTable()).thenReturn("druid_config"); - Mockito.when(mockAuditManager.fetchAuditHistory( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - ArgumentMatchers.any() - ) - ).thenReturn(ImmutableList.of()); - coordinatorCompactionConfigsResource = new CoordinatorCompactionConfigsResource( - new CoordinatorConfigManager(mockJacksonConfigManager, mockConnector, mockConnectorConfig), - mockAuditManager - ); Mockito.when(mockHttpServletRequest.getRemoteAddr()).thenReturn("123"); + final AuditManager auditManager = new TestAuditManager(); + configManager = TestCoordinatorConfigManager.create(auditManager); + resource = new CoordinatorCompactionConfigsResource(configManager, auditManager); + configManager.delegate.start(); + } + + @After + public void tearDown() + { + configManager.delegate.stop(); } @Test - public void testSetCompactionTaskLimitWithExistingConfig() + public void testGetDefaultClusterConfig() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - double compactionTaskSlotRatio = 0.5; - int maxCompactionTaskSlots = 9; - Response result = coordinatorCompactionConfigsResource.setCompactionTaskLimit( - compactionTaskSlotRatio, - maxCompactionTaskSlots, - true, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(newConfigCaptor.getValue().getMaxCompactionTaskSlots(), maxCompactionTaskSlots); - Assert.assertTrue(newConfigCaptor.getValue().isUseAutoScaleSlots()); - Assert.assertEquals(compactionTaskSlotRatio, newConfigCaptor.getValue().getCompactionTaskSlotRatio(), 0); + Response response = resource.getCompactionConfig(); + final DruidCompactionConfig defaultConfig + = verifyAndGetPayload(response, DruidCompactionConfig.class); + + Assert.assertEquals(0.1, defaultConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(Integer.MAX_VALUE, defaultConfig.getMaxCompactionTaskSlots()); + Assert.assertFalse(defaultConfig.isUseAutoScaleSlots()); + Assert.assertTrue(defaultConfig.getCompactionConfigs().isEmpty()); + Assert.assertEquals(CompactionEngine.NATIVE, defaultConfig.getEngine()); } @Test - public void testAddOrUpdateCompactionConfigWithExistingConfig() + public void testUpdateGlobalConfig() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, true), - null, - null, - null, - null, - CompactionEngine.NATIVE, - ImmutableMap.of("key", "val") - ); - Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, + Response response = resource.updateClusterCompactionConfig( + new ClusterCompactionConfig(0.5, 10, true, CompactionEngine.MSQ), mockHttpServletRequest ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(2, newConfigCaptor.getValue().getCompactionConfigs().size()); - Assert.assertEquals(OLD_CONFIG, newConfigCaptor.getValue().getCompactionConfigs().get(0)); - Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(1)); - Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getEngine()); + verifyStatus(Response.Status.OK, response); + + final DruidCompactionConfig updatedConfig = verifyAndGetPayload( + resource.getCompactionConfig(), + DruidCompactionConfig.class + ); + + Assert.assertNotNull(updatedConfig); + Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(10, updatedConfig.getMaxCompactionTaskSlots()); + Assert.assertTrue(updatedConfig.isUseAutoScaleSlots()); + Assert.assertEquals(CompactionEngine.MSQ, updatedConfig.getEngine()); } @Test - public void testDeleteCompactionConfigWithExistingConfig() + public void testSetCompactionTaskLimit() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - final String datasourceName = "dataSource"; - final DataSourceCompactionConfig toDelete = new DataSourceCompactionConfig( - datasourceName, - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete)); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(OLD_CONFIG_IN_BYTES), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(originalConfig); - - Response result = coordinatorCompactionConfigsResource.deleteCompactionConfig( - datasourceName, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(0, newConfigCaptor.getValue().getCompactionConfigs().size()); + final DruidCompactionConfig defaultConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + + Response response = resource.setCompactionTaskLimit(0.5, 9, true, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + final DruidCompactionConfig updatedConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + + // Verify that the task slot fields have been updated + Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(9, updatedConfig.getMaxCompactionTaskSlots()); + Assert.assertTrue(updatedConfig.isUseAutoScaleSlots()); + + // Verify that the other fields are unchanged + Assert.assertEquals(defaultConfig.getCompactionConfigs(), updatedConfig.getCompactionConfigs()); + Assert.assertEquals(defaultConfig.getEngine(), updatedConfig.getEngine()); + } + + @Test + public void testGetUnknownDatasourceConfigThrowsNotFound() + { + Response response = resource.getDatasourceCompactionConfig(DS.WIKI); + verifyStatus(Response.Status.NOT_FOUND, response); + } + + @Test + public void testAddDatasourceConfig() + { + final DataSourceCompactionConfig newDatasourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(newDatasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + final DataSourceCompactionConfig fetchedDatasourceConfig + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + Assert.assertEquals(newDatasourceConfig, fetchedDatasourceConfig); + + final DruidCompactionConfig fullCompactionConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + Assert.assertEquals(1, fullCompactionConfig.getCompactionConfigs().size()); + Assert.assertEquals(newDatasourceConfig, fullCompactionConfig.getCompactionConfigs().get(0)); } @Test - public void testUpdateShouldRetryIfRetryableException() + public void testUpdateDatasourceConfig() { - Mockito.when( - mockJacksonConfigManager.set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + final DataSourceCompactionConfig originalDatasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(Period.hours(1)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, true) ) - ).thenReturn(ConfigManager.SetResult.retryableFailure(new ISE("retryable"))); + .withEngine(CompactionEngine.NATIVE) + .build(); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - NEW_CONFIG, + Response response = resource.addOrUpdateDatasourceCompactionConfig( + originalDatasourceConfig, mockHttpServletRequest ); + verifyStatus(Response.Status.OK, response); + + final DataSourceCompactionConfig updatedDatasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withInputSegmentSizeBytes(1000L) + .withSkipOffsetFromLatest(Period.hours(3)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.DAY, null, true) + ) + .withEngine(CompactionEngine.MSQ) + .build(); - // Verify that the update is retried upto the max number of retries - Mockito.verify( - mockJacksonConfigManager, - Mockito.times(CoordinatorCompactionConfigsResource.UPDATE_NUM_RETRY) - ).set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + response = resource.addOrUpdateDatasourceCompactionConfig(updatedDatasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + final DataSourceCompactionConfig latestDatasourceConfig + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + Assert.assertEquals(updatedDatasourceConfig, latestDatasourceConfig); + + final DruidCompactionConfig fullCompactionConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + Assert.assertEquals(1, fullCompactionConfig.getCompactionConfigs().size()); + Assert.assertEquals(updatedDatasourceConfig, fullCompactionConfig.getCompactionConfigs().get(0)); } @Test - public void testUpdateShouldNotRetryIfNotRetryableException() + public void testDeleteDatasourceConfig() { - Mockito.when( - mockJacksonConfigManager.set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.failure(new ISE("retryable"))); + final DataSourceCompactionConfig datasourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - NEW_CONFIG, + response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + response = resource.getDatasourceCompactionConfig(DS.WIKI); + verifyStatus(Response.Status.NOT_FOUND, response); + } + + @Test + public void testDeleteUnknownDatasourceConfigThrowsNotFound() + { + Response response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + verifyStatus(Response.Status.NOT_FOUND, response); + } + + @Test + public void testUpdateIsRetriedIfFailureIsRetryable() + { + configManager.configUpdateResult + = ConfigManager.SetResult.retryableFailure(new Exception("retryable")); + resource.addOrUpdateDatasourceCompactionConfig( + DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), mockHttpServletRequest ); - // Verify that the update is tried only once - Mockito.verify(mockJacksonConfigManager, Mockito.times(1)).set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + Assert.assertEquals( + CoordinatorCompactionConfigsResource.MAX_UPDATE_RETRIES, + configManager.numUpdateAttempts ); } @Test - public void testSetCompactionTaskLimitWithoutExistingConfig() + public void testUpdateIsNotRetriedIfFailureIsNotRetryable() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - double compactionTaskSlotRatio = 0.5; - int maxCompactionTaskSlots = 9; - Response result = coordinatorCompactionConfigsResource.setCompactionTaskLimit( - compactionTaskSlotRatio, - maxCompactionTaskSlots, - true, + configManager.configUpdateResult + = ConfigManager.SetResult.failure(new Exception("not retryable")); + resource.addOrUpdateDatasourceCompactionConfig( + DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), mockHttpServletRequest ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNull(oldConfigCaptor.getValue()); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(newConfigCaptor.getValue().getMaxCompactionTaskSlots(), maxCompactionTaskSlots); - Assert.assertTrue(newConfigCaptor.getValue().isUseAutoScaleSlots()); - Assert.assertEquals(compactionTaskSlotRatio, newConfigCaptor.getValue().getCompactionTaskSlotRatio(), 0); + + Assert.assertEquals(1, configManager.numUpdateAttempts); } @Test - public void testAddOrUpdateCompactionConfigWithoutExistingConfig() + public void testGetDatasourceConfigHistory() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of("key", "val") - ); - String author = "maytas"; - String comment = "hello"; - Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNull(oldConfigCaptor.getValue()); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(1, newConfigCaptor.getValue().getCompactionConfigs().size()); - Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(0)); - Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); + final DataSourceCompactionConfig.Builder builder + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI); + + final DataSourceCompactionConfig configV1 = builder.build(); + resource.addOrUpdateDatasourceCompactionConfig(configV1, mockHttpServletRequest); + + final DataSourceCompactionConfig configV2 = builder.withEngine(CompactionEngine.NATIVE).build(); + resource.addOrUpdateDatasourceCompactionConfig(configV2, mockHttpServletRequest); + + final DataSourceCompactionConfig configV3 = builder + .withEngine(CompactionEngine.MSQ) + .withSkipOffsetFromLatest(Period.hours(1)) + .build(); + resource.addOrUpdateDatasourceCompactionConfig(configV3, mockHttpServletRequest); + + Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + verifyStatus(Response.Status.OK, response); + + final List history + = (List) response.getEntity(); + Assert.assertEquals(3, history.size()); + Assert.assertEquals(configV1, history.get(0).getCompactionConfig()); + Assert.assertEquals(configV2, history.get(1).getCompactionConfig()); + Assert.assertEquals(configV3, history.get(2).getCompactionConfig()); } @Test - public void testAddOrUpdateCompactionConfigWithoutExistingConfigAndEngineAsNull() + public void testGetHistoryOfUnknownDatasourceReturnsEmpty() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(null, newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); + Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + verifyStatus(Response.Status.OK, response); + Assert.assertTrue(((List) response.getEntity()).isEmpty()); } @Test - public void testAddOrUpdateCompactionConfigWithInvalidMaxNumTasksForMSQEngine() + public void testAddInvalidDatasourceConfigThrowsBadRequest() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - - int maxNumTasks = 1; - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasks) - ); - Response response = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(DruidException.Category.INVALID_INPUT.getExpectedStatus(), response.getStatus()); + final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .withEngine(CompactionEngine.MSQ) + .build(); + + final Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.BAD_REQUEST, response); + Assert.assertTrue(response.getEntity() instanceof ErrorResponse); Assert.assertEquals( - "Compaction config not supported. Reason[MSQ context maxNumTasks [1] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.].", + "Compaction config not supported. Reason[MSQ: Context maxNumTasks[1]" + + " must be at least 2 (1 controller + 1 worker)].", ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() ); } @Test - public void testDeleteCompactionConfigWithoutExistingConfigShouldFailAsDatasourceNotExist() + public void testUpdateEngineToMSQWithInvalidDatasourceConfigThrowsBadRequest() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - Response result = coordinatorCompactionConfigsResource.deleteCompactionConfig( - DATASOURCE_NOT_EXISTS, + final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + response = resource.updateClusterCompactionConfig( + new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ), mockHttpServletRequest ); - Assert.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), result.getStatus()); + verifyStatus(Response.Status.BAD_REQUEST, response); + Assert.assertTrue(response.getEntity() instanceof ErrorResponse); + Assert.assertEquals( + "Cannot update engine to [msq] as it does not support compaction config of DataSource[wiki]." + + " Reason[MSQ: Context maxNumTasks[1] must be at least 2 (1 controller + 1 worker)].", + ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() + ); } - @Test - public void testGetCompactionConfigHistoryForUnknownDataSourceShouldReturnEmptyList() + @SuppressWarnings("unchecked") + private T verifyAndGetPayload(Response response, Class type) { - Response response = coordinatorCompactionConfigsResource.getCompactionConfigHistory( - DATASOURCE_NOT_EXISTS, - null, - null - ); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertTrue(((Collection) response.getEntity()).isEmpty()); + + Assert.assertTrue(type.isInstance(response.getEntity())); + return (T) response.getEntity(); + } + + private void verifyStatus(Response.Status expectedStatus, Response response) + { + Assert.assertEquals(expectedStatus.getStatusCode(), response.getStatus()); + } + + /** + * Test implementation of AuditManager that keeps audit entries in memory. + */ + private static class TestAuditManager implements AuditManager + { + private final List audits = new ArrayList<>(); + + @Override + public void doAudit(AuditEntry event, Handle handle) + { + // do nothing + } + + @Override + public void doAudit(AuditEntry event) + { + final String json; + try { + json = OBJECT_MAPPER.writeValueAsString(event.getPayload().raw()); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + + final AuditEntry eventWithSerializedPayload + = AuditEntry.builder() + .key(event.getKey()) + .type(event.getType()) + .auditInfo(event.getAuditInfo()) + .auditTime(event.getAuditTime()) + .request(event.getRequest()) + .serializedPayload(json) + .build(); + audits.add(eventWithSerializedPayload); + } + + @Override + public List fetchAuditHistory(String key, String type, Interval interval) + { + return audits; + } + + @Override + public List fetchAuditHistory(String type, int limit) + { + return audits; + } + + @Override + public List fetchAuditHistory(String type, Interval interval) + { + return audits; + } + + @Override + public List fetchAuditHistory(String key, String type, int limit) + { + return audits; + } + + @Override + public int removeAuditLogsOlderThan(long timestamp) + { + return 0; + } + } + + /** + * Test implementation of CoordinatorConfigManager to track number of update attempts. + */ + private static class TestCoordinatorConfigManager extends CoordinatorConfigManager + { + private final ConfigManager delegate; + private int numUpdateAttempts; + private ConfigManager.SetResult configUpdateResult; + + static TestCoordinatorConfigManager create(AuditManager auditManager) + { + final MetadataStorageTablesConfig tablesConfig = new TestMetadataStorageTablesConfig() + { + @Override + public String getConfigTable() + { + return "druid_config"; + } + }; + + final TestDBConnector dbConnector = new TestDBConnector(); + final ConfigManager configManager = new ConfigManager( + dbConnector, + Suppliers.ofInstance(tablesConfig), + Suppliers.ofInstance(new TestConfigManagerConfig()) + ); + + return new TestCoordinatorConfigManager(configManager, dbConnector, tablesConfig, auditManager); + } + + TestCoordinatorConfigManager( + ConfigManager configManager, + TestDBConnector dbConnector, + MetadataStorageTablesConfig tablesConfig, + AuditManager auditManager + ) + { + super( + new JacksonConfigManager(configManager, OBJECT_MAPPER, auditManager), + dbConnector, + tablesConfig + ); + this.delegate = configManager; + } + + @Override + public ConfigManager.SetResult getAndUpdateCompactionConfig( + UnaryOperator operator, + AuditInfo auditInfo + ) + { + ++numUpdateAttempts; + if (configUpdateResult == null) { + return super.getAndUpdateCompactionConfig(operator, auditInfo); + } else { + return configUpdateResult; + } + } + } + + /** + * Test implementation for in-memory insert, lookup and compareAndSwap operations. + */ + private static class TestDBConnector extends TestMetadataStorageConnector + { + private final Map, byte[]> values = new HashMap<>(); + + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + values.put( + Arrays.asList(tableName, keyColumn, valueColumn, key), + value + ); + return null; + } + + @Nullable + @Override + public byte[] lookup(String tableName, String keyColumn, String valueColumn, String key) + { + return values.get(Arrays.asList(tableName, keyColumn, valueColumn, key)); + } + + @Override + public boolean compareAndSwap(List updates) + { + for (MetadataCASUpdate update : updates) { + final List key = Arrays.asList( + update.getTableName(), + update.getKeyColumn(), + update.getValueColumn(), + update.getKey() + ); + + final byte[] currentValue = values.get(key); + if (currentValue == null || Arrays.equals(currentValue, update.getOldValue())) { + values.put(key, update.getNewValue()); + } else { + return false; + } + } + + return true; + } + } + + private static class DS + { + static final String WIKI = "wiki"; } } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java index c25417a8d13a..9d2b81f6aa15 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorResourceTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -73,4 +74,29 @@ public void testIsLeader() Assert.assertEquals(ImmutableMap.of("leader", false), response2.getEntity()); Assert.assertEquals(404, response2.getStatus()); } + + @Test + public void testGetLoadStatusSimple() + { + EasyMock.expect(mock.getLoadManagementPeons()) + .andReturn(ImmutableMap.of("hist1", new TestLoadQueuePeon())) + .once(); + EasyMock.replay(mock); + + final Response response = new CoordinatorResource(mock).getLoadQueue("true", null); + Assert.assertEquals( + ImmutableMap.of( + "hist1", + ImmutableMap.of( + "segmentsToDrop", 0, + "segmentsToLoad", 0, + "segmentsToLoadSize", 0L, + "segmentsToDropSize", 0L, + "expectedLoadTimeMillis", 0L + ) + ), + response.getEntity() + ); + Assert.assertEquals(200, response.getStatus()); + } } diff --git a/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java b/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java index ff9fcffb8d99..ad00e5e6dbde 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/WorkerTaskCountStatsMonitorTest.java @@ -120,6 +120,24 @@ public Map getWorkerCompletedTasks() "metrics", 9L ); } + + @Override + public Map getWorkerFailedTasks() + { + return ImmutableMap.of( + "movies", 4L, + "games", 6L + ); + } + + @Override + public Map getWorkerSuccessfulTasks() + { + return ImmutableMap.of( + "games", 23L, + "inventory", 89L + ); + } }; nullStatsProvider = new WorkerTaskCountStatsProvider() @@ -239,7 +257,7 @@ public void testMonitorIndexer() new WorkerTaskCountStatsMonitor(injectorForIndexer, ImmutableSet.of(NodeRole.INDEXER)); final StubServiceEmitter emitter = new StubServiceEmitter("service", "host"); monitor.doMonitor(emitter); - Assert.assertEquals(6, emitter.getEvents().size()); + Assert.assertEquals(10, emitter.getEvents().size()); emitter.verifyValue( "worker/task/running/count", ImmutableMap.of("dataSource", "wikipedia"), @@ -270,6 +288,26 @@ public void testMonitorIndexer() ImmutableMap.of("dataSource", "metrics"), 9L ); + emitter.verifyValue( + "worker/task/failed/count", + ImmutableMap.of("dataSource", "movies"), + 4L + ); + emitter.verifyValue( + "worker/task/failed/count", + ImmutableMap.of("dataSource", "games"), + 6L + ); + emitter.verifyValue( + "worker/task/success/count", + ImmutableMap.of("dataSource", "games"), + 23L + ); + emitter.verifyValue( + "worker/task/success/count", + ImmutableMap.of("dataSource", "inventory"), + 89L + ); } @Test public void testMonitorWithNulls() diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 9369717bf1f6..75c8bb6b143f 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -31,7 +31,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.guice.annotations.Json; import org.apache.druid.initialization.ServerInjectorBuilder; @@ -84,7 +84,7 @@ public class DumpSegmentTest extends InitializedNullHandlingTest public DumpSegmentTest() { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); this.closer = Closer.create(); } @@ -156,15 +156,15 @@ public void testDumpNestedColumn() throws Exception { Injector injector = Mockito.mock(Injector.class); ObjectMapper mapper = TestHelper.makeJsonMapper(); - mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); @@ -196,15 +196,15 @@ public void testDumpNestedColumnPath() throws Exception { Injector injector = Mockito.mock(Injector.class); ObjectMapper mapper = TestHelper.makeJsonMapper(); - mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.registerModules(BuiltInTypesModule.getJacksonModulesList()); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) - .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null)) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null)) ); Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); - Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null)); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index ec518ee4522f..689db77c0faa 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -28,6 +28,7 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexOver; import org.apache.calcite.rex.RexUtil; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; @@ -36,6 +37,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExpressionType; @@ -66,6 +68,7 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.ExpressionParser; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import org.apache.druid.sql.calcite.table.RowSignatures; import org.joda.time.Interval; @@ -238,6 +241,10 @@ public static DruidExpression toDruidExpressionWithPostAggOperands( final SqlKind kind = rexNode.getKind(); if (kind == SqlKind.INPUT_REF) { return inputRefToDruidExpression(rowSignature, rexNode); + } else if (rexNode instanceof RexOver) { + throw new CannotBuildQueryException( + StringUtils.format("Unexpected OVER expression during translation [%s]", rexNode) + ); } else if (rexNode instanceof RexCall) { return rexCallToDruidExpression(plannerContext, rowSignature, rexNode, postAggregatorVisitor); } else if (kind == SqlKind.LITERAL) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java index 8a18ce73df22..33e49697f52e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ScalarInArrayOperatorConversion.java @@ -94,6 +94,9 @@ public DimFilter toDruidFilter( } else if (scalarExpression.isSimpleExtraction() && plannerContext.isUseLegacyInFilter()) { scalarColumn = scalarExpression.getSimpleExtraction().getColumn(); scalarExtractionFn = scalarExpression.getSimpleExtraction().getExtractionFn(); + } else if (virtualColumnRegistry == null) { + // virtual column registry unavailable, fallback to expression filter + return null; } else { scalarColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( scalarExpression, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index 8d3b6a743e38..c42c237ee365 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -40,7 +40,6 @@ import org.apache.calcite.rel.rules.JoinPushThroughJoinRule; import org.apache.calcite.rel.rules.ProjectMergeRule; import org.apache.calcite.rel.rules.PruneEmptyRules; -import org.apache.calcite.runtime.Hook; import org.apache.calcite.sql.SqlExplainFormat; import org.apache.calcite.sql.SqlExplainLevel; import org.apache.calcite.sql2rel.RelDecorrelator; @@ -70,6 +69,7 @@ import org.apache.druid.sql.calcite.rule.logical.DruidAggregateRemoveRedundancyRule; import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules; import org.apache.druid.sql.calcite.run.EngineFeature; +import org.apache.druid.sql.hook.DruidHook; import java.util.ArrayList; import java.util.Collections; @@ -395,7 +395,8 @@ private static class SaveLogicalPlanProgram implements Program public RelNode run(RelOptPlanner planner, RelNode rel, RelTraitSet requiredOutputTraits, List materializations, List lattices) { - Hook.TRIMMED.run(rel); + PlannerContext pctx = planner.getContext().unwrapOrThrow(PlannerContext.class); + pctx.dispatchHook(DruidHook.LOGICAL_PLAN, rel); return rel; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java index 16d3541e96c6..857c8cb0d120 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java @@ -36,6 +36,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.SqlOverOperator; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlSelectKeyword; import org.apache.calcite.sql.SqlUpdate; @@ -46,6 +47,8 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.calcite.sql.util.SqlBasicVisitor; +import org.apache.calcite.sql.util.SqlVisitor; import org.apache.calcite.sql.validate.IdentifierNamespace; import org.apache.calcite.sql.validate.SelectNamespace; import org.apache.calcite.sql.validate.SqlNonNullableAccessors; @@ -83,6 +86,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Predicate; import java.util.regex.Pattern; /** @@ -902,4 +906,50 @@ private boolean isSqlCallDistinct(@Nullable SqlCall call) && call.getFunctionQuantifier() != null && call.getFunctionQuantifier().getValue() == SqlSelectKeyword.DISTINCT; } + + @Override + protected void validateHavingClause(SqlSelect select) + { + super.validateHavingClause(select); + SqlNode having = select.getHaving(); + if (containsOver(having)) { + throw buildCalciteContextException("Window functions are not allowed in HAVING", having); + } + } + + private boolean containsOver(SqlNode having) + { + if (having == null) { + return false; + } + final Predicate callPredicate = call -> call.getOperator() instanceof SqlOverOperator; + return containsCall(having, callPredicate); + } + + // copy of SqlUtil#containsCall + /** Returns whether an AST tree contains a call that matches a given + * predicate. */ + private static boolean containsCall(SqlNode node, + Predicate callPredicate) + { + try { + SqlVisitor visitor = + new SqlBasicVisitor() { + @Override public Void visit(SqlCall call) + { + if (callPredicate.test(call)) { + throw new Util.FoundOne(call); + } + return super.visit(call); + } + }; + node.accept(visitor); + return false; + } + catch (Util.FoundOne e) { + Util.swallow(e, null); + return true; + } + } + } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index b7e2de3e66b2..21ad4c236633 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.hook.DruidHook.HookKey; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; @@ -668,4 +669,9 @@ public LookupExtractor getLookup(final String lookupName) return lookupCache.getLookup(lookupName); } + + public void dispatchHook(HookKey key, T object) + { + plannerToolbox.getHookDispatcher().dispatch(key, object); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index a5cdc028e7fc..514952972d39 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -49,6 +49,8 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.schema.DruidSchemaName; +import org.apache.druid.sql.hook.DruidHook; +import org.apache.druid.sql.hook.DruidHookDispatcher; import java.util.Map; import java.util.Properties; @@ -77,7 +79,8 @@ public PlannerFactory( final CalciteRulesManager calciteRuleManager, final JoinableFactoryWrapper joinableFactoryWrapper, final CatalogResolver catalog, - final AuthConfig authConfig + final AuthConfig authConfig, + final DruidHookDispatcher hookDispatcher ) { super( @@ -91,7 +94,8 @@ public PlannerFactory( druidSchemaName, calciteRuleManager, authorizerMapper, - authConfig + authConfig, + hookDispatcher ); } @@ -112,6 +116,7 @@ public DruidPlanner createPlanner( queryContext, hook ); + context.dispatchHook(DruidHook.SQL, sql); return new DruidPlanner(buildFrameworkConfig(context), context, engine, hook); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerHook.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerHook.java index fd245b096d82..8158a314f511 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerHook.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerHook.java @@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.sql.calcite.rel.DruidRel; +import org.apache.druid.sql.hook.DruidHookDispatcher; /** * Druid-specific version of Calcite's {@link org.apache.calcite.runtime.Hook Hook} @@ -33,8 +34,11 @@ * for test validation. Calcite's hook has multiple low-level events, but, sadly, * none at the points where tests want to verify, except for the opportunity to * capture the native query. + * + * Should be removed ; use {@link DruidHookDispatcher} instead. */ @UnstableApi +@Deprecated public interface PlannerHook { void captureSql(String sql); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerToolbox.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerToolbox.java index f00098261189..d8e3a25f4583 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerToolbox.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerToolbox.java @@ -26,6 +26,7 @@ import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; +import org.apache.druid.sql.hook.DruidHookDispatcher; public class PlannerToolbox { @@ -40,6 +41,7 @@ public class PlannerToolbox protected final CalciteRulesManager calciteRuleManager; protected final AuthorizerMapper authorizerMapper; protected final AuthConfig authConfig; + protected final DruidHookDispatcher hookDispatcher; public PlannerToolbox( final DruidOperatorTable operatorTable, @@ -52,7 +54,8 @@ public PlannerToolbox( final String druidSchemaName, final CalciteRulesManager calciteRuleManager, final AuthorizerMapper authorizerMapper, - final AuthConfig authConfig + final AuthConfig authConfig, + final DruidHookDispatcher hookDispatcher ) { this.operatorTable = operatorTable; @@ -66,6 +69,7 @@ public PlannerToolbox( this.calciteRuleManager = calciteRuleManager; this.authorizerMapper = authorizerMapper; this.authConfig = authConfig; + this.hookDispatcher = hookDispatcher; } public DruidOperatorTable operatorTable() @@ -117,4 +121,9 @@ public AuthConfig getAuthConfig() { return authConfig; } + + public DruidHookDispatcher getHookDispatcher() + { + return hookDispatcher; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index 9f15d3822866..b543be135cc2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -47,7 +47,6 @@ import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.runtime.Hook; import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.sql.SqlExplain; import org.apache.calcite.sql.SqlNode; @@ -76,6 +75,7 @@ import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.table.DruidTable; +import org.apache.druid.sql.hook.DruidHook; import org.apache.druid.utils.Throwables; import javax.annotation.Nullable; @@ -155,7 +155,7 @@ public void prepare() isPrepared = true; SqlNode validatedQueryNode = validatedQueryNode(); rootQueryRel = handlerContext.planner().rel(validatedQueryNode); - Hook.CONVERTED.run(rootQueryRel.rel); + handlerContext.plannerContext().dispatchHook(DruidHook.CONVERTED_PLAN, rootQueryRel.rel); handlerContext.hook().captureQueryRel(rootQueryRel); final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory(); final SqlValidator validator = handlerContext.planner().getValidator(); @@ -563,7 +563,8 @@ protected PlannerResult planWithDruidConvention() throws ValidationException .plus(DruidLogicalConvention.instance()), newRoot ); - Hook.JAVA_PLAN.run(newRoot); + + plannerContext.dispatchHook(DruidHook.DRUID_PLAN, newRoot); DruidQueryGenerator generator = new DruidQueryGenerator(plannerContext, (DruidLogicalNode) newRoot, rexBuilder); DruidQuery baseQuery = generator.buildQuery(); @@ -591,7 +592,7 @@ protected PlannerResult planWithDruidConvention() throws ValidationException handlerContext.hook().captureDruidRel(druidRel); - Hook.JAVA_PLAN.run(druidRel); + plannerContext.dispatchHook(DruidHook.DRUID_PLAN, druidRel); if (explain != null) { return planExplanation(possiblyLimitedRoot, druidRel, true); diff --git a/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java b/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java new file mode 100644 index 000000000000..5969d608f2fb --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java @@ -0,0 +1,75 @@ +/* + * 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.sql.hook; + +import com.google.errorprone.annotations.Immutable; +import org.apache.calcite.rel.RelNode; +import java.util.Objects; + +/** + * Interface for hooks that can be invoked by {@link DruidHookDispatcher}. + * + * HookKey should be added at every place a new hook is needed. + */ +@FunctionalInterface +public interface DruidHook +{ + HookKey CONVERTED_PLAN = new HookKey<>("converted", RelNode.class); + HookKey LOGICAL_PLAN = new HookKey<>("logicalPlan", RelNode.class); + HookKey DRUID_PLAN = new HookKey<>("druidPlan", RelNode.class); + HookKey SQL = new HookKey<>("sql", String.class); + + @Immutable + class HookKey + { + private final String label; + private final Class type; + + public HookKey(String label, Class type) + { + this.label = label; + this.type = type; + } + + @Override + public int hashCode() + { + return Objects.hash(label, type); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + HookKey other = (HookKey) obj; + return Objects.equals(label, other.label) && Objects.equals(type, other.type); + } + } + + void invoke(HookKey key, T object); +} diff --git a/sql/src/main/java/org/apache/druid/sql/hook/DruidHookDispatcher.java b/sql/src/main/java/org/apache/druid/sql/hook/DruidHookDispatcher.java new file mode 100644 index 000000000000..0114699d638b --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/hook/DruidHookDispatcher.java @@ -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. + */ + +package org.apache.druid.sql.hook; + +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.hook.DruidHook.HookKey; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Dispatcher for Druid hooks. + * + * A single instance should live in the system and be used to dispatch hooks. + * Usual way to dispatch should be via + * {@link PlannerContext#dispatchHook(HookKey, Object)}. Access to this class is + * also possible thru DruidConnectionExtras. + */ +@LazySingleton +public class DruidHookDispatcher +{ + Map, List>> hooks = new HashMap<>(); + + @Inject + public DruidHookDispatcher() + { + } + + public void register(HookKey label, DruidHook hook) + { + hooks.computeIfAbsent(label, k -> new ArrayList<>()).add(hook); + } + + public void unregister(HookKey key, DruidHook hook) + { + hooks.get(key).remove(hook); + } + + public Closeable withHook(HookKey key, DruidHook hook) + { + register(key, hook); + return new Closeable() + { + @Override + public void close() + { + unregister(key, hook); + } + }; + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public void dispatch(HookKey key, T object) + { + List> currentHooks = hooks.get(key); + if (currentHooks != null) { + for (DruidHook hook : currentHooks) { + hook.invoke(key, object); + } + } + } + +} diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaDriverTest.java b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaDriverTest.java index efdf782d147f..3f039814c0f5 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaDriverTest.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaDriverTest.java @@ -19,6 +19,7 @@ package org.apache.druid.quidem; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.junit.jupiter.api.Test; import java.sql.Connection; @@ -51,6 +52,6 @@ public void testSelect() throws SQLException @Test public void testURIParse() throws SQLException { - DruidAvaticaTestDriver.buildConfigfromURIParams("druidtest:///"); + SqlTestFrameworkConfig.fromURL("druidtest:///"); } } diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java index 123019dad04f..d2a7b461dfc8 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java @@ -70,25 +70,18 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier; import org.apache.druid.sql.guice.SqlModule; -import org.apache.http.NameValuePair; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.http.client.utils.URIBuilder; -import org.apache.http.client.utils.URLEncodedUtils; import org.eclipse.jetty.server.Server; import java.io.Closeable; import java.io.File; import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.charset.StandardCharsets; import java.sql.Connection; import java.sql.Driver; import java.sql.DriverManager; import java.sql.DriverPropertyInfo; import java.sql.SQLException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.logging.Logger; @@ -98,10 +91,13 @@ public class DruidAvaticaTestDriver implements Driver new DruidAvaticaTestDriver().register(); } - public static final String URI_PREFIX = "druidtest://"; + public static final String SCHEME = "druidtest"; + public static final String URI_PREFIX = SCHEME + "://"; public static final String DEFAULT_URI = URI_PREFIX + "/"; - static final SqlTestFrameworkConfigStore CONFIG_STORE = new SqlTestFrameworkConfigStore(); + static final SqlTestFrameworkConfigStore CONFIG_STORE = new SqlTestFrameworkConfigStore( + x -> new AvaticaBasedTestConnectionSupplier(x) + ); public DruidAvaticaTestDriver() { @@ -114,13 +110,8 @@ public Connection connect(String url, Properties info) throws SQLException return null; } try { - SqlTestFrameworkConfig config = buildConfigfromURIParams(url); - - ConfigurationInstance ci = CONFIG_STORE.getConfigurationInstance( - config, - x -> new AvaticaBasedTestConnectionSupplier(x) - ); - + SqlTestFrameworkConfig config = SqlTestFrameworkConfig.fromURL(url); + ConfigurationInstance ci = CONFIG_STORE.getConfigurationInstance(config); AvaticaJettyServer server = ci.framework.injector().getInstance(AvaticaJettyServer.class); return server.getConnection(info); } @@ -148,9 +139,9 @@ public DruidSchemaCatalog getLookupNodeService(QueryRunnerFactoryConglomerate co @Provides @LazySingleton - public DruidConnectionExtras getConnectionExtras(ObjectMapper objectMapper) + public DruidConnectionExtras getConnectionExtras(ObjectMapper objectMapper, DruidHookDispatcher druidHookDispatcher) { - return new DruidConnectionExtras.DruidConnectionExtrasImpl(objectMapper); + return new DruidConnectionExtras.DruidConnectionExtrasImpl(objectMapper, druidHookDispatcher); } @Provides @@ -349,24 +340,6 @@ public void run() return tempDir; } - public static SqlTestFrameworkConfig buildConfigfromURIParams(String url) throws SQLException - { - Map queryParams; - queryParams = new HashMap<>(); - try { - List params = URLEncodedUtils.parse(new URI(url), StandardCharsets.UTF_8); - for (NameValuePair pair : params) { - queryParams.put(pair.getName(), pair.getValue()); - } - // possible caveat: duplicate entries overwrite earlier ones - } - catch (URISyntaxException e) { - throw new SQLException("Can't decode URI", e); - } - - return new SqlTestFrameworkConfig(queryParams); - } - private void register() { try { diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java index 75bdd4280fab..176411a6bb74 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java @@ -20,18 +20,25 @@ package org.apache.druid.quidem; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.sql.hook.DruidHookDispatcher; + +import java.sql.Connection; public interface DruidConnectionExtras { ObjectMapper getObjectMapper(); + DruidHookDispatcher getDruidHookDispatcher(); + class DruidConnectionExtrasImpl implements DruidConnectionExtras { private final ObjectMapper objectMapper; + private final DruidHookDispatcher druidHookDispatcher; - public DruidConnectionExtrasImpl(ObjectMapper objectMapper) + public DruidConnectionExtrasImpl(ObjectMapper objectMapper, DruidHookDispatcher druidHookDispatcher) { this.objectMapper = objectMapper; + this.druidHookDispatcher = druidHookDispatcher; } @Override @@ -39,5 +46,19 @@ public ObjectMapper getObjectMapper() { return objectMapper; } + + @Override + public DruidHookDispatcher getDruidHookDispatcher() + { + return druidHookDispatcher; + } + } + + static DruidConnectionExtras unwrapOrThrow(Connection connection) + { + if (connection instanceof DruidConnectionExtras) { + return (DruidConnectionExtras) connection; + } + throw new UnsupportedOperationException("Expected DruidConnectionExtras to be implemented by connection!"); } } diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java index 8c2b99d80b6b..7ae8f97650c9 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java @@ -35,12 +35,16 @@ import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.rel.DruidRel; import org.apache.druid.sql.calcite.util.QueryLogHook; +import org.apache.druid.sql.hook.DruidHook; +import org.apache.druid.sql.hook.DruidHook.HookKey; +import org.apache.druid.sql.hook.DruidHookDispatcher; +import java.io.Closeable; +import java.io.IOException; import java.sql.ResultSet; import java.sql.Statement; import java.util.ArrayList; import java.util.List; -import java.util.function.Consumer; import java.util.stream.Collectors; public class DruidQuidemCommandHandler implements CommandHandler @@ -56,7 +60,7 @@ public Command parseCommand(List lines, List content, String lin return new LogicalPlanCommand(lines, content); } if (line.startsWith("druidPlan")) { - return new PhysicalPlanCommand(lines, content); + return new DruidPlanCommand(lines, content); } if (line.startsWith("nativePlan")) { return new NativePlanCommand(lines, content); @@ -155,19 +159,22 @@ protected void executeExplain(Context x) throws Exception */ abstract static class AbstractRelPlanCommand extends AbstractPlanCommand { - Hook hook; + HookKey hook; - AbstractRelPlanCommand(List lines, List content, Hook hook) + AbstractRelPlanCommand(List lines, List content, DruidHook.HookKey hook) { super(lines, content); this.hook = hook; } @Override - protected final void executeExplain(Context x) + protected final void executeExplain(Context x) throws IOException { + DruidHookDispatcher dhp = unwrapDruidHookDispatcher(x); List logged = new ArrayList<>(); - try (final Hook.Closeable unhook = hook.add((Consumer) logged::add)) { + try (Closeable unhook = dhp.withHook(hook, (key, relNode) -> { + logged.add(relNode); + })) { executeQuery(x); } @@ -179,21 +186,26 @@ protected final void executeExplain(Context x) x.echo(ImmutableList.of(str)); } } + + protected final DruidHookDispatcher unwrapDruidHookDispatcher(Context x) + { + return DruidConnectionExtras.unwrapOrThrow(x.connection()).getDruidHookDispatcher(); + } } static class LogicalPlanCommand extends AbstractRelPlanCommand { LogicalPlanCommand(List lines, List content) { - super(lines, content, Hook.TRIMMED); + super(lines, content, DruidHook.LOGICAL_PLAN); } } - static class PhysicalPlanCommand extends AbstractRelPlanCommand + static class DruidPlanCommand extends AbstractRelPlanCommand { - PhysicalPlanCommand(List lines, List content) + DruidPlanCommand(List lines, List content) { - super(lines, content, Hook.JAVA_PLAN); + super(lines, content, DruidHook.DRUID_PLAN); } } @@ -201,7 +213,7 @@ static class ConvertedPlanCommand extends AbstractRelPlanCommand { ConvertedPlanCommand(List lines, List content) { - super(lines, content, Hook.CONVERTED); + super(lines, content, DruidHook.CONVERTED_PLAN); } } } diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemTestBase.java b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemTestBase.java index 6058cb2a8bdb..650af0880f96 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemTestBase.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemTestBase.java @@ -29,12 +29,14 @@ import org.apache.calcite.util.Util; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -76,8 +78,12 @@ * */ @TestInstance(TestInstance.Lifecycle.PER_CLASS) +@ExtendWith(EnabledOnlyInSqlCompatibleMode.class) public abstract class DruidQuidemTestBase { + static { + NullHandling.initializeForTests(); + } public static final String IQ_SUFFIX = ".iq"; /** diff --git a/sql/src/test/java/org/apache/druid/quidem/EnabledOnlyInSqlCompatibleMode.java b/sql/src/test/java/org/apache/druid/quidem/EnabledOnlyInSqlCompatibleMode.java new file mode 100644 index 000000000000..96cf0a72acff --- /dev/null +++ b/sql/src/test/java/org/apache/druid/quidem/EnabledOnlyInSqlCompatibleMode.java @@ -0,0 +1,43 @@ +/* + * 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.quidem; + +import org.apache.druid.common.config.NullHandling; +import org.junit.jupiter.api.extension.ConditionEvaluationResult; +import org.junit.jupiter.api.extension.ExecutionCondition; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class EnabledOnlyInSqlCompatibleMode implements ExecutionCondition +{ + static { + NullHandling.initializeForTests(); + } + + @Override + public ConditionEvaluationResult evaluateExecutionCondition(ExtensionContext context) + { + if (NullHandling.sqlCompatible()) { + return ConditionEvaluationResult.enabled("SQL compatible mode is enabled"); + } else { + return ConditionEvaluationResult.disabled("SQL compatible mode is disabled"); + } + } + +} diff --git a/sql/src/test/java/org/apache/druid/quidem/SqlQuidemTest.java b/sql/src/test/java/org/apache/druid/quidem/SqlQuidemTest.java index d1922472d7f3..9e6a965891ce 100644 --- a/sql/src/test/java/org/apache/druid/quidem/SqlQuidemTest.java +++ b/sql/src/test/java/org/apache/druid/quidem/SqlQuidemTest.java @@ -19,20 +19,10 @@ package org.apache.druid.quidem; -import org.apache.druid.common.config.NullHandling; -import org.junit.jupiter.api.condition.EnabledIf; - import java.io.File; -@EnabledIf(value = "enabled", disabledReason = "These tests are only run in SqlCompatible mode!") public class SqlQuidemTest extends DruidQuidemTestBase { - public static boolean enabled() - { - NullHandling.initializeForTests(); - return NullHandling.sqlCompatible(); - } - public SqlQuidemTest() { super(); diff --git a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java index b332b6dabfa2..1a7b6e3a7c51 100644 --- a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java @@ -58,6 +58,7 @@ import org.apache.druid.sql.calcite.planner.PrepareResult; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.sql.http.SqlQuery; import org.easymock.EasyMock; import org.hamcrest.MatcherAssert; @@ -158,7 +159,8 @@ public void setUp() new CalciteRulesManager(ImmutableSet.of()), joinableFactoryWrapper, CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); this.sqlStatementFactory = new SqlStatementFactory( diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index cccaab75b6f0..518e225c491b 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -89,6 +89,7 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.guice.SqlModule; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.eclipse.jetty.server.Server; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -1048,7 +1049,8 @@ private SqlStatementFactory makeStatementFactory() new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java index 81cd1c692318..ed855f9c5f86 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java @@ -47,6 +47,7 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.junit.Assert; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -112,7 +113,8 @@ public void setUp() new CalciteRulesManager(ImmutableSet.of()), joinableFactoryWrapper, CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); this.sqlStatementFactory = CalciteTests.createSqlStatementFactory( CalciteTests.createMockSqlEngine(walker, conglomerate), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 2fe1919f6a16..2e89c2eeb049 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.calcite.avatica.SqlType; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -130,7 +130,7 @@ public ArraysComponentSupplier(TempDirProducer tempFolderProducer) public void configureGuice(DruidInjectorBuilder builder) { super.configureGuice(builder); - builder.addModule(new NestedDataModule()); + builder.addModule(new BuiltInTypesModule()); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 9e6e5da2bb0d..5efc59420ada 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -34,8 +34,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.guice.DruidInjectorBuilder; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -196,7 +196,7 @@ public NestedComponentSupplier(TempDirProducer tempFolderProducer) public void configureGuice(DruidInjectorBuilder builder) { super.configureGuice(builder); - builder.addModule(new NestedDataModule()); + builder.addModule(new BuiltInTypesModule()); } @SuppressWarnings("resource") @@ -207,7 +207,7 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( final Injector injector ) { - NestedDataModule.registerHandlersAndSerde(); + BuiltInTypesModule.registerHandlersAndSerde(); final QueryableIndex index = IndexBuilder.create() .tmpDir(tempDirProducer.newTempFolder()) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index 3c47eb20491e..6a0f1742155c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.rel.RelNode; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -2139,4 +2140,21 @@ public void testSqlToRelInConversion() ) .run(); } + + @Test + public void testRejectHavingWithWindowExpression() + { + assertEquals( + "1.37.0", + RelNode.class.getPackage().getImplementationVersion(), + "Calcite version changed; check if CALCITE-6473 is fixed and remove:\n * this assertion\n * DruidSqlValidator#validateHavingClause" + ); + + testQueryThrows( + "SELECT cityName,sum(1) OVER () as w FROM wikipedia group by cityName HAVING w > 10", + ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true), + DruidException.class, + invalidSqlContains("Window functions are not allowed in HAVING") + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index c1236162fa02..1452358950ac 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -58,8 +58,12 @@ import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; @@ -92,6 +96,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -1601,6 +1606,36 @@ public void testTimeseriesSubqueryWithEarliestAggregator(String testName, Map queryContext) + { + DimFilter filter = NullHandling.replaceWithDefault() + ? new InDimFilter("v0", new HashSet<>(Arrays.asList("1", "17"))) + : new TypedInFilter("v0", ColumnType.LONG, null, ImmutableList.of(1, 17), null); + testQuery( + "select countryName from " + + "(select countryName, length(countryName) as cname from wikipedia group by countryName) " + + "where SCALAR_IN_ARRAY(cname, ARRAY[17, 1])", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(new TableDataSource(CalciteTests.WIKIPEDIA)) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setVirtualColumns(expressionVirtualColumn("v0", "strlen(\"countryName\")", ColumnType.LONG)) + .setDimFilter(filter) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("countryName", "d0", ColumnType.STRING)) + .setLimitSpec(NoopLimitSpec.instance()) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"Republic of Korea"} + ) + ); + } + public static class SubqueryComponentSupplier extends SqlTestFramework.StandardComponentSupplier { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java index 9eda5849fcf9..1fc39d52ec51 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java @@ -30,15 +30,23 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.topn.TopNQueryConfig; +import org.apache.druid.quidem.DruidAvaticaTestDriver; import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode; import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; +import org.apache.http.NameValuePair; import org.apache.http.client.utils.URIBuilder; +import org.apache.http.client.utils.URLEncodedUtils; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeEachCallback; import org.junit.jupiter.api.extension.ExtensionContext; +import org.reflections.Configuration; import org.reflections.Reflections; +import org.reflections.scanners.SubTypesScanner; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; +import org.reflections.util.FilterBuilder; import javax.annotation.Nonnull; import java.io.Closeable; @@ -51,6 +59,8 @@ import java.lang.reflect.Method; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -224,11 +234,18 @@ public boolean equals(Object obj) public static class SqlTestFrameworkConfigStore implements Closeable { + private final Function queryComponentSupplierWrapper; + + public SqlTestFrameworkConfigStore( + Function queryComponentSupplierWrapper) + { + this.queryComponentSupplierWrapper = queryComponentSupplierWrapper; + } + Map configMap = new HashMap<>(); public ConfigurationInstance getConfigurationInstance( - SqlTestFrameworkConfig config, - Function queryComponentSupplierWrapper) throws Exception + SqlTestFrameworkConfig config) throws Exception { ConfigurationInstance ret = configMap.get(config); if (!configMap.containsKey(config)) { @@ -267,7 +284,7 @@ public static List collectAnnotations(Class testClass, Method met */ public static class Rule implements AfterAllCallback, BeforeEachCallback { - SqlTestFrameworkConfigStore configStore = new SqlTestFrameworkConfigStore(); + SqlTestFrameworkConfigStore configStore = new SqlTestFrameworkConfigStore(Function.identity()); private SqlTestFrameworkConfig config; private Method method; private String testName; @@ -318,7 +335,7 @@ public SqlTestFrameworkConfig getConfig() public SqlTestFramework get() throws Exception { - return configStore.getConfigurationInstance(config, Function.identity()).framework; + return configStore.getConfigurationInstance(config).framework; } public T getAnnotation(Class annotationType) @@ -344,6 +361,7 @@ public static class ConfigurationInstance .minTopNThreshold(config.minTopNThreshold) .mergeBufferCount(config.numMergeBuffers) .withOverrideModule(config.resultCache.makeModule()); + framework = builder.build(); } @@ -393,12 +411,43 @@ private Map getNonDefaultMap() if (def.minTopNThreshold != minTopNThreshold) { map.put("minTopNThreshold", String.valueOf(minTopNThreshold)); } + if (def.componentSupplier != componentSupplier) { + map.put("componentSupplier", componentSupplier.getSimpleName()); + } if (!equals(new SqlTestFrameworkConfig(map))) { throw new IAE("Can't reproduce config via map!"); } return map; } + public static SqlTestFrameworkConfig fromURL(String url) throws SQLException + { + + Map queryParams; + queryParams = new HashMap<>(); + try { + URI uri = new URI(url); + if (!DruidAvaticaTestDriver.SCHEME.equals(uri.getScheme())) { + throw new SQLException( + StringUtils.format("URI [%s] is invalid ; only scheme [%s] is supported.", url, DruidAvaticaTestDriver.SCHEME) + ); + } + if (uri.getHost() != null || uri.getPort() != -1) { + throw new SQLException(StringUtils.format("URI [%s] is invalid ; only query parameters are supported.", url)); + } + List params = URLEncodedUtils.parse(uri, StandardCharsets.UTF_8); + for (NameValuePair pair : params) { + queryParams.put(pair.getName(), pair.getValue()); + } + // possible caveat: duplicate entries overwrite earlier ones + } + catch (URISyntaxException e) { + throw new SQLException("Can't decode URI", e); + } + + return new SqlTestFrameworkConfig(queryParams); + } + abstract static class ConfigOptionProcessor { final Class annotationClass; @@ -459,7 +508,15 @@ public final T fromMap(Map map) throws Exception @Override public Set> load(String pkg) { - return new Reflections(pkg).getSubTypesOf(QueryComponentSupplier.class); + Configuration cfg = new ConfigurationBuilder() + .setScanners(new SubTypesScanner(true)) + .setUrls(ClasspathHelper.forJavaClassPath()) + .filterInputsBy( + new FilterBuilder() + .includePackage(pkg) + .and(s -> s.contains("ComponentSupplier")) + ); + return new Reflections(cfg).getSubTypesOf(QueryComponentSupplier.class); } }); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java index 3a7dd2d22d3b..844251e8ac3d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfigTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.sql.calcite.DrillWindowQueryTest.DrillComponentSupplier; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.MinTopNThreshold; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.NumMergeBuffers; import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.ResultCache; @@ -29,6 +30,7 @@ import org.junit.jupiter.api.Test; import java.lang.annotation.Annotation; +import java.net.URI; import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -139,4 +141,15 @@ public void testInvalidConfigKeySpecified() ); } + @Test + public void testURI() + { + SqlTestFrameworkConfig c = new SqlTestFrameworkConfig( + ImmutableMap.of( + "componentSupplier", DrillComponentSupplier.class.getSimpleName() + ) + ); + URI uri = c.getDruidTestURI(); + assertEquals("druidtest:///?componentSupplier=DrillComponentSupplier", uri.toString()); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index 304400bc3d80..07f57a5deeb3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -49,6 +49,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -152,7 +153,8 @@ public static void setupClass() new CalciteRulesManager(ImmutableSet.of()), joinableFactoryWrapper, CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 6155678c8459..f4ada1f1c179 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -62,6 +62,7 @@ import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -97,7 +98,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) "druid", new CalciteRulesManager(ImmutableSet.of()), CalciteTests.TEST_AUTHORIZER_MAPPER, - AuthConfig.newBuilder().build() + AuthConfig.newBuilder().build(), + new DruidHookDispatcher() ); public static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( PLANNER_TOOLBOX, @@ -336,7 +338,7 @@ void testExpression( } ExprEval result = PLANNER_CONTEXT.parseExpression(expression.getExpression()) - + .eval(expressionBindings); Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index 86b5d38639be..c94eac39703d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -24,6 +24,8 @@ import org.apache.calcite.avatica.util.TimeUnit; import org.apache.calcite.avatica.util.TimeUnitRange; import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexWindowBounds; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.SqlOperator; @@ -72,7 +74,9 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.apache.druid.sql.calcite.util.CalciteTestBase; +import org.hamcrest.core.StringContains; import org.joda.time.DateTimeZone; import org.joda.time.Period; import org.junit.Assert; @@ -84,6 +88,8 @@ import java.util.Collections; import java.util.Map; +import static org.hamcrest.MatcherAssert.assertThat; + public class ExpressionsTest extends CalciteTestBase { private static final RowSignature ROW_SIGNATURE = RowSignature @@ -2827,6 +2833,36 @@ public void testHumanReadableDecimalByteFormat() ); } + @Test + public void testPresenceOfOverIsInvalid() + { + final RexBuilder rexBuilder = new RexBuilder(DruidTypeSystem.TYPE_FACTORY); + final PlannerContext plannerContext = Mockito.mock(PlannerContext.class); + Mockito.when(plannerContext.getTimeZone()).thenReturn(DateTimeZone.UTC); + + RexNode rexNode = rexBuilder.makeOver( + testHelper.createSqlType(SqlTypeName.BIGINT), + SqlStdOperatorTable.SUM, + Collections.emptyList(), + Collections.emptyList(), + ImmutableList.of(), + RexWindowBounds.CURRENT_ROW, + RexWindowBounds.CURRENT_ROW, + false, + true, + false, + false, + false + ); + + CannotBuildQueryException t = Assert.assertThrows( + CannotBuildQueryException.class, + () -> testHelper.testExpression(rexNode, null, plannerContext) + ); + + assertThat(t.getMessage(), StringContains.containsString("Unexpected OVER expression")); + } + @Test public void testCalciteLiteralToDruidLiteral() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java index 64a626f682c4..fe26ba9db8c7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java @@ -39,6 +39,7 @@ import org.apache.druid.sql.calcite.schema.NamedViewSchema; import org.apache.druid.sql.calcite.schema.ViewSchema; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -71,7 +72,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) "druid", new CalciteRulesManager(ImmutableSet.of()), CalciteTests.TEST_AUTHORIZER_MAPPER, - AuthConfig.newBuilder().build() + AuthConfig.newBuilder().build(), + new DruidHookDispatcher() ); final PlannerContext plannerContext = PlannerContext.create( toolbox, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java index 43d47733277f..89df405b7f1f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java @@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.schema.NamedSchema; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.easymock.EasyMock; import org.easymock.EasyMockExtension; import org.easymock.Mock; @@ -192,7 +193,8 @@ public void testExtensionCalciteRule() "druid", new CalciteRulesManager(ImmutableSet.of()), CalciteTests.TEST_AUTHORIZER_MAPPER, - AuthConfig.newBuilder().build() + AuthConfig.newBuilder().build(), + new DruidHookDispatcher() ); PlannerContext context = PlannerContext.create( @@ -224,7 +226,8 @@ public void testConfigurableBloat() "druid", new CalciteRulesManager(ImmutableSet.of()), CalciteTests.TEST_AUTHORIZER_MAPPER, - AuthConfig.newBuilder().build() + AuthConfig.newBuilder().build(), + new DruidHookDispatcher() ); PlannerContext contextWithBloat = PlannerContext.create( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java index 205f3379d71b..072bdd6b6e82 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java @@ -54,6 +54,7 @@ import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; @@ -100,7 +101,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) "druid", new CalciteRulesManager(ImmutableSet.of()), CalciteTests.TEST_AUTHORIZER_MAPPER, - AuthConfig.newBuilder().build() + AuthConfig.newBuilder().build(), + new DruidHookDispatcher() ); private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( PLANNER_TOOLBOX, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/run/DruidHookTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/run/DruidHookTest.java new file mode 100644 index 000000000000..62190fb0c693 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/run/DruidHookTest.java @@ -0,0 +1,36 @@ +/* + * 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.sql.calcite.run; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.sql.hook.DruidHook; +import org.junit.Test; + +public class DruidHookTest +{ + @Test + public void testHookKeyEquals() + { + EqualsVerifier.forClass(DruidHook.HookKey.class) + .withNonnullFields("label", "type") + .usingGetClass() + .verify(); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 3fdbafb71a8d..3d0d0352e602 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -365,18 +365,30 @@ public static DruidOperatorTable createOperatorTable() return QueryFrameworkUtils.createOperatorTable(INJECTOR); } - public static SystemSchema createMockSystemSchema( - final DruidSchema druidSchema, - final SpecificSegmentsQuerySegmentWalker walker, - final AuthorizerMapper authorizerMapper - ) + + public static DruidNode mockCoordinatorNode() + { + return new DruidNode("test-coordinator", "dummy", false, 8081, null, true, false); + } + + public static FakeDruidNodeDiscoveryProvider mockDruidNodeDiscoveryProvider(final DruidNode coordinatorNode) { - final DruidNode coordinatorNode = new DruidNode("test-coordinator", "dummy", false, 8081, null, true, false); FakeDruidNodeDiscoveryProvider provider = new FakeDruidNodeDiscoveryProvider( ImmutableMap.of( NodeRole.COORDINATOR, new FakeDruidNodeDiscovery(ImmutableMap.of(NodeRole.COORDINATOR, coordinatorNode)) ) ); + return provider; + } + + public static SystemSchema createMockSystemSchema( + final DruidSchema druidSchema, + final SpecificSegmentsQuerySegmentWalker walker, + final AuthorizerMapper authorizerMapper + ) + { + final DruidNode coordinatorNode = mockCoordinatorNode(); + FakeDruidNodeDiscoveryProvider provider = mockDruidNodeDiscoveryProvider(coordinatorNode); final DruidNode overlordNode = new DruidNode("test-overlord", "dummy", false, 8090, null, true, false); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index d195e0cefb91..c43cb5555ad3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -39,6 +39,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.topn.TopNQueryConfig; import org.apache.druid.segment.DefaultColumnFormatConfig; @@ -67,8 +68,11 @@ import org.apache.druid.sql.calcite.view.DruidViewMacroFactory; import org.apache.druid.sql.calcite.view.InProcessViewManager; import org.apache.druid.sql.calcite.view.ViewManager; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.apache.druid.timeline.DataSegment; +import javax.inject.Named; + import java.io.Closeable; import java.io.IOException; import java.net.URI; @@ -184,6 +188,11 @@ default CatalogResolver createCatalogResolver() default void close() throws IOException { } + + default void configureGuice(CoreInjectorBuilder injectorBuilder, List overrideModules) + { + configureGuice(injectorBuilder); + } } public interface PlannerComponentSupplier @@ -494,7 +503,8 @@ public PlannerFixture( new CalciteRulesManager(componentSupplier.extensionCalciteRules()), framework.injector.getInstance(JoinableFactoryWrapper.class), framework.builder.catalogResolver, - authConfig != null ? authConfig : new AuthConfig() + authConfig != null ? authConfig : new AuthConfig(), + new DruidHookDispatcher() ); componentSupplier.finalizePlanner(this); this.statementFactory = QueryFrameworkUtils.createSqlStatementFactory( @@ -548,7 +558,7 @@ public void configure(Binder binder) { binder.bind(DruidOperatorTable.class).in(LazySingleton.class); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); - binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null)); + binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null)); } @Provides @@ -569,7 +579,14 @@ public JoinableFactoryWrapper joinableFactoryWrapper(final Injector injector) @Provides @LazySingleton - public SpecificSegmentsQuerySegmentWalker segmentsQuerySegmentWalker(final Injector injector) + public QuerySegmentWalker querySegmentWalker(final Injector injector) + { + return injector.getInstance(SpecificSegmentsQuerySegmentWalker.class); + } + + @Provides + @LazySingleton + public SpecificSegmentsQuerySegmentWalker specificSegmentsQuerySegmentWalker(final Injector injector) { SpecificSegmentsQuerySegmentWalker walker = componentSupplier.createQuerySegmentWalker( injector.getInstance(QueryRunnerFactoryConglomerate.class), @@ -585,10 +602,54 @@ public SpecificSegmentsQuerySegmentWalker segmentsQuerySegmentWalker(final Injec public QueryLifecycleFactory queryLifecycleFactory(final Injector injector) { return QueryFrameworkUtils.createMockQueryLifecycleFactory( - injector.getInstance(SpecificSegmentsQuerySegmentWalker.class), + injector.getInstance(QuerySegmentWalker.class), injector.getInstance(QueryRunnerFactoryConglomerate.class) ); } + + @Provides + @LazySingleton + ViewManager createViewManager() + { + return componentSupplier.getPlannerComponentSupplier().createViewManager(); + } + + @Provides + @LazySingleton + public DruidSchemaCatalog makeCatalog( + final Injector injector, + final PlannerConfig plannerConfig, + final AuthConfig authConfig, + final ViewManager viewManager, + QueryRunnerFactoryConglomerate conglomerate, + QuerySegmentWalker walker + ) + { + final DruidSchemaCatalog rootSchema = QueryFrameworkUtils.createMockRootSchema( + injector, + conglomerate, + (SpecificSegmentsQuerySegmentWalker) walker, + plannerConfig, + viewManager, + componentSupplier.getPlannerComponentSupplier().createSchemaManager(), + authorizerMapper, + builder.catalogResolver + ); + return rootSchema; + } + + @Provides + SqlTestFrameworkConfig getTestConfig() + { + return builder.config; + } + + @Provides + @Named("quidem") + public URI getDruidTestURI() + { + return getTestConfig().getDruidTestURI(); + } } public static final DruidViewMacroFactory DRUID_VIEW_MACRO_FACTORY = new TestDruidViewMacroFactory(); @@ -613,18 +674,18 @@ private SqlTestFramework(Builder builder) // Ignore load scopes. This is a unit test, not a Druid node. If a // test pulls in a module, then pull in that module, even though we are // not the Druid node to which the module is scoped. - .ignoreLoadScopes() - .addModule(binder -> binder.bind(Closer.class).toInstance(resourceCloser)) - .addModule(new LookylooModule()) - .addModule(new SegmentWranglerModule()) - .addModule(new SqlAggregationModule()) - .addModule(new ExpressionModule()) - .addModule(new TestSetupModule(builder)); + .ignoreLoadScopes(); + List overrideModules = new ArrayList<>(builder.overrideModules); + overrideModules.add(new LookylooModule()); + overrideModules.add(new SqlAggregationModule()); + overrideModules.add(new SegmentWranglerModule()); + overrideModules.add(new ExpressionModule()); - builder.componentSupplier.configureGuice(injectorBuilder); + overrideModules.add(testSetupModule()); + builder.componentSupplier.configureGuice(injectorBuilder, overrideModules); ServiceInjectorBuilder serviceInjector = new ServiceInjectorBuilder(injectorBuilder); - serviceInjector.addAll(builder.overrideModules); + serviceInjector.addAll(overrideModules); this.injector = serviceInjector.build(); this.engine = builder.componentSupplier.createEngine(queryLifecycleFactory(), queryJsonMapper(), injector); @@ -632,6 +693,11 @@ private SqlTestFramework(Builder builder) componentSupplier.finalizeTestFramework(this); } + public TestSetupModule testSetupModule() + { + return new TestSetupModule(builder); + } + public Injector injector() { return injector; diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index fa3c1edcea24..665d5873b461 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -102,6 +102,7 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.hook.DruidHookDispatcher; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.jupiter.api.AfterAll; @@ -259,7 +260,8 @@ public void setUp() throws Exception new CalciteRulesManager(ImmutableSet.of()), CalciteTests.createJoinableFactoryWrapper(), CatalogResolver.NULL_RESOLVER, - new AuthConfig() + new AuthConfig(), + new DruidHookDispatcher() ); lifecycleManager = new SqlLifecycleManager() diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 6acefa40e7f0..0d097729cbf4 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -206,7 +206,7 @@ export class ConsoleApplication extends React.PureComponent< private readonly openSupervisorSubmit = () => { this.openSupervisorDialog = true; - location.hash = 'supervisor'; + location.hash = 'supervisors'; this.resetInitialsWithDelay(); }; diff --git a/web-console/src/druid-models/async-query/async-query.mock.ts b/web-console/src/druid-models/async-query/async-query.mock.ts index 906f82a753fe..c3c0e1d1926f 100644 --- a/web-console/src/druid-models/async-query/async-query.mock.ts +++ b/web-console/src/druid-models/async-query/async-query.mock.ts @@ -29,9 +29,9 @@ LIMIT 2 */ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { - queryId: 'query-ad84d20a-c331-4ee9-ac59-83024e369cf1', + queryId: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349', state: 'SUCCESS', - createdAt: '2023-07-05T21:33:19.147Z', + createdAt: '2024-07-27T02:39:22.230Z', schema: [ { name: 'channel', @@ -44,10 +44,10 @@ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { nativeType: 'LONG', }, ], - durationMs: 29168, + durationMs: 7183, result: { numTotalRows: 2, - totalSizeInBytes: 116, + totalSizeInBytes: 150, dataSource: '__query_select', sampleRecords: [ ['#en.wikipedia', 6650], @@ -55,12 +55,395 @@ export const SUCCESS_ASYNC_STATUS: AsyncStatusResponse = { ], pages: [ { - numRows: 2, - sizeInBytes: 116, id: 0, + numRows: 2, + sizeInBytes: 150, }, ], }, + stages: [ + { + stageNumber: 0, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_0', + input: [ + { + type: 'table', + dataSource: 'wikipedia', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + ], + processor: { + type: 'groupByPreShuffle', + query: { + queryType: 'groupBy', + dataSource: { + type: 'inputNumber', + inputNumber: 0, + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + granularity: { + type: 'all', + }, + dimensions: [ + { + type: 'default', + dimension: 'channel', + outputName: 'd0', + outputType: 'STRING', + }, + ], + aggregations: [ + { + type: 'count', + name: 'a0', + }, + ], + limitSpec: { + type: 'default', + columns: [ + { + dimension: 'a0', + direction: 'descending', + dimensionOrder: { + type: 'numeric', + }, + }, + ], + limit: 2, + }, + context: { + __resultFormat: 'array', + __user: 'allowAll', + executionMode: 'async', + finalize: true, + maxNumTasks: 2, + maxParseExceptions: 0, + queryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlOuterLimit: 1001, + sqlQueryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlStringifyArrays: false, + }, + }, + }, + signature: [ + { + name: 'd0', + type: 'STRING', + }, + { + name: 'a0', + type: 'LONG', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'd0', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + aggregate: true, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:24.713Z', + duration: 3384, + sort: true, + }, + { + stageNumber: 1, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_1', + input: [ + { + type: 'stage', + stage: 0, + }, + ], + processor: { + type: 'groupByPostShuffle', + query: { + queryType: 'groupBy', + dataSource: { + type: 'inputNumber', + inputNumber: 0, + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + granularity: { + type: 'all', + }, + dimensions: [ + { + type: 'default', + dimension: 'channel', + outputName: 'd0', + outputType: 'STRING', + }, + ], + aggregations: [ + { + type: 'count', + name: 'a0', + }, + ], + limitSpec: { + type: 'default', + columns: [ + { + dimension: 'a0', + direction: 'descending', + dimensionOrder: { + type: 'numeric', + }, + }, + ], + limit: 2, + }, + context: { + __resultFormat: 'array', + __user: 'allowAll', + executionMode: 'async', + finalize: true, + maxNumTasks: 2, + maxParseExceptions: 0, + queryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlOuterLimit: 1001, + sqlQueryId: '45f1dafd-8a52-4eb7-9a6c-77840cddd349', + sqlStringifyArrays: false, + }, + }, + }, + signature: [ + { + name: 'a0', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'd0', + type: 'STRING', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'a0', + order: 'DESCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:28.089Z', + duration: 26, + sort: true, + }, + { + stageNumber: 2, + definition: { + id: 'query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_2', + input: [ + { + type: 'stage', + stage: 1, + }, + ], + processor: { + type: 'limit', + limit: 2, + }, + signature: [ + { + name: 'a0', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'd0', + type: 'STRING', + }, + ], + shuffleSpec: { + type: 'maxCount', + clusterBy: { + columns: [ + { + columnName: 'a0', + order: 'DESCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + }, + partitions: 1, + }, + maxWorkerCount: 1, + }, + phase: 'FINISHED', + workerCount: 1, + partitionCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-27T02:39:28.112Z', + duration: 12, + sort: true, + }, + ], + counters: { + '0': { + '0': { + input0: { + type: 'channel', + rows: [24433], + bytes: [6525055], + files: [1], + totalFiles: [1], + }, + output: { + type: 'channel', + rows: [51], + bytes: [2335], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [51], + bytes: [2131], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + '1': { + '0': { + input0: { + type: 'channel', + rows: [51], + bytes: [2131], + frames: [1], + }, + output: { + type: 'channel', + rows: [51], + bytes: [2998], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [51], + bytes: [2794], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + '2': { + '0': { + input0: { + type: 'channel', + rows: [51], + bytes: [2794], + frames: [1], + }, + output: { + type: 'channel', + rows: [2], + bytes: [150], + frames: [1], + }, + shuffle: { + type: 'channel', + rows: [2], + bytes: [142], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + levelToMergedBatches: { + '0': 1, + '1': 1, + '2': 1, + }, + totalMergersForUltimateLevel: 1, + progressDigest: 1.0, + }, + }, + }, + }, + warnings: [], }; /* @@ -82,18 +465,285 @@ PARTITIONED BY DAY */ export const FAILED_ASYNC_STATUS: AsyncStatusResponse = { - queryId: 'query-36ea273a-bd6d-48de-b890-2d853d879bf8', + queryId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4', state: 'FAILED', - createdAt: '2023-07-05T21:40:39.986Z', - durationMs: 11217, + createdAt: '2024-07-26T18:04:59.873Z', + durationMs: 6954, errorDetails: { error: 'druidException', - errorCode: 'UnknownError', + errorCode: 'TooManyWarnings', persona: 'USER', category: 'UNCATEGORIZED', - errorMessage: 'java.io.UncheckedIOException: /', + errorMessage: 'Too many warnings of type CannotParseExternalData generated (max = 2)', context: { - message: 'java.io.UncheckedIOException: /', + maxWarnings: '2', + rootErrorCode: 'CannotParseExternalData', + }, + }, + stages: [ + { + stageNumber: 0, + definition: { + id: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4_0', + input: [ + { + type: 'external', + inputSource: { + type: 'http', + uris: ['https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json'], + }, + inputFormat: { + type: 'json', + }, + signature: [ + { + name: 'timestamp', + type: 'STRING', + }, + { + name: 'agent_type', + type: 'STRING', + }, + ], + }, + ], + processor: { + type: 'scan', + query: { + queryType: 'scan', + dataSource: { + type: 'external', + inputSource: { + type: 'http', + uris: [ + 'https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json', + ], + }, + inputFormat: { + type: 'json', + }, + signature: [ + { + name: 'timestamp', + type: 'STRING', + }, + { + name: 'agent_type', + type: 'STRING', + }, + ], + }, + intervals: { + type: 'intervals', + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + virtualColumns: [ + { + type: 'expression', + name: 'v0', + expression: 'timestamp_parse("timestamp",null,\'UTC\')', + outputType: 'LONG', + }, + ], + resultFormat: 'compactedList', + columns: ['agent_type', 'v0'], + context: { + __resultFormat: 'array', + __timeColumn: 'v0', + __user: 'allowAll', + executionMode: 'async', + finalize: false, + finalizeAggregations: false, + groupByEnableMultiValueUnnesting: false, + maxNumTasks: 2, + maxParseExceptions: 2, + queryId: 'ea3e36df-ad67-4870-b136-f5616b17d9c4', + scanSignature: '[{"name":"agent_type","type":"STRING"},{"name":"v0","type":"LONG"}]', + sqlInsertSegmentGranularity: '"DAY"', + sqlQueryId: 'ea3e36df-ad67-4870-b136-f5616b17d9c4', + sqlReplaceTimeChunks: 'all', + sqlStringifyArrays: false, + waitUntilSegmentsLoad: true, + }, + columnTypes: ['STRING', 'LONG'], + granularity: { + type: 'all', + }, + legacy: false, + }, + }, + signature: [ + { + name: '__bucket', + type: 'LONG', + }, + { + name: '__boost', + type: 'LONG', + }, + { + name: 'agent_type', + type: 'STRING', + }, + { + name: 'v0', + type: 'LONG', + }, + ], + shuffleSpec: { + type: 'targetSize', + clusterBy: { + columns: [ + { + columnName: '__bucket', + order: 'ASCENDING', + }, + { + columnName: '__boost', + order: 'ASCENDING', + }, + ], + bucketByCount: 1, + }, + targetSize: 3000000, + }, + maxWorkerCount: 1, + shuffleCheckHasMultipleValues: true, + }, + phase: 'FAILED', + workerCount: 1, + shuffle: 'globalSort', + output: 'localStorage', + startTime: '2024-07-26T18:05:02.399Z', + duration: 4056, + sort: true, + }, + { + stageNumber: 1, + definition: { + id: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4_1', + input: [ + { + type: 'stage', + stage: 0, + }, + ], + processor: { + type: 'segmentGenerator', + dataSchema: { + dataSource: 'kttm-blank-lines', + timestampSpec: { + column: '__time', + format: 'millis', + missingValue: null, + }, + dimensionsSpec: { + dimensions: [ + { + type: 'string', + name: 'agent_type', + multiValueHandling: 'SORTED_ARRAY', + createBitmapIndex: true, + }, + ], + dimensionExclusions: ['__time'], + includeAllDimensions: false, + useSchemaDiscovery: false, + }, + metricsSpec: [], + granularitySpec: { + type: 'arbitrary', + queryGranularity: { + type: 'none', + }, + rollup: false, + intervals: ['-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z'], + }, + transformSpec: { + filter: null, + transforms: [], + }, + }, + columnMappings: [ + { + queryColumn: 'v0', + outputColumn: '__time', + }, + { + queryColumn: 'agent_type', + outputColumn: 'agent_type', + }, + ], + tuningConfig: { + maxNumWorkers: 1, + maxRowsInMemory: 100000, + rowsPerSegment: 3000000, + }, + }, + signature: [], + maxWorkerCount: 1, + }, + }, + ], + counters: { + '0': { + '0': { + input0: { + type: 'channel', + rows: [10], + bytes: [7658], + files: [1], + totalFiles: [1], + }, + output: { + type: 'channel', + rows: [10], + bytes: [712], + frames: [1], + }, + sortProgress: { + type: 'sortProgress', + totalMergingLevels: 3, + levelToTotalBatches: { + '0': 1, + '1': 1, + '2': -1, + }, + levelToMergedBatches: {}, + totalMergersForUltimateLevel: -1, + progressDigest: 0.0, + }, + warnings: { + type: 'warnings', + CannotParseExternalData: 3, + }, + }, }, }, + warnings: [ + { + taskId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0', + host: 'localhost:8101', + stageNumber: 0, + error: { + errorCode: 'CannotParseExternalData', + errorMessage: + 'Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)', + }, + exceptionStackTrace: + 'org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94)\n\tat org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115)\n\tat org.apache.druid.segment.RowWalker.advance(RowWalker.java:75)\n\tat org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110)\n\tat org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273)\n\tat org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157)\n\tat org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138)\n\tat org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838)\n\tat java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)\n\tat java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\n\tat org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\n\tat java.base/java.lang.Thread.run(Thread.java:840)\nCaused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input\n at [Source: (byte[])""; line: 1, column: 0]\n\tat com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59)\n\tat com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688)\n\tat com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586)\n\tat com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71)\n\t... 24 more\n', + }, + { + taskId: 'query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0', + host: 'localhost:8101', + stageNumber: 0, + error: { + errorCode: 'CannotParseExternalData', + errorMessage: + 'Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)', + }, + exceptionStackTrace: + 'org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93)\n\tat org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42)\n\tat org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94)\n\tat org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115)\n\tat org.apache.druid.segment.RowWalker.advance(RowWalker.java:75)\n\tat org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110)\n\tat org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273)\n\tat org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88)\n\tat org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157)\n\tat org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230)\n\tat org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138)\n\tat org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838)\n\tat java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)\n\tat java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\n\tat org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)\n\tat java.base/java.lang.Thread.run(Thread.java:840)\nCaused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input\n at [Source: (byte[])""; line: 1, column: 0]\n\tat com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59)\n\tat com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688)\n\tat com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586)\n\tat com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75)\n\tat org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48)\n\tat org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71)\n\t... 24 more\n', + }, + ], }; diff --git a/web-console/src/druid-models/async-query/async-query.ts b/web-console/src/druid-models/async-query/async-query.ts index e823ba6a17e2..71eec3725155 100644 --- a/web-console/src/druid-models/async-query/async-query.ts +++ b/web-console/src/druid-models/async-query/async-query.ts @@ -17,6 +17,8 @@ */ import type { ErrorResponse } from '../../utils'; +import type { Counters, StageDefinition } from '../stages/stages'; +import type { MsqTaskErrorReport } from '../task/task'; export type AsyncState = 'ACCEPTED' | 'RUNNING' | 'SUCCESS' | 'FAILED'; @@ -37,5 +39,8 @@ export interface AsyncStatusResponse { sizeInBytes: number; }[]; }; + stages?: StageDefinition[]; + counters?: Counters; errorDetails?: ErrorResponse; + warnings?: MsqTaskErrorReport[]; } diff --git a/web-console/src/druid-models/execution/execution.spec.ts b/web-console/src/druid-models/execution/execution.spec.ts index 275de4f1757d..2938351a22a8 100644 --- a/web-console/src/druid-models/execution/execution.spec.ts +++ b/web-console/src/druid-models/execution/execution.spec.ts @@ -622,13 +622,13 @@ describe('Execution', () => { { "id": 0, "numRows": 2, - "sizeInBytes": 116, + "sizeInBytes": 150, }, ], - "duration": 29168, + "duration": 7183, "engine": "sql-msq-task", "error": undefined, - "id": "query-ad84d20a-c331-4ee9-ac59-83024e369cf1", + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349", "nativeQuery": undefined, "queryContext": undefined, "result": _QueryResult { @@ -663,8 +663,453 @@ describe('Execution', () => { }, "segmentStatus": undefined, "sqlQuery": undefined, - "stages": undefined, - "startTime": 2023-07-05T21:33:19.147Z, + "stages": Stages { + "counters": { + "0": { + "0": { + "input0": { + "bytes": [ + 6525055, + ], + "files": [ + 1, + ], + "rows": [ + 24433, + ], + "totalFiles": [ + 1, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 2335, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 2131, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + "1": { + "0": { + "input0": { + "bytes": [ + 2131, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 2998, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 2794, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + "2": { + "0": { + "input0": { + "bytes": [ + 2794, + ], + "frames": [ + 1, + ], + "rows": [ + 51, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 150, + ], + "frames": [ + 1, + ], + "rows": [ + 2, + ], + "type": "channel", + }, + "shuffle": { + "bytes": [ + 142, + ], + "frames": [ + 1, + ], + "rows": [ + 2, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": 1, + }, + "progressDigest": 1, + "totalMergersForUltimateLevel": 1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + }, + }, + }, + "stages": [ + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_0", + "input": [ + { + "dataSource": "wikipedia", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "table", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "aggregations": [ + { + "name": "a0", + "type": "count", + }, + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlOuterLimit": 1001, + "sqlQueryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlStringifyArrays": false, + }, + "dataSource": { + "inputNumber": 0, + "type": "inputNumber", + }, + "dimensions": [ + { + "dimension": "channel", + "outputName": "d0", + "outputType": "STRING", + "type": "default", + }, + ], + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "limitSpec": { + "columns": [ + { + "dimension": "a0", + "dimensionOrder": { + "type": "numeric", + }, + "direction": "descending", + }, + ], + "limit": 2, + "type": "default", + }, + "queryType": "groupBy", + }, + "type": "groupByPreShuffle", + }, + "shuffleSpec": { + "aggregate": true, + "clusterBy": { + "columns": [ + { + "columnName": "d0", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "d0", + "type": "STRING", + }, + { + "name": "a0", + "type": "LONG", + }, + ], + }, + "duration": 3384, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 0, + "startTime": "2024-07-27T02:39:24.713Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_1", + "input": [ + { + "stage": 0, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "aggregations": [ + { + "name": "a0", + "type": "count", + }, + ], + "context": { + "__resultFormat": "array", + "__user": "allowAll", + "executionMode": "async", + "finalize": true, + "maxNumTasks": 2, + "maxParseExceptions": 0, + "queryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlOuterLimit": 1001, + "sqlQueryId": "45f1dafd-8a52-4eb7-9a6c-77840cddd349", + "sqlStringifyArrays": false, + }, + "dataSource": { + "inputNumber": 0, + "type": "inputNumber", + }, + "dimensions": [ + { + "dimension": "channel", + "outputName": "d0", + "outputType": "STRING", + "type": "default", + }, + ], + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "limitSpec": { + "columns": [ + { + "dimension": "a0", + "dimensionOrder": { + "type": "numeric", + }, + "direction": "descending", + }, + ], + "limit": 2, + "type": "default", + }, + "queryType": "groupBy", + }, + "type": "groupByPostShuffle", + }, + "shuffleSpec": { + "clusterBy": { + "columns": [ + { + "columnName": "a0", + "order": "DESCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "a0", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "d0", + "type": "STRING", + }, + ], + }, + "duration": 26, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 1, + "startTime": "2024-07-27T02:39:28.089Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-45f1dafd-8a52-4eb7-9a6c-77840cddd349_2", + "input": [ + { + "stage": 1, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "limit": 2, + "type": "limit", + }, + "shuffleSpec": { + "clusterBy": { + "columns": [ + { + "columnName": "a0", + "order": "DESCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "partitions": 1, + "type": "maxCount", + }, + "signature": [ + { + "name": "a0", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "d0", + "type": "STRING", + }, + ], + }, + "duration": 12, + "output": "localStorage", + "partitionCount": 1, + "phase": "FINISHED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 2, + "startTime": "2024-07-27T02:39:28.112Z", + "workerCount": 1, + }, + ], + }, + "startTime": 2024-07-27T02:39:22.230Z, "status": "SUCCESS", "usageInfo": undefined, "warnings": undefined, @@ -679,32 +1124,395 @@ describe('Execution', () => { "capacityInfo": undefined, "destination": undefined, "destinationPages": undefined, - "duration": 11217, + "duration": 6954, "engine": "sql-msq-task", "error": { "error": { "category": "UNCATEGORIZED", "context": { - "message": "java.io.UncheckedIOException: /", + "maxWarnings": "2", + "rootErrorCode": "CannotParseExternalData", }, "error": "druidException", - "errorCode": "UnknownError", - "errorMessage": "java.io.UncheckedIOException: /", + "errorCode": "TooManyWarnings", + "errorMessage": "Too many warnings of type CannotParseExternalData generated (max = 2)", "persona": "USER", }, - "taskId": "query-36ea273a-bd6d-48de-b890-2d853d879bf8", + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4", }, - "id": "query-36ea273a-bd6d-48de-b890-2d853d879bf8", + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4", "nativeQuery": undefined, "queryContext": undefined, "result": undefined, "segmentStatus": undefined, "sqlQuery": undefined, - "stages": undefined, - "startTime": 2023-07-05T21:40:39.986Z, + "stages": Stages { + "counters": { + "0": { + "0": { + "input0": { + "bytes": [ + 7658, + ], + "files": [ + 1, + ], + "rows": [ + 10, + ], + "totalFiles": [ + 1, + ], + "type": "channel", + }, + "output": { + "bytes": [ + 712, + ], + "frames": [ + 1, + ], + "rows": [ + 10, + ], + "type": "channel", + }, + "sortProgress": { + "levelToMergedBatches": {}, + "levelToTotalBatches": { + "0": 1, + "1": 1, + "2": -1, + }, + "progressDigest": 0, + "totalMergersForUltimateLevel": -1, + "totalMergingLevels": 3, + "type": "sortProgress", + }, + "warnings": { + "CannotParseExternalData": 3, + "type": "warnings", + }, + }, + }, + }, + "stages": [ + { + "definition": { + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4_0", + "input": [ + { + "inputFormat": { + "type": "json", + }, + "inputSource": { + "type": "http", + "uris": [ + "https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json", + ], + }, + "signature": [ + { + "name": "timestamp", + "type": "STRING", + }, + { + "name": "agent_type", + "type": "STRING", + }, + ], + "type": "external", + }, + ], + "maxWorkerCount": 1, + "processor": { + "query": { + "columnTypes": [ + "STRING", + "LONG", + ], + "columns": [ + "agent_type", + "v0", + ], + "context": { + "__resultFormat": "array", + "__timeColumn": "v0", + "__user": "allowAll", + "executionMode": "async", + "finalize": false, + "finalizeAggregations": false, + "groupByEnableMultiValueUnnesting": false, + "maxNumTasks": 2, + "maxParseExceptions": 2, + "queryId": "ea3e36df-ad67-4870-b136-f5616b17d9c4", + "scanSignature": "[{"name":"agent_type","type":"STRING"},{"name":"v0","type":"LONG"}]", + "sqlInsertSegmentGranularity": ""DAY"", + "sqlQueryId": "ea3e36df-ad67-4870-b136-f5616b17d9c4", + "sqlReplaceTimeChunks": "all", + "sqlStringifyArrays": false, + "waitUntilSegmentsLoad": true, + }, + "dataSource": { + "inputFormat": { + "type": "json", + }, + "inputSource": { + "type": "http", + "uris": [ + "https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json", + ], + }, + "signature": [ + { + "name": "timestamp", + "type": "STRING", + }, + { + "name": "agent_type", + "type": "STRING", + }, + ], + "type": "external", + }, + "granularity": { + "type": "all", + }, + "intervals": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "type": "intervals", + }, + "legacy": false, + "queryType": "scan", + "resultFormat": "compactedList", + "virtualColumns": [ + { + "expression": "timestamp_parse("timestamp",null,'UTC')", + "name": "v0", + "outputType": "LONG", + "type": "expression", + }, + ], + }, + "type": "scan", + }, + "shuffleCheckHasMultipleValues": true, + "shuffleSpec": { + "clusterBy": { + "bucketByCount": 1, + "columns": [ + { + "columnName": "__bucket", + "order": "ASCENDING", + }, + { + "columnName": "__boost", + "order": "ASCENDING", + }, + ], + }, + "targetSize": 3000000, + "type": "targetSize", + }, + "signature": [ + { + "name": "__bucket", + "type": "LONG", + }, + { + "name": "__boost", + "type": "LONG", + }, + { + "name": "agent_type", + "type": "STRING", + }, + { + "name": "v0", + "type": "LONG", + }, + ], + }, + "duration": 4056, + "output": "localStorage", + "phase": "FAILED", + "shuffle": "globalSort", + "sort": true, + "stageNumber": 0, + "startTime": "2024-07-26T18:05:02.399Z", + "workerCount": 1, + }, + { + "definition": { + "id": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4_1", + "input": [ + { + "stage": 0, + "type": "stage", + }, + ], + "maxWorkerCount": 1, + "processor": { + "columnMappings": [ + { + "outputColumn": "__time", + "queryColumn": "v0", + }, + { + "outputColumn": "agent_type", + "queryColumn": "agent_type", + }, + ], + "dataSchema": { + "dataSource": "kttm-blank-lines", + "dimensionsSpec": { + "dimensionExclusions": [ + "__time", + ], + "dimensions": [ + { + "createBitmapIndex": true, + "multiValueHandling": "SORTED_ARRAY", + "name": "agent_type", + "type": "string", + }, + ], + "includeAllDimensions": false, + "useSchemaDiscovery": false, + }, + "granularitySpec": { + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z", + ], + "queryGranularity": { + "type": "none", + }, + "rollup": false, + "type": "arbitrary", + }, + "metricsSpec": [], + "timestampSpec": { + "column": "__time", + "format": "millis", + "missingValue": null, + }, + "transformSpec": { + "filter": null, + "transforms": [], + }, + }, + "tuningConfig": { + "maxNumWorkers": 1, + "maxRowsInMemory": 100000, + "rowsPerSegment": 3000000, + }, + "type": "segmentGenerator", + }, + "signature": [], + }, + "stageNumber": 1, + }, + ], + }, + "startTime": 2024-07-26T18:04:59.873Z, "status": "FAILED", "usageInfo": undefined, - "warnings": undefined, + "warnings": [ + { + "error": { + "errorCode": "CannotParseExternalData", + "errorMessage": "Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3)", + }, + "exceptionStackTrace": "org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 3, Line: 3) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94) + at org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115) + at org.apache.druid.segment.RowWalker.advance(RowWalker.java:75) + at org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110) + at org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273) + at org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157) + at org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138) + at org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838) + at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) + at java.base/java.lang.Thread.run(Thread.java:840) + Caused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input + at [Source: (byte[])""; line: 1, column: 0] + at com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59) + at com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688) + at com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586) + at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71) + ... 24 more + ", + "host": "localhost:8101", + "stageNumber": 0, + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0", + }, + { + "error": { + "errorCode": "CannotParseExternalData", + "errorMessage": "Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7)", + }, + "exceptionStackTrace": "org.apache.druid.java.util.common.parsers.ParseException: Unable to parse row [] (Path: https://static.imply.io/example-data/kttm-with-issues/kttm-blank-lines.json, Record: 6, Line: 7) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:80) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.findNextIteratorIfNecessary(CloseableIterator.java:72) + at org.apache.druid.java.util.common.parsers.CloseableIterator$2.hasNext(CloseableIterator.java:93) + at org.apache.druid.java.util.common.parsers.CloseableIterator$1.hasNext(CloseableIterator.java:42) + at org.apache.druid.msq.input.external.ExternalSegment$1$1.hasNext(ExternalSegment.java:94) + at org.apache.druid.java.util.common.guava.BaseSequence$1.next(BaseSequence.java:115) + at org.apache.druid.segment.RowWalker.advance(RowWalker.java:75) + at org.apache.druid.segment.RowBasedCursor.advanceUninterruptibly(RowBasedCursor.java:110) + at org.apache.druid.segment.RowBasedCursor.advance(RowBasedCursor.java:103) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeeded(ScanQueryFrameProcessor.java:374) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.populateFrameWriterAndFlushIfNeededWithExceptionHandling(ScanQueryFrameProcessor.java:334) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runWithSegment(ScanQueryFrameProcessor.java:273) + at org.apache.druid.msq.querykit.BaseLeafFrameProcessor.runIncrementally(BaseLeafFrameProcessor.java:88) + at org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor.runIncrementally(ScanQueryFrameProcessor.java:157) + at org.apache.druid.frame.processor.FrameProcessors$1FrameProcessorWithBaggage.runIncrementally(FrameProcessors.java:75) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.runProcessorNow(FrameProcessorExecutor.java:230) + at org.apache.druid.frame.processor.FrameProcessorExecutor$1ExecutorRunnable.run(FrameProcessorExecutor.java:138) + at org.apache.druid.msq.exec.WorkerImpl$1$2.run(WorkerImpl.java:838) + at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:259) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) + at java.base/java.lang.Thread.run(Thread.java:840) + Caused by: com.fasterxml.jackson.databind.exc.MismatchedInputException: No content to map due to end-of-input + at [Source: (byte[])""; line: 1, column: 0] + at com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:59) + at com.fasterxml.jackson.databind.ObjectMapper._initForReading(ObjectMapper.java:4688) + at com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4586) + at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3609) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:75) + at org.apache.druid.data.input.impl.JsonLineReader.parseInputRows(JsonLineReader.java:48) + at org.apache.druid.data.input.IntermediateRowParsingReader$1.hasNext(IntermediateRowParsingReader.java:71) + ... 24 more + ", + "host": "localhost:8101", + "stageNumber": 0, + "taskId": "query-ea3e36df-ad67-4870-b136-f5616b17d9c4-worker0_0", + }, + ], } `); }); diff --git a/web-console/src/druid-models/execution/execution.ts b/web-console/src/druid-models/execution/execution.ts index dcb5b2d1b939..45bfc5175389 100644 --- a/web-console/src/druid-models/execution/execution.ts +++ b/web-console/src/druid-models/execution/execution.ts @@ -194,7 +194,7 @@ export interface ExecutionValue { export class Execution { static USE_TASK_PAYLOAD = true; - static USE_TASK_REPORTS = true; + static USE_TASK_REPORTS = false; static INLINE_DATASOURCE_MARKER = '__query_select'; static getClusterCapacity: (() => Promise) | undefined = @@ -235,7 +235,7 @@ export class Execution { sqlQuery?: string, queryContext?: QueryContext, ): Execution { - const { queryId, schema, result, errorDetails } = asyncSubmitResult; + const { queryId, schema, result, errorDetails, stages, counters, warnings } = asyncSubmitResult; let queryResult: QueryResult | undefined; if (schema && result?.sampleRecords) { @@ -263,6 +263,8 @@ export class Execution { status: Execution.normalizeAsyncState(asyncSubmitResult.state), sqlQuery, queryContext, + stages: Array.isArray(stages) && counters ? new Stages(stages, counters) : undefined, + warnings: Array.isArray(warnings) ? warnings : undefined, error: executionError, destination: typeof result?.dataSource === 'string' diff --git a/web-console/src/druid-models/stages/stages.ts b/web-console/src/druid-models/stages/stages.ts index 04cf22be8f98..0d263a405535 100644 --- a/web-console/src/druid-models/stages/stages.ts +++ b/web-console/src/druid-models/stages/stages.ts @@ -74,6 +74,8 @@ export interface StageDefinition { startTime?: string; duration?: number; sort?: boolean; + shuffle?: string; + output?: string; } export interface ClusterBy { @@ -169,7 +171,7 @@ export type SegmentGenerationProgressFields = | 'rowsPushed'; export interface WarningCounter { - type: 'warning'; + type: 'warnings'; CannotParseExternalData?: number; // More types of warnings might be added later } @@ -192,6 +194,8 @@ function zeroChannelFields(): Record { }; } +export type Counters = Record>; + export class Stages { static readonly QUERY_START_FACTOR = 0.05; static readonly QUERY_END_FACTOR = 0.05; @@ -205,12 +209,9 @@ export class Stages { } public readonly stages: StageDefinition[]; - private readonly counters?: Record>; + private readonly counters?: Counters; - constructor( - stages: StageDefinition[], - counters?: Record>, - ) { + constructor(stages: StageDefinition[], counters?: Counters) { this.stages = stages; this.counters = counters; } diff --git a/web-console/src/druid-models/task/task.ts b/web-console/src/druid-models/task/task.ts index f8e329c34058..1a245508e8de 100644 --- a/web-console/src/druid-models/task/task.ts +++ b/web-console/src/druid-models/task/task.ts @@ -18,7 +18,7 @@ import { C } from '@druid-toolkit/query'; -import type { StageDefinition } from '../stages/stages'; +import type { Counters, StageDefinition } from '../stages/stages'; export type TaskStatus = 'WAITING' | 'PENDING' | 'RUNNING' | 'FAILED' | 'SUCCESS'; export type TaskStatusWithCanceled = TaskStatus | 'CANCELED'; @@ -112,7 +112,7 @@ export interface MsqTaskReportResponse { segmentLoadWaiterStatus?: SegmentLoadWaiterStatus; }; stages: StageDefinition[]; - counters: Record>; + counters: Counters; }; }; error?: any; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.ts b/web-console/src/druid-models/workbench-query/workbench-query.ts index 05818b27606f..76bd29a44482 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.ts @@ -151,11 +151,12 @@ export class WorkbenchQuery { return WorkbenchQuery.enabledQueryEngines; } - static fromEffectiveQueryAndContext(queryString: string, context: QueryContext): WorkbenchQuery { + static fromTaskQueryAndContext(queryString: string, context: QueryContext): WorkbenchQuery { const noSqlOuterLimit = typeof context['sqlOuterLimit'] === 'undefined'; const cleanContext = deleteKeys(context, ['sqlOuterLimit']); let retQuery = WorkbenchQuery.blank() + .changeEngine('sql-msq-task') .changeQueryString(queryString) .changeQueryContext(cleanContext); diff --git a/web-console/src/helpers/execution/sql-task-execution.ts b/web-console/src/helpers/execution/sql-task-execution.ts index ca7c0485bd4a..0fa9b0909597 100644 --- a/web-console/src/helpers/execution/sql-task-execution.ts +++ b/web-console/src/helpers/execution/sql-task-execution.ts @@ -184,7 +184,7 @@ export async function getTaskExecution( if (!execution) { const statusResp = await Api.instance.get( - `/druid/v2/sql/statements/${encodedId}`, + `/druid/v2/sql/statements/${encodedId}?detail=true`, { cancelToken, }, diff --git a/web-console/src/utils/download.spec.ts b/web-console/src/utils/download.spec.ts index 85d6669acc9c..71c051500d0d 100644 --- a/web-console/src/utils/download.spec.ts +++ b/web-console/src/utils/download.spec.ts @@ -20,7 +20,10 @@ import { formatForFormat } from './download'; describe('download', () => { it('.formatForFormat', () => { - expect(formatForFormat(null, 'csv')).toEqual('"null"'); + expect(formatForFormat(null, 'csv')).toEqual(''); + expect(formatForFormat(null, 'tsv')).toEqual(''); + expect(formatForFormat('', 'csv')).toEqual('""'); + expect(formatForFormat('null', 'csv')).toEqual('"null"'); expect(formatForFormat('hello\nworld', 'csv')).toEqual('"hello world"'); expect(formatForFormat(123, 'csv')).toEqual('"123"'); expect(formatForFormat(new Date('2021-01-02T03:04:05.678Z'), 'csv')).toEqual( diff --git a/web-console/src/utils/download.ts b/web-console/src/utils/download.ts index aabe2fc1c66b..4fb3342e523c 100644 --- a/web-console/src/utils/download.ts +++ b/web-console/src/utils/download.ts @@ -43,6 +43,8 @@ export function downloadUrl(url: string, filename: string) { } export function formatForFormat(s: null | string | number | Date, format: 'csv' | 'tsv'): string { + if (s == null) return ''; + // stringify and remove line break const str = stringifyValue(s).replace(/(?:\r\n|\r|\n)/g, ' '); diff --git a/web-console/src/utils/druid-query.ts b/web-console/src/utils/druid-query.ts index 154103297044..fba63b946000 100644 --- a/web-console/src/utils/druid-query.ts +++ b/web-console/src/utils/druid-query.ts @@ -23,7 +23,7 @@ import axios from 'axios'; import { Api } from '../singletons'; import type { RowColumn } from './general'; -import { assemble } from './general'; +import { assemble, lookupBy } from './general'; const CANCELED_MESSAGE = 'Query canceled by user.'; @@ -345,10 +345,24 @@ export async function queryDruidSql( export interface QueryExplanation { query: any; signature: { name: string; type: string }[]; + columnMappings: { + queryColumn: string; + outputColumn: string; + }[]; } -export function formatSignature(queryExplanation: QueryExplanation): string { - return queryExplanation.signature - .map(({ name, type }) => `${C.optionalQuotes(name)}::${type}`) +export function formatColumnMappingsAndSignature(queryExplanation: QueryExplanation): string { + const columnNameToType = lookupBy( + queryExplanation.signature, + c => c.name, + c => c.type, + ); + return queryExplanation.columnMappings + .map(({ queryColumn, outputColumn }) => { + const type = columnNameToType[queryColumn]; + return `${C.optionalQuotes(queryColumn)}${type ? `::${type}` : ''}→${C.optionalQuotes( + outputColumn, + )}`; + }) .join(', '); } diff --git a/web-console/src/views/workbench-view/explain-dialog/__snapshots__/explain-dialog.spec.tsx.snap b/web-console/src/views/workbench-view/explain-dialog/__snapshots__/explain-dialog.spec.tsx.snap index e2a5a6c3ded5..806f209ef638 100644 --- a/web-console/src/views/workbench-view/explain-dialog/__snapshots__/explain-dialog.spec.tsx.snap +++ b/web-console/src/views/workbench-view/explain-dialog/__snapshots__/explain-dialog.spec.tsx.snap @@ -185,7 +185,7 @@ exports[`ExplainDialog matches snapshot on some data (many queries) 1`] = ` label="Signature" > @@ -287,7 +287,7 @@ exports[`ExplainDialog matches snapshot on some data (many queries) 1`] = ` label="Signature" > @@ -473,7 +473,7 @@ exports[`ExplainDialog matches snapshot on some data (one query) 1`] = ` label="Signature" > diff --git a/web-console/src/views/workbench-view/explain-dialog/explain-dialog.spec.tsx b/web-console/src/views/workbench-view/explain-dialog/explain-dialog.spec.tsx index 0d6a878d424c..bf3b1ece6ed8 100644 --- a/web-console/src/views/workbench-view/explain-dialog/explain-dialog.spec.tsx +++ b/web-console/src/views/workbench-view/explain-dialog/explain-dialog.spec.tsx @@ -160,6 +160,16 @@ describe('ExplainDialog', () => { type: 'LONG', }, ], + columnMappings: [ + { + queryColumn: 'd0', + outputColumn: 'channel', + }, + { + queryColumn: 'a0', + outputColumn: 'Count', + }, + ], }, ], }); @@ -199,6 +209,12 @@ describe('ExplainDialog', () => { type: 'STRING', }, ], + columnMappings: [ + { + queryColumn: 'channel', + outputColumn: 'channel', + }, + ], }, { query: { @@ -234,6 +250,12 @@ describe('ExplainDialog', () => { type: 'STRING', }, ], + columnMappings: [ + { + queryColumn: 'channel', + outputColumn: 'channel', + }, + ], }, ], }); diff --git a/web-console/src/views/workbench-view/explain-dialog/explain-dialog.tsx b/web-console/src/views/workbench-view/explain-dialog/explain-dialog.tsx index 4bab7e7bfb05..7f01436babbd 100644 --- a/web-console/src/views/workbench-view/explain-dialog/explain-dialog.tsx +++ b/web-console/src/views/workbench-view/explain-dialog/explain-dialog.tsx @@ -40,7 +40,7 @@ import { Api } from '../../../singletons'; import type { QueryExplanation } from '../../../utils'; import { deepGet, - formatSignature, + formatColumnMappingsAndSignature, getDruidErrorMessage, nonEmptyArray, queryDruidSql, @@ -141,7 +141,7 @@ export const ExplainDialog = React.memo(function ExplainDialog(props: ExplainDia /> - + {openQueryLabel && (