Clone Calcite planner to access validator (#12708)

Done in preparation for the "single-pass" planner.
This commit is contained in:
Paul Rogers 2022-07-14 18:10:33 -07:00 committed by GitHub
parent 50f1f5840d
commit ee15c238cc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 534 additions and 74 deletions

View File

@ -70,9 +70,9 @@ public class GuavaUtils
}
/**
* If first argument is not null, return it, else return the other argument. Sort of like
* {@link com.google.common.base.Objects#firstNonNull(T, T)} except will not explode if both arguments are
* null.
* If first argument is not null, return it, else return the other argument.
* Sort of like {@link <T> com.google.common.base.Objects#firstNonNull(T, T)}
* except will not explode if both arguments are null.
*/
@Nullable
public static <T> T firstNonNull(@Nullable T arg1, @Nullable T arg2)
@ -84,10 +84,11 @@ public class GuavaUtils
}
/**
* Cancel futures manually, because sometime we can't cancel all futures in {@link com.google.common.util.concurrent.Futures.CombinedFuture}
* Cancel futures manually, because sometime we can't cancel all futures in
* {code com.google.common.util.concurrent.Futures.CombinedFuture}
* automatically. Especially when we call
* {@link static <V> ListenableFuture<List<V>> com.google.common.util.concurrent.Futures#allAsList(Iterable<? extends ListenableFuture <? extends V>> futures)} to create a batch of
* future.
* {@link static <V> ListenableFuture<List<V>> com.google.common.util.concurrent.Futures#allAsList(Iterable<? extends ListenableFuture <? extends V>> futures)}
* to create a batch of future.
* @param mayInterruptIfRunning {@code true} if the thread executing this
* task should be interrupted; otherwise, in-progress tasks are allowed
* to complete

View File

@ -82,7 +82,9 @@
<avatica.version>1.17.0</avatica.version>
<avro.version>1.9.2</avro.version>
<!-- sql/src/main/codegen/config.fmpp is based on a file from calcite-core, and needs to be
updated when upgrading Calcite. Refer to the top-level comments in that file for details. -->
updated when upgrading Calcite. Refer to the top-level comments in that file for details.
Also, CalcitePlanner is a clone of Calcite's PlannerImpl and may require updates when
Calcite is upgrade. -->
<calcite.version>1.21.0</calcite.version>
<datasketches.version>3.2.0</datasketches.version>
<datasketches.memory.version>2.0.0</datasketches.memory.version>

View File

@ -0,0 +1,44 @@
/*
* 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.
*/
//CHECKSTYLE.OFF: PackageName - Must be in Calcite
package org.apache.calcite.prepare;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.sql.SqlOperatorTable;
import org.apache.calcite.sql.validate.SqlConformance;
/**
* Extend {@link CalciteSqlValidator} to make it visible to Druid.
* {code CalciteSqlValidator} itself is protected which is why
* this class is in a Calcite name space.
*/
public class DruidSqlValidator extends CalciteSqlValidator
{
public DruidSqlValidator(
SqlOperatorTable opTab,
CalciteCatalogReader catalogReader,
JavaTypeFactory typeFactory,
SqlConformance conformance)
{
super(opTab, catalogReader, typeFactory, conformance);
}
}

View File

