Wire up the catalog resolver (#13788)

Introduces the catalog resolver interface
Wires the resolver up to the planner factory
Refactors planner factory
This commit is contained in:
Paul Rogers 2023-02-22 11:42:32 -08:00 committed by GitHub
parent 1595653e6f
commit 914eebb4b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 440 additions and 100 deletions

View File

@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
@ -485,7 +486,8 @@ public class SqlBenchmark
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
);
}

View File

@ -39,6 +39,7 @@ import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -322,7 +323,8 @@ public class SqlExpressionBenchmark
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
);
try {

View File

@ -47,6 +47,7 @@ import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -311,7 +312,8 @@ public class SqlNestedDataBenchmark
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
);
try {

View File

@ -40,6 +40,7 @@ import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -128,7 +129,8 @@ public class SqlVsNativeBenchmark
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
);
groupByQuery = GroupByQuery
.builder()

View File

@ -141,6 +141,7 @@ import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.external.ExternalDataSource;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.rel.DruidQuery;
@ -174,6 +175,7 @@ import org.mockito.Mockito;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
@ -258,7 +260,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
public static final String DEFAULT = "default";
public static final String SEQUENTIAL_MERGE = "sequential_merge";
public final boolean useDefault = NullHandling.replaceWithDefault();
protected File localFileStorageDir;
@ -293,7 +294,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
builder.addModule(new DruidModule()
{
// Small subset of MsqSqlModule
@Override
public void configure(Binder binder)
@ -478,6 +478,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
testTaskActionClient,
workerMemoryParameters
);
CatalogResolver catalogResolver = createMockCatalogResolver();
final InProcessViewManager viewManager = new InProcessViewManager(SqlTestFramework.DRUID_VIEW_MACRO_FACTORY);
DruidSchemaCatalog rootSchema = QueryFrameworkUtils.createMockRootSchema(
CalciteTests.INJECTOR,
@ -503,12 +504,18 @@ public class MSQTestBase extends BaseCalciteQueryTest
objectMapper,
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
catalogResolver
);
sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory);
}
protected CatalogResolver createMockCatalogResolver()
{
return CatalogResolver.NULL_RESOLVER;
}
/**
* Returns query context expected for a scan query. Same as {@link #DEFAULT_MSQ_CONTEXT}, but
* includes {@link DruidQuery#CTX_SCAN_SIGNATURE}.
@ -732,7 +739,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
}
return payload.getStatus().getErrorReport();
}
private void assertMSQSpec(MSQSpec expectedMSQSpec, MSQSpec querySpecForTask)
@ -781,7 +787,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
throw new ISE("Unable to get results from the report");
}
return Optional.of(new Pair(rowSignature, rows));
return Optional.of(new Pair<RowSignature, List<Object[]>>(rowSignature, rows));
}
}

View File

@ -287,7 +287,7 @@ public class Expressions
{
final SqlOperator operator = ((RexCall) rexNode).getOperator();
final SqlOperatorConversion conversion = plannerContext.getOperatorTable()
final SqlOperatorConversion conversion = plannerContext.getPlannerToolbox().operatorTable()
.lookupOperatorConversion(operator);
if (conversion == null) {
@ -703,7 +703,7 @@ public class Expressions
return filter;
} else if (rexNode instanceof RexCall) {
final SqlOperator operator = ((RexCall) rexNode).getOperator();
final SqlOperatorConversion conversion = plannerContext.getOperatorTable().lookupOperatorConversion(operator);
final SqlOperatorConversion conversion = plannerContext.getPlannerToolbox().operatorTable().lookupOperatorConversion(operator);
if (conversion == null) {
return null;

View File

@ -268,7 +268,7 @@ public class OperatorConversions
);
} else if (rexNode instanceof RexCall) {
final SqlOperator operator = ((RexCall) rexNode).getOperator();
final SqlOperatorConversion conversion = plannerContext.getOperatorTable()
final SqlOperatorConversion conversion = plannerContext.getPlannerToolbox().operatorTable()
.lookupOperatorConversion(operator);
if (conversion != null) {

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite.planner;
import org.apache.druid.sql.calcite.table.DatasourceTable;
import org.apache.druid.sql.calcite.table.DruidTable;
import java.util.Set;
/**
* Facade onto the (optional) Druid catalog. Configured in Guice to be
* the {@link CatalogResolver.NullCatalogResolver} or to an actual catalog.
*/
public interface CatalogResolver
{
CatalogResolver NULL_RESOLVER = new NullCatalogResolver();
/**
* Catalog resolver for when the catalog is not available.
*/
class NullCatalogResolver implements CatalogResolver
{
@Override
public boolean ingestRequiresExistingTable()
{
return false;
}
@Override
public DruidTable resolveDatasource(
final String tableName,
final DatasourceTable.PhysicalDatasourceMetadata dsMetadata
)
{
return dsMetadata == null ? null : new DatasourceTable(dsMetadata);
}
@Override
public Set<String> getTableNames(Set<String> datasourceNames)
{
return datasourceNames;
}
}
/**
* Global option to determine whether ingest requires an existing datasource, or
* can automatically create a new datasource.
*/
boolean ingestRequiresExistingTable();
DruidTable resolveDatasource(
String tableName,
DatasourceTable.PhysicalDatasourceMetadata dsMetadata
);
Set<String> getTableNames(Set<String> datasourceNames);
}

