You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/01/27 21:31:33 UTC

[GitHub] [druid] paul-rogers commented on a change in pull request #12163: Add syntax support for PARTITION BY/CLUSTER BY in INSERT queries

paul-rogers commented on a change in pull request #12163:
URL: https://github.com/apache/druid/pull/12163#discussion_r793999444



##########
File path: sql/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,443 @@
+# 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.
+
+# This file is an FMPP (http://fmpp.sourceforge.net/) configuration file to
+# allow clients to extend Calcite's SQL parser to support application specific
+# SQL statements, literals or data types.
+#
+# Calcite's parser grammar file (Parser.jj) is written in javacc
+# (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
+# to allow clients to:
+#   1. have custom parser implementation class and package name.
+#   2. insert new parser method implementations written in javacc to parse
+#      custom:
+#      a) SQL statements.
+#      b) literals.
+#      c) data types.
+#   3. add new keywords to support custom SQL constructs added as part of (2).
+#   4. add import statements needed by inserted custom parser implementations.
+#
+# Parser template file (Parser.jj) along with this file are packaged as
+# part of the calcite-core-<version>.jar under "codegen" directory.
+
+# This file is directly copied from calite-core-1.21.0.jar/codegen/config.fmpp, and then modified slightly.
+# While not a necessary requirement, it would be ideal if it is kept in line with calcite-core's version. In the newer
+# Calcite versions, there is a default_config.fmpp which will free us from maintaining this file.
+#
+# Following clauses are modified in the file:
+#   1. data.parser.package & data.parser.class
+#   2. data.parser.imports
+#   3. data.parser.keywords (Added "CLUSTER")
+#   4. data.parser.statementParserMethods
+#   5. data.parser.implementationFiles
+data: {

Review comment:
       This is the right approach with Calcite. For reference, here's how its done in Apache Drill: https://github.com/apache/drill/blob/9cf896a69e559cad8d1cd30d98a0555973da1ada/exec/java-exec/src/main/codegen/data/Parser.tdd

##########
File path: sql/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,443 @@
+# 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.
+
+# This file is an FMPP (http://fmpp.sourceforge.net/) configuration file to
+# allow clients to extend Calcite's SQL parser to support application specific
+# SQL statements, literals or data types.
+#
+# Calcite's parser grammar file (Parser.jj) is written in javacc
+# (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
+# to allow clients to:
+#   1. have custom parser implementation class and package name.
+#   2. insert new parser method implementations written in javacc to parse
+#      custom:
+#      a) SQL statements.
+#      b) literals.
+#      c) data types.
+#   3. add new keywords to support custom SQL constructs added as part of (2).
+#   4. add import statements needed by inserted custom parser implementations.
+#
+# Parser template file (Parser.jj) along with this file are packaged as
+# part of the calcite-core-<version>.jar under "codegen" directory.
+
+# This file is directly copied from calite-core-1.21.0.jar/codegen/config.fmpp, and then modified slightly.
+# While not a necessary requirement, it would be ideal if it is kept in line with calcite-core's version. In the newer
+# Calcite versions, there is a default_config.fmpp which will free us from maintaining this file.
+#
+# Following clauses are modified in the file:
+#   1. data.parser.package & data.parser.class
+#   2. data.parser.imports
+#   3. data.parser.keywords (Added "CLUSTER")
+#   4. data.parser.statementParserMethods
+#   5. data.parser.implementationFiles
+data: {
+  parser: {
+    # Generated parser implementation package and class name.
+    package: "org.apache.druid.sql.calcite.parser",
+    class: "DruidSqlParserImpl",
+
+    # List of additional classes and packages to import.
+    # Example. "org.apache.calcite.sql.*", "java.util.List".
+    imports: [
+      "org.apache.calcite.sql.SqlNode"
+      "org.apache.calcite.sql.SqlInsert"
+      "org.apache.druid.sql.calcite.parser.DruidSqlInsert"
+    ]
+
+    # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved
+    # keyword add it to 'nonReservedKeywords' section.
+    keywords: [
+      "CLUSTER"
+    ]
+
+    # List of keywords from "keywords" section that are not reserved.
+    nonReservedKeywords: [
+        "A"
+        "ABSENT"
+        "ABSOLUTE"
+        "ACTION"
+        "ADA"
+        "ADD"
+        "ADMIN"
+        "AFTER"
+        "ALWAYS"
+        "APPLY"
+        "ASC"
+        "ASSERTION"
+        "ASSIGNMENT"
+        "ATTRIBUTE"
+        "ATTRIBUTES"
+        "BEFORE"
+        "BERNOULLI"
+        "BREADTH"
+        "C"
+        "CASCADE"
+        "CATALOG"
+        "CATALOG_NAME"
+        "CENTURY"
+        "CHAIN"
+        "CHARACTER_SET_CATALOG"
+        "CHARACTER_SET_NAME"
+        "CHARACTER_SET_SCHEMA"
+        "CHARACTERISTICS"
+        "CHARACTERS"
+        "CLASS_ORIGIN"
+        "COBOL"
+        "COLLATION"
+        "COLLATION_CATALOG"
+        "COLLATION_NAME"
+        "COLLATION_SCHEMA"
+        "COLUMN_NAME"
+        "COMMAND_FUNCTION"
+        "COMMAND_FUNCTION_CODE"
+        "COMMITTED"
+        "CONDITION_NUMBER"
+        "CONDITIONAL"
+        "CONNECTION"
+        "CONNECTION_NAME"
+        "CONSTRAINT_CATALOG"
+        "CONSTRAINT_NAME"
+        "CONSTRAINT_SCHEMA"
+        "CONSTRAINTS"
+        "CONSTRUCTOR"
+        "CONTINUE"
+        "CURSOR_NAME"
+        "DATA"
+        "DATABASE"
+        "DATETIME_INTERVAL_CODE"
+        "DATETIME_INTERVAL_PRECISION"
+        "DECADE"
+        "DEFAULTS"
+        "DEFERRABLE"
+        "DEFERRED"
+        "DEFINED"
+        "DEFINER"
+        "DEGREE"
+        "DEPTH"
+        "DERIVED"
+        "DESC"
+        "DESCRIPTION"
+        "DESCRIPTOR"
+        "DIAGNOSTICS"
+        "DISPATCH"
+        "DOMAIN"
+        "DOW"
+        "DOY"
+        "DYNAMIC_FUNCTION"
+        "DYNAMIC_FUNCTION_CODE"
+        "ENCODING"
+        "EPOCH"
+        "ERROR"
+        "EXCEPTION"
+        "EXCLUDE"
+        "EXCLUDING"
+        "FINAL"
+        "FIRST"
+        "FOLLOWING"
+        "FORMAT"
+        "FORTRAN"
+        "FOUND"
+        "FRAC_SECOND"
+        "G"
+        "GENERAL"
+        "GENERATED"
+        "GEOMETRY"
+        "GO"
+        "GOTO"
+        "GRANTED"
+        "HIERARCHY"
+        "IGNORE"
+        "IMMEDIATE"
+        "IMMEDIATELY"
+        "IMPLEMENTATION"
+        "INCLUDING"
+        "INCREMENT"
+        "INITIALLY"
+        "INPUT"
+        "INSTANCE"
+        "INSTANTIABLE"
+        "INVOKER"
+        "ISODOW"
+        "ISOYEAR"
+        "ISOLATION"
+        "JAVA"
+        "JSON"
+        "K"
+        "KEY"
+        "KEY_MEMBER"
+        "KEY_TYPE"
+        "LABEL"
+        "LAST"
+        "LENGTH"
+        "LEVEL"
+        "LIBRARY"
+        "LOCATOR"
+        "M"
+        "MAP"
+        "MATCHED"
+        "MAXVALUE"
+        "MICROSECOND"
+        "MESSAGE_LENGTH"
+        "MESSAGE_OCTET_LENGTH"
+        "MESSAGE_TEXT"
+        "MILLISECOND"
+        "MILLENNIUM"
+        "MINVALUE"
+        "MORE_"
+        "MUMPS"
+        "NAME"
+        "NAMES"
+        "NANOSECOND"
+        "NESTING"
+        "NORMALIZED"
+        "NULLABLE"
+        "NULLS"
+        "NUMBER"
+        "OBJECT"
+        "OCTETS"
+        "OPTION"
+        "OPTIONS"
+        "ORDERING"
+        "ORDINALITY"
+        "OTHERS"
+        "OUTPUT"
+        "OVERRIDING"
+        "PAD"
+        "PARAMETER_MODE"
+        "PARAMETER_NAME"
+        "PARAMETER_ORDINAL_POSITION"
+        "PARAMETER_SPECIFIC_CATALOG"
+        "PARAMETER_SPECIFIC_NAME"
+        "PARAMETER_SPECIFIC_SCHEMA"
+        "PARTIAL"
+        "PASCAL"
+        "PASSING"
+        "PASSTHROUGH"
+        "PAST"
+        "PATH"
+        "PLACING"
+        "PLAN"
+        "PLI"
+        "PRECEDING"
+        "PRESERVE"
+        "PRIOR"
+        "PRIVILEGES"
+        "PUBLIC"
+        "QUARTER"
+        "READ"
+        "RELATIVE"
+        "REPEATABLE"
+        "REPLACE"
+        "RESPECT"
+        "RESTART"
+        "RESTRICT"
+        "RETURNED_CARDINALITY"
+        "RETURNED_LENGTH"
+        "RETURNED_OCTET_LENGTH"
+        "RETURNED_SQLSTATE"
+        "RETURNING"
+        "ROLE"
+        "ROUTINE"
+        "ROUTINE_CATALOG"
+        "ROUTINE_NAME"
+        "ROUTINE_SCHEMA"
+        "ROW_COUNT"
+        "SCALAR"
+        "SCALE"
+        "SCHEMA"
+        "SCHEMA_NAME"
+        "SCOPE_CATALOGS"
+        "SCOPE_NAME"
+        "SCOPE_SCHEMA"
+        "SECTION"
+        "SECURITY"
+        "SELF"
+        "SEQUENCE"
+        "SERIALIZABLE"
+        "SERVER"
+        "SERVER_NAME"
+        "SESSION"
+        "SETS"
+        "SIMPLE"
+        "SIZE"
+        "SOURCE"
+        "SPACE"
+        "SPECIFIC_NAME"
+        "SQL_BIGINT"
+        "SQL_BINARY"
+        "SQL_BIT"
+        "SQL_BLOB"
+        "SQL_BOOLEAN"
+        "SQL_CHAR"
+        "SQL_CLOB"
+        "SQL_DATE"
+        "SQL_DECIMAL"
+        "SQL_DOUBLE"
+        "SQL_FLOAT"
+        "SQL_INTEGER"
+        "SQL_INTERVAL_DAY"
+        "SQL_INTERVAL_DAY_TO_HOUR"
+        "SQL_INTERVAL_DAY_TO_MINUTE"
+        "SQL_INTERVAL_DAY_TO_SECOND"
+        "SQL_INTERVAL_HOUR"
+        "SQL_INTERVAL_HOUR_TO_MINUTE"
+        "SQL_INTERVAL_HOUR_TO_SECOND"
+        "SQL_INTERVAL_MINUTE"
+        "SQL_INTERVAL_MINUTE_TO_SECOND"
+        "SQL_INTERVAL_MONTH"
+        "SQL_INTERVAL_SECOND"
+        "SQL_INTERVAL_YEAR"
+        "SQL_INTERVAL_YEAR_TO_MONTH"
+        "SQL_LONGVARBINARY"
+        "SQL_LONGVARNCHAR"
+        "SQL_LONGVARCHAR"
+        "SQL_NCHAR"
+        "SQL_NCLOB"
+        "SQL_NUMERIC"
+        "SQL_NVARCHAR"
+        "SQL_REAL"
+        "SQL_SMALLINT"
+        "SQL_TIME"
+        "SQL_TIMESTAMP"
+        "SQL_TINYINT"
+        "SQL_TSI_DAY"
+        "SQL_TSI_FRAC_SECOND"
+        "SQL_TSI_HOUR"
+        "SQL_TSI_MICROSECOND"
+        "SQL_TSI_MINUTE"
+        "SQL_TSI_MONTH"
+        "SQL_TSI_QUARTER"
+        "SQL_TSI_SECOND"
+        "SQL_TSI_WEEK"
+        "SQL_TSI_YEAR"
+        "SQL_VARBINARY"
+        "SQL_VARCHAR"
+        "STATE"
+        "STATEMENT"
+        "STRUCTURE"
+        "STYLE"
+        "SUBCLASS_ORIGIN"
+        "SUBSTITUTE"
+        "TABLE_NAME"
+        "TEMPORARY"
+        "TIES"
+        "TIMESTAMPADD"
+        "TIMESTAMPDIFF"
+        "TOP_LEVEL_COUNT"
+        "TRANSACTION"
+        "TRANSACTIONS_ACTIVE"
+        "TRANSACTIONS_COMMITTED"
+        "TRANSACTIONS_ROLLED_BACK"
+        "TRANSFORM"
+        "TRANSFORMS"
+        "TRIGGER_CATALOG"
+        "TRIGGER_NAME"
+        "TRIGGER_SCHEMA"
+        "TYPE"
+        "UNBOUNDED"
+        "UNCOMMITTED"
+        "UNCONDITIONAL"
+        "UNDER"
+        "UNNAMED"
+        "USAGE"
+        "USER_DEFINED_TYPE_CATALOG"
+        "USER_DEFINED_TYPE_CODE"
+        "USER_DEFINED_TYPE_NAME"
+        "USER_DEFINED_TYPE_SCHEMA"
+        "UTF8"
+        "UTF16"
+        "UTF32"
+        "VERSION"
+        "VIEW"
+        "WEEK"
+        "WRAPPER"
+        "WORK"
+        "WRITE"
+        "XML"
+        "ZONE"
+    ]
+
+    # List of additional join types. Each is a method with no arguments.
+    # Example: LeftSemiJoin()
+    joinTypes: [
+    ]
+
+    # List of methods for parsing custom SQL statements.
+    # Return type of method implementation should be 'SqlNode'.
+    # Example: SqlShowDatabases(), SqlShowTables().
+    statementParserMethods: [
+      "DruidSqlInsert()"
+    ]
+
+    # List of methods for parsing custom literals.
+    # Return type of method implementation should be "SqlNode".
+    # Example: ParseJsonLiteral().
+    literalParserMethods: [
+    ]
+
+    # List of methods for parsing custom data types.
+    # Return type of method implementation should be "SqlTypeNameSpec".
+    # Example: SqlParseTimeStampZ().
+    dataTypeParserMethods: [
+    ]
+
+    # List of methods for parsing builtin function calls.
+    # Return type of method implementation should be "SqlNode".
+    # Example: DateFunctionCall().
+    builtinFunctionCallMethods: [
+    ]
+
+    # List of methods for parsing extensions to "ALTER <scope>" calls.
+    # Each must accept arguments "(SqlParserPos pos, String scope)".
+    # Example: "SqlUploadJarNode"
+    alterStatementParserMethods: [
+    ]
+
+    # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+    # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+    createStatementParserMethods: [
+    ]
+
+    # List of methods for parsing extensions to "DROP" calls.
+    # Each must accept arguments "(SqlParserPos pos)".
+    dropStatementParserMethods: [
+    ]
+
+    # Binary operators tokens
+    binaryOperatorsTokens: [
+    ]
+
+    # Binary operators initialization
+    extraBinaryExpressions: [
+    ]
+
+    # List of files in @includes directory that have parser method
+    # implementations for parsing custom SQL statements, literals or types
+    # given as part of "statementParserMethods", "literalParserMethods" or
+    # "dataTypeParserMethods".
+    implementationFiles: [
+      "insert.ftl"
+    ]
+
+    includePosixOperators: false
+    includeCompoundIdentifier: true
+    includeBraces: true
+    includeAdditionalDeclarations: false
+  }
+}
+
+freemarkerLinks: {
+  includes: includes/
+}

