This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new d52abe7b38 Today is that day - Single pass through Calcite planner 
(#12636)
d52abe7b38 is described below

commit d52abe7b38508908ef45e016446bd1f7c0c1c7a7
Author: Paul Rogers <[email protected]>
AuthorDate: Fri Jul 29 18:53:21 2022 -0700

    Today is that day - Single pass through Calcite planner (#12636)
    
    * Druid planner now makes only one pass through Calcite planner
    
    Resolves the issue that required two parse/plan cycles: one
    for validate, another for plan. Creates a clone of the Calcite
    planner and validator to resolve the conflict that prevented
    the merger.
---
 .../apache/druid/tests/query/ITJdbcQueryTest.java  |   2 +-
 ...qlValidator.java => BaseDruidSqlValidator.java} |  11 +-
 .../java/org/apache/druid/sql/SqlLifecycle.java    | 124 ++++++----
 .../apache/druid/sql/avatica/DruidConnection.java  |   1 -
 .../druid/sql/calcite/planner/CalcitePlanner.java  |   6 +-
 .../druid/sql/calcite/planner/DruidPlanner.java    | 258 +++++++++++++--------
 ...alidationResult.java => DruidSqlValidator.java} |  34 ++-
 .../druid/sql/calcite/planner/PlannerContext.java  |   2 +-
 .../druid/sql/calcite/planner/PlannerFactory.java  |  14 +-
 .../calcite/planner/RelParameterizerShuttle.java   |   4 +-
 .../calcite/planner/SqlParameterizerShuttle.java   |  77 +++---
 .../druid/sql/calcite/view/DruidViewMacro.java     |   3 +-
 .../org/apache/druid/sql/SqlLifecycleTest.java     |  48 ++--
 .../druid/sql/calcite/BaseCalciteQueryTest.java    |   4 +-
 .../druid/sql/calcite/CalciteInsertDmlTest.java    |  16 +-
 .../sql/calcite/CalciteParameterQueryTest.java     |  15 +-
 .../calcite/SqlVectorizedExpressionSanityTest.java |   7 +-
 .../druid/sql/calcite/util/QueryLogHook.java       |   4 +-
 18 files changed, 356 insertions(+), 274 deletions(-)

diff --git 
a/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java
 
b/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java
index 6e3944029e..fe95c27bc7 100644
--- 
a/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java
+++ 
b/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java
@@ -211,7 +211,7 @@ public class ITJdbcQueryTest
     }
   }
 
-  @Test(expectedExceptions = AvaticaSqlException.class, 
expectedExceptionsMessageRegExp = ".* Parameter at position\\[0] is not bound")
+  @Test(expectedExceptions = AvaticaSqlException.class, 
expectedExceptionsMessageRegExp = ".* Parameter at position \\[0] is not bound")
   public void testJdbcPrepareStatementQueryMissingParameters() throws 