@ -0,0 +1,437 @@
/*
* 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.ImmutableList;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.calcite.config.CalciteConnectionConfigImpl;
import org.apache.calcite.config.CalciteConnectionProperty;
import org.apache.calcite.config.CalciteSystemProperty;
import org.apache.calcite.jdbc.CalciteSchema;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.plan.Context;
import org.apache.calcite.plan.ConventionTraitDef;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptTable.ViewExpander;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.plan.RelTraitDef;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.plan.volcano.VolcanoPlanner;
import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.prepare.DruidSqlValidator;
import org.apache.calcite.rel.RelCollationTraitDef;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexExecutor;
import org.apache.calcite.runtime.Hook;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlOperatorTable;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.calcite.sql.validate.SqlConformance;
import org.apache.calcite.sql.validate.SqlValidator;
import org.apache.calcite.sql2rel.RelDecorrelator;
import org.apache.calcite.sql2rel.SqlRexConvertletTable;
import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Planner;
import org.apache.calcite.tools.Program;
import org.apache.calcite.tools.RelBuilder;
import org.apache.calcite.tools.ValidationException;
import org.apache.calcite.util.Pair;
import javax.annotation.Nullable;
import java.io.Reader;
import java.util.List;
import java.util.Properties;
/**
* Calcite planner. Clone of Calcite's
* {@link org.apache.calcite.prepare.PlannerImpl}, as of version 1.21,
* but with the validator made accessible, and with the minimum of formatting
* changes needed to pass Druid's static checks. Note that the resulting code
* is more Calcite-like than Druid-like. There seemed no value in restructuring
* the code just to be more Druid-like.
*/
public class CalcitePlanner implements Planner, ViewExpander
{
private final SqlOperatorTable operatorTable;
private final ImmutableList<Program> programs;
private final FrameworkConfig frameworkConfig;
private final Context context;
private final CalciteConnectionConfig connectionConfig;
/** Holds the trait definitions to be registered with planner. May be null. */
private @Nullable final List<RelTraitDef> traitDefs;
private final SqlParser.Config parserConfig;
private final SqlToRelConverter.Config sqlToRelConverterConfig;
private final SqlRexConvertletTable convertletTable;
private State state;
// set in STATE_2_READY
private SchemaPlus defaultSchema;
private JavaTypeFactory typeFactory;
private RelOptPlanner planner;
private RexExecutor executor;
// set in STATE_4_VALIDATE
private SqlValidator validator;
private SqlNode validatedSqlNode;
// set in STATE_5_CONVERT
private RelRoot root;
public CalcitePlanner(FrameworkConfig config)
{
this.frameworkConfig = config;
this.defaultSchema = config.getDefaultSchema();
this.operatorTable = config.getOperatorTable();
this.programs = config.getPrograms();
this.parserConfig = config.getParserConfig();
this.sqlToRelConverterConfig = config.getSqlToRelConverterConfig();
this.state = State.STATE_0_CLOSED;
this.traitDefs = config.getTraitDefs();
this.convertletTable = config.getConvertletTable();
this.executor = config.getExecutor();
this.context = config.getContext();
this.connectionConfig = connConfig();
reset();
}
private CalciteConnectionConfig connConfig()
{
CalciteConnectionConfig unwrapped = context.unwrap(CalciteConnectionConfig.class);
if (unwrapped != null) {
return unwrapped;
}
Properties properties = new Properties();
properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
String.valueOf(parserConfig.caseSensitive()));
properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
String.valueOf(frameworkConfig.getParserConfig().conformance()));
return new CalciteConnectionConfigImpl(properties);
}
/** Makes sure that the state is at least the given state. */
private void ensure(State state)
{
if (state == this.state) {
return;
}
if (state.ordinal() < this.state.ordinal()) {
throw new IllegalArgumentException("cannot move to " + state + " from "
+ this.state);
}
state.from(this);
}
@Override
public RelTraitSet getEmptyTraitSet()
{
return planner.emptyTraitSet();
}
public FrameworkConfig frameworkConfig()
{
return frameworkConfig;
}
@Override
public void close()
{
typeFactory = null;
state = State.STATE_0_CLOSED;
}
@Override
public void reset()
{
ensure(State.STATE_0_CLOSED);
state = State.STATE_1_RESET;
}
private void ready()
{
switch (state) {
case STATE_0_CLOSED:
reset();
break;
default:
}
ensure(State.STATE_1_RESET);
RelDataTypeSystem typeSystem =
connectionConfig.typeSystem(RelDataTypeSystem.class,
RelDataTypeSystem.DEFAULT);
typeFactory = new JavaTypeFactoryImpl(typeSystem);
planner = new VolcanoPlanner(frameworkConfig.getCostFactory(), context);
RelOptUtil.registerDefaultRules(planner,
connectionConfig.materializationsEnabled(),
Hook.ENABLE_BINDABLE.get(false));
planner.setExecutor(executor);
state = State.STATE_2_READY;
// If user specify own traitDef, instead of default default trait,
// register the trait def specified in traitDefs.
if (this.traitDefs == null) {
planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
if (CalciteSystemProperty.ENABLE_COLLATION_TRAIT.value()) {
planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
}
} else {
for (RelTraitDef def : this.traitDefs) {
planner.addRelTraitDef(def);
}
}
}
@Override
public SqlNode parse(final Reader reader) throws SqlParseException
{
switch (state) {
case STATE_0_CLOSED:
case STATE_1_RESET:
ready();
break;
default:
}
ensure(State.STATE_2_READY);
SqlParser parser = SqlParser.create(reader, parserConfig);
SqlNode sqlNode = parser.parseStmt();
state = State.STATE_3_PARSED;
return sqlNode;
}
@Override
public SqlNode validate(SqlNode sqlNode) throws ValidationException
{
ensure(State.STATE_3_PARSED);
final SqlConformance conformance = conformance();
final CalciteCatalogReader catalogReader = createCatalogReader();
this.validator =
new DruidSqlValidator(operatorTable, catalogReader, typeFactory,
conformance);
this.validator.setIdentifierExpansion(true);
try {
validatedSqlNode = validator.validate(sqlNode);
}
catch (RuntimeException e) {
throw new ValidationException(e);
}
state = State.STATE_4_VALIDATED;
return validatedSqlNode;
}
public SqlValidator getValidator()
{
return validator;
}
private SqlConformance conformance()
{
return connectionConfig.conformance();
}
@Override
public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode)
throws ValidationException
{
final SqlNode validatedNode = this.validate(sqlNode);
final RelDataType type =
this.validator.getValidatedNodeType(validatedNode);
return Pair.of(validatedNode, type);
}
@Override
public final RelNode convert(SqlNode sql)
{
return rel(sql).rel;
}
@Override
public RelRoot rel(SqlNode sql)
{
ensure(State.STATE_4_VALIDATED);
assert validatedSqlNode != null;
final RexBuilder rexBuilder = createRexBuilder();
final RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
final SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
.withConfig(sqlToRelConverterConfig)
.withTrimUnusedFields(false)
.withConvertTableAccess(false)
.build();
final SqlToRelConverter sqlToRelConverter =
new SqlToRelConverter(this, validator,
createCatalogReader(), cluster, convertletTable, config);
root =
sqlToRelConverter.convertQuery(validatedSqlNode, false, true);
root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
final RelBuilder relBuilder =
config.getRelBuilderFactory().create(cluster, null);
root = root.withRel(
RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
state = State.STATE_5_CONVERTED;
return root;
}
@Override public RelRoot expandView(
RelDataType rowType,
String queryString,
List<String> schemaPath,
List<String> viewPath)
{
if (planner == null) {
ready();
}
SqlParser parser = SqlParser.create(queryString, parserConfig);
SqlNode sqlNode;
try {
sqlNode = parser.parseQuery();
}
catch (SqlParseException e) {
throw new RuntimeException("parse failed", e);
}
final SqlConformance conformance = conformance();
final CalciteCatalogReader catalogReader =
createCatalogReader().withSchemaPath(schemaPath);
final SqlValidator validator =
new DruidSqlValidator(operatorTable, catalogReader, typeFactory,
conformance);
validator.setIdentifierExpansion(true);
final RexBuilder rexBuilder = createRexBuilder();
final RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
final SqlToRelConverter.Config config = SqlToRelConverter
.configBuilder()
.withConfig(sqlToRelConverterConfig)
.withTrimUnusedFields(false)
.withConvertTableAccess(false)
.build();
final SqlToRelConverter sqlToRelConverter =
new SqlToRelConverter(this, validator,
catalogReader, cluster, convertletTable, config);
final RelRoot root =
sqlToRelConverter.convertQuery(sqlNode, true, false);
final RelRoot root2 =
root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
final RelBuilder relBuilder =
config.getRelBuilderFactory().create(cluster, null);
return root2.withRel(
RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
}
// CalciteCatalogReader is stateless; no need to store one
private CalciteCatalogReader createCatalogReader()
{
return new CalciteCatalogReader(
CalciteSchema.from(rootSchema(defaultSchema)),
CalciteSchema.from(defaultSchema).path(null),
typeFactory,
connectionConfig
);
}
private static SchemaPlus rootSchema(SchemaPlus schema)
{
for (;;) {
if (schema.getParentSchema() == null) {
return schema;
}
schema = schema.getParentSchema();
}
}
// RexBuilder is stateless; no need to store one
private RexBuilder createRexBuilder()
{
return new RexBuilder(typeFactory);
}
@Override
public JavaTypeFactory getTypeFactory()
{
return typeFactory;
}
@Override
public RelNode transform(
int ruleSetIndex,
RelTraitSet requiredOutputTraits,
RelNode rel
)
{
ensure(State.STATE_5_CONVERTED);
rel.getCluster().setMetadataProvider(
new CachingRelMetadataProvider(
rel.getCluster().getMetadataProvider(),
rel.getCluster().getPlanner()));
Program program = programs.get(ruleSetIndex);
return program.run(planner, rel, requiredOutputTraits, ImmutableList.of(),
ImmutableList.of());
}
/** Stage of a statement in the query-preparation lifecycle. */
private enum State
{
STATE_0_CLOSED {
@Override void from(CalcitePlanner planner)
{
planner.close();
}
},
STATE_1_RESET {
@Override void from(CalcitePlanner planner)
{
planner.ensure(STATE_0_CLOSED);
planner.reset();
}
},
STATE_2_READY {
@Override void from(CalcitePlanner planner)
{
STATE_1_RESET.from(planner);
planner.ready();
}
},
STATE_3_PARSED,
STATE_4_VALIDATED,
STATE_5_CONVERTED;
/** Moves planner's state to this state. This must be a higher state. */
void from(CalcitePlanner planner)
{
throw new IllegalArgumentException("cannot move from " + planner.state
+ " to " + this);
}
}
}

View File

@ -106,9 +106,18 @@ public class Calcites
// system properties, so we'll have to set those...
final String charset = ConversionUtil.NATIVE_UTF16_CHARSET_NAME;
// Deprecated in Calcite 1.19. See:
// https://calcite.apache.org/javadocAggregate/org/apache/calcite/util/SaffronProperties.html
System.setProperty("saffron.default.charset", Calcites.defaultCharset().name());
System.setProperty("saffron.default.nationalcharset", Calcites.defaultCharset().name());
System.setProperty("saffron.default.collation.name", StringUtils.format("%s$en_US", charset));
// The following are the current names. See org.apache.calcite.config.CalciteSystemProperty
// https://github.com/apache/calcite/blob/master/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
System.setProperty("calcite.default.charset", Calcites.defaultCharset().name());
System.setProperty("calcite.default.nationalcharset", Calcites.defaultCharset().name());
System.setProperty("calcite.default.collation.name", StringUtils.format("%s$en_US", charset));
}
public static Charset defaultCharset()