Review comment:
       Nit: missing newline.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
##########
@@ -790,5 +862,11 @@ public SqlNode getQueryNode()
     {
       return query;
     }
+
+    @Nullable
+    public String getIngestionGranularity()
+    {
+      return ingestionGranularity;

Review comment:
       Would recommend "semanticising" this into something structured, such as a `PERIOD`. That tells the consumer that the value has been validated and that the consumer need not do their own validation.

##########
File path: sql/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,443 @@
+# 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.
+
+# This file is an FMPP (http://fmpp.sourceforge.net/) configuration file to
+# allow clients to extend Calcite's SQL parser to support application specific
+# SQL statements, literals or data types.
+#
+# Calcite's parser grammar file (Parser.jj) is written in javacc
+# (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
+# to allow clients to:
+#   1. have custom parser implementation class and package name.
+#   2. insert new parser method implementations written in javacc to parse
+#      custom:
+#      a) SQL statements.
+#      b) literals.
+#      c) data types.
+#   3. add new keywords to support custom SQL constructs added as part of (2).
+#   4. add import statements needed by inserted custom parser implementations.
+#
+# Parser template file (Parser.jj) along with this file are packaged as
+# part of the calcite-core-<version>.jar under "codegen" directory.
+
+# This file is directly copied from calite-core-1.21.0.jar/codegen/config.fmpp, and then modified slightly.
+# While not a necessary requirement, it would be ideal if it is kept in line with calcite-core's version. In the newer
+# Calcite versions, there is a default_config.fmpp which will free us from maintaining this file.
+#
+# Following clauses are modified in the file:
+#   1. data.parser.package & data.parser.class
+#   2. data.parser.imports
+#   3. data.parser.keywords (Added "CLUSTER")
+#   4. data.parser.statementParserMethods
+#   5. data.parser.implementationFiles
+data: {
+  parser: {
+    # Generated parser implementation package and class name.
+    package: "org.apache.druid.sql.calcite.parser",
+    class: "DruidSqlParserImpl",
+
+    # List of additional classes and packages to import.
+    # Example. "org.apache.calcite.sql.*", "java.util.List".
+    imports: [
+      "org.apache.calcite.sql.SqlNode"
+      "org.apache.calcite.sql.SqlInsert"
+      "org.apache.druid.sql.calcite.parser.DruidSqlInsert"
+    ]
+
+    # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved
+    # keyword add it to 'nonReservedKeywords' section.
+    keywords: [
+      "CLUSTER"

Review comment:
       As noted in a general comment, consider following the Hive syntax: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingvaluesintotablesfromSQL
   
   That is `CLUSTERED` and `PARTITIONED` (verb forms) which are less likely to collide with columns named `cluster` or `partition`.

##########
File path: sql/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,433 @@
+# 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.
+
+# This file is an FMPP (http://fmpp.sourceforge.net/) configuration file to
+# allow clients to extend Calcite's SQL parser to support application specific
+# SQL statements, literals or data types.
+#
+# Calcite's parser grammar file (Parser.jj) is written in javacc
+# (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
+# to allow clients to:
+#   1. have custom parser implementation class and package name.
+#   2. insert new parser method implementations written in javacc to parse
+#      custom:
+#      a) SQL statements.
+#      b) literals.
+#      c) data types.
+#   3. add new keywords to support custom SQL constructs added as part of (2).
+#   4. add import statements needed by inserted custom parser implementations.
+#
+# Parser template file (Parser.jj) along with this file are packaged as
+# part of the calcite-core-<version>.jar under "codegen" directory.
+
+data: {
+  parser: {
+    # Generated parser implementation package and class name.
+    package: "org.apache.druid.sql.calcite.parser",
+    class: "DruidSqlParserImpl",
+
+    # List of additional classes and packages to import.
+    # Example. "org.apache.calcite.sql.*", "java.util.List".
+    imports: [
+      "org.apache.calcite.sql.SqlNode"
+      "org.apache.calcite.sql.SqlInsert"
+      "org.apache.druid.sql.calcite.parser.DruidSqlInsert"
+    ]
+
+    # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved
+    # keyword add it to 'nonReservedKeywords' section.
+    keywords: [

Review comment:
       Having two parsers creates great usability and testing issues: All tests would have to be run in both. Users would need a per-query way to switch parsers, which means the poor user has to understand which is in each one, which is a hard task even for developers. Automated tools that generate Druid SQL would not know what to do.
   
   So, better is to get the solution right within a single parser.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
##########
@@ -765,13 +785,53 @@ static ParsedNodes create(final SqlNode node) throws ValidationException
       if (query.getKind() == SqlKind.INSERT) {
         insert = (SqlInsert) query;
         query = insert.getSource();
+
+        // Processing to be done when the original query has either of the PARTITION BY or CLUSTER BY clause
+        if (insert instanceof DruidSqlInsert) {
+          DruidSqlInsert druidSqlInsert = (DruidSqlInsert) insert;
+
+          ingestionGranularity = druidSqlInsert.getPartitionBy();
+
+          if (druidSqlInsert.getClusterBy() != null) {
+            // If we have a CLUSTER BY clause, extract the information in that CLUSTER BY and create a new SqlOrderBy
+            // node
+            SqlNode offset = null;
+            SqlNode fetch = null;
+            SqlNodeList orderByList = null;
+
+            if (query instanceof SqlOrderBy) {
+              SqlOrderBy sqlOrderBy = (SqlOrderBy) query;
+              // Extract the query present inside the SqlOrderBy (which is free of ORDER BY, OFFSET and FETCH clauses)
+              query = sqlOrderBy.query;
+
+              offset = sqlOrderBy.offset;
+              fetch = sqlOrderBy.fetch;
+              orderByList = sqlOrderBy.orderList;
+              // If the orderList is non-empty (i.e. there existed an ORDER BY clause in the query) and CLUSTER BY clause
+              // is also non-empty, throw an error
+              if (!(orderByList == null || orderByList.equals(SqlNodeList.EMPTY))
+                  && druidSqlInsert.getClusterBy() != null) {
+                throw new ValidationException(
+                    "Cannot have both ORDER BY and CLUSTER BY clauses in the same INSERT query");
+              }
+            }
+            // Creates a new SqlOrderBy query, which may have our CLUSTER BY overwritten
+            query = new SqlOrderBy(

Review comment:
       As a counter argument, it may be good to do as much semantic analysis as possible in the SQL layer. In the fullness of time, this allows us to leverage the SQL logical plan without the conversion to native. (There is some prototyping happening in that area.) The traditional approach is to do all semantic analysis in the SQL layer so that the logical plan is validated prior to the next step (in this case, prior to generating a native query.)
   
   The one caveat, of course, is if the information is (not yet) available to the SQL layer. 

##########
File path: sql/src/main/codegen/includes/insert.ftl
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SqlNode DruidSqlInsert() :
+{
+ SqlNode insertNode;
+ SqlNode partitionBy = null;
+ SqlNodeList clusterBy = null;
+}
+{
+    insertNode = SqlInsert()
+    [
+      <PARTITION> <BY>
+      partitionBy = StringLiteral()

Review comment:
       Also, though I personally cringe at the idea, seems some folks like using the archaic`GROUP BY 3, 2, 5` form in `SELECT` statements. Should we allow ordinals here also? I don't like encouraging their use, but if folks already use them...

##########
File path: sql/src/main/codegen/includes/insert.ftl
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SqlNode DruidSqlInsert() :
+{
+ SqlNode insertNode;
+ SqlNode partitionBy = null;
+ SqlNodeList clusterBy = null;
+}
+{
+    insertNode = SqlInsert()
+    [
+      <PARTITION> <BY>
+      partitionBy = StringLiteral()

Review comment:
       Per Gian (and Hive: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingvaluesintotablesfromSQL) the arguments should be expressions, not literals. Druid may have a very limited set of expressions (such as those Gian showed), but they are still expressions.

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java
##########
@@ -280,6 +283,242 @@ public void testInsertFromExternal()
         .verify();
   }
 
+  @Test
+  public void testInsertWithPartitionBy()
+  {
+    // Test correctness of the query when only PARTITION BY clause is present
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .build();
+
+    Map<String, Object> queryContext = new HashMap<>(DEFAULT_CONTEXT);
+    queryContext.put(QueryContexts.INGESTION_GRANULARITY, "day");
+
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo PARTITION BY 'day'")
+        .expectTarget("dst", targetRowSignature)
+        .expectResources(dataSourceRead("foo"), dataSourceWrite("dst"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource("foo")
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("__time", "dim1", "v0")
+                .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT))
+                .context(queryContext)
+                .build()
+        )
+        .verify();
+  }
+
+  @Test
+  public void testInsertWithClusterBy()
+  {
+    // Test correctness of the query when only CLUSTER BY clause is present
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .add("EXPR$3", ColumnType.DOUBLE)
+                                                  .build();
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst "
+            + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) FROM foo "
+            + "CLUSTER BY 2, dim1 DESC, CEIL(m2)"
+        )
+        .expectTarget("dst", targetRowSignature)
+        .expectResources(dataSourceRead("foo"), dataSourceWrite("dst"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource("foo")
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("__time", "dim1", "v0", "v1")
+                .virtualColumns(
+                    expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT),
+                    expressionVirtualColumn("v1", "ceil(\"m2\")", ColumnType.DOUBLE)
+                )
+                .orderBy(
+                    ImmutableList.of(
+                        new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING),
+                        new ScanQuery.OrderBy("dim1", ScanQuery.Order.DESCENDING),
+                        new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING)
+                    )
+                )
+                .build()
+        )
+        .verify();
+  }
+
+  @Test
+  public void testInsertWithPartitionByAndClusterBy()
+  {
+    // Test correctness of the query when both PARTITION BY and CLUSTER BY clause is present
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .build();
+
+    Map<String, Object> queryContext = new HashMap<>(DEFAULT_CONTEXT);
+    queryContext.put(QueryContexts.INGESTION_GRANULARITY, "day");
+
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo PARTITION BY 'day' CLUSTER BY 2, dim1")
+        .expectTarget("dst", targetRowSignature)
+        .expectResources(dataSourceRead("foo"), dataSourceWrite("dst"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource("foo")
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("__time", "dim1", "v0")
+                .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT))
+                .orderBy(
+                    ImmutableList.of(
+                        new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING),
+                        new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING)
+                    )
+                )
+                .context(queryContext)
+                .build()
+        )
+        .verify();
+  }
+
+  @Test
+  public void testInsertWithPartitionByAndLimitOffset()
+  {
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .build();
+
+    Map<String, Object> queryContext = new HashMap<>(DEFAULT_CONTEXT);
+    queryContext.put(QueryContexts.INGESTION_GRANULARITY, "day");
+
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo LIMIT 10 OFFSET 20 PARTITION BY 'day'")
+        .expectTarget("dst", targetRowSignature)
+        .expectResources(dataSourceRead("foo"), dataSourceWrite("dst"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource("foo")
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("__time", "dim1", "v0")
+                .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT))
+                .limit(10)
+                .offset(20)
+                .context(queryContext)
+                .build()
+        )
+        .verify();
+  }
+
+  @Test
+  public void testInsertWithPartitionByAndOrderBy()
+  {
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .build();
+
+    Map<String, Object> queryContext = new HashMap<>(DEFAULT_CONTEXT);
+    queryContext.put(QueryContexts.INGESTION_GRANULARITY, "day");
+
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo ORDER BY 2, dim1 PARTITION BY 'day'")
+        .expectTarget("dst", targetRowSignature)
+        .expectResources(dataSourceRead("foo"), dataSourceWrite("dst"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource("foo")
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("__time", "dim1", "v0")
+                .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT))
+                .orderBy(
+                    ImmutableList.of(
+                        new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING),
+                        new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING)
+                    )
+                )
+                .context(queryContext)
+                .build()
+        )
+        .verify();
+  }
+
+  @Test
+  public void testInsertWithClusterByAndOrderBy() throws Exception
+  {
+    try {
+      testQuery(
+          StringUtils.format(
+              "INSERT INTO dst SELECT * FROM %s ORDER BY 2 CLUSTER BY 3",
+              externSql(externalDataSource)
+          ),
+          ImmutableList.of(),
+          ImmutableList.of()
+      );
+      Assert.fail("Exception should be thrown");
+    }
+    catch (SqlPlanningException e) {
+      Assert.assertEquals(
+          "Cannot have both ORDER BY and CLUSTER BY clauses in the same INSERT query",
+          e.getMessage()
+      );
+    }
+    didTest = true;
+  }
+
+  @Test
+  public void testInsertWithPartitionByContainingInvalidGranularity() throws Exception
+  {
+    // Throws a ValidationException, which gets converted to a SqlPlanningException before throwing to end user
+    try {
+      testQuery(
+          "INSERT INTO dst SELECT * FROM foo PARTITION BY 'invalid_granularity'",
+          ImmutableList.of(),
+          ImmutableList.of()
+      );
+      Assert.fail("Exception should be thrown");
+    }
+    catch (SqlPlanningException e) {
+      Assert.assertEquals(
+          "Granularity passed in PARTITION BY clause is invalid",
+          e.getMessage()
+      );
+    }
+    didTest = true;
+  }
+
+
+  // Currently EXPLAIN PLAN FOR doesn't work with the modified syntax
+  @Ignore
+  @Test
+  public void testExplainInsertWithPartitionByAndClusterBy()
+  {
+    Assert.assertThrows(
+        SqlPlanningException.class,
+        () ->
+            testQuery(
+                StringUtils.format(
+                    "EXPLAIN PLAN FOR INSERT INTO dst SELECT * FROM %s PARTITION BY 'day' CLUSTER BY 1",

Review comment:
       Just as general practice, it is good that `CLUSTER` supports ordinals, but, for many reasons, best if users don't actually use them.

##########
File path: sql/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,443 @@
+# 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.
+
+# This file is an FMPP (http://fmpp.sourceforge.net/) configuration file to
+# allow clients to extend Calcite's SQL parser to support application specific
+# SQL statements, literals or data types.
+#
+# Calcite's parser grammar file (Parser.jj) is written in javacc
+# (http://javacc.java.net/) with Freemarker (http://freemarker.org/) variables
+# to allow clients to:
+#   1. have custom parser implementation class and package name.
+#   2. insert new parser method implementations written in javacc to parse
+#      custom:
+#      a) SQL statements.
+#      b) literals.
+#      c) data types.
+#   3. add new keywords to support custom SQL constructs added as part of (2).
+#   4. add import statements needed by inserted custom parser implementations.
+#
+# Parser template file (Parser.jj) along with this file are packaged as
+# part of the calcite-core-<version>.jar under "codegen" directory.
+
+# This file is directly copied from calite-core-1.21.0.jar/codegen/config.fmpp, and then modified slightly.
+# While not a necessary requirement, it would be ideal if it is kept in line with calcite-core's version. In the newer
+# Calcite versions, there is a default_config.fmpp which will free us from maintaining this file.
+#
+# Following clauses are modified in the file:
+#   1. data.parser.package & data.parser.class
+#   2. data.parser.imports
+#   3. data.parser.keywords (Added "CLUSTER")
+#   4. data.parser.statementParserMethods
+#   5. data.parser.implementationFiles
+data: {
+  parser: {
+    # Generated parser implementation package and class name.
+    package: "org.apache.druid.sql.calcite.parser",
+    class: "DruidSqlParserImpl",
+
+    # List of additional classes and packages to import.
+    # Example. "org.apache.calcite.sql.*", "java.util.List".
+    imports: [
+      "org.apache.calcite.sql.SqlNode"
+      "org.apache.calcite.sql.SqlInsert"
+      "org.apache.druid.sql.calcite.parser.DruidSqlInsert"
+    ]
+
+    # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved
+    # keyword add it to 'nonReservedKeywords' section.
+    keywords: [
+      "CLUSTER"
+    ]
+
+    # List of keywords from "keywords" section that are not reserved.
+    nonReservedKeywords: [
+        "A"
+        "ABSENT"
+        "ABSOLUTE"
+        "ACTION"
+        "ADA"
+        "ADD"
+        "ADMIN"
+        "AFTER"
+        "ALWAYS"
+        "APPLY"
+        "ASC"
+        "ASSERTION"
+        "ASSIGNMENT"
+        "ATTRIBUTE"
+        "ATTRIBUTES"
+        "BEFORE"
+        "BERNOULLI"
+        "BREADTH"
+        "C"
+        "CASCADE"
+        "CATALOG"
+        "CATALOG_NAME"
+        "CENTURY"
+        "CHAIN"
+        "CHARACTER_SET_CATALOG"
+        "CHARACTER_SET_NAME"
+        "CHARACTER_SET_SCHEMA"
+        "CHARACTERISTICS"
+        "CHARACTERS"
+        "CLASS_ORIGIN"
+        "COBOL"
+        "COLLATION"
+        "COLLATION_CATALOG"
+        "COLLATION_NAME"
+        "COLLATION_SCHEMA"
+        "COLUMN_NAME"
+        "COMMAND_FUNCTION"
+        "COMMAND_FUNCTION_CODE"
+        "COMMITTED"
+        "CONDITION_NUMBER"
+        "CONDITIONAL"
+        "CONNECTION"
+        "CONNECTION_NAME"
+        "CONSTRAINT_CATALOG"
+        "CONSTRAINT_NAME"
+        "CONSTRAINT_SCHEMA"
+        "CONSTRAINTS"
+        "CONSTRUCTOR"
+        "CONTINUE"
+        "CURSOR_NAME"
+        "DATA"
+        "DATABASE"
+        "DATETIME_INTERVAL_CODE"
+        "DATETIME_INTERVAL_PRECISION"
+        "DECADE"
+        "DEFAULTS"
+        "DEFERRABLE"
+        "DEFERRED"
+        "DEFINED"
+        "DEFINER"
+        "DEGREE"
+        "DEPTH"
+        "DERIVED"
+        "DESC"
+        "DESCRIPTION"
+        "DESCRIPTOR"
+        "DIAGNOSTICS"
+        "DISPATCH"
+        "DOMAIN"
+        "DOW"
+        "DOY"
+        "DYNAMIC_FUNCTION"
+        "DYNAMIC_FUNCTION_CODE"
+        "ENCODING"
+        "EPOCH"
+        "ERROR"
+        "EXCEPTION"
+        "EXCLUDE"
+        "EXCLUDING"
+        "FINAL"
+        "FIRST"
+        "FOLLOWING"
+        "FORMAT"
+        "FORTRAN"
+        "FOUND"
+        "FRAC_SECOND"
+        "G"
+        "GENERAL"
+        "GENERATED"
+        "GEOMETRY"
+        "GO"
+        "GOTO"
+        "GRANTED"
+        "HIERARCHY"
+        "IGNORE"
+        "IMMEDIATE"
+        "IMMEDIATELY"
+        "IMPLEMENTATION"
+        "INCLUDING"
+        "INCREMENT"
+        "INITIALLY"
+        "INPUT"
+        "INSTANCE"
+        "INSTANTIABLE"
+        "INVOKER"
+        "ISODOW"
+        "ISOYEAR"
+        "ISOLATION"
+        "JAVA"
+        "JSON"
+        "K"
+        "KEY"
+        "KEY_MEMBER"
+        "KEY_TYPE"
+        "LABEL"
+        "LAST"
+        "LENGTH"
+        "LEVEL"
+        "LIBRARY"
+        "LOCATOR"
+        "M"
+        "MAP"
+        "MATCHED"
+        "MAXVALUE"
+        "MICROSECOND"
+        "MESSAGE_LENGTH"
+        "MESSAGE_OCTET_LENGTH"
+        "MESSAGE_TEXT"
+        "MILLISECOND"
+        "MILLENNIUM"
+        "MINVALUE"
+        "MORE_"
+        "MUMPS"
+        "NAME"
+        "NAMES"
+        "NANOSECOND"
+        "NESTING"
+        "NORMALIZED"
+        "NULLABLE"
+        "NULLS"
+        "NUMBER"
+        "OBJECT"
+        "OCTETS"
+        "OPTION"
+        "OPTIONS"
+        "ORDERING"
+        "ORDINALITY"
+        "OTHERS"
+        "OUTPUT"
+        "OVERRIDING"
+        "PAD"
+        "PARAMETER_MODE"
+        "PARAMETER_NAME"
+        "PARAMETER_ORDINAL_POSITION"
+        "PARAMETER_SPECIFIC_CATALOG"
+        "PARAMETER_SPECIFIC_NAME"
+        "PARAMETER_SPECIFIC_SCHEMA"
+        "PARTIAL"
+        "PASCAL"
+        "PASSING"
+        "PASSTHROUGH"
+        "PAST"
+        "PATH"
+        "PLACING"
+        "PLAN"
+        "PLI"
+        "PRECEDING"
+        "PRESERVE"
+        "PRIOR"
+        "PRIVILEGES"
+        "PUBLIC"
+        "QUARTER"
+        "READ"
+        "RELATIVE"
+        "REPEATABLE"
+        "REPLACE"
+        "RESPECT"
+        "RESTART"
+        "RESTRICT"
+        "RETURNED_CARDINALITY"
+        "RETURNED_LENGTH"
+        "RETURNED_OCTET_LENGTH"
+        "RETURNED_SQLSTATE"
+        "RETURNING"
+        "ROLE"
+        "ROUTINE"
+        "ROUTINE_CATALOG"
+        "ROUTINE_NAME"
+        "ROUTINE_SCHEMA"
+        "ROW_COUNT"
+        "SCALAR"
+        "SCALE"
+        "SCHEMA"
+        "SCHEMA_NAME"
+        "SCOPE_CATALOGS"
+        "SCOPE_NAME"
+        "SCOPE_SCHEMA"
+        "SECTION"
+        "SECURITY"
+        "SELF"
+        "SEQUENCE"
+        "SERIALIZABLE"
+        "SERVER"
+        "SERVER_NAME"
+        "SESSION"
+        "SETS"
+        "SIMPLE"
+        "SIZE"
+        "SOURCE"
+        "SPACE"
+        "SPECIFIC_NAME"
+        "SQL_BIGINT"
+        "SQL_BINARY"
+        "SQL_BIT"
+        "SQL_BLOB"
+        "SQL_BOOLEAN"
+        "SQL_CHAR"
+        "SQL_CLOB"
+        "SQL_DATE"
+        "SQL_DECIMAL"
+        "SQL_DOUBLE"
+        "SQL_FLOAT"
+        "SQL_INTEGER"
+        "SQL_INTERVAL_DAY"
+        "SQL_INTERVAL_DAY_TO_HOUR"
+        "SQL_INTERVAL_DAY_TO_MINUTE"
+        "SQL_INTERVAL_DAY_TO_SECOND"
+        "SQL_INTERVAL_HOUR"
+        "SQL_INTERVAL_HOUR_TO_MINUTE"
+        "SQL_INTERVAL_HOUR_TO_SECOND"
+        "SQL_INTERVAL_MINUTE"
+        "SQL_INTERVAL_MINUTE_TO_SECOND"
+        "SQL_INTERVAL_MONTH"
+        "SQL_INTERVAL_SECOND"
+        "SQL_INTERVAL_YEAR"
+        "SQL_INTERVAL_YEAR_TO_MONTH"
+        "SQL_LONGVARBINARY"
+        "SQL_LONGVARNCHAR"
+        "SQL_LONGVARCHAR"
+        "SQL_NCHAR"
+        "SQL_NCLOB"
+        "SQL_NUMERIC"
+        "SQL_NVARCHAR"
+        "SQL_REAL"
+        "SQL_SMALLINT"
+        "SQL_TIME"
+        "SQL_TIMESTAMP"
+        "SQL_TINYINT"
+        "SQL_TSI_DAY"
+        "SQL_TSI_FRAC_SECOND"
+        "SQL_TSI_HOUR"
+        "SQL_TSI_MICROSECOND"
+        "SQL_TSI_MINUTE"
+        "SQL_TSI_MONTH"
+        "SQL_TSI_QUARTER"
+        "SQL_TSI_SECOND"
+        "SQL_TSI_WEEK"
+        "SQL_TSI_YEAR"
+        "SQL_VARBINARY"
+        "SQL_VARCHAR"
+        "STATE"
+        "STATEMENT"
+        "STRUCTURE"
+        "STYLE"
+        "SUBCLASS_ORIGIN"
+        "SUBSTITUTE"
+        "TABLE_NAME"
+        "TEMPORARY"
+        "TIES"
+        "TIMESTAMPADD"
+        "TIMESTAMPDIFF"
+        "TOP_LEVEL_COUNT"
+        "TRANSACTION"
+        "TRANSACTIONS_ACTIVE"
+        "TRANSACTIONS_COMMITTED"
+        "TRANSACTIONS_ROLLED_BACK"
+        "TRANSFORM"
+        "TRANSFORMS"
+        "TRIGGER_CATALOG"
+        "TRIGGER_NAME"
+        "TRIGGER_SCHEMA"
+        "TYPE"
+        "UNBOUNDED"
+        "UNCOMMITTED"
+        "UNCONDITIONAL"
+        "UNDER"
+        "UNNAMED"
+        "USAGE"
+        "USER_DEFINED_TYPE_CATALOG"
+        "USER_DEFINED_TYPE_CODE"
+        "USER_DEFINED_TYPE_NAME"
+        "USER_DEFINED_TYPE_SCHEMA"
+        "UTF8"
+        "UTF16"
+        "UTF32"
+        "VERSION"
+        "VIEW"
+        "WEEK"
+        "WRAPPER"
+        "WORK"
+        "WRITE"
+        "XML"
+        "ZONE"
+    ]
+
+    # List of additional join types. Each is a method with no arguments.
+    # Example: LeftSemiJoin()
+    joinTypes: [
+    ]
+
+    # List of methods for parsing custom SQL statements.
+    # Return type of method implementation should be 'SqlNode'.
+    # Example: SqlShowDatabases(), SqlShowTables().
+    statementParserMethods: [
+      "DruidSqlInsert()"
+    ]
+
+    # List of methods for parsing custom literals.
+    # Return type of method implementation should be "SqlNode".
+    # Example: ParseJsonLiteral().
+    literalParserMethods: [
+    ]
+
+    # List of methods for parsing custom data types.
+    # Return type of method implementation should be "SqlTypeNameSpec".
+    # Example: SqlParseTimeStampZ().
+    dataTypeParserMethods: [
+    ]
+
+    # List of methods for parsing builtin function calls.
+    # Return type of method implementation should be "SqlNode".
+    # Example: DateFunctionCall().
+    builtinFunctionCallMethods: [
+    ]
+
+    # List of methods for parsing extensions to "ALTER <scope>" calls.
+    # Each must accept arguments "(SqlParserPos pos, String scope)".
+    # Example: "SqlUploadJarNode"
+    alterStatementParserMethods: [
+    ]
+
+    # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+    # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+    createStatementParserMethods: [
+    ]
+
+    # List of methods for parsing extensions to "DROP" calls.
+    # Each must accept arguments "(SqlParserPos pos)".
+    dropStatementParserMethods: [
+    ]
+
+    # Binary operators tokens
+    binaryOperatorsTokens: [
+    ]
+
+    # Binary operators initialization
+    extraBinaryExpressions: [
+    ]
+
+    # List of files in @includes directory that have parser method
+    # implementations for parsing custom SQL statements, literals or types
+    # given as part of "statementParserMethods", "literalParserMethods" or
+    # "dataTypeParserMethods".
+    implementationFiles: [
+      "insert.ftl"
+    ]
+
+    includePosixOperators: false
+    includeCompoundIdentifier: true
+    includeBraces: true
+    includeAdditionalDeclarations: false

Review comment:
       `identifierMaxLength: 1024` (or some other number)?

##########
File path: sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java
##########
@@ -280,6 +283,242 @@ public void testInsertFromExternal()
         .verify();
   }
 
+  @Test
+  public void testInsertWithPartitionBy()
+  {
+    // Test correctness of the query when only PARTITION BY clause is present
+    RowSignature targetRowSignature = RowSignature.builder()
+                                                  .add("__time", ColumnType.LONG)
+                                                  .add("floor_m1", ColumnType.FLOAT)
+                                                  .add("dim1", ColumnType.STRING)
+                                                  .build();
+
+    Map<String, Object> queryContext = new HashMap<>(DEFAULT_CONTEXT);
+    queryContext.put(QueryContexts.INGESTION_GRANULARITY, "day");
+
+    testInsertQuery()
+        .sql(
+            "INSERT INTO druid.dst SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo PARTITION BY 'day'")

Review comment:
       Per earlier; this form of `PARTITION BY` is too implicit; where is `day` coming from? We know it is `__time`, but this makes too many assumptions. Better to use the expression Gian suggested. That way, we can have `PARTITIONED BY FLOOR(__time TO DAY), "cluster", "partition",  "node"`. (Would be good to have a better name for the function; `DAY(__time)`, say, but that's another discussion.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org