View File

@ -71,7 +71,10 @@ public class DruidRexExecutor implements RexExecutor
reducedValues.add(constExp);
} else {
final SqlTypeName sqlTypeName = constExp.getType().getSqlTypeName();
final Expr expr = Parser.parse(druidExpression.getExpression(), plannerContext.getExprMacroTable());
final Expr expr = Parser.parse(
druidExpression.getExpression(),
plannerContext.getPlannerToolbox().exprMacroTable()
);
final ExprEval exprResult = expr.eval(
InputBindings.forFunction(

View File

@ -43,12 +43,12 @@ import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import org.apache.druid.sql.calcite.run.EngineFeature;
import org.apache.druid.sql.calcite.run.QueryMaker;
import org.apache.druid.sql.calcite.run.SqlEngine;
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -57,8 +57,9 @@ import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
/**
* Like {@link PlannerConfig}, but that has static configuration and this class contains dynamic, per-query
* configuration.
* Like {@link PlannerConfig}, but that has static configuration and this class
* contains dynamic, per-query configuration. Additional Druid-specific static
* configuration resides in the {@link PlannerToolbox} class.
*/
public class PlannerContext
{
@ -75,19 +76,16 @@ public class PlannerContext
// DataContext keys
public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult";
private final PlannerToolbox plannerToolbox;
private final String sql;
private final DruidOperatorTable operatorTable;
private final ExprMacroTable macroTable;
private final JoinableFactoryWrapper joinableFactoryWrapper;
private final ObjectMapper jsonMapper;
private final PlannerConfig plannerConfig;
private final DateTime localNow;
private final DruidSchemaCatalog rootSchema;
private final SqlEngine engine;
private final Map<String, Object> queryContext;
private final String sqlQueryId;
private final boolean stringifyArrays;
private final CopyOnWriteArrayList<String> nativeQueryIds = new CopyOnWriteArrayList<>();
private final PlannerHook hook;
// bindings for dynamic parameters to bind during planning
private List<TypedValue> parameters = Collections.emptyList();
// result of authentication, providing identity to authorize set of resources produced by validation
@ -103,30 +101,24 @@ public class PlannerContext
private VirtualColumnRegistry joinExpressionVirtualColumnRegistry;
private PlannerContext(
final PlannerToolbox plannerToolbox,
final String sql,
final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable,
final ObjectMapper jsonMapper,
final PlannerConfig plannerConfig,
final DateTime localNow,
final boolean stringifyArrays,
final DruidSchemaCatalog rootSchema,
final SqlEngine engine,
final Map<String, Object> queryContext,
final JoinableFactoryWrapper joinableFactoryWrapper
final PlannerHook hook
)
{
this.plannerToolbox = plannerToolbox;
this.sql = sql;
this.operatorTable = operatorTable;
this.macroTable = macroTable;
this.jsonMapper = jsonMapper;
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
this.rootSchema = rootSchema;
this.engine = engine;
this.queryContext = queryContext;
this.localNow = Preconditions.checkNotNull(localNow, "localNow");
this.stringifyArrays = stringifyArrays;
this.joinableFactoryWrapper = joinableFactoryWrapper;
this.hook = hook == null ? NoOpPlannerHook.INSTANCE : hook;
String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID);
// special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle
@ -137,15 +129,11 @@ public class PlannerContext
}
public static PlannerContext create(
final PlannerToolbox plannerToolbox,
final String sql,
final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable,
final ObjectMapper jsonMapper,
final PlannerConfig plannerConfig,
final DruidSchemaCatalog rootSchema,
final SqlEngine engine,
final Map<String, Object> queryContext,
final JoinableFactoryWrapper joinableFactoryWrapper
final PlannerHook hook
)
{
final DateTime utcNow;
@ -165,7 +153,7 @@ public class PlannerContext
if (tzParam != null) {
timeZone = DateTimes.inferTzFromString(String.valueOf(tzParam));
} else {
timeZone = plannerConfig.getSqlTimeZone();
timeZone = plannerToolbox.plannerConfig().getSqlTimeZone();
}
if (stringifyParam != null) {
@ -175,33 +163,32 @@ public class PlannerContext
}
return new PlannerContext(
plannerToolbox,
sql,
operatorTable,
macroTable,
jsonMapper,
plannerConfig.withOverrides(queryContext),
plannerToolbox.plannerConfig().withOverrides(queryContext),
utcNow.withZone(timeZone),
stringifyArrays,
rootSchema,
engine,
queryContext,
joinableFactoryWrapper
hook
);
}
public DruidOperatorTable getOperatorTable()
public PlannerToolbox getPlannerToolbox()
{
return operatorTable;
return plannerToolbox;
}
// Deprecated: prefer using the toolbox
public ExprMacroTable getExprMacroTable()
{
return macroTable;
return plannerToolbox.exprMacroTable();
}
// Deprecated: prefer using the toolbox
public ObjectMapper getJsonMapper()
{
return jsonMapper;
return plannerToolbox.jsonMapper();
}
public PlannerConfig getPlannerConfig()
@ -221,13 +208,13 @@ public class PlannerContext
public JoinableFactoryWrapper getJoinableFactoryWrapper()
{
return joinableFactoryWrapper;
return plannerToolbox.joinableFactoryWrapper();
}
@Nullable
public String getSchemaResourceType(String schema, String resourceName)
{
return rootSchema.getResourceType(schema, resourceName);
return plannerToolbox.rootSchema().getResourceType(schema, resourceName);
}
/**
@ -268,6 +255,11 @@ public class PlannerContext
return sql;
}
public PlannerHook getPlannerHook()
{
return hook;
}
public String getSqlQueryId()
{
return sqlQueryId;

View File

@ -52,7 +52,7 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaName;
import java.util.Map;
import java.util.Properties;
public class PlannerFactory
public class PlannerFactory extends PlannerToolbox
{
static final SqlParser.Config PARSER_CONFIG = SqlParser
.configBuilder()
@ -61,19 +61,9 @@ public class PlannerFactory
.setQuotedCasing(Casing.UNCHANGED)
.setQuoting(Quoting.DOUBLE_QUOTE)
.setConformance(DruidConformance.instance())
.setParserFactory(new DruidSqlParserImplFactory()) // Custom sql parser factory
.setParserFactory(new DruidSqlParserImplFactory()) // Custom SQL parser factory
.build();
private final DruidSchemaCatalog rootSchema;
private final DruidOperatorTable operatorTable;
private final ExprMacroTable macroTable;
private final PlannerConfig plannerConfig;
private final ObjectMapper jsonMapper;
private final AuthorizerMapper authorizerMapper;
private final String druidSchemaName;
private final CalciteRulesManager calciteRuleManager;
private final JoinableFactoryWrapper joinableFactoryWrapper;
@Inject
public PlannerFactory(
final DruidSchemaCatalog rootSchema,
@ -84,18 +74,22 @@ public class PlannerFactory
final @Json ObjectMapper jsonMapper,
final @DruidSchemaName String druidSchemaName,
final CalciteRulesManager calciteRuleManager,
final JoinableFactoryWrapper joinableFactoryWrapper
final JoinableFactoryWrapper joinableFactoryWrapper,
final CatalogResolver catalog
)
{
this.rootSchema = rootSchema;
this.operatorTable = operatorTable;
this.macroTable = macroTable;
this.plannerConfig = plannerConfig;
this.authorizerMapper = authorizerMapper;
this.jsonMapper = jsonMapper;
this.druidSchemaName = druidSchemaName;
this.calciteRuleManager = calciteRuleManager;
this.joinableFactoryWrapper = joinableFactoryWrapper;
super(
operatorTable,
macroTable,
jsonMapper,
plannerConfig,
rootSchema,
joinableFactoryWrapper,
catalog,
druidSchemaName,
calciteRuleManager,
authorizerMapper
);
}
/**
@ -109,15 +103,11 @@ public class PlannerFactory
)
{
final PlannerContext context = PlannerContext.create(
this,
sql,
operatorTable,
macroTable,
jsonMapper,
plannerConfig,
rootSchema,
engine,
queryContext,
joinableFactoryWrapper
hook
);
return new DruidPlanner(buildFrameworkConfig(context), context, engine, hook);

View File

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite.planner;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
public class PlannerToolbox
{
protected final DruidOperatorTable operatorTable;
protected final ExprMacroTable macroTable;
protected final JoinableFactoryWrapper joinableFactoryWrapper;
protected final ObjectMapper jsonMapper;
protected final PlannerConfig plannerConfig;
protected final DruidSchemaCatalog rootSchema;
protected final CatalogResolver catalog;
protected final String druidSchemaName;
protected final CalciteRulesManager calciteRuleManager;
protected final AuthorizerMapper authorizerMapper;
public PlannerToolbox(
final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable,
final ObjectMapper jsonMapper,
final PlannerConfig plannerConfig,
final DruidSchemaCatalog rootSchema,
final JoinableFactoryWrapper joinableFactoryWrapper,
final CatalogResolver catalog,
final String druidSchemaName,
final CalciteRulesManager calciteRuleManager,
final AuthorizerMapper authorizerMapper
)
{
this.operatorTable = operatorTable;
this.macroTable = macroTable;
this.jsonMapper = jsonMapper;
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
this.rootSchema = rootSchema;
this.joinableFactoryWrapper = joinableFactoryWrapper;
this.catalog = catalog;
this.druidSchemaName = druidSchemaName;
this.calciteRuleManager = calciteRuleManager;
this.authorizerMapper = authorizerMapper;
}
public DruidOperatorTable operatorTable()
{
return operatorTable;
}
public ExprMacroTable exprMacroTable()
{
return macroTable;
}
public ObjectMapper jsonMapper()
{
return jsonMapper;
}
public DruidSchemaCatalog rootSchema()
{
return rootSchema;
}
public JoinableFactoryWrapper joinableFactoryWrapper()
{
return joinableFactoryWrapper;
}
public CatalogResolver catalogResolver()
{
return catalog;
}
public String druidSchemaName()
{
return druidSchemaName;
}
public CalciteRulesManager calciteRuleManager()
{
return calciteRuleManager;
}
public PlannerConfig plannerConfig()
{
return plannerConfig;
}
}

View File

@ -91,7 +91,7 @@ public class GroupByRules
filter = null;
}
final SqlAggregator sqlAggregator = plannerContext.getOperatorTable()
final SqlAggregator sqlAggregator = plannerContext.getPlannerToolbox().operatorTable()
.lookupAggregator(call.getAggregation());
if (sqlAggregator == null) {

View File

@ -42,6 +42,7 @@ 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.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.run.NativeSqlEngine;
import org.apache.druid.sql.calcite.schema.DruidCalciteSchemaModule;
@ -121,6 +122,9 @@ public class SqlModule implements Module
if (isAvaticaEnabled()) {
binder.install(new AvaticaModule());
}
// Default do-nothing catalog resolver
binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
}
private boolean isEnabled()

View File

@ -49,6 +49,7 @@ import org.apache.druid.server.security.ForbiddenException;
import org.apache.druid.sql.DirectStatement.ResultSet;
import org.apache.druid.sql.SqlPlanningException.PlanningError;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -156,7 +157,8 @@ public class SqlStatementTest
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
joinableFactoryWrapper
joinableFactoryWrapper,
CatalogResolver.NULL_RESOLVER
);
this.sqlStatementFactory = new SqlStatementFactory(

View File

@ -75,6 +75,7 @@ import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcher;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -291,6 +292,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())));
binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of()));
binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper());
binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
}
)
.build();
@ -999,7 +1001,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
)
);
}

View File

@ -38,6 +38,7 @@ import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -114,7 +115,8 @@ public class DruidStatementTest extends CalciteTestBase
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
joinableFactoryWrapper
joinableFactoryWrapper,
CatalogResolver.NULL_RESOLVER
);
this.sqlStatementFactory = CalciteTests.createSqlStatementFactory(
CalciteTests.createMockSqlEngine(walker, conglomerate),

View File

@ -41,6 +41,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -152,7 +153,8 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
joinableFactoryWrapper
joinableFactoryWrapper,
CatalogResolver.NULL_RESOLVER
);
}

View File

@ -20,6 +20,7 @@
package org.apache.druid.sql.calcite.expression;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
@ -44,10 +45,13 @@ import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerToolbox;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import org.apache.druid.sql.calcite.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
@ -63,6 +67,7 @@ import org.joda.time.DateTimeZone;
import org.junit.Assert;
import javax.annotation.Nullable;
import java.math.BigDecimal;
import java.util.Arrays;
import java.util.Collections;
@ -75,8 +80,7 @@ import java.util.stream.Collectors;
class ExpressionTestHelper
{
private static final JoinableFactoryWrapper JOINABLE_FACTORY_WRAPPER = CalciteTests.createJoinableFactoryWrapper();
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
"SELECT 1", // The actual query isn't important for this test
private static final PlannerToolbox PLANNER_TOOLBOX = new PlannerToolbox(
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
CalciteTests.getJsonMapper(),
@ -88,9 +92,18 @@ class ExpressionTestHelper
NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class))
)
),
JOINABLE_FACTORY_WRAPPER,
CatalogResolver.NULL_RESOLVER,
"druid",
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.TEST_AUTHORIZER_MAPPER
);
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
PLANNER_TOOLBOX,
"SELECT 1", // The actual query isn't important for this test
null, /* Don't need engine */
Collections.emptyMap(),
JOINABLE_FACTORY_WRAPPER
null
);
private final RowSignature rowSignature;

View File

@ -20,13 +20,17 @@
package org.apache.druid.sql.calcite.external;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerToolbox;
import org.apache.druid.sql.calcite.run.NativeSqlEngine;
import org.apache.druid.sql.calcite.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
@ -49,8 +53,7 @@ public class ExternalTableScanRuleTest
EasyMock.createMock(QuerySegmentWalker.class),
EasyMock.createMock(QueryRunnerFactoryConglomerate.class)
);
final PlannerContext plannerContext = PlannerContext.create(
"DUMMY", // The actual query isn't important for this test
final PlannerToolbox toolbox = new PlannerToolbox(
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
CalciteTests.getJsonMapper(),
@ -62,9 +65,18 @@ public class ExternalTableScanRuleTest
NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class))
)
),
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER,
"druid",
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.TEST_AUTHORIZER_MAPPER
);
final PlannerContext plannerContext = PlannerContext.create(
toolbox,
"DUMMY", // The actual query isn't important for this test
engine,
Collections.emptyMap(),
CalciteTests.createJoinableFactoryWrapper()
null
);
plannerContext.setQueryMaker(
engine.buildQueryMakerForSelect(EasyMock.createMock(RelRoot.class), plannerContext)

View File

@ -45,6 +45,7 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
import org.apache.druid.sql.calcite.schema.NamedSchema;
import org.apache.druid.sql.calcite.util.CalciteTestBase;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.easymock.EasyMock;
import org.easymock.EasyMockRunner;
import org.easymock.Mock;
@ -112,7 +113,6 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
@Override
public void onMatch(RelOptRuleCall call)
{
}
};
injector = Guice.createInjector(
@ -128,6 +128,7 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>() {})).toInstance(operatorConversions);
binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema);
binder.bind(JoinableFactoryWrapper.class).toInstance(joinableFactoryWrapper);
binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
},
target,
binder -> {
@ -170,16 +171,24 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
@Test
public void testExtensionCalciteRule()
{
PlannerContext context = PlannerContext.create(
"SELECT 1",
PlannerToolbox toolbox = new PlannerToolbox(
injector.getInstance(DruidOperatorTable.class),
macroTable,
new DefaultObjectMapper(),
injector.getInstance(PlannerConfig.class),
rootSchema,
joinableFactoryWrapper,
CatalogResolver.NULL_RESOLVER,
"druid",
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.TEST_AUTHORIZER_MAPPER
);
PlannerContext context = PlannerContext.create(
toolbox,
"SELECT 1",
null,
Collections.emptyMap(),
joinableFactoryWrapper
null
);
boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class)
.druidConventionRuleSet(context)

View File

@ -79,8 +79,7 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
.build();
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
"SELECT 1", // The actual query isn't important for this test
private static final PlannerToolbox PLANNER_TOOLBOX = new PlannerToolbox(
new DruidOperatorTable(
Collections.emptySet(),
ImmutableSet.of(new DirectOperatorConversion(OPERATOR, "hyper_unique"))
@ -95,9 +94,18 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class))
)
),
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER,
"druid",
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.TEST_AUTHORIZER_MAPPER
);
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
PLANNER_TOOLBOX,
"SELECT 1", // The actual query isn't important for this test
null, /* Don't need an engine */
Collections.emptyMap(),
CalciteTests.createJoinableFactoryWrapper()
null
);
private final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl());