View File

@ -29,10 +29,6 @@ import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.calcite.DataContext;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.calcite.config.CalciteConnectionConfigImpl;
import org.apache.calcite.config.CalciteConnectionProperty;
import org.apache.calcite.interpreter.BindableConvention;
import org.apache.calcite.interpreter.BindableRel;
import org.apache.calcite.interpreter.Bindables;
@ -41,8 +37,6 @@ import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.prepare.Prepare;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.core.Sort;
@ -51,6 +45,7 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlDynamicParam;
import org.apache.calcite.sql.SqlExplain;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlInsert;
@ -62,10 +57,7 @@ import org.apache.calcite.sql.SqlOrderBy;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.validate.SqlValidator;
import org.apache.calcite.sql.validate.SqlValidatorUtil;
import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Frameworks;
import org.apache.calcite.tools.Planner;
import org.apache.calcite.tools.RelConversionException;
import org.apache.calcite.tools.ValidationException;
import org.apache.calcite.util.Pair;
@ -96,12 +88,12 @@ import org.apache.druid.utils.Throwables;
import org.joda.time.DateTimeZone;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@ -112,7 +104,7 @@ public class DruidPlanner implements Closeable
private static final Pattern UNNAMED_COLUMN_PATTERN = Pattern.compile("^EXPR\\$\\d+$", Pattern.CASE_INSENSITIVE);
private final FrameworkConfig frameworkConfig;
private final Planner planner;
private final CalcitePlanner planner;
private final PlannerContext plannerContext;
private final QueryMakerFactory queryMakerFactory;
@ -125,11 +117,18 @@ public class DruidPlanner implements Closeable
)
{
this.frameworkConfig = frameworkConfig;
this.planner = Frameworks.getPlanner(frameworkConfig);
this.planner = new CalcitePlanner(frameworkConfig);
this.plannerContext = plannerContext;
this.queryMakerFactory = queryMakerFactory;
}
private ParsedNodes parse() throws SqlParseException, ValidationException
{
resetPlanner();
SqlNode root = planner.parse(plannerContext.getSql());
return ParsedNodes.create(root, plannerContext.getTimeZone());
}
/**
* Validates a SQL query and populates {@link PlannerContext#getResourceActions()}.
*
@ -137,18 +136,17 @@ public class DruidPlanner implements Closeable
*/
public ValidationResult validate(boolean authorizeContextParams) throws SqlParseException, ValidationException
{
resetPlanner();
final ParsedNodes parsed = ParsedNodes.create(planner.parse(plannerContext.getSql()), plannerContext.getTimeZone());
final SqlValidator validator = getValidator();
final ParsedNodes parsed = parse();
final SqlNode validatedQueryNode;
try {
validatedQueryNode = validator.validate(rewriteDynamicParameters(parsed.getQueryNode()));
validatedQueryNode = planner.validate(rewriteDynamicParameters(parsed.getQueryNode()));
}
catch (RuntimeException e) {
throw new ValidationException(e);
}
final SqlValidator validator = planner.getValidator();
SqlResourceCollectorShuttle resourceCollectorShuttle = new SqlResourceCollectorShuttle(validator, plannerContext);
validatedQueryNode.accept(resourceCollectorShuttle);
@ -172,18 +170,19 @@ public class DruidPlanner implements Closeable
* Prepare an SQL query for execution, including some initial parsing and validation and any dynamic parameter type
* resolution, to support prepared statements via JDBC.
*
* In some future this could perhaps re-use some of the work done by {@link #validate(boolean)}
* In some future this could perhaps re-use some work done by {@link #validate(boolean)}
* instead of repeating it, but that day is not today.
*/
// RelConversionException is no longer thrown, but removing it causes
// cascading Intellij warnings in many files. Leave well enough alone.
@SuppressWarnings("RedundantThrows")
public PrepareResult prepare() throws SqlParseException, ValidationException, RelConversionException
{
resetPlanner();
final ParsedNodes parsed = ParsedNodes.create(planner.parse(plannerContext.getSql()), plannerContext.getTimeZone());
final ParsedNodes parsed = parse();
final SqlNode validatedQueryNode = planner.validate(parsed.getQueryNode());
final RelRoot rootQueryRel = planner.rel(validatedQueryNode);
final SqlValidator validator = getValidator();
final SqlValidator validator = planner.getValidator();
final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory();
final RelDataType parameterTypes = validator.getParameterRowType(validator.validate(validatedQueryNode));
final RelDataType returnedRowType;
@ -201,11 +200,14 @@ public class DruidPlanner implements Closeable
* Plan an SQL query for execution, returning a {@link PlannerResult} which can be used to actually execute the query.
*
* Ideally, the query can be planned into a native Druid query, using {@link #planWithDruidConvention}, but will
* fall-back to {@link #planWithBindableConvention} if this is not possible.
* fall back to {@link #planWithBindableConvention} if this is not possible.
*
* In some future this could perhaps re-use some of the work done by {@link #validate(boolean)}
* In some future this could perhaps re-use some work done by {@link #validate(boolean)}
* instead of repeating it, but that day is not today.
*/
// RelConversionException is no longer thrown, but removing it causes
// cascading Intellij warnings in many files. Leave well enough alone.
@SuppressWarnings("RedundantThrows")
public PlannerResult plan() throws SqlParseException, ValidationException, RelConversionException
{
resetPlanner();
@ -307,7 +309,7 @@ public class DruidPlanner implements Closeable
final RelRoot root,
@Nullable final SqlExplain explain,
@Nullable final SqlInsert insertOrReplace
) throws ValidationException, RelConversionException
) throws ValidationException
{
final RelRoot possiblyLimitedRoot = possiblyWrapRootWithOuterLimitFromContext(root);
final QueryMaker queryMaker = buildQueryMaker(root, insertOrReplace);
@ -359,7 +361,7 @@ public class DruidPlanner implements Closeable
private PlannerResult planWithBindableConvention(
final RelRoot root,
@Nullable final SqlExplain explain
) throws RelConversionException
)
{
BindableRel bindableRel = (BindableRel) planner.transform(
CalciteRulesManager.BINDABLE_CONVENTION_RULES,
@ -388,8 +390,8 @@ public class DruidPlanner implements Closeable
} else {
final BindableRel theRel = bindableRel;
final DataContext dataContext = plannerContext.createDataContext(
(JavaTypeFactory) planner.getTypeFactory(),
plannerContext.getParameters()
planner.getTypeFactory(),
plannerContext.getParameters()
);
final Supplier<Sequence<Object[]>> resultsSupplier = () -> {
final Enumerable<?> enumerable = theRel.bind(dataContext);
@ -411,7 +413,10 @@ public class DruidPlanner implements Closeable
@Override
public Object[] next()
{
return (Object[]) enumerator.current();
// Avoids an Intellij IteratorNextCanNotThrowNoSuchElementException
// warning.
Object[] temp = (Object[]) enumerator.current();
return temp;
}
});
}
@ -595,47 +600,9 @@ public class DruidPlanner implements Closeable
return new RelRoot(newRootRel, root.validatedRowType, root.kind, root.fields, root.collation);
}
/**
* Constructs an SQL validator, just like papa {@link #planner} uses.
*/
private SqlValidator getValidator()
{
// this is sort of lame, planner won't cough up its validator, which is nice and seeded after validating a query,
// but it is private and has no accessors, so make another one so we can get the parameter types... but i suppose
// beats creating our own Prepare and Planner implementations
Preconditions.checkNotNull(planner.getTypeFactory());
final CalciteConnectionConfig connectionConfig;
if (frameworkConfig.getContext() != null) {
connectionConfig = frameworkConfig.getContext().unwrap(CalciteConnectionConfig.class);
} else {
Properties properties = new Properties();
properties.setProperty(
CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
String.valueOf(PlannerFactory.PARSER_CONFIG.caseSensitive())
);
connectionConfig = new CalciteConnectionConfigImpl(properties);
}
Prepare.CatalogReader catalogReader = new CalciteCatalogReader(
CalciteSchema.from(frameworkConfig.getDefaultSchema().getParentSchema()),
CalciteSchema.from(frameworkConfig.getDefaultSchema()).path(null),
planner.getTypeFactory(),
connectionConfig
);
return SqlValidatorUtil.newValidator(
frameworkConfig.getOperatorTable(),
catalogReader,
planner.getTypeFactory(),
DruidConformance.instance()
);
}
/**
* Uses {@link SqlParameterizerShuttle} to rewrite {@link SqlNode} to swap out any
* {@link org.apache.calcite.sql.SqlDynamicParam} early for their {@link SqlLiteral}
* {@link SqlDynamicParam} early for their {@link SqlLiteral}
* replacement
*/
private SqlNode rewriteDynamicParameters(SqlNode parsed)