mirror of https://github.com/apache/druid.git
Guicify druid sql module (#9279)
* Guicify druid sql module Break up the SQLModule in to smaller modules and provide a binding that modules can use to register schemas with druid sql. * fix some tests * address code review * tests compile * Working tests * Add all the tests * fix up licenses and dependencies * add calcite dependency to druid-benchmarks * tests pass * rename the schemas
This commit is contained in:
parent
a085685182
commit
33a97dfaae
|
@ -105,6 +105,11 @@
|
|||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite</groupId>
|
||||
<artifactId>calcite-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.wnameless</groupId>
|
||||
<artifactId>json-flattener</artifactId>
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.benchmark.query;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
|
||||
import org.apache.druid.benchmark.datagen.SegmentGenerator;
|
||||
|
@ -40,8 +41,6 @@ import org.apache.druid.sql.calcite.planner.DruidPlanner;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerResult;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
@ -193,19 +192,17 @@ public class SqlBenchmark
|
|||
);
|
||||
closer.register(walker);
|
||||
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate.lhs, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
|
||||
final SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate.lhs, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate.lhs),
|
||||
CalciteTests.createOperatorTable(),
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.benchmark.query;
|
||||
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
|
||||
import org.apache.druid.benchmark.datagen.SegmentGenerator;
|
||||
|
@ -44,8 +45,6 @@ import org.apache.druid.sql.calcite.planner.DruidPlanner;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerResult;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
@ -116,19 +115,17 @@ public class SqlVsNativeBenchmark
|
|||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
|
||||
this.walker = closer.register(new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index));
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
|
||||
final SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
CalciteTests.createOperatorTable(),
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
);
|
||||
groupByQuery = GroupByQuery
|
||||
.builder()
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
|
@ -53,8 +54,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -151,23 +150,22 @@ public class TDigestSketchSqlAggregatorTest extends CalciteTestBase
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new TDigestSketchQuantileSqlAggregator(), new TDigestGenerateSketchSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -68,8 +69,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -170,8 +169,6 @@ public class HllSketchSqlAggregatorTest extends CalciteTestBase
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(
|
||||
new HllSketchApproxCountDistinctSqlAggregator(),
|
||||
|
@ -185,17 +182,18 @@ public class HllSketchSqlAggregatorTest extends CalciteTestBase
|
|||
)
|
||||
);
|
||||
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -71,8 +72,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -169,8 +168,6 @@ public class DoublesSketchSqlAggregatorTest extends CalciteTestBase
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(
|
||||
new DoublesSketchApproxQuantileSqlAggregator(),
|
||||
|
@ -185,18 +182,19 @@ public class DoublesSketchSqlAggregatorTest extends CalciteTestBase
|
|||
new DoublesSketchSummaryOperatorConversion()
|
||||
)
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -65,8 +66,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -167,8 +166,6 @@ public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(
|
||||
new ThetaSketchApproxCountDistinctSqlAggregator(),
|
||||
|
@ -182,18 +179,19 @@ public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
|
|||
new ThetaSketchSetNotOperatorConversion()
|
||||
)
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
|
@ -72,8 +73,6 @@ import org.apache.druid.sql.calcite.filtration.Filtration;
|
|||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
|
@ -192,24 +191,24 @@ public class BloomFilterSqlAggregatorTest extends InitializedNullHandlingTest
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new BloomFilterSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
jsonMapper
|
||||
jsonMapper,
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -62,8 +63,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -162,24 +161,23 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends CalciteTestB
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -61,8 +62,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -161,24 +160,23 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new QuantileSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.query.aggregation.variance.sql;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
|
@ -58,8 +59,6 @@ import org.apache.druid.sql.calcite.filtration.Filtration;
|
|||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
|
@ -157,8 +156,6 @@ public class VarianceSqlAggregatorTest extends InitializedNullHandlingTest
|
|||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(
|
||||
new BaseVarianceSqlAggregator.VarPopSqlAggregator(),
|
||||
|
@ -170,18 +167,19 @@ public class VarianceSqlAggregatorTest extends InitializedNullHandlingTest
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -296,6 +296,7 @@ libraries:
|
|||
- com.google.inject: guice
|
||||
- com.google.inject.extensions: guice-multibindings
|
||||
- com.google.inject.extensions: guice-servlet
|
||||
- com.google.inject.extensions: guice-assistedinject
|
||||
notices:
|
||||
- guice: |
|
||||
Google Guice - Core Library
|
||||
|
@ -306,6 +307,9 @@ notices:
|
|||
- guice-servlet: |
|
||||
Google Guice - Extensions - Servlet
|
||||
Copyright 2006-2016 Google, Inc.
|
||||
- guice-assistedinject: |
|
||||
Google Guice - Extensions - AssistedInect
|
||||
Copyright 2006-2016 Google, Inc.
|
||||
|
||||
---
|
||||
|
||||
|
|
5
pom.xml
5
pom.xml
|
@ -387,6 +387,11 @@
|
|||
<artifactId>guice-multibindings</artifactId>
|
||||
<version>${guice.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-assistedinject</artifactId>
|
||||
<version>${guice.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.errorprone</groupId>
|
||||
<artifactId>error_prone_annotations</artifactId>
|
||||
|
|
|
@ -36,7 +36,7 @@ public class MapJoinableFactory implements JoinableFactory
|
|||
private final Map<Class<? extends DataSource>, JoinableFactory> joinableFactories;
|
||||
|
||||
@Inject
|
||||
public MapJoinableFactory(Map<Class<? extends DataSource>, JoinableFactory> joinableFactories)
|
||||
MapJoinableFactory(Map<Class<? extends DataSource>, JoinableFactory> joinableFactories)
|
||||
{
|
||||
// Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
|
||||
// Class doesn't override Object.equals().
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.join;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
|
@ -30,11 +31,20 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class MapJoinableFactoryTest
|
||||
{
|
||||
/**
|
||||
* A utility to create a {@link MapJoinableFactory} to be used by tests.
|
||||
*/
|
||||
public static MapJoinableFactory fromMap(Map<Class<? extends DataSource>, JoinableFactory> map)
|
||||
{
|
||||
return new MapJoinableFactory(map);
|
||||
}
|
||||
|
||||
@Mock
|
||||
private InlineDataSource inlineDataSource;
|
||||
@Mock(MockType.NICE)
|
||||
|
|
|
@ -118,6 +118,10 @@
|
|||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-assistedinject</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
|
@ -172,6 +176,11 @@
|
|||
<version>${checkerframework.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.validation</groupId>
|
||||
<artifactId>validation-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
|
|
|
@ -157,7 +157,7 @@ public class SqlLifecycle
|
|||
{
|
||||
synchronized (lock) {
|
||||
Preconditions.checkState(plannerResult != null,
|
||||
"must be called after sql has been planned");
|
||||
"must be called after SQL has been planned");
|
||||
return plannerResult.rowType();
|
||||
}
|
||||
}
|
||||
|
@ -332,7 +332,7 @@ public class SqlLifecycle
|
|||
);
|
||||
}
|
||||
catch (Exception ex) {
|
||||
log.error(ex, "Unable to log sql [%s]!", sql);
|
||||
log.error(ex, "Unable to log SQL [%s]!", sql);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.avatica;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.server.initialization.jetty.JettyBindings;
|
||||
import org.apache.druid.server.metrics.MetricsModule;
|
||||
|
||||
/**
|
||||
* The module responsible for providing bindings to Avatica.
|
||||
*/
|
||||
public class AvaticaModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
JsonConfigProvider.bind(binder, "druid.sql.avatica", AvaticaServerConfig.class);
|
||||
binder.bind(AvaticaMonitor.class).in(LazySingleton.class);
|
||||
JettyBindings.addHandler(binder, DruidAvaticaHandler.class);
|
||||
MetricsModule.register(binder, AvaticaMonitor.class);
|
||||
}
|
||||
}
|
|
@ -22,7 +22,7 @@ package org.apache.druid.sql.avatica;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Period;
|
||||
|
||||
public class AvaticaServerConfig
|
||||
class AvaticaServerConfig
|
||||
{
|
||||
@JsonProperty
|
||||
public int maxConnections = 25;
|
||||
|
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* 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.aggregation;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.multibindings.Multibinder;
|
||||
|
||||
/**
|
||||
* Module that provides SQL aggregations.
|
||||
* To add an aggregation use {@link org.apache.druid.sql.guice.SqlBindings#addAggregator(Binder, Class)}
|
||||
*/
|
||||
public class SqlAggregationModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
// Add empty SqlAggregator binder.
|
||||
Multibinder.newSetBinder(binder, SqlAggregator.class);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* 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.planner;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
|
||||
/**
|
||||
* The module responsible for provide bindings for the Calcite Planner.
|
||||
*/
|
||||
public class CalcitePlannerModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
JsonConfigProvider.bind(binder, "druid.sql.planner", PlannerConfig.class);
|
||||
binder.bind(PlannerFactory.class);
|
||||
binder.bind(DruidOperatorTable.class);
|
||||
}
|
||||
}
|
|
@ -23,13 +23,11 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.io.BaseEncoding;
|
||||
import com.google.common.primitives.Chars;
|
||||
import org.apache.calcite.jdbc.CalciteSchema;
|
||||
import org.apache.calcite.rel.core.Sort;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.sql.SqlCollation;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
|
@ -44,11 +42,6 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.query.ordering.StringComparator;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.InformationSchema;
|
||||
import org.apache.druid.sql.calcite.schema.LookupSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Days;
|
||||
|
@ -106,21 +99,6 @@ public class Calcites
|
|||
return DEFAULT_CHARSET;
|
||||
}
|
||||
|
||||
public static SchemaPlus createRootSchema(
|
||||
final DruidSchema druidSchema,
|
||||
final LookupSchema lookupSchema,
|
||||
final SystemSchema systemSchema,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
)
|
||||
{
|
||||
final SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
|
||||
rootSchema.add(DruidSchema.NAME, druidSchema);
|
||||
rootSchema.add(LookupSchema.NAME, lookupSchema);
|
||||
rootSchema.add(InformationSchema.NAME, new InformationSchema(rootSchema, authorizerMapper));
|
||||
rootSchema.add(SystemSchema.NAME, systemSchema);
|
||||
return rootSchema;
|
||||
}
|
||||
|
||||
public static String escapeStringLiteral(final String s)
|
||||
{
|
||||
Preconditions.checkNotNull(s);
|
||||
|
|
|
@ -41,9 +41,7 @@ import org.apache.druid.server.QueryLifecycleFactory;
|
|||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.sql.calcite.rel.QueryMaker;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.LookupSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
@ -59,38 +57,35 @@ public class PlannerFactory
|
|||
.setConformance(DruidConformance.instance())
|
||||
.build();
|
||||
|
||||
private final DruidSchema druidSchema;
|
||||
private final LookupSchema lookupSchema;
|
||||
private final SystemSchema systemSchema;
|
||||
private final SchemaPlus rootSchema;
|
||||
private final QueryLifecycleFactory queryLifecycleFactory;
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final ExprMacroTable macroTable;
|
||||
private final PlannerConfig plannerConfig;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final AuthorizerMapper authorizerMapper;
|
||||
private final String druidSchemaName;
|
||||
|
||||
@Inject
|
||||
public PlannerFactory(
|
||||
final DruidSchema druidSchema,
|
||||
final LookupSchema lookupSchema,
|
||||
final SystemSchema systemSchema,
|
||||
final SchemaPlus rootSchema,
|
||||
final QueryLifecycleFactory queryLifecycleFactory,
|
||||
final DruidOperatorTable operatorTable,
|
||||
final ExprMacroTable macroTable,
|
||||
final PlannerConfig plannerConfig,
|
||||
final AuthorizerMapper authorizerMapper,
|
||||
final @Json ObjectMapper jsonMapper
|
||||
final @Json ObjectMapper jsonMapper,
|
||||
final @DruidSchemaName String druidSchemaName
|
||||
)
|
||||
{
|
||||
this.druidSchema = druidSchema;
|
||||
this.lookupSchema = lookupSchema;
|
||||
this.systemSchema = systemSchema;
|
||||
this.rootSchema = rootSchema;
|
||||
this.queryLifecycleFactory = queryLifecycleFactory;
|
||||
this.operatorTable = operatorTable;
|
||||
this.macroTable = macroTable;
|
||||
this.plannerConfig = plannerConfig;
|
||||
this.authorizerMapper = authorizerMapper;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.druidSchemaName = druidSchemaName;
|
||||
}
|
||||
|
||||
public DruidPlanner createPlanner(
|
||||
|
@ -98,12 +93,6 @@ public class PlannerFactory
|
|||
final AuthenticationResult authenticationResult
|
||||
)
|
||||
{
|
||||
final SchemaPlus rootSchema = Calcites.createRootSchema(
|
||||
druidSchema,
|
||||
lookupSchema,
|
||||
systemSchema,
|
||||
authorizerMapper
|
||||
);
|
||||
final PlannerContext plannerContext = PlannerContext.create(
|
||||
operatorTable,
|
||||
macroTable,
|
||||
|
@ -129,7 +118,7 @@ public class PlannerFactory
|
|||
.executor(new DruidRexExecutor(plannerContext))
|
||||
.context(Contexts.EMPTY_CONTEXT)
|
||||
.typeSystem(DruidTypeSystem.INSTANCE)
|
||||
.defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME))
|
||||
.defaultSchema(rootSchema.getSubSchema(druidSchemaName))
|
||||
.sqlToRelConverterConfig(sqlToRelConverterConfig)
|
||||
.context(new Context()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.Provides;
|
||||
import com.google.inject.Scopes;
|
||||
import com.google.inject.Singleton;
|
||||
import com.google.inject.name.Named;
|
||||
import com.google.inject.name.Names;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.guice.LifecycleModule;
|
||||
import org.apache.druid.sql.guice.SqlBindings;
|
||||
|
||||
/**
|
||||
* The module responsible for providing bindings to Calcite schemas.
|
||||
*/
|
||||
public class DruidCalciteSchemaModule implements Module
|
||||
{
|
||||
private static final String DRUID_SCHEMA_NAME = "druid";
|
||||
private static final String INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA";
|
||||
static final String INCOMPLETE_SCHEMA = "INCOMPLETE_SCHEMA";
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(String.class).annotatedWith(DruidSchemaName.class).toInstance(DRUID_SCHEMA_NAME);
|
||||
|
||||
// Should only be used by the information schema
|
||||
binder.bind(SchemaPlus.class)
|
||||
.annotatedWith(Names.named(INCOMPLETE_SCHEMA))
|
||||
.toProvider(RootSchemaProvider.class)
|
||||
.in(Scopes.SINGLETON);
|
||||
|
||||
// DruidSchema needs to listen to changes for incoming segments
|
||||
LifecycleModule.register(binder, DruidSchema.class);
|
||||
binder.bind(SystemSchema.class).in(Scopes.SINGLETON);
|
||||
binder.bind(InformationSchema.class).in(Scopes.SINGLETON);
|
||||
binder.bind(LookupSchema.class).in(Scopes.SINGLETON);
|
||||
|
||||
// Binder to inject different schema to Calcite
|
||||
SqlBindings.addSchema(binder, NamedDruidSchema.class);
|
||||
SqlBindings.addSchema(binder, NamedSystemSchema.class);
|
||||
SqlBindings.addSchema(binder, NamedLookupSchema.class);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Singleton
|
||||
private SchemaPlus getRootSchema(@Named(INCOMPLETE_SCHEMA) SchemaPlus rootSchema, InformationSchema informationSchema)
|
||||
{
|
||||
rootSchema.add(INFORMATION_SCHEMA_NAME, informationSchema);
|
||||
return rootSchema;
|
||||
}
|
||||
}
|
|
@ -95,8 +95,6 @@ public class DruidSchema extends AbstractSchema
|
|||
.reversed()
|
||||
.thenComparing(Function.identity());
|
||||
|
||||
public static final String NAME = "druid";
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(DruidSchema.class);
|
||||
private static final int MAX_SEGMENTS_PER_QUERY = 15000;
|
||||
private static final long DEFAULT_NUM_ROWS = 0;
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.inject.BindingAnnotation;
|
||||
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* An annotation to get the name of the schema to access Druid tables in SQL.
|
||||
*/
|
||||
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@BindingAnnotation
|
||||
public @interface DruidSchemaName
|
||||
{
|
||||
}
|
|
@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.name.Named;
|
||||
import org.apache.calcite.DataContext;
|
||||
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
|
||||
import org.apache.calcite.linq4j.Enumerable;
|
||||
|
@ -61,8 +62,6 @@ import java.util.Set;
|
|||
|
||||
public class InformationSchema extends AbstractSchema
|
||||
{
|
||||
public static final String NAME = "INFORMATION_SCHEMA";
|
||||
|
||||
private static final String CATALOG_NAME = "druid";
|
||||
private static final String SCHEMATA_TABLE = "SCHEMATA";
|
||||
private static final String TABLES_TABLE = "TABLES";
|
||||
|
@ -112,11 +111,13 @@ public class InformationSchema extends AbstractSchema
|
|||
private final SchemaPlus rootSchema;
|
||||
private final Map<String, Table> tableMap;
|
||||
private final AuthorizerMapper authorizerMapper;
|
||||
private final String druidSchemaName;
|
||||
|
||||
@Inject
|
||||
public InformationSchema(
|
||||
final SchemaPlus rootSchema,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
@Named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA) final SchemaPlus rootSchema,
|
||||
final AuthorizerMapper authorizerMapper,
|
||||
@DruidSchemaName String druidSchemaName
|
||||
)
|
||||
{
|
||||
this.rootSchema = Preconditions.checkNotNull(rootSchema, "rootSchema");
|
||||
|
@ -126,6 +127,7 @@ public class InformationSchema extends AbstractSchema
|
|||
COLUMNS_TABLE, new ColumnsTable()
|
||||
);
|
||||
this.authorizerMapper = authorizerMapper;
|
||||
this.druidSchemaName = druidSchemaName;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -451,7 +453,7 @@ public class InformationSchema extends AbstractSchema
|
|||
final AuthenticationResult authenticationResult
|
||||
)
|
||||
{
|
||||
if (DruidSchema.NAME.equals(subSchema.getName())) {
|
||||
if (druidSchemaName.equals(subSchema.getName())) {
|
||||
// The "druid" schema's tables represent Druid datasources which require authorization
|
||||
return ImmutableSet.copyOf(
|
||||
AuthorizationUtils.filterAuthorizedResources(
|
||||
|
@ -472,7 +474,7 @@ public class InformationSchema extends AbstractSchema
|
|||
final AuthenticationResult authenticationResult
|
||||
)
|
||||
{
|
||||
if (DruidSchema.NAME.equals(subSchema.getName())) {
|
||||
if (druidSchemaName.equals(subSchema.getName())) {
|
||||
// The "druid" schema's functions represent views on Druid datasources, authorize them as if they were
|
||||
// datasources for now
|
||||
return ImmutableSet.copyOf(
|
||||
|
|
|
@ -37,8 +37,6 @@ import java.util.Map;
|
|||
*/
|
||||
public class LookupSchema extends AbstractSchema
|
||||
{
|
||||
public static final String NAME = "lookup";
|
||||
|
||||
private static final RowSignature ROW_SIGNATURE =
|
||||
RowSignature.builder()
|
||||
.add(LookupColumnSelectorFactory.KEY_COLUMN, ValueType.STRING)
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
|
||||
/**
|
||||
* The schema for Druid tables to be accessible via SQL.
|
||||
*/
|
||||
class NamedDruidSchema implements NamedSchema
|
||||
{
|
||||
private final DruidSchema druidSchema;
|
||||
private final String druidSchemaName;
|
||||
|
||||
@Inject
|
||||
NamedDruidSchema(DruidSchema druidSchema, @DruidSchemaName String druidSchemaName)
|
||||
{
|
||||
this.druidSchema = druidSchema;
|
||||
this.druidSchemaName = druidSchemaName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSchemaName()
|
||||
{
|
||||
return druidSchemaName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema()
|
||||
{
|
||||
return druidSchema;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
|
||||
/**
|
||||
* The schema for Druid lookup tables to be accessible via SQL.
|
||||
*/
|
||||
public class NamedLookupSchema implements NamedSchema
|
||||
{
|
||||
private static final String NAME = "lookup";
|
||||
|
||||
private final LookupSchema lookupSchema;
|
||||
|
||||
@Inject
|
||||
NamedLookupSchema(LookupSchema lookupSchema)
|
||||
{
|
||||
this.lookupSchema = lookupSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSchemaName()
|
||||
{
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema()
|
||||
{
|
||||
return lookupSchema;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import org.apache.calcite.schema.Schema;
|
||||
|
||||
/**
|
||||
* This interface provides everything that is needed to register a {@link Schema} as a sub schema to the root schema
|
||||
* of Druid SQL. The {@link #getSchemaName()} will be used to access the provided {@link Schema} via SQL.
|
||||
*/
|
||||
public interface NamedSchema
|
||||
{
|
||||
/**
|
||||
* @return The name that this schema should be registered to.
|
||||
*/
|
||||
String getSchemaName();
|
||||
|
||||
/**
|
||||
* @return The Schema that Calcite should use.
|
||||
*/
|
||||
Schema getSchema();
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
|
||||
/**
|
||||
* The schema for Druid system tables to be accessible via SQL.
|
||||
*/
|
||||
class NamedSystemSchema implements NamedSchema
|
||||
{
|
||||
private static final String NAME = "sys";
|
||||
|
||||
private final SystemSchema systemSchema;
|
||||
|
||||
@Inject
|
||||
NamedSystemSchema(SystemSchema systemSchema)
|
||||
{
|
||||
this.systemSchema = systemSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSchemaName()
|
||||
{
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema()
|
||||
{
|
||||
return systemSchema;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* 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.schema;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.Provider;
|
||||
import org.apache.calcite.jdbc.CalciteSchema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Provides the RootSchema for calcite with
|
||||
* - metadata schema disabled because it's not needed
|
||||
* - caching disabled because druid's caching is better.
|
||||
*
|
||||
* All the provided schema are added to the rootSchema.
|
||||
*/
|
||||
public class RootSchemaProvider implements Provider<SchemaPlus>
|
||||
{
|
||||
private final Set<NamedSchema> namedSchemas;
|
||||
|
||||
@Inject
|
||||
RootSchemaProvider(Set<NamedSchema> namedSchemas)
|
||||
{
|
||||
this.namedSchemas = namedSchemas;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SchemaPlus get()
|
||||
{
|
||||
SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
|
||||
List<String> schemaNames = namedSchemas.stream()
|
||||
.map(NamedSchema::getSchemaName)
|
||||
.collect(Collectors.toList());
|
||||
Set<String> uniqueSchemaNames = new HashSet<>(schemaNames);
|
||||
if (uniqueSchemaNames.size() < schemaNames.size()) {
|
||||
throw new ISE("Found multiple schemas registered to the same name. "
|
||||
+ "The list of registered schemas are %s", schemaNames);
|
||||
}
|
||||
namedSchemas.forEach(schema -> rootSchema.add(schema.getSchemaName(), schema.getSchema()));
|
||||
return rootSchema;
|
||||
}
|
||||
}
|
|
@ -95,7 +95,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class SystemSchema extends AbstractSchema
|
||||
{
|
||||
public static final String NAME = "sys";
|
||||
private static final String SEGMENTS_TABLE = "segments";
|
||||
private static final String SERVERS_TABLE = "servers";
|
||||
private static final String SERVER_SEGMENTS_TABLE = "server_segments";
|
||||
|
@ -285,7 +284,7 @@ public class SystemSchema extends AbstractSchema
|
|||
partialSegmentDataMap.put(h.getSegment().getId(), partialSegmentData);
|
||||
}
|
||||
|
||||
// Get published segments from metadata segment cache (if enabled in sql planner config), else directly from
|
||||
// Get published segments from metadata segment cache (if enabled in SQL planner config), else directly from
|
||||
// Coordinator.
|
||||
final Iterator<SegmentWithOvershadowedStatus> metadataStoreSegments = metadataView.getPublishedSegments();
|
||||
|
||||
|
@ -456,7 +455,7 @@ public class SystemSchema extends AbstractSchema
|
|||
}
|
||||
|
||||
/**
|
||||
* This table contains row per server. It contains all the discovered servers in druid cluster.
|
||||
* This table contains row per server. It contains all the discovered servers in Druid cluster.
|
||||
* Some columns like tier and size are only applicable to historical nodes which contain segments.
|
||||
*/
|
||||
static class ServersTable extends AbstractTable implements ScannableTable
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.druid.sql.calcite.view;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.assistedinject.Assisted;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeImpl;
|
||||
import org.apache.calcite.schema.FunctionParameter;
|
||||
|
@ -30,7 +32,7 @@ import org.apache.druid.server.security.AuthenticationResult;
|
|||
import org.apache.druid.server.security.Escalator;
|
||||
import org.apache.druid.sql.calcite.planner.DruidPlanner;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -39,12 +41,20 @@ public class DruidViewMacro implements TableMacro
|
|||
private final PlannerFactory plannerFactory;
|
||||
private final Escalator escalator;
|
||||
private final String viewSql;
|
||||
private final String druidSchemaName;
|
||||
|
||||
public DruidViewMacro(final PlannerFactory plannerFactory, final Escalator escalator, final String viewSql)
|
||||
@Inject
|
||||
public DruidViewMacro(
|
||||
@Assisted final PlannerFactory plannerFactory,
|
||||
@Assisted final Escalator escalator,
|
||||
@Assisted final String viewSql,
|
||||
@DruidSchemaName String druidSchemaName
|
||||
)
|
||||
{
|
||||
this.plannerFactory = plannerFactory;
|
||||
this.escalator = escalator;
|
||||
this.viewSql = viewSql;
|
||||
this.druidSchemaName = druidSchemaName;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -66,7 +76,7 @@ public class DruidViewMacro implements TableMacro
|
|||
null,
|
||||
RelDataTypeImpl.proto(rowType),
|
||||
viewSql,
|
||||
ImmutableList.of(DruidSchema.NAME),
|
||||
ImmutableList.of(druidSchemaName),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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.sql.calcite.view;
|
||||
|
||||
import org.apache.druid.server.security.Escalator;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
|
||||
/**
|
||||
* A factory to create a {@link DruidViewMacro} that is used by Guice for Assisted injection.
|
||||
*/
|
||||
public interface DruidViewMacroFactory
|
||||
{
|
||||
/**
|
||||
* Creates an instance of {@link DruidViewMacro}
|
||||
*/
|
||||
DruidViewMacro create(PlannerFactory plannerFactory, Escalator escalator, String viewSql);
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.view;
|
||||
|
||||
import com.google.inject.AbstractModule;
|
||||
import com.google.inject.assistedinject.FactoryModuleBuilder;
|
||||
|
||||
/**
|
||||
* This module is responsible for providing bindings for views in Druid.
|
||||
*/
|
||||
public class DruidViewModule extends AbstractModule
|
||||
{
|
||||
@Override
|
||||
public void configure()
|
||||
{
|
||||
install(new FactoryModuleBuilder().build(DruidViewMacroFactory.class));
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.sql.calcite.view;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.calcite.schema.TableMacro;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -37,20 +38,25 @@ public class InProcessViewManager implements ViewManager
|
|||
{
|
||||
private final ConcurrentMap<String, DruidViewMacro> views;
|
||||
private final Escalator escalator;
|
||||
private final DruidViewMacroFactory druidViewMacroFactory;
|
||||
|
||||
@Inject
|
||||
@VisibleForTesting
|
||||
public InProcessViewManager(
|
||||
final Escalator escalator
|
||||
final Escalator escalator,
|
||||
final DruidViewMacroFactory druidViewMacroFactory
|
||||
)
|
||||
{
|
||||
this.views = new ConcurrentHashMap<>();
|
||||
this.escalator = escalator;
|
||||
this.druidViewMacroFactory = druidViewMacroFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createView(final PlannerFactory plannerFactory, final String viewName, final String viewSql)
|
||||
{
|
||||
final TableMacro oldValue = views.putIfAbsent(viewName, new DruidViewMacro(plannerFactory, escalator, viewSql));
|
||||
final TableMacro oldValue =
|
||||
views.putIfAbsent(viewName, druidViewMacroFactory.create(plannerFactory, escalator, viewSql));
|
||||
if (oldValue != null) {
|
||||
throw new ISE("View[%s] already exists", viewName);
|
||||
}
|
||||
|
@ -59,7 +65,8 @@ public class InProcessViewManager implements ViewManager
|
|||
@Override
|
||||
public void alterView(final PlannerFactory plannerFactory, final String viewName, final String viewSql)
|
||||
{
|
||||
final TableMacro oldValue = views.replace(viewName, new DruidViewMacro(plannerFactory, escalator, viewSql));
|
||||
final TableMacro oldValue =
|
||||
views.replace(viewName, druidViewMacroFactory.create(plannerFactory, escalator, viewSql));
|
||||
if (oldValue != null) {
|
||||
throw new ISE("View[%s] does not exist", viewName);
|
||||
}
|
||||
|
|
|
@ -20,10 +20,15 @@
|
|||
package org.apache.druid.sql.guice;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Scopes;
|
||||
import com.google.inject.multibindings.Multibinder;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.schema.NamedSchema;
|
||||
|
||||
/**
|
||||
* Utility class that provides bindings to extendable components in the SqlModule
|
||||
*/
|
||||
public class SqlBindings
|
||||
{
|
||||
public static void addAggregator(
|
||||
|
@ -41,4 +46,16 @@ public class SqlBindings
|
|||
{
|
||||
Multibinder.newSetBinder(binder, SqlOperatorConversion.class).addBinding().to(clazz);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a multiBinder that can modules can use to bind {@link NamedSchema} to be used by the SqlModule
|
||||
*/
|
||||
public static void addSchema(
|
||||
final Binder binder,
|
||||
final Class<? extends NamedSchema> clazz
|
||||
)
|
||||
{
|
||||
binder.bind(clazz).in(Scopes.SINGLETON);
|
||||
Multibinder.newSetBinder(binder, NamedSchema.class).addBinding().to(clazz);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,24 +23,17 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.inject.Binder;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.multibindings.Multibinder;
|
||||
import org.apache.druid.guice.Jerseys;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.LifecycleModule;
|
||||
import org.apache.druid.server.initialization.jetty.JettyBindings;
|
||||
import org.apache.druid.server.metrics.MetricsModule;
|
||||
import org.apache.druid.sql.avatica.AvaticaMonitor;
|
||||
import org.apache.druid.sql.avatica.AvaticaServerConfig;
|
||||
import org.apache.druid.sql.avatica.DruidAvaticaHandler;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.avatica.AvaticaModule;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.planner.CalcitePlannerModule;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.DruidCalciteSchemaModule;
|
||||
import org.apache.druid.sql.calcite.view.DruidViewModule;
|
||||
import org.apache.druid.sql.calcite.view.NoopViewManager;
|
||||
import org.apache.druid.sql.calcite.view.ViewManager;
|
||||
import org.apache.druid.sql.http.SqlResource;
|
||||
import org.apache.druid.sql.http.SqlHttpModule;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
|
@ -63,25 +56,22 @@ public class SqlModule implements Module
|
|||
if (isEnabled()) {
|
||||
Calcites.setSystemProperties();
|
||||
|
||||
JsonConfigProvider.bind(binder, "druid.sql.planner", PlannerConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.sql.avatica", AvaticaServerConfig.class);
|
||||
LifecycleModule.register(binder, DruidSchema.class);
|
||||
binder.bind(ViewManager.class).to(NoopViewManager.class).in(LazySingleton.class);
|
||||
|
||||
// Add empty SqlAggregator binder.
|
||||
Multibinder.newSetBinder(binder, SqlAggregator.class);
|
||||
binder.install(new DruidCalciteSchemaModule());
|
||||
binder.install(new CalcitePlannerModule());
|
||||
binder.install(new SqlAggregationModule());
|
||||
binder.install(new DruidViewModule());
|
||||
|
||||
// QueryLookupOperatorConversion isn't in DruidOperatorTable since it needs a LookupExtractorFactoryContainerProvider injected.
|
||||
SqlBindings.addOperatorConversion(binder, QueryLookupOperatorConversion.class);
|
||||
|
||||
if (isJsonOverHttpEnabled()) {
|
||||
Jerseys.addResource(binder, SqlResource.class);
|
||||
binder.install(new SqlHttpModule());
|
||||
}
|
||||
|
||||
if (isAvaticaEnabled()) {
|
||||
binder.bind(AvaticaMonitor.class).in(LazySingleton.class);
|
||||
JettyBindings.addHandler(binder, DruidAvaticaHandler.class);
|
||||
MetricsModule.register(binder, AvaticaMonitor.class);
|
||||
binder.install(new AvaticaModule());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.http;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import org.apache.druid.guice.Jerseys;
|
||||
|
||||
/**
|
||||
* The Module responsible for providing bindings to the SQL http endpoint
|
||||
*/
|
||||
public class SqlHttpModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
Jerseys.addResource(binder, SqlResource.class);
|
||||
}
|
||||
}
|
|
@ -151,7 +151,7 @@ public class SqlResource
|
|||
}
|
||||
catch (Exception ex) {
|
||||
e = ex;
|
||||
log.error(ex, "Unable to send sql response [%s]", sqlQueryId);
|
||||
log.error(ex, "Unable to send SQL response [%s]", sqlQueryId);
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
finally {
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.avatica;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Scopes;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.annotations.Self;
|
||||
import org.apache.druid.jackson.JacksonModule;
|
||||
import org.apache.druid.java.util.metrics.Monitor;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.eclipse.jetty.server.Handler;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import javax.validation.Validation;
|
||||
import javax.validation.Validator;
|
||||
import java.util.Set;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class AvaticaModuleTest
|
||||
{
|
||||
private static final String HOST_AND_PORT = "HOST_AND_PORT";
|
||||
|
||||
@Mock
|
||||
private DruidNode druidNode;
|
||||
@Mock
|
||||
private DruidMeta druidMeta;
|
||||
|
||||
private AvaticaModule target;
|
||||
private Injector injector;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EasyMock.expect(druidNode.getHostAndPortToUse()).andStubReturn(HOST_AND_PORT);
|
||||
EasyMock.replay(druidNode);
|
||||
target = new AvaticaModule();
|
||||
injector = Guice.createInjector(
|
||||
new JacksonModule(),
|
||||
binder -> {
|
||||
binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator());
|
||||
binder.bindScope(LazySingleton.class, Scopes.SINGLETON);
|
||||
binder.bind(DruidNode.class).annotatedWith(Self.class).toInstance(druidNode);
|
||||
binder.bind(DruidMeta.class).toInstance(druidMeta);
|
||||
},
|
||||
target
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvaticaMonitorIsInjectedAsSingleton()
|
||||
{
|
||||
AvaticaMonitor monitor = injector.getInstance(AvaticaMonitor.class);
|
||||
Assert.assertNotNull(monitor);
|
||||
AvaticaMonitor other = injector.getInstance(AvaticaMonitor.class);
|
||||
Assert.assertSame(monitor, other);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvaticaMonitorIsRegisterdWithMetricsModule()
|
||||
{
|
||||
Set<Class<? extends Monitor>> monitors =
|
||||
injector.getInstance(Key.get(new TypeLiteral<Set<Class<? extends Monitor>>>(){}));
|
||||
Assert.assertTrue(monitors.contains(AvaticaMonitor.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvaticaServerConfigIsInjectable()
|
||||
{
|
||||
AvaticaServerConfig config = injector.getInstance(AvaticaServerConfig.class);
|
||||
Assert.assertNotNull(config);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidAvaticaHandlerIsInjected()
|
||||
{
|
||||
DruidAvaticaHandler handler = injector.getInstance(DruidAvaticaHandler.class);
|
||||
Assert.assertNotNull(handler);
|
||||
DruidAvaticaHandler other = injector.getInstance(DruidAvaticaHandler.class);
|
||||
Assert.assertNotSame(handler, other);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidAvaticaHandlerIsRegisterdWithJerseyModule()
|
||||
{
|
||||
Set<Handler> handlers =
|
||||
injector.getInstance(Key.get(new TypeLiteral<Set<Handler>>(){}));
|
||||
Assert.assertTrue(handlers.stream().anyMatch(h -> DruidAvaticaHandler.class.equals(h.getClass())));
|
||||
}
|
||||
}
|
|
@ -35,6 +35,7 @@ import org.apache.calcite.avatica.AvaticaClientRuntimeException;
|
|||
import org.apache.calcite.avatica.Meta;
|
||||
import org.apache.calcite.avatica.MissingResultsException;
|
||||
import org.apache.calcite.avatica.NoSuchStatementException;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.GuiceInjectors;
|
||||
import org.apache.druid.initialization.Initialization;
|
||||
|
@ -42,23 +43,24 @@ import org.apache.druid.java.util.common.DateTimes;
|
|||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.server.RequestLogLine;
|
||||
import org.apache.druid.server.log.RequestLogger;
|
||||
import org.apache.druid.server.log.TestRequestLogger;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.server.security.AuthenticatorMapper;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.server.security.Escalator;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -160,10 +162,11 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
{
|
||||
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
final SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, CalciteTests.TEST_AUTHORIZER_MAPPER);
|
||||
testRequestLogger = new TestRequestLogger();
|
||||
|
||||
injector = Initialization.makeInjectorWithModules(
|
||||
GuiceInjectors.makeStartupInjector(),
|
||||
|
@ -179,32 +182,24 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
binder.bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER);
|
||||
binder.bind(AuthorizerMapper.class).toInstance(CalciteTests.TEST_AUTHORIZER_MAPPER);
|
||||
binder.bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR);
|
||||
binder.bind(RequestLogger.class).toInstance(testRequestLogger);
|
||||
binder.bind(SchemaPlus.class).toInstance(rootSchema);
|
||||
binder.bind(QueryLifecycleFactory.class)
|
||||
.toInstance(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate));
|
||||
binder.bind(DruidOperatorTable.class).toInstance(operatorTable);
|
||||
binder.bind(ExprMacroTable.class).toInstance(macroTable);
|
||||
binder.bind(PlannerConfig.class).toInstance(plannerConfig);
|
||||
binder.bind(String.class)
|
||||
.annotatedWith(DruidSchemaName.class)
|
||||
.toInstance(CalciteTests.DRUID_SCHEMA_NAME);
|
||||
binder.bind(AvaticaServerConfig.class).toInstance(AVATICA_CONFIG);
|
||||
binder.bind(ServiceEmitter.class).to(NoopServiceEmitter.class);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
testRequestLogger = new TestRequestLogger();
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
macroTable,
|
||||
plannerConfig,
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
);
|
||||
druidMeta = new DruidMeta(
|
||||
new SqlLifecycleFactory(
|
||||
plannerFactory,
|
||||
new NoopServiceEmitter(),
|
||||
testRequestLogger
|
||||
),
|
||||
AVATICA_CONFIG,
|
||||
injector
|
||||
);
|
||||
druidMeta = injector.getInstance(DruidMeta.class);
|
||||
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
|
||||
druidMeta,
|
||||
new DruidNode("dummy", "dummy", false, 1, null, true, false),
|
||||
|
@ -819,23 +814,22 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
};
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
final List<Meta.Frame> frames = new ArrayList<>();
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
DruidMeta smallFrameDruidMeta = new DruidMeta(
|
||||
CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
macroTable,
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
)
|
||||
),
|
||||
smallFrameConfig,
|
||||
|
@ -911,7 +905,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
testRequestLogger.clear();
|
||||
try {
|
||||
client.createStatement().executeQuery("SELECT notexist FROM druid.foo");
|
||||
Assert.fail("invalid sql should throw SQLException");
|
||||
Assert.fail("invalid SQL should throw SQLException");
|
||||
}
|
||||
catch (SQLException e) {
|
||||
}
|
||||
|
@ -925,7 +919,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
testRequestLogger.clear();
|
||||
try {
|
||||
client.createStatement().executeQuery("SELECT count(*) FROM druid.forbiddenDatasource");
|
||||
Assert.fail("unauthorzed sql should throw SQLException");
|
||||
Assert.fail("unauthorzed SQL should throw SQLException");
|
||||
}
|
||||
catch (SQLException e) {
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.apache.calcite.avatica.ColumnMetaData;
|
||||
import org.apache.calcite.avatica.Meta;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
|
@ -35,8 +36,6 @@ import org.apache.druid.sql.SqlLifecycleFactory;
|
|||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -87,20 +86,19 @@ public class DruidStatementTest extends CalciteTestBase
|
|||
{
|
||||
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
SchemaPlus rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
macroTable,
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
);
|
||||
this.sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.hll.VersionOneHyperLogLogCollector;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -72,8 +73,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
|
@ -632,21 +631,25 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
final ObjectMapper objectMapper
|
||||
) throws Exception
|
||||
{
|
||||
final InProcessViewManager viewManager = new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR);
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig, viewManager);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
|
||||
final InProcessViewManager viewManager =
|
||||
new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR, CalciteTests.DRUID_VIEW_MACRO_FACTORY);
|
||||
SchemaPlus rootSchema = CalciteTests.createMockRootSchema(
|
||||
conglomerate,
|
||||
walker,
|
||||
plannerConfig,
|
||||
viewManager,
|
||||
authorizerMapper
|
||||
);
|
||||
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
macroTable,
|
||||
plannerConfig,
|
||||
authorizerMapper,
|
||||
objectMapper
|
||||
objectMapper,
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
);
|
||||
final SqlLifecycleFactory sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory);
|
||||
|
||||
|
|
|
@ -1338,7 +1338,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
|
||||
// This test the on-heap version of the AnyAggregator (Double/Float/Long) against numeric columns
|
||||
// that have null values (when run in sql compatible null mode)
|
||||
// that have null values (when run in SQL compatible null mode)
|
||||
@Test
|
||||
public void testAnyAggregatorsOnHeapNumericNulls() throws Exception
|
||||
{
|
||||
|
@ -1369,7 +1369,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
|
||||
// This test the off-heap (buffer) version of the AnyAggregator (Double/Float/Long) against numeric columns
|
||||
// that have null values (when run in sql compatible null mode)
|
||||
// that have null values (when run in SQL compatible null mode)
|
||||
@Test
|
||||
public void testAnyAggregatorsOffHeapNumericNulls() throws Exception
|
||||
{
|
||||
|
@ -1580,7 +1580,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
//[, 10.1]
|
||||
//[a, ]
|
||||
//[abc, def]
|
||||
// sql compatible mode subquery results:
|
||||
// SQL compatible mode subquery results:
|
||||
//[null, 10.1]
|
||||
//[, 2]
|
||||
//[a, ]
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.aggregation;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
public class SqlAggregationModuleTest
|
||||
{
|
||||
private SqlAggregationModule target;
|
||||
private Injector injector;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new SqlAggregationModule();
|
||||
injector = Guice.createInjector(target);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptySqlAggregatorsAreBound()
|
||||
{
|
||||
Set<SqlAggregator> sqlAggregators = injector.getInstance(Key.get(new TypeLiteral<Set<SqlAggregator>>(){}));
|
||||
Assert.assertNotNull(sqlAggregators);
|
||||
Assert.assertTrue(sqlAggregators.isEmpty());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,136 @@
|
|||
/*
|
||||
* 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.planner;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Scopes;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.jackson.JacksonModule;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
|
||||
import org.apache.druid.sql.calcite.schema.NamedSchema;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import javax.validation.Validation;
|
||||
import javax.validation.Validator;
|
||||
import java.util.Set;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class CalcitePlannerModuleTest
|
||||
{
|
||||
private static final String SCHEMA_1 = "SCHEMA_1";
|
||||
private static final String SCHEMA_2 = "SCHEMA_2";
|
||||
private static final String DRUID_SCHEMA_NAME = "DRUID_SCHEMA_NAME";
|
||||
|
||||
@Mock
|
||||
private NamedSchema druidSchema1;
|
||||
@Mock
|
||||
private NamedSchema druidSchema2;
|
||||
@Mock
|
||||
private Schema schema1;
|
||||
@Mock
|
||||
private Schema schema2;
|
||||
@Mock
|
||||
private QueryLifecycleFactory queryLifecycleFactory;
|
||||
@Mock
|
||||
private ExprMacroTable macroTable;
|
||||
@Mock
|
||||
private AuthorizerMapper authorizerMapper;
|
||||
@Mock
|
||||
private SchemaPlus rootSchema;
|
||||
|
||||
private Set<SqlAggregator> aggregators;
|
||||
private Set<SqlOperatorConversion> operatorConversions;
|
||||
private Set<NamedSchema> calciteSchemas;
|
||||
|
||||
private CalcitePlannerModule target;
|
||||
private Injector injector;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EasyMock.expect(druidSchema1.getSchema()).andStubReturn(schema1);
|
||||
EasyMock.expect(druidSchema2.getSchema()).andStubReturn(schema2);
|
||||
EasyMock.expect(druidSchema1.getSchemaName()).andStubReturn(SCHEMA_1);
|
||||
EasyMock.expect(druidSchema2.getSchemaName()).andStubReturn(SCHEMA_2);
|
||||
EasyMock.replay(druidSchema1, druidSchema2);
|
||||
calciteSchemas = ImmutableSet.of(druidSchema1, druidSchema2);
|
||||
aggregators = ImmutableSet.of();
|
||||
operatorConversions = ImmutableSet.of();
|
||||
target = new CalcitePlannerModule();
|
||||
injector = Guice.createInjector(
|
||||
new JacksonModule(),
|
||||
binder -> {
|
||||
binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator());
|
||||
binder.bindScope(LazySingleton.class, Scopes.SINGLETON);
|
||||
binder.bind(Key.get(new TypeLiteral<Set<NamedSchema>>(){})).toInstance(calciteSchemas);
|
||||
binder.bind(QueryLifecycleFactory.class).toInstance(queryLifecycleFactory);
|
||||
binder.bind(ExprMacroTable.class).toInstance(macroTable);
|
||||
binder.bind(AuthorizerMapper.class).toInstance(authorizerMapper);
|
||||
binder.bind(String.class).annotatedWith(DruidSchemaName.class).toInstance(DRUID_SCHEMA_NAME);
|
||||
binder.bind(Key.get(new TypeLiteral<Set<SqlAggregator>>(){})).toInstance(aggregators);
|
||||
binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>(){})).toInstance(operatorConversions);
|
||||
binder.bind(SchemaPlus.class).toInstance(rootSchema);
|
||||
},
|
||||
target
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidOperatorTableIsInjectable()
|
||||
{
|
||||
DruidOperatorTable operatorTable = injector.getInstance(DruidOperatorTable.class);
|
||||
Assert.assertNotNull(operatorTable);
|
||||
DruidOperatorTable other = injector.getInstance(DruidOperatorTable.class);
|
||||
Assert.assertNotSame(other, operatorTable);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPlannerFactoryIsInjectable()
|
||||
{
|
||||
PlannerFactory plannerFactory = injector.getInstance(PlannerFactory.class);
|
||||
Assert.assertNotNull(PlannerFactory.class);
|
||||
PlannerFactory other = injector.getInstance(PlannerFactory.class);
|
||||
Assert.assertNotSame(other, plannerFactory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPlannerConfigIsInjected()
|
||||
{
|
||||
PlannerConfig plannerConfig = injector.getInstance(PlannerConfig.class);
|
||||
Assert.assertNotNull(plannerConfig);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,226 @@
|
|||
/*
|
||||
* 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.schema;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Scopes;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import com.google.inject.name.Names;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.client.InventoryView;
|
||||
import org.apache.druid.client.TimelineServerView;
|
||||
import org.apache.druid.client.coordinator.Coordinator;
|
||||
import org.apache.druid.client.indexing.IndexingService;
|
||||
import org.apache.druid.discovery.DruidLeaderClient;
|
||||
import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
|
||||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.LifecycleModule;
|
||||
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
|
||||
import org.apache.druid.query.lookup.LookupReferencesManager;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.server.security.Escalator;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.view.ViewManager;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class DruidCalciteSchemaModuleTest
|
||||
{
|
||||
private static final String DRUID_SCHEMA_NAME = "druid";
|
||||
|
||||
@Mock
|
||||
private QueryLifecycleFactory queryLifecycleFactory;
|
||||
@Mock
|
||||
private TimelineServerView serverView;
|
||||
@Mock
|
||||
private PlannerConfig plannerConfig;
|
||||
@Mock
|
||||
private ViewManager viewManager;
|
||||
@Mock
|
||||
private Escalator escalator;
|
||||
@Mock
|
||||
AuthorizerMapper authorizerMapper;
|
||||
@Mock
|
||||
private InventoryView serverInventoryView;
|
||||
@Mock
|
||||
private DruidLeaderClient coordinatorDruidLeaderClient;
|
||||
@Mock
|
||||
private DruidLeaderClient overlordDruidLeaderClient;
|
||||
@Mock
|
||||
private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider;
|
||||
@Mock
|
||||
private ObjectMapper objectMapper;
|
||||
@Mock
|
||||
private LookupReferencesManager lookupReferencesManager;
|
||||
|
||||
private DruidCalciteSchemaModule target;
|
||||
private Injector injector;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EasyMock.expect(plannerConfig.isMetadataSegmentCacheEnable()).andStubReturn(false);
|
||||
EasyMock.expect(plannerConfig.getMetadataSegmentPollPeriod()).andStubReturn(6000L);
|
||||
EasyMock.replay(plannerConfig);
|
||||
target = new DruidCalciteSchemaModule();
|
||||
injector = Guice.createInjector(
|
||||
binder -> {
|
||||
binder.bind(QueryLifecycleFactory.class).toInstance(queryLifecycleFactory);
|
||||
binder.bind(TimelineServerView.class).toInstance(serverView);
|
||||
binder.bind(PlannerConfig.class).toInstance(plannerConfig);
|
||||
binder.bind(ViewManager.class).toInstance(viewManager);
|
||||
binder.bind(Escalator.class).toInstance(escalator);
|
||||
binder.bind(AuthorizerMapper.class).toInstance(authorizerMapper);
|
||||
binder.bind(InventoryView.class).toInstance(serverInventoryView);
|
||||
binder.bind(DruidLeaderClient.class)
|
||||
.annotatedWith(Coordinator.class)
|
||||
.toInstance(coordinatorDruidLeaderClient);
|
||||
binder.bind(DruidLeaderClient.class)
|
||||
.annotatedWith(IndexingService.class)
|
||||
.toInstance(overlordDruidLeaderClient);
|
||||
binder.bind(DruidNodeDiscoveryProvider.class).toInstance(druidNodeDiscoveryProvider);
|
||||
binder.bind(ObjectMapper.class).toInstance(objectMapper);
|
||||
binder.bindScope(LazySingleton.class, Scopes.SINGLETON);
|
||||
binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager);
|
||||
},
|
||||
new LifecycleModule(),
|
||||
target);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidSchemaNameIsInjected()
|
||||
{
|
||||
String schemaName = injector.getInstance(Key.get(String.class, DruidSchemaName.class));
|
||||
Assert.assertEquals(DRUID_SCHEMA_NAME, schemaName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidSqlSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
NamedDruidSchema namedDruidSchema = injector.getInstance(NamedDruidSchema.class);
|
||||
Assert.assertNotNull(namedDruidSchema);
|
||||
NamedDruidSchema other = injector.getInstance(NamedDruidSchema.class);
|
||||
Assert.assertSame(other, namedDruidSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSystemSqlSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
NamedSystemSchema namedSystemSchema = injector.getInstance(NamedSystemSchema.class);
|
||||
Assert.assertNotNull(namedSystemSchema);
|
||||
NamedSystemSchema other = injector.getInstance(NamedSystemSchema.class);
|
||||
Assert.assertSame(other, namedSystemSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidCalciteSchemasAreInjected()
|
||||
{
|
||||
Set<NamedSchema> sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<Set<NamedSchema>>(){}));
|
||||
Set<Class<? extends NamedSchema>> expectedSchemas =
|
||||
ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class);
|
||||
Assert.assertEquals(expectedSchemas.size(), sqlSchemas.size());
|
||||
Assert.assertEquals(
|
||||
expectedSchemas,
|
||||
sqlSchemas.stream().map(NamedSchema::getClass).collect(Collectors.toSet()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
DruidSchema schema = injector.getInstance(DruidSchema.class);
|
||||
Assert.assertNotNull(schema);
|
||||
DruidSchema other = injector.getInstance(DruidSchema.class);
|
||||
Assert.assertSame(other, schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSystemSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
SystemSchema schema = injector.getInstance(SystemSchema.class);
|
||||
Assert.assertNotNull(schema);
|
||||
SystemSchema other = injector.getInstance(SystemSchema.class);
|
||||
Assert.assertSame(other, schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInformationSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
InformationSchema schema = injector.getInstance(InformationSchema.class);
|
||||
Assert.assertNotNull(schema);
|
||||
InformationSchema other = injector.getInstance(InformationSchema.class);
|
||||
Assert.assertSame(other, schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLookupSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
LookupSchema schema = injector.getInstance(LookupSchema.class);
|
||||
Assert.assertNotNull(schema);
|
||||
LookupSchema other = injector.getInstance(LookupSchema.class);
|
||||
Assert.assertSame(other, schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRootSchemaAnnotatedIsInjectedAsSingleton()
|
||||
{
|
||||
SchemaPlus rootSchema = injector.getInstance(
|
||||
Key.get(SchemaPlus.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA))
|
||||
);
|
||||
Assert.assertNotNull(rootSchema);
|
||||
SchemaPlus other = injector.getInstance(
|
||||
Key.get(SchemaPlus.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA))
|
||||
);
|
||||
Assert.assertSame(other, rootSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRootSchemaIsInjectedAsSingleton()
|
||||
{
|
||||
SchemaPlus rootSchema = injector.getInstance(Key.get(SchemaPlus.class));
|
||||
Assert.assertNotNull(rootSchema);
|
||||
SchemaPlus other = injector.getInstance(
|
||||
Key.get(SchemaPlus.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA))
|
||||
);
|
||||
Assert.assertSame(other, rootSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRootSchemaIsInjectedAndHasInformationSchema()
|
||||
{
|
||||
SchemaPlus rootSchema = injector.getInstance(Key.get(SchemaPlus.class));
|
||||
InformationSchema expectedSchema = injector.getInstance(InformationSchema.class);
|
||||
Assert.assertNotNull(rootSchema);
|
||||
Assert.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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.schema;
|
||||
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class NamedDruidSchemaTest
|
||||
{
|
||||
private static final String SCHEMA_NAME = "SCHEMA_NAME";
|
||||
|
||||
@Mock
|
||||
private DruidSchema druidSchema;
|
||||
|
||||
private NamedDruidSchema target;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new NamedDruidSchema(druidSchema, SCHEMA_NAME);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaNameShouldReturnName()
|
||||
{
|
||||
Assert.assertEquals(SCHEMA_NAME, target.getSchemaName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaShouldReturnSchema()
|
||||
{
|
||||
Assert.assertEquals(druidSchema, target.getSchema());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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.schema;
|
||||
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class NamedLookupSchemaTest
|
||||
{
|
||||
private static final String SCHEMA_NAME = "lookup";
|
||||
|
||||
@Mock
|
||||
private LookupSchema lookupSchema;
|
||||
|
||||
private NamedLookupSchema target;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new NamedLookupSchema(lookupSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaNameShouldReturnName()
|
||||
{
|
||||
Assert.assertEquals(SCHEMA_NAME, target.getSchemaName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaShouldReturnSchema()
|
||||
{
|
||||
Assert.assertEquals(lookupSchema, target.getSchema());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class NamedSystemSchemaTest
|
||||
{
|
||||
private static final String SCHEMA_NAME = "sys";
|
||||
|
||||
@Mock
|
||||
private SystemSchema systemSchema;
|
||||
|
||||
private NamedSystemSchema target;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new NamedSystemSchema(systemSchema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaNameShouldReturnName()
|
||||
{
|
||||
Assert.assertEquals(SCHEMA_NAME, target.getSchemaName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSchemaShouldReturnSchema()
|
||||
{
|
||||
Assert.assertEquals(systemSchema, target.getSchema());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* 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.schema;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class RootSchemaProviderTest
|
||||
{
|
||||
private static final String SCHEMA_1 = "SCHEMA_1";
|
||||
private static final String SCHEMA_2 = "SCHEMA_2";
|
||||
@Mock
|
||||
private NamedSchema druidSchema1;
|
||||
@Mock
|
||||
private NamedSchema druidSchema2;
|
||||
@Mock
|
||||
private NamedSchema duplicateSchema1;
|
||||
@Mock
|
||||
private Schema schema1;
|
||||
@Mock
|
||||
private Schema schema2;
|
||||
@Mock
|
||||
private Schema schema3;
|
||||
private Set<NamedSchema> druidSchemas;
|
||||
|
||||
private RootSchemaProvider target;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EasyMock.expect(druidSchema1.getSchema()).andStubReturn(schema1);
|
||||
EasyMock.expect(druidSchema2.getSchema()).andStubReturn(schema2);
|
||||
EasyMock.expect(duplicateSchema1.getSchema()).andStubReturn(schema3);
|
||||
EasyMock.expect(druidSchema1.getSchemaName()).andStubReturn(SCHEMA_1);
|
||||
EasyMock.expect(druidSchema2.getSchemaName()).andStubReturn(SCHEMA_2);
|
||||
EasyMock.expect(duplicateSchema1.getSchemaName()).andStubReturn(SCHEMA_1);
|
||||
EasyMock.replay(druidSchema1, druidSchema2, duplicateSchema1);
|
||||
|
||||
druidSchemas = ImmutableSet.of(druidSchema1, druidSchema2);
|
||||
target = new RootSchemaProvider(druidSchemas);
|
||||
}
|
||||
@Test
|
||||
public void testGetShouldReturnRootSchemaWithProvidedSchemasRegistered()
|
||||
{
|
||||
SchemaPlus rootSchema = target.get();
|
||||
Assert.assertEquals("", rootSchema.getName());
|
||||
Assert.assertFalse(rootSchema.isCacheEnabled());
|
||||
// metadata schema should not be added
|
||||
Assert.assertEquals(druidSchemas.size(), rootSchema.getSubSchemaNames().size());
|
||||
|
||||
Assert.assertEquals(schema1, rootSchema.getSubSchema(SCHEMA_1).unwrap(schema1.getClass()));
|
||||
Assert.assertEquals(schema2, rootSchema.getSubSchema(SCHEMA_2).unwrap(schema2.getClass()));
|
||||
}
|
||||
|
||||
@Test(expected = ISE.class)
|
||||
public void testGetWithDuplicateSchemasShouldThrowISE()
|
||||
{
|
||||
target = new RootSchemaProvider(ImmutableSet.of(druidSchema1, druidSchema2, duplicateSchema1));
|
||||
target.get();
|
||||
}
|
||||
}
|
|
@ -27,6 +27,8 @@ import com.google.inject.Guice;
|
|||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Module;
|
||||
import org.apache.calcite.jdbc.CalciteSchema;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.curator.x.discovery.ServiceProvider;
|
||||
import org.apache.druid.client.BrokerSegmentWatcherConfig;
|
||||
import org.apache.druid.client.ServerInventoryView;
|
||||
|
@ -118,9 +120,11 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.InformationSchema;
|
||||
import org.apache.druid.sql.calcite.schema.LookupSchema;
|
||||
import org.apache.druid.sql.calcite.schema.MetadataSegmentView;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.view.DruidViewMacroFactory;
|
||||
import org.apache.druid.sql.calcite.view.NoopViewManager;
|
||||
import org.apache.druid.sql.calcite.view.ViewManager;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
@ -151,6 +155,10 @@ public class CalciteTests
|
|||
public static final String DATASOURCE4 = "foo4";
|
||||
public static final String DATASOURCE5 = "lotsocolumns";
|
||||
public static final String FORBIDDEN_DATASOURCE = "forbiddenDatasource";
|
||||
public static final String DRUID_SCHEMA_NAME = "druid";
|
||||
public static final String INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA";
|
||||
public static final String SYSTEM_SCHEMA_NAME = "sys";
|
||||
public static final String LOOKUP_SCHEMA_NAME = "lookup";
|
||||
|
||||
public static final String TEST_SUPERUSER_NAME = "testSuperuser";
|
||||
public static final AuthorizerMapper TEST_AUTHORIZER_MAPPER = new AuthorizerMapper(null)
|
||||
|
@ -530,6 +538,8 @@ public class CalciteTests
|
|||
// No instantiation.
|
||||
}
|
||||
|
||||
public static final DruidViewMacroFactory DRUID_VIEW_MACRO_FACTORY = new TestDruidViewMacroFactory();
|
||||
|
||||
/**
|
||||
* Returns a new {@link QueryRunnerFactoryConglomerate} and a {@link Closer} which should be closed at the end of the
|
||||
* test.
|
||||
|
@ -803,42 +813,6 @@ public class CalciteTests
|
|||
}
|
||||
}
|
||||
|
||||
public static DruidSchema createMockSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig
|
||||
)
|
||||
{
|
||||
return createMockSchema(conglomerate, walker, plannerConfig, new NoopViewManager());
|
||||
}
|
||||
|
||||
public static DruidSchema createMockSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig,
|
||||
final ViewManager viewManager
|
||||
)
|
||||
{
|
||||
final DruidSchema schema = new DruidSchema(
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
new TestServerInventoryView(walker.getSegments()),
|
||||
plannerConfig,
|
||||
viewManager,
|
||||
TEST_AUTHENTICATOR_ESCALATOR
|
||||
);
|
||||
|
||||
try {
|
||||
schema.start();
|
||||
schema.awaitInitialization();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
schema.stop();
|
||||
return schema;
|
||||
}
|
||||
|
||||
public static InputRow createRow(final ImmutableMap<String, ?> map)
|
||||
{
|
||||
return PARSER.parseBatch((Map<String, Object>) map).get(0);
|
||||
|
@ -869,7 +843,8 @@ public class CalciteTests
|
|||
public static SystemSchema createMockSystemSchema(
|
||||
final DruidSchema druidSchema,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig
|
||||
final PlannerConfig plannerConfig,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
)
|
||||
{
|
||||
final DruidLeaderClient druidLeaderClient = new DruidLeaderClient(
|
||||
|
@ -892,7 +867,7 @@ public class CalciteTests
|
|||
),
|
||||
new TestServerInventoryView(walker.getSegments()),
|
||||
EasyMock.createMock(ServerInventoryView.class),
|
||||
TEST_AUTHORIZER_MAPPER,
|
||||
authorizerMapper,
|
||||
druidLeaderClient,
|
||||
druidLeaderClient,
|
||||
EasyMock.createMock(DruidNodeDiscoveryProvider.class),
|
||||
|
@ -901,6 +876,50 @@ public class CalciteTests
|
|||
return schema;
|
||||
}
|
||||
|
||||
public static SchemaPlus createMockRootSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
)
|
||||
{
|
||||
DruidSchema druidSchema = createMockSchema(conglomerate, walker, plannerConfig);
|
||||
SystemSchema systemSchema =
|
||||
CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig, authorizerMapper);
|
||||
|
||||
SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
|
||||
InformationSchema informationSchema =
|
||||
new InformationSchema(rootSchema, authorizerMapper, CalciteTests.DRUID_SCHEMA_NAME);
|
||||
LookupSchema lookupSchema = CalciteTests.createMockLookupSchema();
|
||||
rootSchema.add(CalciteTests.DRUID_SCHEMA_NAME, druidSchema);
|
||||
rootSchema.add(CalciteTests.INFORMATION_SCHEMA_NAME, informationSchema);
|
||||
rootSchema.add(CalciteTests.SYSTEM_SCHEMA_NAME, systemSchema);
|
||||
rootSchema.add(CalciteTests.LOOKUP_SCHEMA_NAME, lookupSchema);
|
||||
return rootSchema;
|
||||
}
|
||||
|
||||
public static SchemaPlus createMockRootSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig,
|
||||
final ViewManager viewManager,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
)
|
||||
{
|
||||
DruidSchema druidSchema = createMockSchema(conglomerate, walker, plannerConfig, viewManager);
|
||||
SystemSchema systemSchema =
|
||||
CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig, authorizerMapper);
|
||||
SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
|
||||
InformationSchema informationSchema =
|
||||
new InformationSchema(rootSchema, authorizerMapper, CalciteTests.DRUID_SCHEMA_NAME);
|
||||
LookupSchema lookupSchema = CalciteTests.createMockLookupSchema();
|
||||
rootSchema.add(CalciteTests.DRUID_SCHEMA_NAME, druidSchema);
|
||||
rootSchema.add(CalciteTests.INFORMATION_SCHEMA_NAME, informationSchema);
|
||||
rootSchema.add(CalciteTests.SYSTEM_SCHEMA_NAME, systemSchema);
|
||||
rootSchema.add(CalciteTests.LOOKUP_SCHEMA_NAME, lookupSchema);
|
||||
return rootSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Some Calcite exceptions (such as that thrown by
|
||||
* {@link org.apache.druid.sql.calcite.CalciteQueryTest#testCountStarWithTimeFilterUsingStringLiteralsInvalid)},
|
||||
|
@ -915,4 +934,40 @@ public class CalciteTests
|
|||
}
|
||||
return curThrowable;
|
||||
}
|
||||
|
||||
private static DruidSchema createMockSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig
|
||||
)
|
||||
{
|
||||
return createMockSchema(conglomerate, walker, plannerConfig, new NoopViewManager());
|
||||
}
|
||||
|
||||
private static DruidSchema createMockSchema(
|
||||
final QueryRunnerFactoryConglomerate conglomerate,
|
||||
final SpecificSegmentsQuerySegmentWalker walker,
|
||||
final PlannerConfig plannerConfig,
|
||||
final ViewManager viewManager
|
||||
)
|
||||
{
|
||||
final DruidSchema schema = new DruidSchema(
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
new TestServerInventoryView(walker.getSegments()),
|
||||
plannerConfig,
|
||||
viewManager,
|
||||
TEST_AUTHENTICATOR_ESCALATOR
|
||||
);
|
||||
|
||||
try {
|
||||
schema.start();
|
||||
schema.awaitInitialization();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
schema.stop();
|
||||
return schema;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ import org.apache.druid.segment.join.InlineJoinableFactory;
|
|||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.segment.join.Joinables;
|
||||
import org.apache.druid.segment.join.LookupJoinableFactory;
|
||||
import org.apache.druid.segment.join.MapJoinableFactory;
|
||||
import org.apache.druid.segment.join.MapJoinableFactoryTest;
|
||||
import org.apache.druid.server.ClientQuerySegmentWalker;
|
||||
import org.apache.druid.server.initialization.ServerConfig;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
|
@ -109,7 +109,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
|
|||
)
|
||||
{
|
||||
this.conglomerate = conglomerate;
|
||||
this.joinableFactory = new MapJoinableFactory(
|
||||
this.joinableFactory = MapJoinableFactoryTest.fromMap(
|
||||
ImmutableMap.<Class<? extends DataSource>, JoinableFactory>builder()
|
||||
.put(InlineDataSource.class, new InlineJoinableFactory())
|
||||
.put(LookupDataSource.class, new LookupJoinableFactory(lookupProvider))
|
||||
|
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* 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.util;
|
||||
|
||||
import org.apache.druid.server.security.Escalator;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.view.DruidViewMacro;
|
||||
import org.apache.druid.sql.calcite.view.DruidViewMacroFactory;
|
||||
|
||||
public class TestDruidViewMacroFactory implements DruidViewMacroFactory
|
||||
{
|
||||
@Override
|
||||
public DruidViewMacro create(
|
||||
PlannerFactory plannerFactory,
|
||||
Escalator escalator,
|
||||
String viewSql
|
||||
)
|
||||
{
|
||||
return new DruidViewMacro(plannerFactory, escalator, viewSql, CalciteTests.DRUID_SCHEMA_NAME);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* 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.http;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import org.apache.druid.guice.annotations.JSR311Resource;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
@RunWith(EasyMockRunner.class)
|
||||
public class SqlHttpModuleTest
|
||||
{
|
||||
@Mock
|
||||
private ObjectMapper jsonMpper;
|
||||
@Mock
|
||||
private SqlLifecycleFactory sqlLifecycleFactory;
|
||||
|
||||
private SqlHttpModule target;
|
||||
private Injector injector;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new SqlHttpModule();
|
||||
injector = Guice.createInjector(
|
||||
binder -> {
|
||||
binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMpper);
|
||||
binder.bind(SqlLifecycleFactory.class).toInstance(sqlLifecycleFactory);
|
||||
},
|
||||
target);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSqlResourceIsInjectedAndNotSingleton()
|
||||
{
|
||||
SqlResource sqlResource = injector.getInstance(SqlResource.class);
|
||||
Assert.assertNotNull(sqlResource);
|
||||
SqlResource other = injector.getInstance(SqlResource.class);
|
||||
Assert.assertNotSame(other, sqlResource);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSqlResourceIsAvailableViaJersey()
|
||||
{
|
||||
Set<Class<?>> jerseyResourceClasses =
|
||||
injector.getInstance(Key.get(new TypeLiteral<Set<Class<?>>>(){}, JSR311Resource.class));
|
||||
Assert.assertEquals(1, jerseyResourceClasses.size());
|
||||
Assert.assertEquals(SqlResource.class, jerseyResourceClasses.iterator().next());
|
||||
}
|
||||
}
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.http;
|
||||
package org.apache.druid.sql.http;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
@ -25,6 +25,7 @@ import com.google.common.base.Splitter;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.tools.ValidationException;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -45,15 +46,10 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.sql.http.ResultFormat;
|
||||
import org.apache.druid.sql.http.SqlQuery;
|
||||
import org.apache.druid.sql.http.SqlResource;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -120,8 +116,7 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
return false;
|
||||
}
|
||||
};
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final SchemaPlus rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, CalciteTests.TEST_AUTHORIZER_MAPPER);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
req = EasyMock.createStrictMock(HttpServletRequest.class);
|
||||
|
@ -146,15 +141,14 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
testRequestLogger = new TestRequestLogger();
|
||||
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
CalciteTests.createMockLookupSchema(),
|
||||
systemSchema,
|
||||
rootSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
macroTable,
|
||||
plannerConfig,
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME
|
||||
);
|
||||
|
||||
resource = new SqlResource(
|
Loading…
Reference in New Issue