mirror of https://github.com/apache/druid.git
Quidem record (#16624)
* enables to launch a fake broker based on test resources (druidtest uri) * could record queries into new testfiles during usage * instead of re-purpose Calcite's Hook migrates to use DruidHook which we can add further keys * added a quidem-ut module which could be the place for tests which could iteract with modules/etc
This commit is contained in:
parent
08f9ec1cae
commit
26e3c44f4b
|
@ -27,7 +27,7 @@ README
|
|||
.pmdruleset.xml
|
||||
.java-version
|
||||
integration-tests/gen-scripts/
|
||||
/bin/
|
||||
**/bin/
|
||||
*.hprof
|
||||
**/.ipynb_checkpoints/
|
||||
website/.yarn/
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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 class InPlanningBenchmark
|
|||
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 ");
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.apache.druid.sql.calcite.util.CalciteTests;
|
|||
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 class SqlBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(injector)),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
return Pair.of(plannerFactory, engine);
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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 class SqlExpressionBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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;
|
||||
|
@ -347,7 +348,8 @@ public class SqlGroupByBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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 class SqlNestedDataBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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 class SqlVsNativeBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
groupByQuery = GroupByQuery
|
||||
.builder()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -166,6 +166,11 @@
|
|||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
|
|
@ -164,6 +164,11 @@
|
|||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
|
|
|
@ -170,6 +170,11 @@
|
|||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
|
|
|
@ -139,6 +139,11 @@
|
|||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
|
|
|
@ -177,6 +177,11 @@
|
|||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
|
|
|
@ -123,6 +123,11 @@
|
|||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
|
|
|
@ -233,6 +233,11 @@
|
|||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.druid.sql.calcite.schema.NamedDruidSchema;
|
|||
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.Before;
|
||||
|
@ -75,7 +76,8 @@ public class ResultsContextSerdeTest
|
|||
"druid",
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
AuthConfig.newBuilder().build()
|
||||
AuthConfig.newBuilder().build(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
final NativeSqlEngine engine = CalciteTests.createMockSqlEngine(
|
||||
EasyMock.createMock(QuerySegmentWalker.class),
|
||||
|
|
|
@ -186,6 +186,7 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSuppl
|
|||
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;
|
||||
|
@ -562,7 +563,8 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
catalogResolver,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory);
|
||||
|
|
|
@ -123,7 +123,11 @@
|
|||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
|
|
|
@ -145,6 +145,11 @@
|
|||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
|
|
1
pom.xml
1
pom.xml
|
@ -242,6 +242,7 @@
|
|||
<module>integration-tests-ex/tools</module>
|
||||
<module>integration-tests-ex/image</module>
|
||||
<module>integration-tests-ex/cases</module>
|
||||
<module>quidem-ut</module>
|
||||
</modules>
|
||||
|
||||
<repositories>
|
||||
|
|
|
@ -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.
|
||||
-->
|
||||
|
||||
# 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
|
||||
|
|
@ -0,0 +1,546 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Licensed to the Apache Software Foundation (ASF) under one
|
||||
~ or more contributor license agreements. See the NOTICE file
|
||||
~ distributed with this work for additional information
|
||||
~ regarding copyright ownership. The ASF licenses this file
|
||||
~ to you under the Apache License, Version 2.0 (the
|
||||
~ "License"); you may not use this file except in compliance
|
||||
~ with the License. You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing,
|
||||
~ software distributed under the License is distributed on an
|
||||
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
~ KIND, either express or implied. See the License for the
|
||||
~ specific language governing permissions and limitations
|
||||
~ under the License.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>druid-quidem-ut</artifactId>
|
||||
<name>druid-quidem-ut</name>
|
||||
<description>druid-quidem-ut</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>31.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<repositories>
|
||||
<repository>
|
||||
<id>confluent</id>
|
||||
<url>https://packages.confluent.io/maven/</url>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
<properties>
|
||||
<quidem.uri>druidtest:///</quidem.uri>
|
||||
<quidem.record.autostart>false</quidem.record.autostart>
|
||||
</properties>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>amazon-kinesis-producer</artifactId>
|
||||
<version>0.13.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.opencsv</groupId>
|
||||
<artifactId>opencsv</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpcore</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpclient</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.rvesse</groupId>
|
||||
<artifactId>airline</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-kinesis</artifactId>
|
||||
<version>${aws.sdk.version}</version>
|
||||
<exclusions>
|
||||
<!-- aws-java-sdk-core is provided by Druid core. -->
|
||||
<exclusion>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-core</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-core</artifactId>
|
||||
<version>${aws.sdk.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-codec</groupId>
|
||||
<artifactId>commons-codec</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-s3</artifactId>
|
||||
<version>${aws.sdk.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-orc-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>servlet-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.squareup.okhttp</groupId>
|
||||
<artifactId>okhttp</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-parquet-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-avro-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-protobuf-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-s3-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-kinesis-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-azure-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-google-extensions</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-hdfs-storage</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-bundle</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-datasketches</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-histogram</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-aws-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>mysql-metadata-storage</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-kafka-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-clients</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-basic-security</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-lookups-cached-global</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-testing-tools</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>simple-client-sslcontext</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-multi-stage-query</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-services</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-multibindings</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-clients</artifactId>
|
||||
<version>${apache.kafka.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.ws.rs</groupId>
|
||||
<artifactId>jsr311-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.jaxrs</groupId>
|
||||
<artifactId>jackson-jaxrs-smile-provider</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>jakarta.validation</groupId>
|
||||
<artifactId>jakarta.validation-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>jakarta.inject</groupId>
|
||||
<artifactId>jakarta.inject-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-slf4j-impl</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-transport-jersey</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-transport-netty</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.docker-java</groupId>
|
||||
<artifactId>docker-java-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.hydromatic</groupId>
|
||||
<artifactId>quidem</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-transport-native-kqueue</artifactId>
|
||||
<classifier>osx-x86_64</classifier>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-transport-native-kqueue</artifactId>
|
||||
<classifier>osx-aarch_64</classifier>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-servlet</artifactId>
|
||||
<version>${jetty.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-server</artifactId>
|
||||
<version>${jetty.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-servlet</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.confluent</groupId>
|
||||
<artifactId>kafka-schema-registry-client</artifactId>
|
||||
<version>6.2.12</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.ws.rs</groupId>
|
||||
<artifactId>javax.ws.rs-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.ws.rs</groupId>
|
||||
<artifactId>javax.ws.rs-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.ws.rs</groupId>
|
||||
<artifactId>jsr311-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>jakarta.ws.rs</groupId>
|
||||
<artifactId>jakarta.ws.rs-api</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.confluent</groupId>
|
||||
<artifactId>kafka-protobuf-provider</artifactId>
|
||||
<version>6.2.12</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.protobuf</groupId>
|
||||
<artifactId>protobuf-java</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<type>test-jar</type>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<type>test-jar</type>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<type>test-jar</type>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-datasketches</artifactId>
|
||||
<type>test-jar</type>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-simple</artifactId>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.reflections</groupId>
|
||||
<artifactId>reflections</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite.avatica</groupId>
|
||||
<artifactId>avatica</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite.avatica</groupId>
|
||||
<artifactId>avatica-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite</groupId>
|
||||
<artifactId>calcite-testkit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.os72</groupId>
|
||||
<artifactId>protobuf-dynamic</artifactId>
|
||||
<version>0.9.3</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-sts</artifactId>
|
||||
<version>${aws.sdk.version}</version>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.datasketches</groupId>
|
||||
<artifactId>datasketches-java</artifactId>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-all</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-core</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.owasp</groupId>
|
||||
<artifactId>dependency-check-maven</artifactId>
|
||||
<configuration>
|
||||
<skip>true</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<configuration>
|
||||
<skip>true</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>exec-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<executable>java</executable>
|
||||
<arguments>
|
||||
<argument>-classpath</argument>
|
||||
<classpath />
|
||||
<argument>-Dquidem.uri=${quidem.uri}</argument>
|
||||
<argument>-Dquidem.record.autostart=${quidem.record.autostart}</argument>
|
||||
<argument>org.apache.druid.quidem.Launcher</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -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<Module> 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<? extends Module> 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));
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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<Object, Object> 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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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<String>
|
||||
{
|
||||
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<String> 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 + "]";
|
||||
}
|
||||
|
||||
}
|
|
@ -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\":\""));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
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 class StatusResource
|
|||
|
||||
private String getDruidVersion()
|
||||
{
|
||||
return Status.class.getPackage().getImplementationVersion();
|
||||
return GuavaUtils.firstNonNull(Status.class.getPackage().getImplementationVersion(), "unknown");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.calcite.rel.rules.DateRangeRules;
|
|||
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.SortCollapseRule;
|
|||
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 @@ public class CalciteRulesManager
|
|||
public RelNode run(RelOptPlanner planner, RelNode rel, RelTraitSet requiredOutputTraits,
|
||||
List<RelOptMaterialization> materializations, List<RelOptLattice> lattices)
|
||||
{
|
||||
Hook.TRIMMED.run(rel);
|
||||
PlannerContext pctx = planner.getContext().unwrapOrThrow(PlannerContext.class);
|
||||
pctx.dispatchHook(DruidHook.LOGICAL_PLAN, rel);
|
||||
return rel;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.rule.ReverseLookupRule;
|
|||
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 class PlannerContext
|
|||
|
||||
return lookupCache.getLookup(lookupName);
|
||||
}
|
||||
|
||||
public <T> void dispatchHook(HookKey<T> key, T object)
|
||||
{
|
||||
plannerToolbox.getHookDispatcher().dispatch(key, object);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,6 +49,8 @@ import org.apache.druid.sql.calcite.planner.convertlet.DruidConvertletTable;
|
|||
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 class PlannerFactory extends PlannerToolbox
|
|||
final CalciteRulesManager calciteRuleManager,
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper,
|
||||
final CatalogResolver catalog,
|
||||
final AuthConfig authConfig
|
||||
final AuthConfig authConfig,
|
||||
final DruidHookDispatcher hookDispatcher
|
||||
)
|
||||
{
|
||||
super(
|
||||
|
@ -91,7 +94,8 @@ public class PlannerFactory extends PlannerToolbox
|
|||
druidSchemaName,
|
||||
calciteRuleManager,
|
||||
authorizerMapper,
|
||||
authConfig
|
||||
authConfig,
|
||||
hookDispatcher
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -112,6 +116,7 @@ public class PlannerFactory extends PlannerToolbox
|
|||
queryContext,
|
||||
hook
|
||||
);
|
||||
context.dispatchHook(DruidHook.SQL, sql);
|
||||
|
||||
return new DruidPlanner(buildFrameworkConfig(context), context, engine, hook);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlInsert;
|
|||
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 @@ import org.apache.druid.sql.calcite.rel.DruidRel;
|
|||
* 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);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
|||
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 class 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 class PlannerToolbox
|
|||
this.calciteRuleManager = calciteRuleManager;
|
||||
this.authorizerMapper = authorizerMapper;
|
||||
this.authConfig = authConfig;
|
||||
this.hookDispatcher = hookDispatcher;
|
||||
}
|
||||
|
||||
public DruidOperatorTable operatorTable()
|
||||
|
@ -117,4 +121,9 @@ public class PlannerToolbox
|
|||
{
|
||||
return authConfig;
|
||||
}
|
||||
|
||||
public DruidHookDispatcher getHookDispatcher()
|
||||
{
|
||||
return hookDispatcher;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.calcite.rel.type.RelDataType;
|
|||
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.rel.logical.DruidLogicalNode;
|
|||
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 abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
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 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
.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 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
|
||||
handlerContext.hook().captureDruidRel(druidRel);
|
||||
|
||||
Hook.JAVA_PLAN.run(druidRel);
|
||||
plannerContext.dispatchHook(DruidHook.DRUID_PLAN, druidRel);
|
||||
|
||||
if (explain != null) {
|
||||
return planExplanation(possiblyLimitedRoot, druidRel, true);
|
||||
|
|
|
@ -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<T>
|
||||
{
|
||||
HookKey<RelNode> CONVERTED_PLAN = new HookKey<>("converted", RelNode.class);
|
||||
HookKey<RelNode> LOGICAL_PLAN = new HookKey<>("logicalPlan", RelNode.class);
|
||||
HookKey<RelNode> DRUID_PLAN = new HookKey<>("druidPlan", RelNode.class);
|
||||
HookKey<String> SQL = new HookKey<>("sql", String.class);
|
||||
|
||||
@Immutable
|
||||
class HookKey<T>
|
||||
{
|
||||
private final String label;
|
||||
private final Class<T> type;
|
||||
|
||||
public HookKey(String label, Class<T> 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<T> key, T object);
|
||||
}
|
|
@ -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<HookKey<?>, List<DruidHook<?>>> 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 <T> Closeable withHook(HookKey<T> key, DruidHook<T> hook)
|
||||
{
|
||||
register(key, hook);
|
||||
return new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
unregister(key, hook);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
public <T> void dispatch(HookKey<T> key, T object)
|
||||
{
|
||||
List<DruidHook<?>> currentHooks = hooks.get(key);
|
||||
if (currentHooks != null) {
|
||||
for (DruidHook hook : currentHooks) {
|
||||
hook.invoke(key, object);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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 class DruidAvaticaDriverTest
|
|||
@Test
|
||||
public void testURIParse() throws SQLException
|
||||
{
|
||||
DruidAvaticaTestDriver.buildConfigfromURIParams("druidtest:///");
|
||||
SqlTestFrameworkConfig.fromURL("druidtest:///");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -70,25 +70,18 @@ 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.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 class DruidAvaticaTestDriver implements Driver
|
|||
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 class DruidAvaticaTestDriver implements Driver
|
|||
|
||||
@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 class DruidAvaticaTestDriver implements Driver
|
|||
return tempDir;
|
||||
}
|
||||
|
||||
public static SqlTestFrameworkConfig buildConfigfromURIParams(String url) throws SQLException
|
||||
{
|
||||
Map<String, String> queryParams;
|
||||
queryParams = new HashMap<>();
|
||||
try {
|
||||
List<NameValuePair> 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 {
|
||||
|
|
|
@ -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 interface DruidConnectionExtras
|
|||
{
|
||||
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!");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,12 +35,16 @@ import org.apache.druid.query.Query;
|
|||
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 class DruidQuidemCommandHandler implements CommandHandler
|
|||
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 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
*/
|
||||
abstract static class AbstractRelPlanCommand extends AbstractPlanCommand
|
||||
{
|
||||
Hook hook;
|
||||
HookKey<RelNode> hook;
|
||||
|
||||
AbstractRelPlanCommand(List<String> lines, List<String> content, Hook hook)
|
||||
AbstractRelPlanCommand(List<String> lines, List<String> content, DruidHook.HookKey<RelNode> 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<RelNode> logged = new ArrayList<>();
|
||||
try (final Hook.Closeable unhook = hook.add((Consumer<RelNode>) logged::add)) {
|
||||
try (Closeable unhook = dhp.withHook(hook, (key, relNode) -> {
|
||||
logged.add(relNode);
|
||||
})) {
|
||||
executeQuery(x);
|
||||
}
|
||||
|
||||
|
@ -179,21 +186,26 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
x.echo(ImmutableList.of(str));
|
||||
}
|
||||
}
|
||||
|
||||
protected final DruidHookDispatcher unwrapDruidHookDispatcher(Context x)
|
||||
{
|
||||
return DruidConnectionExtras.unwrapOrThrow(x.connection()).getDruidHookDispatcher();
|
||||
}
|
||||
}
|
||||
|
||||
static class LogicalPlanCommand extends AbstractRelPlanCommand
|
||||
{
|
||||
LogicalPlanCommand(List<String> lines, List<String> content)
|
||||
{
|
||||
super(lines, content, Hook.TRIMMED);
|
||||
super(lines, content, DruidHook.LOGICAL_PLAN);
|
||||
}
|
||||
}
|
||||
|
||||
static class PhysicalPlanCommand extends AbstractRelPlanCommand
|
||||
static class DruidPlanCommand extends AbstractRelPlanCommand
|
||||
{
|
||||
PhysicalPlanCommand(List<String> lines, List<String> content)
|
||||
DruidPlanCommand(List<String> lines, List<String> content)
|
||||
{
|
||||
super(lines, content, Hook.JAVA_PLAN);
|
||||
super(lines, content, DruidHook.DRUID_PLAN);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -201,7 +213,7 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
{
|
||||
ConvertedPlanCommand(List<String> lines, List<String> content)
|
||||
{
|
||||
super(lines, content, Hook.CONVERTED);
|
||||
super(lines, content, DruidHook.CONVERTED_PLAN);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,12 +29,14 @@ import org.apache.calcite.util.Closer;
|
|||
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 @@ import static org.junit.jupiter.api.Assertions.fail;
|
|||
*
|
||||
*/
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
@ExtendWith(EnabledOnlyInSqlCompatibleMode.class)
|
||||
public abstract class DruidQuidemTestBase
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
public static final String IQ_SUFFIX = ".iq";
|
||||
/**
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -58,6 +58,7 @@ import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
|||
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 class SqlStatementTest
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper,
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
this.sqlStatementFactory = new SqlStatementFactory(
|
||||
|
|
|
@ -89,6 +89,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.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 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
|||
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 class DruidStatementTest extends CalciteTestBase
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper,
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
this.sqlStatementFactory = CalciteTests.createSqlStatementFactory(
|
||||
CalciteTests.createMockSqlEngine(walker, conglomerate),
|
||||
|
|
|
@ -30,15 +30,23 @@ import com.google.common.collect.Sets;
|
|||
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.Constructor;
|
|||
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 class SqlTestFrameworkConfig
|
|||
|
||||
public static class SqlTestFrameworkConfigStore implements Closeable
|
||||
{
|
||||
private final Function<QueryComponentSupplier, QueryComponentSupplier> queryComponentSupplierWrapper;
|
||||
|
||||
public SqlTestFrameworkConfigStore(
|
||||
Function<QueryComponentSupplier, QueryComponentSupplier> queryComponentSupplierWrapper)
|
||||
{
|
||||
this.queryComponentSupplierWrapper = queryComponentSupplierWrapper;
|
||||
}
|
||||
|
||||
Map<SqlTestFrameworkConfig, ConfigurationInstance> configMap = new HashMap<>();
|
||||
|
||||
public ConfigurationInstance getConfigurationInstance(
|
||||
SqlTestFrameworkConfig config,
|
||||
Function<QueryComponentSupplier, QueryComponentSupplier> queryComponentSupplierWrapper) throws Exception
|
||||
SqlTestFrameworkConfig config) throws Exception
|
||||
{
|
||||
ConfigurationInstance ret = configMap.get(config);
|
||||
if (!configMap.containsKey(config)) {
|
||||
|
@ -267,7 +284,7 @@ public class SqlTestFrameworkConfig
|
|||
*/
|
||||
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 class SqlTestFrameworkConfig
|
|||
|
||||
public SqlTestFramework get() throws Exception
|
||||
{
|
||||
return configStore.getConfigurationInstance(config, Function.identity()).framework;
|
||||
return configStore.getConfigurationInstance(config).framework;
|
||||
}
|
||||
|
||||
public <T extends Annotation> T getAnnotation(Class<T> annotationType)
|
||||
|
@ -344,6 +361,7 @@ public class SqlTestFrameworkConfig
|
|||
.minTopNThreshold(config.minTopNThreshold)
|
||||
.mergeBufferCount(config.numMergeBuffers)
|
||||
.withOverrideModule(config.resultCache.makeModule());
|
||||
|
||||
framework = builder.build();
|
||||
}
|
||||
|
||||
|
@ -393,12 +411,43 @@ public class SqlTestFrameworkConfig
|
|||
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<String, String> 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<NameValuePair> 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<T>
|
||||
{
|
||||
final Class<? extends Annotation> annotationClass;
|
||||
|
@ -459,7 +508,15 @@ public class SqlTestFrameworkConfig
|
|||
@Override
|
||||
public Set<Class<? extends QueryComponentSupplier>> 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);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite;
|
|||
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.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode;
|
|||
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 class SqlTestFrameworkConfigTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testURI()
|
||||
{
|
||||
SqlTestFrameworkConfig c = new SqlTestFrameworkConfig(
|
||||
ImmutableMap.of(
|
||||
"componentSupplier", DrillComponentSupplier.class.getSimpleName()
|
||||
)
|
||||
);
|
||||
URI uri = c.getDruidTestURI();
|
||||
assertEquals("druidtest:///?componentSupplier=DrillComponentSupplier", uri.toString());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
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 class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper,
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.druid.sql.calcite.schema.ViewSchema;
|
|||
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 @@ public class ExpressionTestHelper
|
|||
"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 @@ public class ExpressionTestHelper
|
|||
}
|
||||
|
||||
ExprEval<?> result = PLANNER_CONTEXT.parseExpression(expression.getExpression())
|
||||
|
||||
|
||||
.eval(expressionBindings);
|
||||
|
||||
Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value());
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.sql.calcite.schema.NamedDruidSchema;
|
|||
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 @@ public class ExternalTableScanRuleTest
|
|||
"druid",
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
AuthConfig.newBuilder().build()
|
||||
AuthConfig.newBuilder().build(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
final PlannerContext plannerContext = PlannerContext.create(
|
||||
toolbox,
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaName;
|
|||
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 class CalcitePlannerModuleTest extends CalciteTestBase
|
|||
"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 class CalcitePlannerModuleTest extends CalciteTestBase
|
|||
"druid",
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
AuthConfig.newBuilder().build()
|
||||
AuthConfig.newBuilder().build(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
PlannerContext contextWithBloat = PlannerContext.create(
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.druid.sql.calcite.schema.ViewSchema;
|
|||
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 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
|
|||
"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,
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -365,18 +365,30 @@ public class CalciteTests
|
|||
return QueryFrameworkUtils.createOperatorTable(INJECTOR);
|
||||
}
|
||||
|
||||
|
||||
public static DruidNode mockCoordinatorNode()
|
||||
{
|
||||
return new DruidNode("test-coordinator", "dummy", false, 8081, null, true, false);
|
||||
}
|
||||
|
||||
public static FakeDruidNodeDiscoveryProvider mockDruidNodeDiscoveryProvider(final DruidNode coordinatorNode)
|
||||
{
|
||||
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 = 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))
|
||||
)
|
||||
);
|
||||
final DruidNode coordinatorNode = mockCoordinatorNode();
|
||||
FakeDruidNodeDiscoveryProvider provider = mockDruidNodeDiscoveryProvider(coordinatorNode);
|
||||
|
||||
final DruidNode overlordNode = new DruidNode("test-overlord", "dummy", false, 8090, null, true, false);
|
||||
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
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.schema.NoopDruidSchemaManager;
|
|||
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 @@ public class SqlTestFramework
|
|||
default void close() throws IOException
|
||||
{
|
||||
}
|
||||
|
||||
default void configureGuice(CoreInjectorBuilder injectorBuilder, List<Module> overrideModules)
|
||||
{
|
||||
configureGuice(injectorBuilder);
|
||||
}
|
||||
}
|
||||
|
||||
public interface PlannerComponentSupplier
|
||||
|
@ -494,7 +503,8 @@ public class SqlTestFramework
|
|||
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(
|
||||
|
@ -569,7 +579,14 @@ public class SqlTestFramework
|
|||
|
||||
@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 class SqlTestFramework
|
|||
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,17 +674,18 @@ public class SqlTestFramework
|
|||
// 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));
|
||||
builder.componentSupplier.configureGuice(injectorBuilder);
|
||||
.ignoreLoadScopes();
|
||||
List<Module> overrideModules = new ArrayList<>(builder.overrideModules);
|
||||
overrideModules.add(new LookylooModule());
|
||||
overrideModules.add(new SqlAggregationModule());
|
||||
overrideModules.add(new SegmentWranglerModule());
|
||||
overrideModules.add(new ExpressionModule());
|
||||
|
||||
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);
|
||||
|
@ -631,6 +693,11 @@ public class SqlTestFramework
|
|||
componentSupplier.finalizeTestFramework(this);
|
||||
}
|
||||
|
||||
public TestSetupModule testSetupModule()
|
||||
{
|
||||
return new TestSetupModule(builder);
|
||||
}
|
||||
|
||||
public Injector injector()
|
||||
{
|
||||
return injector;
|
||||
|
|
|
@ -102,6 +102,7 @@ import org.apache.druid.sql.calcite.run.NativeSqlEngine;
|
|||
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 class SqlResourceTest extends CalciteTestBase
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
lifecycleManager = new SqlLifecycleManager()
|
||||
|
|
Loading…
Reference in New Issue