SQLException
   {
     for (String url : connections) {
diff --git 
a/sql/src/main/java/org/apache/calcite/prepare/DruidSqlValidator.java 
b/sql/src/main/java/org/apache/calcite/prepare/BaseDruidSqlValidator.java
similarity index 80%
rename from sql/src/main/java/org/apache/calcite/prepare/DruidSqlValidator.java
rename to 
sql/src/main/java/org/apache/calcite/prepare/BaseDruidSqlValidator.java
index 0ee436dd0b..d9f714824c 100644
--- a/sql/src/main/java/org/apache/calcite/prepare/DruidSqlValidator.java
+++ b/sql/src/main/java/org/apache/calcite/prepare/BaseDruidSqlValidator.java
@@ -26,13 +26,13 @@ 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.
+ * Extend the Java-focused {@link CalciteSqlValidator} to make it visible
+ * to Druid. {@link CalciteSqlValidator} itself is protected and can be
+ * extended only if this class is in the same Calcite package.
  */
-public class DruidSqlValidator extends CalciteSqlValidator
+public class BaseDruidSqlValidator extends CalciteSqlValidator
 {
-  public DruidSqlValidator(
+  public BaseDruidSqlValidator(
       SqlOperatorTable opTab,
       CalciteCatalogReader catalogReader,
       JavaTypeFactory typeFactory,
@@ -40,5 +40,4 @@ public class DruidSqlValidator extends CalciteSqlValidator
   {
     super(opTab, catalogReader, typeFactory, conformance);
   }
-
 }
diff --git a/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java 
b/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java
index 002fd3045a..9007c77535 100644
--- a/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java
+++ b/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java
@@ -55,7 +55,6 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.planner.PlannerFactory;
 import org.apache.druid.sql.calcite.planner.PlannerResult;
 import org.apache.druid.sql.calcite.planner.PrepareResult;
-import org.apache.druid.sql.calcite.planner.ValidationResult;
 import org.apache.druid.sql.http.SqlParameter;
 import org.apache.druid.sql.http.SqlQuery;
 
@@ -69,6 +68,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 
 /**
@@ -106,14 +106,20 @@ public class SqlLifecycle
   @GuardedBy("stateLock")
   private State state = State.NEW;
 
-  // init during intialize
+  // init during initialize
   private String sql;
   private QueryContext queryContext;
   private List<TypedValue> parameters;
+
   // init during plan
+  /**
+   * The Druid planner follows the SQL statement through the lifecycle.
+   * The planner's state is start --> validate --> (prepare | plan).
+   */
+  private DruidPlanner planner;
   private PlannerContext plannerContext;
-  private ValidationResult validationResult;
   private PrepareResult prepareResult;
+  private Set<ResourceAction> resourceActions;
   private PlannerResult plannerResult;
 
   public SqlLifecycle(
@@ -196,14 +202,13 @@ public class SqlLifecycle
     }
     transition(State.INITIALIZED, State.AUTHORIZING);
     validate(authenticationResult);
-    Access access = doAuthorize(
+    doAuthorize(resourceActions ->
         AuthorizationUtils.authorizeAllResourceActions(
             authenticationResult,
-            validationResult.getResourceActions(),
+            resourceActions,
             plannerFactory.getAuthorizerMapper()
         )
     );
-    checkAccess(access);
   }
 
   /**
@@ -218,26 +223,33 @@ public class SqlLifecycle
     transition(State.INITIALIZED, State.AUTHORIZING);
     AuthenticationResult authResult = 
AuthorizationUtils.authenticationResultFromRequest(req);
     validate(authResult);
-    Access access = doAuthorize(
+    doAuthorize(resourceActions ->
         AuthorizationUtils.authorizeAllResourceActions(
             req,
-            validationResult.getResourceActions(),
+            resourceActions,
             plannerFactory.getAuthorizerMapper()
         )
     );
-    checkAccess(access);
   }
 
-  private ValidationResult validate(AuthenticationResult authenticationResult)
+  /**
+   * Perform the validation step on the Druid planner, leaving the planner
+   * ready to perform either prepare or plan.
+   */
+  private void validate(AuthenticationResult authenticationResult)
   {
-    try (DruidPlanner planner = plannerFactory.createPlanner(sql, 
queryContext)) {
+    try {
+      planner = plannerFactory.createPlanner(sql, queryContext);
       // set planner context for logs/metrics in case something explodes early
-      this.plannerContext = planner.getPlannerContext();
-      this.plannerContext.setAuthenticationResult(authenticationResult);
+      plannerContext = planner.getPlannerContext();
+      plannerContext.setAuthenticationResult(authenticationResult);
       // set parameters on planner context, if parameters have already been set
-      this.plannerContext.setParameters(parameters);
-      this.validationResult = 
planner.validate(authConfig.authorizeQueryContextParams());
-      return validationResult;
+      plannerContext.setParameters(parameters);
+      planner.validate();
+
+      // Capture the resource actions as these are reference past the
+      // life of the planner itself.
+      resourceActions = 
planner.resourceActions(authConfig.authorizeQueryContextParams());
     }
     // we can't collapse catch clauses since SqlPlanningException has 
type-sensitive constructors.
     catch (SqlParseException e) {
@@ -248,46 +260,44 @@ public class SqlLifecycle
     }
   }
 
-  private Access doAuthorize(final Access authorizationResult)
+  private void doAuthorize(Function<Set<ResourceAction>, Access> authorizer)
   {
+    Access authorizationResult = planner.authorize(
+        authorizer,
+        authConfig.authorizeQueryContextParams()
+    );
     if (!authorizationResult.isAllowed()) {
       // Not authorized; go straight to Jail, do not pass Go.
       transition(State.AUTHORIZING, State.UNAUTHORIZED);
     } else {
       transition(State.AUTHORIZING, State.AUTHORIZED);
     }
-    return authorizationResult;
-  }
-
-  private void checkAccess(Access access)
-  {
-    plannerContext.setAuthorizationResult(access);
-    if (!access.isAllowed()) {
-      throw new ForbiddenException(access.toString());
+    if (!authorizationResult.isAllowed()) {
+      throw new ForbiddenException(authorizationResult.toString());
     }
   }
 
   /**
-   * Prepare the query lifecycle for execution, without completely planning 
into something that is executable, but
-   * including some initial parsing and validation and any dyanmic parameter 
type resolution, to support prepared
+   * Prepare the query lifecycle for execution, without completely planning 
into
+   * something that is executable, but including some initial parsing and
+   * validation and any dynamic parameter type resolution, to support prepared
    * statements via JDBC.
+   *
+   * The planner must have already performed the validation step: the planner
+   * state is reused here.
    */
-  public PrepareResult prepare() throws RelConversionException
+  public PrepareResult prepare()
   {
     synchronized (stateLock) {
       if (state != State.AUTHORIZED) {
-        throw new ISE("Cannot prepare because current state[%s] is not [%s].", 
state, State.AUTHORIZED);
+        throw new ISE("Cannot prepare because current state [%s] is not 
[%s].", state, State.AUTHORIZED);
       }
     }
     Preconditions.checkNotNull(plannerContext, "Cannot prepare, plannerContext 
is null");
-    try (DruidPlanner planner = 
plannerFactory.createPlannerWithContext(plannerContext)) {
+    try {
       this.prepareResult = planner.prepare();
       return prepareResult;
     }
-    // we can't collapse catch clauses since SqlPlanningException has 
type-sensitive constructors.
-    catch (SqlParseException e) {
-      throw new SqlPlanningException(e);
-    }
     catch (ValidationException e) {
       throw new SqlPlanningException(e);
     }
@@ -296,22 +306,27 @@ public class SqlLifecycle
   /**
    * Plan the query to enable execution.
    *
-   * If successful, the lifecycle will first transition from {@link 
State#AUTHORIZED} to {@link State#PLANNED}.
+   * The planner must have already performed the validation step: the planner
+   * state is reused here.
+   *
+   * If successful, the lifecycle will first transition from
+   * {@link State#AUTHORIZED} to {@link State#PLANNED}.
    */
   public void plan() throws RelConversionException
   {
     transition(State.AUTHORIZED, State.PLANNED);
     Preconditions.checkNotNull(plannerContext, "Cannot plan, plannerContext is 
null");
-    try (DruidPlanner planner = 
plannerFactory.createPlannerWithContext(plannerContext)) {
+    try {
       this.plannerResult = planner.plan();
     }
-    // we can't collapse catch clauses since SqlPlanningException has 
type-sensitive constructors.
-    catch (SqlParseException e) {
-      throw new SqlPlanningException(e);
-    }
     catch (ValidationException e) {
       throw new SqlPlanningException(e);
     }
+    finally {
+      // Done with the planner, close it.
+      planner.close();
+      planner = null;
+    }
   }
 
   /**
@@ -376,20 +391,20 @@ public class SqlLifecycle
     });
   }
 
-
   @VisibleForTesting
-  public ValidationResult runAnalyzeResources(AuthenticationResult 
authenticationResult)
+  public Set<ResourceAction> runAnalyzeResources(AuthenticationResult 
authenticationResult)
   {
-    return validate(authenticationResult);
+    validate(authenticationResult);
+    return getRequiredResourceActions();
   }
 
   public Set<ResourceAction> getRequiredResourceActions()
   {
-    return Preconditions.checkNotNull(validationResult, 
"validationResult").getResourceActions();
+    return resourceActions;
   }
 
   /**
-   * Cancel all native queries associated to this lifecycle.
+   * Cancel all native queries associated with this lifecycle.
    *
    * This method is thread-safe.
    */
@@ -405,7 +420,7 @@ public class SqlLifecycle
     final CopyOnWriteArrayList<String> nativeQueryIds = 
plannerContext.getNativeQueryIds();
 
     for (String nativeQueryId : nativeQueryIds) {
-      log.debug("canceling native query [%s]", nativeQueryId);
+      log.debug("Canceling native query [%s]", nativeQueryId);
       queryScheduler.cancelQuery(nativeQueryId);
     }
   }
@@ -433,13 +448,22 @@ public class SqlLifecycle
 
       if (state != State.CANCELLED) {
         if (state == State.DONE) {
-          log.warn("Tried to emit logs and metrics twice for query[%s]!", 
sqlQueryId());
+          log.warn("Tried to emit logs and metrics twice for query [%s]!", 
sqlQueryId());
         }
 
         state = State.DONE;
       }
     }
 
+    final Set<ResourceAction> actions;
+    if (planner != null) {
+      actions = getRequiredResourceActions();
+      planner.close();
+      planner = null;
+    } else {
+      actions = null;
+    }
+
     final boolean success = e == null;
     final long queryTimeNs = System.nanoTime() - startNs;
 
@@ -449,10 +473,10 @@ public class SqlLifecycle
         metricBuilder.setDimension("id", plannerContext.getSqlQueryId());
         metricBuilder.setDimension("nativeQueryIds", 
plannerContext.getNativeQueryIds().toString());
       }
-      if (validationResult != null) {
+      if (actions != null) {
         metricBuilder.setDimension(
             "dataSource",
-            validationResult.getResourceActions()
+            actions
                             .stream()
                             .map(action -> action.getResource().getName())
                             .collect(Collectors.toList())
@@ -527,7 +551,7 @@ public class SqlLifecycle
       }
       if (state != from) {
         throw new ISE(
-            "Cannot transition from[%s] to[%s] because current state[%s] is 
not [%s].",
+            "Cannot transition from [%s] to [%s] because current state [%s] is 
not [%s].",
             from,
             to,
             state,
diff --git 
a/sql/src/main/java/org/apache/druid/sql/avatica/DruidConnection.java 
b/sql/src/main/java/org/apache/druid/sql/avatica/DruidConnection.java
index ab6ae65a98..6c5238fc45 100644
--- a/sql/src/main/java/org/apache/druid/sql/avatica/DruidConnection.java
+++ b/sql/src/main/java/org/apache/druid/sql/avatica/DruidConnection.java
@@ -141,7 +141,6 @@ public class DruidConnection
         throw DruidMeta.logFailure(new ISE("Too many open statements, limit is 
[%,d]", maxStatements));
       }
 
-      @SuppressWarnings("GuardedBy")
       final DruidJdbcPreparedStatement jdbcStmt = new 
DruidJdbcPreparedStatement(
           this,
           statementId,
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlanner.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlanner.java
index bc3a3616be..49d2b5f431 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlanner.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlanner.java
@@ -36,8 +36,8 @@ 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.BaseDruidSqlValidator;
 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;
@@ -237,7 +237,7 @@ public class CalcitePlanner implements Planner, ViewExpander
     final SqlConformance conformance = conformance();
     final CalciteCatalogReader catalogReader = createCatalogReader();
     this.validator =
-        new DruidSqlValidator(operatorTable, catalogReader, typeFactory,
+        new BaseDruidSqlValidator(operatorTable, catalogReader, typeFactory,
             conformance);
     this.validator.setIdentifierExpansion(true);
     try {
@@ -324,7 +324,7 @@ public class CalcitePlanner implements Planner, ViewExpander
     final CalciteCatalogReader catalogReader =
         createCatalogReader().withSchemaPath(schemaPath);
     final SqlValidator validator =
-        new DruidSqlValidator(operatorTable, catalogReader, typeFactory,
+        new BaseDruidSqlValidator(operatorTable, catalogReader, typeFactory,
             conformance);
     validator.setIdentifierExpansion(true);
 
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
index abf602f4c0..e3feaa2a2f 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
@@ -45,12 +46,10 @@ 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;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOrderBy;
@@ -58,7 +57,6 @@ 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.tools.FrameworkConfig;
-import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.calcite.util.Pair;
 import org.apache.druid.common.utils.IdUtils;
@@ -71,6 +69,7 @@ import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.query.Query;
 import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.server.security.Access;
 import org.apache.druid.server.security.Action;
 import org.apache.druid.server.security.Resource;
 import org.apache.druid.server.security.ResourceAction;
@@ -95,19 +94,46 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
+/**
+ * Druid SQL planner. Wraps the underlying Calcite planner with Druid-specific
+ * actions around resource validation and conversion of the Calcite logical
+ * plan into a Druid native query.
+ * <p>
+ * The planner is designed to use once: it makes one trip through its
+ * lifecycle defined as:
+ * <p>
+ * start --> validate [--> prepare] --> plan
+ */
 public class DruidPlanner implements Closeable
 {
+  public enum State
+  {
+    START, VALIDATED, PREPARED, PLANNED
+  }
+
   private static final EmittingLogger log = new 
EmittingLogger(DruidPlanner.class);
   private static final Pattern UNNAMED_COLUMN_PATTERN = 
Pattern.compile("^EXPR\\$\\d+$", Pattern.CASE_INSENSITIVE);
+  @VisibleForTesting
+  public static final String UNNAMED_INGESTION_COLUMN_ERROR =
+      "Cannot ingest expressions that do not have an alias "
+          + "or columns with names like EXPR$[digit].\n"
+          + "E.g. if you are ingesting \"func(X)\", then you can rewrite it as 
"
+          + "\"func(X) as myColumn\"";
 
   private final FrameworkConfig frameworkConfig;
   private final CalcitePlanner planner;
   private final PlannerContext plannerContext;
   private final QueryMakerFactory queryMakerFactory;
-
+  private State state = State.START;
+  private ParsedNodes parsed;
+  private SqlNode validatedQueryNode;
+  private boolean authorized;
+  private Set<ResourceAction> resourceActions;
+  private RelRoot rootQueryRel;
   private RexBuilder rexBuilder;
 
   DruidPlanner(
@@ -132,15 +158,49 @@ public class DruidPlanner implements Closeable
   /**
    * Validates a SQL query and populates {@link 
PlannerContext#getResourceActions()}.
    *
-   * @return set of {@link Resource} corresponding to any Druid datasources or 
views which are taking part in the query.
+   * @return set of {@link Resource} corresponding to any Druid datasources
+   * or views which are taking part in the query.
    */
-  public ValidationResult validate(boolean authorizeContextParams) throws 
SqlParseException, ValidationException
+  public void validate() throws SqlParseException, ValidationException
   {
-    final ParsedNodes parsed = parse();
-    final SqlNode validatedQueryNode;
+    Preconditions.checkState(state == State.START);
+    resetPlanner();
+    SqlNode root = planner.parse(plannerContext.getSql());
+    parsed = ParsedNodes.create(root, plannerContext.getTimeZone());
+
+    try {
+      if (parsed.getIngestionGranularity() != null) {
+        plannerContext.getQueryContext().addSystemParam(
+            DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY,
+            
plannerContext.getJsonMapper().writeValueAsString(parsed.getIngestionGranularity())
+        );
+      }
+    }
+    catch (JsonProcessingException e) {
+      throw new ValidationException("Unable to serialize partition 
granularity.");
+    }
+
+    if (parsed.getReplaceIntervals() != null) {
+      plannerContext.getQueryContext().addSystemParam(
+          DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS,
+          String.join(",", parsed.getReplaceIntervals())
+      );
+    }
 
     try {
-      validatedQueryNode = 
planner.validate(rewriteDynamicParameters(parsed.getQueryNode()));
+      // Uses {@link SqlParameterizerShuttle} to rewrite {@link SqlNode} to 
swap out any
+      // {@link org.apache.calcite.sql.SqlDynamicParam} early for their {@link 
SqlLiteral}
+      // replacement.
+      //
+      // Parameter replacement is done only if the client provides parameter 
values.
+      // If this is a PREPARE-only, then there will be no values even if the 
statement contains
+      // parameters. If this is a PLAN, then we'll catch later the case that 
the statement
+      // contains parameters, but no values were provided.
+      SqlNode queryNode = parsed.getQueryNode();
+      if (!plannerContext.getParameters().isEmpty()) {
+        queryNode = queryNode.accept(new 
SqlParameterizerShuttle(plannerContext));
+      }
+      validatedQueryNode = planner.validate(queryNode);
     }
     catch (RuntimeException e) {
       throw new ValidationException(e);
@@ -150,7 +210,7 @@ public class DruidPlanner implements Closeable
     SqlResourceCollectorShuttle resourceCollectorShuttle = new 
SqlResourceCollectorShuttle(validator, plannerContext);
     validatedQueryNode.accept(resourceCollectorShuttle);
 
-    final Set<ResourceAction> resourceActions = new 
HashSet<>(resourceCollectorShuttle.getResourceActions());
+    resourceActions = new 
HashSet<>(resourceCollectorShuttle.getResourceActions());
 
     if (parsed.getInsertOrReplace() != null) {
       // Check if CTX_SQL_OUTER_LIMIT is specified and fail the query if it 
is. CTX_SQL_OUTER_LIMIT being provided causes
@@ -161,35 +221,33 @@ public class DruidPlanner implements Closeable
       final String targetDataSource = 
validateAndGetDataSourceForIngest(parsed.getInsertOrReplace());
       resourceActions.add(new ResourceAction(new Resource(targetDataSource, 
ResourceType.DATASOURCE), Action.WRITE));
     }
-    if (authorizeContextParams) {
-      
plannerContext.getQueryContext().getUserParams().keySet().forEach(contextParam 
-> resourceActions.add(
-          new ResourceAction(new Resource(contextParam, 
ResourceType.QUERY_CONTEXT), Action.WRITE)
-      ));
-    }
 
+    state = State.VALIDATED;
     plannerContext.setResourceActions(resourceActions);
-    return new ValidationResult(resourceActions);
   }
 
   /**
-   * Prepare an SQL query for execution, including some initial parsing and 
validation and any dynamic parameter type
-   * resolution, to support prepared statements via JDBC.
+   * Prepare a 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 work done by {@link 
#validate(boolean)}
-   * instead of repeating it, but that day is not today.
+   * Prepare reuses the validation done in {@link #validate()} which must be
+   * called first.
+   *
+   * A query can be prepared on a data source without having permissions on
+   * that data source. This odd state of affairs is necessary because
+   * {@link org.apache.druid.sql.calcite.view.DruidViewMacro} prepares
+   * a view while having no information about the user of that view.
    */
-  // 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
+  public PrepareResult prepare() throws ValidationException
   {
-    final ParsedNodes parsed = parse();
-    final SqlNode validatedQueryNode = planner.validate(parsed.getQueryNode());
-    final RelRoot rootQueryRel = planner.rel(validatedQueryNode);
+    Preconditions.checkState(state == State.VALIDATED);
+
+    rootQueryRel = planner.rel(validatedQueryNode);
 
-    final SqlValidator validator = planner.getValidator();
     final RelDataTypeFactory typeFactory = 
rootQueryRel.rel.getCluster().getTypeFactory();
-    final RelDataType parameterTypes = 
validator.getParameterRowType(validator.validate(validatedQueryNode));
+    final SqlValidator validator = planner.getValidator();
+    final RelDataType parameterTypes = 
validator.getParameterRowType(validatedQueryNode);
     final RelDataType returnedRowType;
 
     if (parsed.getExplainNode() != null) {
@@ -198,51 +256,72 @@ public class DruidPlanner implements Closeable
       returnedRowType = buildQueryMaker(rootQueryRel, 
parsed.getInsertOrReplace()).getResultType();
     }
 
+    state = State.PREPARED;
     return new PrepareResult(returnedRowType, parameterTypes);
   }
 
+  /**
+   * Authorizes the statement. Done within the planner to enforce the 
authorization
+   * step within the planner's state machine.
+   *
+   * @param authorizer a function from resource actions to a {@link Access} 
result.
+   * @return the return value from the authorizer
+   */
+  public Access authorize(Function<Set<ResourceAction>, Access> authorizer, 
boolean authorizeContextParams)
+  {
+    Preconditions.checkState(state == State.VALIDATED);
+    Access access = authorizer.apply(resourceActions(authorizeContextParams));
+    plannerContext.setAuthorizationResult(access);
+
+    // Authorization is done as a flag, not a state, alas.
+    // Views do prepare without authorize, Avatica does authorize, then 
prepare,
+    // so the only constraint is that authorize be done after validation, 
before plan.
+    authorized = true;
+    return access;
+  }
+
+  /**
+   * Return the resource actions corresponding to the datasources and views 
which
+   * an authenticated request must be authorized for to process the
+   * query. The actions will be {@code null} if the
+   * planner has not yet advanced to the validation step. This may occur if
+   * validation fails and the caller ({@code SqlLifecycle}) accesses the 
resource
+   * actions as part of clean-up.
+   */
+  public Set<ResourceAction> resourceActions(boolean includeContext)
+  {
+    Set<ResourceAction> actions;
+    if (includeContext) {
+      actions = new HashSet<>(resourceActions);
+      
plannerContext.getQueryContext().getUserParams().keySet().forEach(contextParam 
-> actions.add(
+          new ResourceAction(new Resource(contextParam, 
ResourceType.QUERY_CONTEXT), Action.WRITE)
+      ));
+    } else {
+      actions = resourceActions;
+    }
+    return actions;
+  }
+
   /**
    * 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.
    *
-   * 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.
+   * Planning reuses the validation done in `validate()` which must be called 
first.
    */
-  // 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
+  public PlannerResult plan() throws ValidationException
   {
-    resetPlanner();
-
-    final ParsedNodes parsed = 
ParsedNodes.create(planner.parse(plannerContext.getSql()), 
plannerContext.getTimeZone());
-
-    try {
-      if (parsed.getIngestionGranularity() != null) {
-        plannerContext.getQueryContext().addSystemParam(
-            DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY,
-            
plannerContext.getJsonMapper().writeValueAsString(parsed.getIngestionGranularity())
-        );
-      }
-    }
-    catch (JsonProcessingException e) {
-      throw new ValidationException("Unable to serialize partition 
granularity.");
-    }
-
-    if (parsed.getReplaceIntervals() != null) {
-      plannerContext.getQueryContext().addSystemParam(
-          DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS,
-          String.join(",", parsed.getReplaceIntervals())
-      );
+    Preconditions.checkState(state == State.VALIDATED || state == 
State.PREPARED);
+    Preconditions.checkState(authorized);
+    if (state == State.VALIDATED) {
+      rootQueryRel = planner.rel(validatedQueryNode);
     }
 
     // the planner's type factory is not available until after parsing
     this.rexBuilder = new RexBuilder(planner.getTypeFactory());
-    final SqlNode parameterizedQueryNode = 
rewriteDynamicParameters(parsed.getQueryNode());
-    final SqlNode validatedQueryNode = 
planner.validate(parameterizedQueryNode);
-    final RelRoot rootQueryRel = planner.rel(validatedQueryNode);
+    state = State.PLANNED;
 
     try {
       return planWithDruidConvention(rootQueryRel, parsed.getExplainNode(), 
parsed.getInsertOrReplace());
@@ -250,7 +329,7 @@ public class DruidPlanner implements Closeable
     catch (Exception e) {
       Throwable cannotPlanException = Throwables.getCauseOfType(e, 
RelOptPlanner.CannotPlanException.class);
       if (null == cannotPlanException) {
-        // Not a CannotPlanningException, rethrow without trying with bindable
+        // Not a CannotPlanException, rethrow without trying with bindable
         throw e;
       }
 
@@ -317,10 +396,20 @@ public class DruidPlanner implements Closeable
   ) throws ValidationException
   {
     final RelRoot possiblyLimitedRoot = 
possiblyWrapRootWithOuterLimitFromContext(root);
-    final QueryMaker queryMaker = buildQueryMaker(root, insertOrReplace);
+    final QueryMaker queryMaker = buildQueryMaker(possiblyLimitedRoot, 
insertOrReplace);
     plannerContext.setQueryMaker(queryMaker);
 
-    RelNode parameterized = 
rewriteRelDynamicParameters(possiblyLimitedRoot.rel);
+    // Fall-back dynamic parameter substitution using {@link 
RelParameterizerShuttle}
+    // in the event that {@link #rewriteDynamicParameters(SqlNode)} was unable 
to
+    // successfully substitute all parameter values, and will cause a failure 
if any
+    // dynamic a parameters are not bound. This occurs at least for DATE 
parameters
+    // with integer values.
+    //
+    // This check also catches the case where we did not do a parameter check 
earlier
+    // because no values were provided. (Values are not required in the 
PREPARE case
+    // but now that we're planning, we require them.)
+    RelNode parameterized = possiblyLimitedRoot.rel.accept(
+        new RelParameterizerShuttle(plannerContext));
     final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
         CalciteRulesManager.DRUID_CONVENTION_RULES,
         planner.getEmptyTraitSet()
@@ -333,13 +422,13 @@ public class DruidPlanner implements Closeable
       return planExplanation(druidRel, explain, true);
     } else {
       final Supplier<Sequence<Object[]>> resultsSupplier = () -> {
+
         // sanity check
         final Set<ResourceAction> readResourceActions =
             plannerContext.getResourceActions()
                           .stream()
                           .filter(action -> action.getAction() == Action.READ)
                           .collect(Collectors.toSet());
-
         Preconditions.checkState(
             readResourceActions.isEmpty() == 
druidRel.getDataSourceNames().isEmpty()
             // The resources found in the plannerContext can be less than the 
datasources in
@@ -357,11 +446,16 @@ public class DruidPlanner implements Closeable
   }
 
   /**
-   * Construct a {@link PlannerResult} for a fall-back 'bindable' rel, for 
things that are not directly translatable
-   * to native Druid queries such as system tables and just a general purpose 
(but definitely not optimized) fall-back.
+   * Construct a {@link PlannerResult} for a fall-back 'bindable' rel, for
+   * things that are not directly translatable to native Druid queries such
+   * as system tables and just a general purpose (but definitely not optimized)
+   * fall-back.
    *
-   * See {@link #planWithDruidConvention} which will handle things which are 
directly translatable
-   * to native Druid queries.
+   * See {@link #planWithDruidConvention} which will handle things which are
+   * directly translatable to native Druid queries.
+   *
+   * The bindable path handles parameter substitution of any values not
+   * bound by the earlier steps.
    */
   private PlannerResult planWithBindableConvention(
       final RelRoot root,
@@ -479,7 +573,7 @@ public class DruidPlanner implements Closeable
   /**
    * This method doesn't utilize the Calcite's internal {@link 
RelOptUtil#dumpPlan} since that tends to be verbose
    * and not indicative of the native Druid Queries which will get executed
-   * This method assumes that the Planner has converted the RelNodes to 
DruidRels, and thereby we can implictly cast it
+   * This method assumes that the Planner has converted the RelNodes to 
DruidRels, and thereby we can implicitly cast it
    *
    * @param rel Instance of the root {@link DruidRel} which is formed by 
running the planner transformations on it
    * @return A string representing an array of native queries that correspond 
to the given SQL query, in JSON format
@@ -605,31 +699,6 @@ public class DruidPlanner implements Closeable
     return new RelRoot(newRootRel, root.validatedRowType, root.kind, 
root.fields, root.collation);
   }
 
-  /**
-   * Uses {@link SqlParameterizerShuttle} to rewrite {@link SqlNode} to swap 
out any
-   * {@link SqlDynamicParam} early for their {@link SqlLiteral}
-   * replacement
-   */
-  private SqlNode rewriteDynamicParameters(SqlNode parsed)
-  {
-    if (!plannerContext.getParameters().isEmpty()) {
-      SqlParameterizerShuttle sshuttle = new 
SqlParameterizerShuttle(plannerContext);
-      return parsed.accept(sshuttle);
-    }
-    return parsed;
-  }
-
-  /**
-   * Fall-back dynamic parameter substitution using {@link 
RelParameterizerShuttle} in the event that
-   * {@link #rewriteDynamicParameters(SqlNode)} was unable to successfully 
substitute all parameter values, and will
-   * cause a failure if any dynamic a parameters are not bound.
-   */
-  private RelNode rewriteRelDynamicParameters(RelNode rootRel)
-  {
-    RelParameterizerShuttle parameterizer = new 
RelParameterizerShuttle(plannerContext);
-    return rootRel.accept(parameterizer);
-  }
-
   private QueryMaker buildQueryMaker(
       final RelRoot rootQueryRel,
       @Nullable final SqlInsert insertOrReplace
@@ -708,10 +777,7 @@ public class DruidPlanner implements Closeable
     // Check that there are no unnamed columns in the insert.
     for (Pair<Integer, String> field : rootQueryRel.fields) {
       if (UNNAMED_COLUMN_PATTERN.matcher(field.right).matches()) {
-        throw new ValidationException("Cannot ingest expressions that do not 
have an alias "
-                                      + "or columns with names like 
EXPR$[digit]."
-                                      + "E.g. if you are ingesting 
\"func(X)\", then you can rewrite it as "
-                                      + "\"func(X) as myColumn\"");
+        throw new ValidationException(UNNAMED_INGESTION_COLUMN_ERROR);
       }
     }
   }
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java
similarity index 54%
rename from 
sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java
rename to 
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java
index 206bd28436..a4b9a9ecd8 100644
--- 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java
+++ 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java
@@ -19,30 +19,24 @@
 
 package org.apache.druid.sql.calcite.planner;
 
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.server.security.Resource;
-import org.apache.druid.server.security.ResourceAction;
-
-import java.util.Set;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.prepare.BaseDruidSqlValidator;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 
 /**
- * If an SQL query can be validated by {@link DruidPlanner}, the resulting 
artifact is the set of {@link Resource}
- * corresponding to the datasources and views which an authenticated request 
must be authorized for to process the
- * query.
+ * Druid extended SQL validator. (At present, it doesn't actually
+ * have any extensions yet, but it will soon.)
  */
-public class ValidationResult
+class DruidSqlValidator extends BaseDruidSqlValidator
 {
-  private final Set<ResourceAction> resourceActions;
-
-  public ValidationResult(
-      final Set<ResourceAction> resourceActions
-  )
-  {
-    this.resourceActions = ImmutableSet.copyOf(resourceActions);
-  }
-
-  public Set<ResourceAction> getResourceActions()
+  protected DruidSqlValidator(
+      SqlOperatorTable opTab,
+      CalciteCatalogReader catalogReader,
+      JavaTypeFactory typeFactory,
+      SqlConformance conformance)
   {
-    return resourceActions;
+    super(opTab, catalogReader, typeFactory, conformance);
   }
 }
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java
index 41b5a6340c..1aae09c4ae 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java
@@ -88,7 +88,7 @@ public class PlannerContext
   // result of authentication, providing identity to authorize set of 
resources produced by validation
   private AuthenticationResult authenticationResult;
   // set of datasources and views which must be authorized, initialized to 
null so we can detect if it has been set.
-  private Set<ResourceAction> resourceActions = null;
+  private Set<ResourceAction> resourceActions;
   // result of authorizing set of resources against authentication identity
   private Access authorizationResult;
   // error messages encountered while planning the query
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java
index 12ed756190..7b6f507947 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java
@@ -113,15 +113,7 @@ public class PlannerFactory
         queryContext
     );
 
-    return createPlannerWithContext(context);
-  }
-
-  /**
-   * Create a new Druid query planner, re-using a previous {@link 
PlannerContext}
-   */
-  public DruidPlanner createPlannerWithContext(final PlannerContext 
plannerContext)
-  {
-    return new DruidPlanner(buildFrameworkConfig(plannerContext), 
plannerContext, queryMakerFactory);
+    return new DruidPlanner(buildFrameworkConfig(context), context, 
queryMakerFactory);
   }
 
   /**
@@ -135,12 +127,12 @@ public class PlannerFactory
     thePlanner.getPlannerContext()
               
.setAuthenticationResult(NoopEscalator.getInstance().createEscalatedAuthenticationResult());
     try {
-      thePlanner.validate(false);
+      thePlanner.validate();
     }
     catch (SqlParseException | ValidationException e) {
       throw new RuntimeException(e);
     }
-    thePlanner.getPlannerContext().setAuthorizationResult(Access.OK);
+    thePlanner.authorize(ra -> Access.OK, false);
     return thePlanner;
   }
 
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java
 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java
index 8029a89d97..cd9b1c2d21 100644
--- 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java
+++ 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java
@@ -203,7 +203,7 @@ public class RelParameterizerShuttle implements RelShuttle
         if (param == null) {
           throw new SqlPlanningException(
               PlanningError.VALIDATION_ERROR,
-              StringUtils.format("Parameter at position[%s] is not bound", 
dynamicParam.getIndex())
+              StringUtils.format("Parameter at position [%s] is not bound", 
dynamicParam.getIndex())
           );
         }
         if (param.value == null) {
@@ -218,7 +218,7 @@ public class RelParameterizerShuttle implements RelShuttle
       } else {
         throw new SqlPlanningException(
             PlanningError.VALIDATION_ERROR,
-            StringUtils.format("Parameter at position[%s] is not bound", 
dynamicParam.getIndex())
+            StringUtils.format("Parameter at position [%s] is not bound", 
dynamicParam.getIndex())
         );
       }
     }
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java
 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java
index f73ff1de58..456415ee67 100644
--- 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java
+++ 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java
@@ -29,13 +29,26 @@ import org.apache.calcite.util.TimestampString;
 import org.apache.druid.java.util.common.IAE;
 
 /**
- * Replaces all {@link SqlDynamicParam} encountered in an {@link SqlNode} tree 
with a {@link SqlLiteral} if a value
- * binding exists for the parameter, if possible. This is used in tandem with 
{@link RelParameterizerShuttle}.
+ * Replaces all {@link SqlDynamicParam} encountered in an {@link SqlNode} tree
+ * with a {@link SqlLiteral} if a value binding exists for the parameter, if
+ * possible. This is used in tandem with {@link RelParameterizerShuttle}.
  *
- * It is preferable that all parameters are placed here to pick up as many 
optimizations as possible, but the facilities
- * to convert jdbc types to {@link SqlLiteral} are a bit less rich here than 
exist for converting a
- * {@link org.apache.calcite.rex.RexDynamicParam} to {@link 
org.apache.calcite.rex.RexLiteral}, which is why
- * {@link SqlParameterizerShuttle} and {@link RelParameterizerShuttle} both 
exist.
+ * It is preferable that all parameters are placed here to pick up as many
+ * optimizations as possible, but the facilities to convert jdbc types to
+ * {@link SqlLiteral} are a bit less rich here than exist for converting a
+ * {@link org.apache.calcite.rex.RexDynamicParam} to
+ * {@link org.apache.calcite.rex.RexLiteral}, which is why
+ * {@link SqlParameterizerShuttle} and {@link RelParameterizerShuttle}
+ * both exist.
+ *
+ * As it turns out, most parameters will be replaced in this shuttle.
+ * The one exception are DATE types expressed as integers. For reasons
+ * known only to Calcite, the {@code RexBuilder.clean()} method, used by
+ * {@code RelParameterizerShuttle}, handles integer values for dates,
+ * but the {@code SqlTypeName.createLiteral()} method used here does
+ * not. As a result, DATE parameters will be left as parameters to be
+ * filled in later. Fortunately, this does not affect optimizations as
+ * there are no rules that optimize based on the value of a DATE.
  */
 public class SqlParameterizerShuttle extends SqlShuttle
 {
@@ -49,35 +62,37 @@ public class SqlParameterizerShuttle extends SqlShuttle
   @Override
   public SqlNode visit(SqlDynamicParam param)
   {
-    try {
-      if (plannerContext.getParameters().size() > param.getIndex()) {
-        TypedValue paramBinding = 
plannerContext.getParameters().get(param.getIndex());
-        if (paramBinding == null) {
-          throw new IAE("Parameter at position[%s] is not bound", 
param.getIndex());
-        }
-        if (paramBinding.value == null) {
-          return SqlLiteral.createNull(param.getParserPosition());
-        }
-        SqlTypeName typeName = 
SqlTypeName.getNameForJdbcType(paramBinding.type.typeId);
-        if (SqlTypeName.APPROX_TYPES.contains(typeName)) {
-          return SqlLiteral.createApproxNumeric(paramBinding.value.toString(), 
param.getParserPosition());
-        }
-        if (SqlTypeName.TIMESTAMP.equals(typeName) && paramBinding.value 
instanceof Long) {
-          return SqlLiteral.createTimestamp(
-              TimestampString.fromMillisSinceEpoch((Long) paramBinding.value),
-              0,
-              param.getParserPosition()
-          );
-        }
+    if (plannerContext.getParameters().size() <= param.getIndex()) {
+      throw new IAE("Parameter at position [%s] is not bound", 
param.getIndex());
+    }
+    TypedValue paramBinding = 
plannerContext.getParameters().get(param.getIndex());
+    if (paramBinding == null) {
+      throw new IAE("Parameter at position [%s] is not bound", 
param.getIndex());
+    }
+    if (paramBinding.value == null) {
+      return SqlLiteral.createNull(param.getParserPosition());
+    }
+    SqlTypeName typeName = 
SqlTypeName.getNameForJdbcType(paramBinding.type.typeId);
+    if (SqlTypeName.APPROX_TYPES.contains(typeName)) {
+      return SqlLiteral.createApproxNumeric(paramBinding.value.toString(), 
param.getParserPosition());
+    }
+    if (SqlTypeName.TIMESTAMP.equals(typeName) && paramBinding.value 
instanceof Long) {
+      return SqlLiteral.createTimestamp(
+          TimestampString.fromMillisSinceEpoch((Long) paramBinding.value),
+          0,
+          param.getParserPosition()
+      );
+    }
 
-        return typeName.createLiteral(paramBinding.value, 
param.getParserPosition());
-      } else {
-        throw new IAE("Parameter at position[%s] is not bound", 
param.getIndex());
-      }
+    try {
+      // This throws ClassCastException for a DATE parameter given as
+      // an Integer. The parameter is left in place and is replaced
+      // properly later by RelParameterizerShuttle.
+      return typeName.createLiteral(paramBinding.value, 
param.getParserPosition());
     }
     catch (ClassCastException ignored) {
       // suppress
+      return param;
     }
-    return param;
   }
 }
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java
index 1d0f70493a..b4739cb9a4 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java
@@ -58,7 +58,8 @@ public class DruidViewMacro implements TableMacro
   {
     final RelDataType rowType;
     try (final DruidPlanner planner = plannerFactory.createPlanner(viewSql, 
new QueryContext())) {
-      rowType = planner.plan().rowType();
+      planner.validate();
+      rowType = planner.prepare().getRowType();
     }
     catch (Exception e) {
       throw new RuntimeException(e);
diff --git a/sql/src/test/java/org/apache/druid/sql/SqlLifecycleTest.java 
b/sql/src/test/java/org/apache/druid/sql/SqlLifecycleTest.java
index 64f2e2dcfe..b30772dc45 100644
--- a/sql/src/test/java/org/apache/druid/sql/SqlLifecycleTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/SqlLifecycleTest.java
@@ -36,13 +36,16 @@ import org.apache.druid.query.QueryContexts;
 import org.apache.druid.server.QueryStackTests;
 import org.apache.druid.server.log.RequestLogger;
 import org.apache.druid.server.security.Access;
+import org.apache.druid.server.security.Action;
 import org.apache.druid.server.security.AuthConfig;
+import org.apache.druid.server.security.Resource;
+import org.apache.druid.server.security.ResourceAction;
+import org.apache.druid.server.security.ResourceType;
 import org.apache.druid.sql.calcite.planner.DruidPlanner;
 import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.planner.PlannerFactory;
 import org.apache.druid.sql.calcite.planner.PlannerResult;
 import org.apache.druid.sql.calcite.planner.PrepareResult;
-import org.apache.druid.sql.calcite.planner.ValidationResult;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.apache.druid.sql.http.SqlParameter;
 import org.easymock.EasyMock;
@@ -51,10 +54,12 @@ import org.junit.Before;
 import org.junit.Test;
 
 import javax.servlet.http.HttpServletRequest;
+
 import java.io.IOException;
-import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 public class SqlLifecycleTest
@@ -137,18 +142,19 @@ public class SqlLifecycleTest
     // test authorization
     DruidPlanner mockPlanner = EasyMock.createMock(DruidPlanner.class);
     PlannerContext mockPlannerContext = 
EasyMock.createMock(PlannerContext.class);
-    ValidationResult validationResult = new 
ValidationResult(Collections.emptySet());
     EasyMock.expect(plannerFactory.createPlanner(EasyMock.eq(sql), 
EasyMock.anyObject())).andReturn(mockPlanner).once();
     
EasyMock.expect(mockPlanner.getPlannerContext()).andReturn(mockPlannerContext).once();
     
mockPlannerContext.setAuthenticationResult(CalciteTests.REGULAR_USER_AUTH_RESULT);
     EasyMock.expectLastCall();
     mockPlannerContext.setParameters(parameters);
     EasyMock.expectLastCall();
-    
EasyMock.expect(plannerFactory.getAuthorizerMapper()).andReturn(CalciteTests.TEST_AUTHORIZER_MAPPER).once();
-    mockPlannerContext.setAuthorizationResult(Access.OK);
+    mockPlanner.validate();
     EasyMock.expectLastCall();
-    
EasyMock.expect(mockPlanner.validate(false)).andReturn(validationResult).once();
-    mockPlanner.close();
+    Set<ResourceAction> mockActions = new HashSet<>();
+    mockActions.add(new ResourceAction(new Resource("dummy", 
ResourceType.DATASOURCE), Action.READ));
+    
EasyMock.expect(mockPlanner.resourceActions(EasyMock.eq(false))).andReturn(mockActions).once();
+    EasyMock.expectLastCall();
+    EasyMock.expect(mockPlanner.authorize(EasyMock.anyObject(), 
EasyMock.eq(false))).andReturn(Access.OK).once();
     EasyMock.expectLastCall();
 
     EasyMock.replay(plannerFactory, serviceEmitter, requestLogger, 
mockPlanner, mockPlannerContext);
@@ -160,9 +166,7 @@ public class SqlLifecycleTest
 
     // test prepare
     PrepareResult mockPrepareResult = EasyMock.createMock(PrepareResult.class);
-    
EasyMock.expect(plannerFactory.createPlannerWithContext(EasyMock.eq(mockPlannerContext))).andReturn(mockPlanner).once();
     EasyMock.expect(mockPlanner.prepare()).andReturn(mockPrepareResult).once();
-    mockPlanner.close();
     EasyMock.expectLastCall();
     EasyMock.replay(plannerFactory, serviceEmitter, requestLogger, 
mockPlanner, mockPlannerContext, mockPrepareResult);
     lifecycle.prepare();
@@ -173,7 +177,6 @@ public class SqlLifecycleTest
 
     // test plan
     PlannerResult mockPlanResult = EasyMock.createMock(PlannerResult.class);
-    
EasyMock.expect(plannerFactory.createPlannerWithContext(EasyMock.eq(mockPlannerContext))).andReturn(mockPlanner).once();
     EasyMock.expect(mockPlanner.plan()).andReturn(mockPlanResult).once();
     mockPlanner.close();
     EasyMock.expectLastCall();
@@ -216,8 +219,8 @@ public class SqlLifecycleTest
   public void testStateTransitionHttpRequest()
       throws ValidationException, SqlParseException, RelConversionException, 
IOException
   {
-    // this test is a duplicate of testStateTransition except with a slight 
variation of how validate and authorize
-    // is run
+    // this test is a duplicate of testStateTransition except with a slight
+    // variation of how validate and authorize is run
     SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
     final String sql = "select 1 + ?";
     Assert.assertEquals(SqlLifecycle.State.NEW, lifecycle.getState());
@@ -233,25 +236,25 @@ public class SqlLifecycleTest
     // test authorization
     DruidPlanner mockPlanner = EasyMock.createMock(DruidPlanner.class);
     PlannerContext mockPlannerContext = 
EasyMock.createMock(PlannerContext.class);
-    ValidationResult validationResult = new 
ValidationResult(Collections.emptySet());
     EasyMock.expect(plannerFactory.createPlanner(EasyMock.eq(sql), 
EasyMock.anyObject())).andReturn(mockPlanner).once();
     
EasyMock.expect(mockPlanner.getPlannerContext()).andReturn(mockPlannerContext).once();
     
mockPlannerContext.setAuthenticationResult(CalciteTests.REGULAR_USER_AUTH_RESULT);
     EasyMock.expectLastCall();
     mockPlannerContext.setParameters(parameters);
     EasyMock.expectLastCall();
-    
EasyMock.expect(plannerFactory.getAuthorizerMapper()).andReturn(CalciteTests.TEST_AUTHORIZER_MAPPER).once();
-    mockPlannerContext.setAuthorizationResult(Access.OK);
+    mockPlanner.validate();
     EasyMock.expectLastCall();
-    
EasyMock.expect(mockPlanner.validate(false)).andReturn(validationResult).once();
-    mockPlanner.close();
+    Set<ResourceAction> mockActions = new HashSet<>();
+    mockActions.add(new ResourceAction(new Resource("dummy", 
ResourceType.DATASOURCE), Action.READ));
+    
EasyMock.expect(mockPlanner.resourceActions(EasyMock.eq(false))).andReturn(mockActions).once();
+    EasyMock.expectLastCall();
+    EasyMock.expect(mockPlanner.authorize(EasyMock.anyObject(), 
EasyMock.eq(false))).andReturn(Access.OK).once();
     EasyMock.expectLastCall();
 
+    // Note: can't check the request usage with mocks: the code is run
+    // in a function which the mock doesn't actually call.
     HttpServletRequest request = EasyMock.createMock(HttpServletRequest.class);
-    
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(CalciteTests.REGULAR_USER_AUTH_RESULT).times(2);
-    
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).once();
-    
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once();
-    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(CalciteTests.REGULAR_USER_AUTH_RESULT).once();
     EasyMock.expectLastCall();
     EasyMock.replay(plannerFactory, serviceEmitter, requestLogger, 
mockPlanner, mockPlannerContext, request);
 
@@ -262,9 +265,7 @@ public class SqlLifecycleTest
 
     // test prepare
     PrepareResult mockPrepareResult = EasyMock.createMock(PrepareResult.class);
-    
EasyMock.expect(plannerFactory.createPlannerWithContext(EasyMock.eq(mockPlannerContext))).andReturn(mockPlanner).once();
     EasyMock.expect(mockPlanner.prepare()).andReturn(mockPrepareResult).once();
-    mockPlanner.close();
     EasyMock.expectLastCall();
     EasyMock.replay(plannerFactory, serviceEmitter, requestLogger, 
mockPlanner, mockPlannerContext, mockPrepareResult);
     lifecycle.prepare();
@@ -275,7 +276,6 @@ public class SqlLifecycleTest
 
     // test plan
     PlannerResult mockPlanResult = EasyMock.createMock(PlannerResult.class);
-    
EasyMock.expect(plannerFactory.createPlannerWithContext(EasyMock.eq(mockPlannerContext))).andReturn(mockPlanner).once();
     EasyMock.expect(mockPlanner.plan()).andReturn(mockPlanResult).once();
     mockPlanner.close();
     EasyMock.expectLastCall();
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index cf542d81db..8faee4fdf6 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -894,7 +894,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
   )
   {
     if (expectedQueries != null) {
-      final List<Query> recordedQueries = queryLogHook.getRecordedQueries();
+      final List<Query<?>> recordedQueries = queryLogHook.getRecordedQueries();
 
       Assert.assertEquals(
           StringUtils.format("query count: %s", sql),
@@ -989,7 +989,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
 
     SqlLifecycle lifecycle = lifecycleFactory.factorize();
     lifecycle.initialize(sql, new QueryContext(contexts));
-    return 
lifecycle.runAnalyzeResources(authenticationResult).getResourceActions();
+    return lifecycle.runAnalyzeResources(authenticationResult);
   }
 
   public SqlLifecycleFactory getSqlLifecycleFactory(
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java
index 4f56eb9fa1..1470b10291 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java
@@ -41,6 +41,7 @@ import 
org.apache.druid.sql.calcite.external.ExternalDataSource;
 import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
 import org.apache.druid.sql.calcite.filtration.Filtration;
 import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
 import org.apache.druid.sql.calcite.planner.PlannerConfig;
 import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.util.CalciteTests;
@@ -704,10 +705,7 @@ public class CalciteInsertDmlTest extends 
CalciteIngestionDmlTest
         .sql("INSERT INTO t SELECT dim1, dim2 || '-lol' FROM foo PARTITIONED 
BY ALL")
         .expectValidationError(
             SqlPlanningException.class,
-            "Cannot ingest expressions that do not have an alias "
-            + "or columns with names like EXPR$[digit]."
-            + "E.g. if you are ingesting \"func(X)\", then you can rewrite it 
as "
-            + "\"func(X) as myColumn\""
+            DruidPlanner.UNNAMED_INGESTION_COLUMN_ERROR
         )
         .verify();
   }
@@ -719,10 +717,7 @@ public class CalciteInsertDmlTest extends 
CalciteIngestionDmlTest
         .sql("INSERT INTO t SELECT __time, dim1 AS EXPR$0 FROM foo PARTITIONED 
BY ALL")
         .expectValidationError(
             SqlPlanningException.class,
-            "Cannot ingest expressions that do not have an alias "
-            + "or columns with names like EXPR$[digit]."
-            + "E.g. if you are ingesting \"func(X)\", then you can rewrite it 
as "
-            + "\"func(X) as myColumn\""
+            DruidPlanner.UNNAMED_INGESTION_COLUMN_ERROR
         )
         .verify();
   }
@@ -736,10 +731,7 @@ public class CalciteInsertDmlTest extends 
CalciteIngestionDmlTest
              + "(SELECT __time, LOWER(dim1) FROM foo) PARTITIONED BY ALL TIME")
         .expectValidationError(
             SqlPlanningException.class,
-            "Cannot ingest expressions that do not have an alias "
-            + "or columns with names like EXPR$[digit]."
-            + "E.g. if you are ingesting \"func(X)\", then you can rewrite it 
as "
-            + "\"func(X) as myColumn\""
+            DruidPlanner.UNNAMED_INGESTION_COLUMN_ERROR
         )
         .verify();
   }
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
index 2750032387..179e690c1b 100644
--- 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
+++ 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java
@@ -47,10 +47,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * This class has copied a subset of the tests in {@link CalciteQueryTest} and 
replaced various parts of queries with
- * dynamic parameters. It is NOT important that this file remains in sync with 
{@link CalciteQueryTest}, the tests
- * were merely chosen to produce a selection of parameter types and positions 
within query expressions and have been
- * renamed to reflect this
+ * This class has copied a subset of the tests in {@link CalciteQueryTest} and
+ * replaced various parts of queries with dynamic parameters. It is NOT
+ * important that this file remains in sync with {@link CalciteQueryTest}, the
+ * tests were merely chosen to produce a selection of parameter types and
+ * positions within query expressions and have been renamed to reflect this
  */
 public class CalciteParameterQueryTest extends BaseCalciteQueryTest
 {
@@ -577,7 +578,7 @@ public class CalciteParameterQueryTest extends 
BaseCalciteQueryTest
   public void testMissingParameter() throws Exception
   {
     expectedException.expect(SqlPlanningException.class);
-    expectedException.expectMessage("Parameter at position[0] is not bound");
+    expectedException.expectMessage("Parameter at position [0] is not bound");
     testQuery(
         "SELECT COUNT(*)\n"
         + "FROM druid.numfoo\n"
@@ -592,7 +593,7 @@ public class CalciteParameterQueryTest extends 
BaseCalciteQueryTest
   public void testPartiallyMissingParameter() throws Exception
   {
     expectedException.expect(SqlPlanningException.class);
-    expectedException.expectMessage("Parameter at position[1] is not bound");
+    expectedException.expectMessage("Parameter at position [1] is not bound");
     testQuery(
         "SELECT COUNT(*)\n"
         + "FROM druid.numfoo\n"
@@ -610,7 +611,7 @@ public class CalciteParameterQueryTest extends 
BaseCalciteQueryTest
     params.add(null);
     params.add(new SqlParameter(SqlType.INTEGER, 1));
     expectedException.expect(SqlPlanningException.class);
-    expectedException.expectMessage("Parameter at position[0] is not bound");
+    expectedException.expectMessage("Parameter at position [0] is not bound");
     testQuery(
         "SELECT 1 + ?, dim1 FROM foo LIMIT ?",
         ImmutableList.of(),
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
 
b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
index 1aca4f1adc..ec088ce998 100644
--- 
a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
+++ 
b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
@@ -22,8 +22,6 @@ package org.apache.druid.sql.calcite;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -61,6 +59,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -175,13 +174,13 @@ public class SqlVectorizedExpressionSanityTest extends 
InitializedNullHandlingTe
   }
 
   @Test
-  public void testQuery() throws SqlParseException, RelConversionException, 
ValidationException
+  public void testQuery() throws ValidationException
   {
     sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query);
   }
 
   public static void sanityTestVectorizedSqlQueries(PlannerFactory 
plannerFactory, String query)
-      throws ValidationException, RelConversionException, SqlParseException
+      throws ValidationException
   {
     final Map<String, Object> vector = ImmutableMap.of(
         QueryContexts.VECTORIZE_KEY, "force",
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java
index c68dbc89b7..4f384e5747 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java
@@ -41,7 +41,7 @@ public class QueryLogHook implements TestRule
   private static final Logger log = new Logger(QueryLogHook.class);
 
   private final ObjectMapper objectMapper;
-  private final List<Query> recordedQueries = Lists.newCopyOnWriteArrayList();
+  private final List<Query<?>> recordedQueries = 
Lists.newCopyOnWriteArrayList();
 
   public QueryLogHook(final ObjectMapper objectMapper)
   {
@@ -63,7 +63,7 @@ public class QueryLogHook implements TestRule
     recordedQueries.clear();
   }
 
-  public List<Query> getRecordedQueries()
+  public List<Query<?>> getRecordedQueries()
   {
     return ImmutableList.copyOf(recordedQueries);
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to