View File

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite.planner;
import org.apache.druid.sql.calcite.table.DatasourceTable.PhysicalDatasourceMetadata;
import org.apache.druid.sql.calcite.table.DruidTable;
import java.util.Collections;
import java.util.Set;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
/**
* Test class to reference items that are otherwise unused in this package, and
* only used in extensions. Prevents GitHub CodeQL warnings.
*/
public class TrivialTest
{
private static class DummyCatalogResolver implements CatalogResolver
{
@Override
public boolean ingestRequiresExistingTable()
{
return true;
}
@Override
public DruidTable resolveDatasource(String tableName, PhysicalDatasourceMetadata dsMetadata)
{
assertTrue("foo".equals(tableName));
return null;
}
@Override
public Set<String> getTableNames(Set<String> datasourceNames)
{
return Collections.emptySet();
}
}
/**
* This is a silly test. It exists only to create references to otherwise-unused
* method parameters.
*/
public void testResolver()
{
CatalogResolver resolver = new DummyCatalogResolver();
assertTrue(resolver.ingestRequiresExistingTable());
assertNull(resolver.resolveDatasource("foo", null));
assertTrue(resolver.getTableNames(Collections.emptySet()).isEmpty());
}
}

View File

@ -46,6 +46,7 @@ import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Escalator;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.util.CalciteTestBase;
import org.apache.druid.sql.calcite.view.ViewManager;
@ -122,6 +123,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase
binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(objectMapper);
binder.bindScope(LazySingleton.class, Scopes.SINGLETON);
binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager);
binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null));
},
new LifecycleModule(),

