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/02/04 23:17:39 UTC

[GitHub] [druid] gianm commented on a change in pull request #12163: Add syntax support for PARTITIONED BY/CLUSTERED BY in INSERT queries

gianm commented on a change in pull request #12163:
URL: https://github.com/apache/druid/pull/12163#discussion_r799847786



##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java
##########
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.druid.java.util.common.granularity.Granularity;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+/**
+ * Extends the 'insert' call to hold custom parameters specific to Druid i.e. PARTITIONED BY and CLUSTERED BY
+ * This class extends the {@link SqlInsert} so that this SqlNode can be used in
+ * {@link org.apache.calcite.sql2rel.SqlToRelConverter} for getting converted into RelNode, and further processing
+ */
+public class DruidSqlInsert extends SqlInsert
+{
+  public static final String SQL_INSERT_SEGMENT_GRANULARITY = "sqlInsertSegmentGranularity";
+
+  // This allows reusing super.unparse
+  public static final SqlOperator OPERATOR = SqlInsert.OPERATOR;
+
+  private final Granularity partitionedBy;
+
+  @Nullable
+  private final SqlNodeList clusteredBy;
+
+  public DruidSqlInsert(
+      @Nonnull SqlInsert insertNode,
+      @Nonnull Granularity partitionedBy,
+      @Nullable SqlNodeList clusteredBy
+  )
+  {
+    super(
+        insertNode.getParserPosition(),
+        (SqlNodeList) insertNode.getOperandList().get(0), // No better getter to extract this
+        insertNode.getTargetTable(),
+        insertNode.getSource(),
+        insertNode.getTargetColumnList()
+    );
+    Preconditions.checkNotNull(partitionedBy); // Shouldn't hit due to how the parser is written
+    this.partitionedBy = partitionedBy;
+    this.clusteredBy = clusteredBy;
+  }
+
+  @Nullable
+  public SqlNodeList getClusteredBy()
+  {
+    return clusteredBy;
+  }
+
+  public Granularity getPartitionedBy()
+  {
+    return partitionedBy;
+  }
+
+  @Nonnull
+  @Override
+  public SqlOperator getOperator()
+  {
+    return OPERATOR;
+  }
+
+  @Override
+  public void unparse(SqlWriter writer, int leftPrec, int rightPrec)
+  {
+    super.unparse(writer, leftPrec, rightPrec);
+    writer.keyword("PARTITIONED BY");
+    writer.keyword(getPartitionedBy().toString()); // TODO: Make it cleaner by directly unparsing the SqlNode

Review comment:
       Does this `unparse` method get used anywhere? If so, we need to fix this because it's going to be incorrect in a bunch of cases. Or if it doesn't get used anywhere, IMO it's better to throw UnsupportedOperationException rather than have a broken implementation. At least that way it's clear that the method isn't reliable.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java
##########
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.sql.calcite.expression.TimeUnits;
+import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
+import org.joda.time.Period;
+
+import java.util.List;
+
+public class DruidSqlParserUtils
+{
+  /**
+   * Delegates to {@code convertSqlNodeToGranularity} and converts the exceptions to {@link ParseException}
+   * with the underlying message
+   */
+  public static Granularity convertSqlNodeToGranularityThrowingParseExceptions(SqlNode sqlNode) throws ParseException
+  {
+    Granularity g;
+    try {
+      g = convertSqlNodeToGranularity(sqlNode);
+    }
+    catch (Exception e) {
+      throw new ParseException(e.getMessage());
+    }
+    return g;
+  }
+
+  /**
+   * This method is used to extract the granularity from a SqlNode representing following function calls:
+   * 1. FLOOR(__time TO TimeUnit)
+   * 2. TIME_FLOOR(__time, 'PT1H')
+   *
+   * Validation on the sqlNode is contingent to following conditions:
+   * 1. sqlNode is an instance of SqlCall
+   * 2. Operator is either one of TIME_FLOOR or FLOOR
+   * 3. Number of operands in the call are 2
+   * 4. First operand is a SimpleIdentifier representing __time
+   * 5. If operator is TIME_FLOOR, the second argument is a literal, and can be converted to the Granularity class
+   * 6. If operator is FLOOR, the second argument is a TimeUnit, and can be mapped using {@link TimeUnits}
+   *
+   * Since it is to be used primarily while parsing the SqlNode, it is wrapped in {@code convertSqlNodeToGranularityThrowingParseExceptions}
+   *
+   * @param sqlNode SqlNode representing a call to a function
+   * @return Granularity as intended by the function call
+   * @throws ParseException SqlNode cannot be converted a granularity
+   */
+  public static Granularity convertSqlNodeToGranularity(SqlNode sqlNode) throws ParseException
+  {
+
+    final String genericParseFailedMessageFormatString = "Unable to parse the granularity from %s. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or %s function";

Review comment:
       "parse the granularity" sounds strange to me, grammatically. How about this:
   
   ```
   Encountered <%s> after PARTITIONED BY. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR(__time TO <timeUnit>) or %s(__time, '<period>').
   ```

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java
##########
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.sql.calcite.expression.TimeUnits;
+import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
+import org.joda.time.Period;
+
+import java.util.List;
+
+public class DruidSqlParserUtils
+{
+  /**
+   * Delegates to {@code convertSqlNodeToGranularity} and converts the exceptions to {@link ParseException}
+   * with the underlying message
+   */
+  public static Granularity convertSqlNodeToGranularityThrowingParseExceptions(SqlNode sqlNode) throws ParseException
+  {
+    Granularity g;
+    try {
+      g = convertSqlNodeToGranularity(sqlNode);
+    }
+    catch (Exception e) {
+      throw new ParseException(e.getMessage());
+    }
+    return g;
+  }
+
+  /**
+   * This method is used to extract the granularity from a SqlNode representing following function calls:
+   * 1. FLOOR(__time TO TimeUnit)
+   * 2. TIME_FLOOR(__time, 'PT1H')
+   *
+   * Validation on the sqlNode is contingent to following conditions:
+   * 1. sqlNode is an instance of SqlCall
+   * 2. Operator is either one of TIME_FLOOR or FLOOR
+   * 3. Number of operands in the call are 2
+   * 4. First operand is a SimpleIdentifier representing __time
+   * 5. If operator is TIME_FLOOR, the second argument is a literal, and can be converted to the Granularity class
+   * 6. If operator is FLOOR, the second argument is a TimeUnit, and can be mapped using {@link TimeUnits}
+   *
+   * Since it is to be used primarily while parsing the SqlNode, it is wrapped in {@code convertSqlNodeToGranularityThrowingParseExceptions}
+   *
+   * @param sqlNode SqlNode representing a call to a function
+   * @return Granularity as intended by the function call
+   * @throws ParseException SqlNode cannot be converted a granularity
+   */
+  public static Granularity convertSqlNodeToGranularity(SqlNode sqlNode) throws ParseException
+  {
+
+    final String genericParseFailedMessageFormatString = "Unable to parse the granularity from %s. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or %s function";
+
+    if (!(sqlNode instanceof SqlCall)) {
+      throw new ParseException(StringUtils.format(
+          genericParseFailedMessageFormatString,
+          sqlNode.toString(),
+          TimeFloorOperatorConversion.SQL_FUNCTION_NAME
+      ));
+    }
+    SqlCall sqlCall = (SqlCall) sqlNode;
+
+    String operatorName = sqlCall.getOperator().getName();
+
+    Preconditions.checkArgument(
+        "FLOOR".equalsIgnoreCase(operatorName)
+        || TimeFloorOperatorConversion.SQL_FUNCTION_NAME.equalsIgnoreCase(operatorName),
+        StringUtils.format(
+            "PARTITIONED BY clause can only parse FLOOR and %s functions.",
+            TimeFloorOperatorConversion.SQL_FUNCTION_NAME
+        )
+    );
+
+    List<SqlNode> operandList = sqlCall.getOperandList();
+    Preconditions.checkArgument(
+        operandList.size() == 2,
+        StringUtils.format("Invalid number of arguments passed to %s in PARTIITONED BY clause", operatorName)

Review comment:
       How about:
   
   > %s in PARTITIONED BY must have two arguments.
   
   (Even if you don't change this, PARTIITONED is misspelled, so please fix that at least.)

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
##########
@@ -765,13 +786,65 @@ 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) {

Review comment:
       Makes sense to me to do it in a follow up. We do need to do it, though, so EXPLAIN on INSERT works properly.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java
##########
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.sql.calcite.expression.TimeUnits;
+import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
+import org.joda.time.Period;
+
+import java.util.List;
+
+public class DruidSqlParserUtils
+{
+  /**
+   * Delegates to {@code convertSqlNodeToGranularity} and converts the exceptions to {@link ParseException}
+   * with the underlying message
+   */
+  public static Granularity convertSqlNodeToGranularityThrowingParseExceptions(SqlNode sqlNode) throws ParseException
+  {
+    Granularity g;
+    try {
+      g = convertSqlNodeToGranularity(sqlNode);

Review comment:
       You could simplify this to `return convertSqlNodeToGranularity(sqlNode);`. The variable `g` isn't necessary.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java
##########
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.druid.java.util.common.granularity.Granularity;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+/**
+ * Extends the 'insert' call to hold custom parameters specific to Druid i.e. PARTITIONED BY and CLUSTERED BY
+ * This class extends the {@link SqlInsert} so that this SqlNode can be used in
+ * {@link org.apache.calcite.sql2rel.SqlToRelConverter} for getting converted into RelNode, and further processing
+ */
+public class DruidSqlInsert extends SqlInsert
+{
+  public static final String SQL_INSERT_SEGMENT_GRANULARITY = "sqlInsertSegmentGranularity";
+
+  // This allows reusing super.unparse
+  public static final SqlOperator OPERATOR = SqlInsert.OPERATOR;
+
+  private final Granularity partitionedBy;
+
+  @Nullable
+  private final SqlNodeList clusteredBy;
+
+  public DruidSqlInsert(
+      @Nonnull SqlInsert insertNode,
+      @Nonnull Granularity partitionedBy,
+      @Nullable SqlNodeList clusteredBy
+  )
+  {
+    super(
+        insertNode.getParserPosition(),
+        (SqlNodeList) insertNode.getOperandList().get(0), // No better getter to extract this
+        insertNode.getTargetTable(),
+        insertNode.getSource(),
+        insertNode.getTargetColumnList()
+    );
+    Preconditions.checkNotNull(partitionedBy); // Shouldn't hit due to how the parser is written
+    this.partitionedBy = partitionedBy;
+    this.clusteredBy = clusteredBy;
+  }
+
+  @Nullable
+  public SqlNodeList getClusteredBy()
+  {
+    return clusteredBy;
+  }
+
+  public Granularity getPartitionedBy()
+  {
+    return partitionedBy;
+  }
+
+  @Nonnull
+  @Override
+  public SqlOperator getOperator()
+  {
+    return OPERATOR;
+  }
+
+  @Override
+  public void unparse(SqlWriter writer, int leftPrec, int rightPrec)
+  {
+    super.unparse(writer, leftPrec, rightPrec);
+    writer.keyword("PARTITIONED BY");
+    writer.keyword(getPartitionedBy().toString()); // TODO: Make it cleaner by directly unparsing the SqlNode
+    if (getClusteredBy() != null) {
+      writer.sep("CLUSTERED BY");

Review comment:
       Why `sep` rather than `keyword`? `sep` is supposed to be for list separators, which doesn't seem right here.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java
##########
@@ -744,18 +755,28 @@ public T next()
 
     private SqlNode query;
 
-    private ParsedNodes(@Nullable SqlExplain explain, @Nullable SqlInsert insert, SqlNode query)
+    @Nullable
+    private String ingestionGranularity;
+
+    private ParsedNodes(
+        @Nullable SqlExplain explain,
+        @Nullable SqlInsert insert,

Review comment:
       I see. But we'll need EXPLAIN on INSERT to work eventually, so we'll need to clean this up at some point.

##########
File path: sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java
##########
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.sql.calcite.expression.TimeUnits;
+import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion;
+import org.joda.time.Period;
+
+import java.util.List;
+
+public class DruidSqlParserUtils
+{
+  /**
+   * Delegates to {@code convertSqlNodeToGranularity} and converts the exceptions to {@link ParseException}
+   * with the underlying message
+   */
+  public static Granularity convertSqlNodeToGranularityThrowingParseExceptions(SqlNode sqlNode) throws ParseException
+  {
+    Granularity g;
+    try {
+      g = convertSqlNodeToGranularity(sqlNode);
+    }
+    catch (Exception e) {
+      throw new ParseException(e.getMessage());
+    }
+    return g;
+  }
+
+  /**
+   * This method is used to extract the granularity from a SqlNode representing following function calls:
+   * 1. FLOOR(__time TO TimeUnit)
+   * 2. TIME_FLOOR(__time, 'PT1H')
+   *
+   * Validation on the sqlNode is contingent to following conditions:
+   * 1. sqlNode is an instance of SqlCall
+   * 2. Operator is either one of TIME_FLOOR or FLOOR
+   * 3. Number of operands in the call are 2
+   * 4. First operand is a SimpleIdentifier representing __time
+   * 5. If operator is TIME_FLOOR, the second argument is a literal, and can be converted to the Granularity class
+   * 6. If operator is FLOOR, the second argument is a TimeUnit, and can be mapped using {@link TimeUnits}
+   *
+   * Since it is to be used primarily while parsing the SqlNode, it is wrapped in {@code convertSqlNodeToGranularityThrowingParseExceptions}
+   *
+   * @param sqlNode SqlNode representing a call to a function
+   * @return Granularity as intended by the function call
+   * @throws ParseException SqlNode cannot be converted a granularity
+   */
+  public static Granularity convertSqlNodeToGranularity(SqlNode sqlNode) throws ParseException
+  {
+
+    final String genericParseFailedMessageFormatString = "Unable to parse the granularity from %s. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or %s function";
+
+    if (!(sqlNode instanceof SqlCall)) {
+      throw new ParseException(StringUtils.format(
+          genericParseFailedMessageFormatString,
+          sqlNode.toString(),
+          TimeFloorOperatorConversion.SQL_FUNCTION_NAME
+      ));
+    }
+    SqlCall sqlCall = (SqlCall) sqlNode;
+
+    String operatorName = sqlCall.getOperator().getName();
+
+    Preconditions.checkArgument(
+        "FLOOR".equalsIgnoreCase(operatorName)
+        || TimeFloorOperatorConversion.SQL_FUNCTION_NAME.equalsIgnoreCase(operatorName),
+        StringUtils.format(
+            "PARTITIONED BY clause can only parse FLOOR and %s functions.",

Review comment:
       The verb "parse" here seems out of place. The clause doesn't do the parsing; the parser does the parsing.
   
   How about including the same error message as above? I don't think this message needs to be different; the earlier one makes total sense here.




-- 
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