View File

@ -139,7 +139,13 @@ public class QueryFrameworkUtils
final AuthorizerMapper authorizerMapper
)
{
DruidSchema druidSchema = createMockSchema(injector, conglomerate, walker, plannerConfig, druidSchemaManager);
DruidSchema druidSchema = createMockSchema(
injector,
conglomerate,
walker,
plannerConfig,
druidSchemaManager
);
SystemSchema systemSchema =
CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig, authorizerMapper);
@ -185,7 +191,15 @@ public class QueryFrameworkUtils
final AuthorizerMapper authorizerMapper
)
{
return createMockRootSchema(injector, conglomerate, walker, plannerConfig, null, new NoopDruidSchemaManager(), authorizerMapper);
return createMockRootSchema(
injector,
conglomerate,
walker,
plannerConfig,
null,
new NoopDruidSchemaManager(),
authorizerMapper
);
}
private static DruidSchema createMockSchema(

View File

@ -47,6 +47,7 @@ import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -357,6 +358,7 @@ public class SqlTestFramework
private final QueryComponentSupplier componentSupplier;
private int minTopNThreshold = TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD;
private int mergeBufferCount;
private CatalogResolver catalogResolver = CatalogResolver.NULL_RESOLVER;
public Builder(QueryComponentSupplier componentSupplier)
{
@ -375,6 +377,12 @@ public class SqlTestFramework
return this;
}
public Builder catalogResolver(CatalogResolver catalogResolver)
{
this.catalogResolver = catalogResolver;
return this;
}
public SqlTestFramework build()
{
return new SqlTestFramework(this);
@ -420,7 +428,8 @@ public class SqlTestFramework
framework.queryJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(componentSupplier.extensionCalciteRules()),
framework.injector.getInstance(JoinableFactoryWrapper.class)
framework.injector.getInstance(JoinableFactoryWrapper.class),
framework.builder.catalogResolver
);
componentSupplier.finalizePlanner(this);
this.statementFactory = QueryFrameworkUtils.createSqlStatementFactory(
@ -523,6 +532,7 @@ public class SqlTestFramework
public static final DruidViewMacroFactory DRUID_VIEW_MACRO_FACTORY = new TestDruidViewMacroFactory();
private final Builder builder;
private final QueryComponentSupplier componentSupplier;
private final Closer resourceCloser = Closer.create();
private final Injector injector;
@ -531,6 +541,7 @@ public class SqlTestFramework
private SqlTestFramework(Builder builder)
{
this.builder = builder;
this.componentSupplier = builder.componentSupplier;
Properties properties = new Properties();
this.componentSupplier.gatherProperties(properties);

View File

@ -60,6 +60,7 @@ import org.apache.druid.server.log.RequestLogger;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Escalator;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.view.DruidViewMacro;
@ -204,6 +205,7 @@ public class SqlModuleTest
.toProvider(QuerySchedulerProvider.class)
.in(LazySingleton.class);
binder.bind(ResponseContextConfig.class).toInstance(SqlResourceTest.TEST_RESPONSE_CONTEXT_CONFIG);
binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
},
sqlModule,
new TestViewManagerModule()

View File

@ -87,6 +87,7 @@ import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.SqlToolbox;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
@ -236,7 +237,8 @@ public class SqlResourceTest extends CalciteTestBase
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
CalciteTests.createJoinableFactoryWrapper(),
CatalogResolver.NULL_RESOLVER
);
lifecycleManager = new SqlLifecycleManager()