You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2016/12/22 23:46:40 UTC

[07/15] incubator-impala git commit: IMPALA-4163: Add sortby() query hint

IMPALA-4163: Add sortby() query hint

This change introduces the sortby() query plan hint for insert
statements. When specified, sortby(a, b) will add an additional sort
step to the plan to order data by columns a, b before inserting it into
the target table.

Change-Id: I37a3ffab99aaa5d5a4fd1ac674b3e8b394a3c4c0
Reviewed-on: http://gerrit.cloudera.org:8080/5051
Reviewed-by: Marcel Kornacker <ma...@cloudera.com>
Tested-by: Internal Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/ce9b332e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/ce9b332e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/ce9b332e

Branch: refs/heads/hadoop-next
Commit: ce9b332ee9e640d79c8ae35e7abb8c7d787ddf78
Parents: 68131b3
Author: Lars Volker <lv...@cloudera.com>
Authored: Tue Nov 8 14:03:59 2016 +0100
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Dec 17 05:37:43 2016 +0000

----------------------------------------------------------------------
 fe/src/main/cup/sql-parser.cup                  | 65 ++++++++------
 .../org/apache/impala/analysis/InsertStmt.java  | 90 ++++++++++++++++----
 .../org/apache/impala/analysis/PlanHint.java    | 75 ++++++++++++++++
 .../org/apache/impala/analysis/SelectList.java  | 24 +++---
 .../org/apache/impala/analysis/TableRef.java    | 51 ++++++-----
 .../org/apache/impala/analysis/ToSqlUtils.java  |  5 +-
 .../java/org/apache/impala/planner/Planner.java | 29 ++++---
 fe/src/main/jflex/sql-scanner.flex              | 51 +++++++----
 .../impala/analysis/AnalyzeStmtsTest.java       | 51 +++++++++--
 .../org/apache/impala/analysis/ParserTest.java  | 84 ++++++++++++------
 .../org/apache/impala/analysis/ToSqlTest.java   |  4 +-
 .../queries/PlannerTest/insert.test             | 73 ++++++++++++++++
 .../queries/QueryTest/insert.test               | 26 ++++++
 13 files changed, 489 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 012da42..33f0591 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -276,10 +276,10 @@ terminal COLON, SEMICOLON, COMMA, DOT, DOTDOTDOT, STAR, LPAREN, RPAREN, LBRACKET
 terminal BITAND, BITOR, BITXOR, BITNOT;
 terminal EQUAL, NOT, NOTEQUAL, LESSTHAN, GREATERTHAN;
 terminal FACTORIAL; // Placeholder terminal for postfix factorial operator
+terminal COMMENTED_PLAN_HINT_START, COMMENTED_PLAN_HINT_END;
 terminal String IDENT;
 terminal String EMPTY_IDENT;
 terminal String NUMERIC_OVERFLOW;
-terminal String COMMENTED_PLAN_HINTS;
 terminal BigDecimal INTEGER_LITERAL;
 terminal BigDecimal DECIMAL_LITERAL;
 terminal String STRING_LITERAL;
@@ -367,7 +367,8 @@ nonterminal TableRef table_ref;
 nonterminal Subquery subquery;
 nonterminal JoinOperator join_operator;
 nonterminal opt_inner, opt_outer;
-nonterminal ArrayList<String> opt_plan_hints;
+nonterminal PlanHint plan_hint;
+nonterminal List<PlanHint> opt_plan_hints, plan_hint_list;
 nonterminal TypeDef type_def;
 nonterminal Type type;
 nonterminal Expr sign_chain_expr;
@@ -2300,41 +2301,41 @@ table_ref_list ::=
     list.add(table);
     RESULT = list;
   :}
-  | table_ref_list:list KW_CROSS KW_JOIN opt_plan_hints:hints table_ref:table
+  | table_ref_list:list KW_CROSS KW_JOIN opt_plan_hints:join_hints table_ref:table
     opt_plan_hints:table_hints
   {:
     table.setJoinOp(JoinOperator.CROSS_JOIN);
     // We will throw an AnalysisException if there are join hints so that we can provide
     // a better error message than a parser exception.
-    table.setJoinHints(hints);
+    table.setJoinHints(join_hints);
     table.setTableHints(table_hints);
     list.add(table);
     RESULT = list;
   :}
-  | table_ref_list:list join_operator:op opt_plan_hints:hints table_ref:table
+  | table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
     opt_plan_hints:table_hints
   {:
     table.setJoinOp((JoinOperator) op);
-    table.setJoinHints(hints);
+    table.setJoinHints(join_hints);
     table.setTableHints(table_hints);
     list.add(table);
     RESULT = list;
   :}
-  | table_ref_list:list join_operator:op opt_plan_hints:hints table_ref:table
+  | table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
     opt_plan_hints:table_hints KW_ON expr:e
   {:
     table.setJoinOp((JoinOperator) op);
-    table.setJoinHints(hints);
+    table.setJoinHints(join_hints);
     table.setTableHints(table_hints);
     table.setOnClause(e);
     list.add(table);
     RESULT = list;
   :}
-  | table_ref_list:list join_operator:op opt_plan_hints:hints table_ref:table
+  | table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
     opt_plan_hints:table_hints KW_USING LPAREN ident_list:colNames RPAREN
   {:
     table.setJoinOp((JoinOperator) op);
-    table.setJoinHints(hints);
+    table.setJoinHints(join_hints);
     table.setTableHints(table_hints);
     table.setUsingClause(colNames);
     list.add(table);
@@ -2381,28 +2382,40 @@ opt_outer ::=
   ;
 
 opt_plan_hints ::=
-  COMMENTED_PLAN_HINTS:l
+  COMMENTED_PLAN_HINT_START plan_hint_list:hints COMMENTED_PLAN_HINT_END
+  {: RESULT = hints; :}
+  /* legacy straight_join hint style */
+  | KW_STRAIGHT_JOIN
+  {: RESULT = Lists.newArrayList(new PlanHint("straight_join")); :}
+  /* legacy plan-hint style */
+  | LBRACKET plan_hint_list:hints RBRACKET
+  {: RESULT = hints; :}
+  | /* empty */
+  {: RESULT = Lists.newArrayList(); :}
+  ;
+
+plan_hint ::=
+  KW_STRAIGHT_JOIN
+  {: RESULT = new PlanHint("straight_join"); :}
+  | IDENT:name
+  {: RESULT = new PlanHint(name); :}
+  | IDENT:name LPAREN ident_list:args RPAREN
+  {: RESULT = new PlanHint(name, args); :}
+  | /* empty */
+  {: RESULT = null; :}
+  ;
+
+plan_hint_list ::=
+  plan_hint:hint
   {:
-    ArrayList<String> hints = new ArrayList<String>();
-    String[] tokens = l.split(",");
-    for (String token: tokens) {
-      String trimmedToken = token.trim();
-      if (trimmedToken.length() > 0) hints.add(trimmedToken);
-    }
+    ArrayList<PlanHint> hints = Lists.newArrayList(hint);
     RESULT = hints;
   :}
-  /* legacy straight_join hint style */
-  | KW_STRAIGHT_JOIN
+  | plan_hint_list:hints COMMA plan_hint:hint
   {:
-    ArrayList<String> hints = new ArrayList<String>();
-    hints.add("straight_join");
+    if (hint != null) hints.add(hint);
     RESULT = hints;
   :}
-  /* legacy plan-hint style */
-  | LBRACKET ident_list:l RBRACKET
-  {: RESULT = l; :}
-  | /* empty */
-  {: RESULT = null; :}
   ;
 
 ident_list ::=

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
index 1dacf48..902d100 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
@@ -64,7 +64,7 @@ public class InsertStmt extends StatementBase {
   private final List<PartitionKeyValue> partitionKeyValues_;
 
   // User-supplied hints to control hash partitioning before the table sink in the plan.
-  private final List<String> planHints_;
+  private List<PlanHint> planHints_ = Lists.newArrayList();
 
   // False if the original insert statement had a query statement, true if we need to
   // auto-generate one (for insert into tbl()) during analysis.
@@ -124,6 +124,14 @@ public class InsertStmt extends StatementBase {
   // clustering step.
   private boolean hasClusteredHint_ = false;
 
+  // For every column of the target table that is referenced in the optional 'sortby()'
+  // hint, this list will contain the corresponding result expr from 'resultExprs_'.
+  // Before insertion, all rows will be sorted by these exprs. If the list is empty, no
+  // additional sorting by non-partitioning columns will be performed. For Hdfs tables,
+  // the 'sortby()' hint must not contain partition columns. For Kudu tables, it must not
+  // contain primary key columns.
+  private List<Expr> sortByExprs_ = Lists.newArrayList();
+
   // Output expressions that produce the final results to write to the target table. May
   // include casts. Set in prepareExpressions().
   // If this is an INSERT on a non-Kudu table, it will contain one Expr for all
@@ -153,19 +161,19 @@ public class InsertStmt extends StatementBase {
 
   public static InsertStmt createInsert(WithClause withClause, TableName targetTable,
       boolean overwrite, List<PartitionKeyValue> partitionKeyValues,
-      List<String> planHints, QueryStmt queryStmt, List<String> columnPermutation) {
+      List<PlanHint> planHints, QueryStmt queryStmt, List<String> columnPermutation) {
     return new InsertStmt(withClause, targetTable, overwrite, partitionKeyValues,
         planHints, queryStmt, columnPermutation, false);
   }
 
   public static InsertStmt createUpsert(WithClause withClause, TableName targetTable,
-      List<String> planHints, QueryStmt queryStmt, List<String> columnPermutation) {
+      List<PlanHint> planHints, QueryStmt queryStmt, List<String> columnPermutation) {
     return new InsertStmt(withClause, targetTable, false, null, planHints, queryStmt,
         columnPermutation, true);
   }
 
   protected InsertStmt(WithClause withClause, TableName targetTable, boolean overwrite,
-      List<PartitionKeyValue> partitionKeyValues, List<String> planHints,
+      List<PartitionKeyValue> partitionKeyValues, List<PlanHint> planHints,
       QueryStmt queryStmt, List<String> columnPermutation, boolean isUpsert) {
     Preconditions.checkState(!isUpsert || (!overwrite && partitionKeyValues == null));
     withClause_ = withClause;
@@ -173,7 +181,7 @@ public class InsertStmt extends StatementBase {
     originalTableName_ = targetTableName_;
     overwrite_ = overwrite;
     partitionKeyValues_ = partitionKeyValues;
-    planHints_ = planHints;
+    planHints_ = (planHints != null) ? planHints : new ArrayList<PlanHint>();
     queryStmt_ = queryStmt;
     needsGeneratedQueryStatement_ = (queryStmt == null);
     columnPermutation_ = columnPermutation;
@@ -210,6 +218,7 @@ public class InsertStmt extends StatementBase {
     hasShuffleHint_ = false;
     hasNoShuffleHint_ = false;
     hasClusteredHint_ = false;
+    sortByExprs_.clear();
     resultExprs_.clear();
     mentionedColumns_.clear();
     primaryKeyExprs_.clear();
@@ -729,25 +738,27 @@ public class InsertStmt extends StatementBase {
   }
 
   private void analyzePlanHints(Analyzer analyzer) throws AnalysisException {
-    if (planHints_ == null) return;
     if (!planHints_.isEmpty() && table_ instanceof HBaseTable) {
-      throw new AnalysisException("INSERT hints are only supported for inserting into " +
-          "Hdfs and Kudu tables.");
+      throw new AnalysisException(String.format("INSERT hints are only supported for " +
+          "inserting into Hdfs and Kudu tables: %s", getTargetTableName()));
     }
     boolean hasNoClusteredHint = false;
-    for (String hint: planHints_) {
-      if (hint.equalsIgnoreCase("SHUFFLE")) {
+    for (PlanHint hint: planHints_) {
+      if (hint.is("SHUFFLE")) {
         hasShuffleHint_ = true;
         analyzer.setHasPlanHints();
-      } else if (hint.equalsIgnoreCase("NOSHUFFLE")) {
+      } else if (hint.is("NOSHUFFLE")) {
         hasNoShuffleHint_ = true;
         analyzer.setHasPlanHints();
-      } else if (hint.equalsIgnoreCase("CLUSTERED")) {
+      } else if (hint.is("CLUSTERED")) {
         hasClusteredHint_ = true;
         analyzer.setHasPlanHints();
-      } else if (hint.equalsIgnoreCase("NOCLUSTERED")) {
+      } else if (hint.is("NOCLUSTERED")) {
         hasNoClusteredHint = true;
         analyzer.setHasPlanHints();
+      } else if (hint.is("SORTBY")) {
+        analyzeSortByHint(hint);
+        analyzer.setHasPlanHints();
       } else {
         analyzer.addWarning("INSERT hint not recognized: " + hint);
       }
@@ -761,6 +772,51 @@ public class InsertStmt extends StatementBase {
     }
   }
 
+  private void analyzeSortByHint(PlanHint hint) throws AnalysisException {
+    // HBase tables don't support insert hints at all (must be enforced by the caller).
+    Preconditions.checkState(!(table_ instanceof HBaseTable));
+
+    if (isUpsert_) {
+      throw new AnalysisException("SORTBY hint is not supported in UPSERT statements.");
+    }
+
+    List<String> columnNames = hint.getArgs();
+    Preconditions.checkState(!columnNames.isEmpty());
+    for (String columnName: columnNames) {
+      // Make sure it's not a Kudu primary key column or Hdfs partition column.
+      if (table_ instanceof KuduTable) {
+        KuduTable kuduTable = (KuduTable) table_;
+        if (kuduTable.isPrimaryKeyColumn(columnName)) {
+          throw new AnalysisException(String.format("SORTBY hint column list must not " +
+              "contain Kudu primary key column: '%s'", columnName));
+        }
+      } else {
+        for (Column tableColumn: table_.getClusteringColumns()) {
+          if (tableColumn.getName().equals(columnName)) {
+            throw new AnalysisException(String.format("SORTBY hint column list must " +
+                "not contain Hdfs partition column: '%s'", columnName));
+          }
+        }
+      }
+
+      // Find the matching column in the target table's column list (by name) and store
+      // the corresponding result expr in sortByExprs_.
+      boolean foundColumn = false;
+      List<Column> columns = table_.getNonClusteringColumns();
+      for (int i = 0; i < columns.size(); ++i) {
+        if (columns.get(i).getName().equals(columnName)) {
+          sortByExprs_.add(resultExprs_.get(i));
+          foundColumn = true;
+          break;
+        }
+      }
+      if (!foundColumn) {
+        throw new AnalysisException(String.format("Could not find SORTBY hint column " +
+            "'%s' in table.", columnName));
+      }
+    }
+  }
+
   @Override
   public ArrayList<Expr> getResultExprs() { return resultExprs_; }
 
@@ -772,7 +828,7 @@ public class InsertStmt extends StatementBase {
 
   private String getOpName() { return isUpsert_ ? "UPSERT" : "INSERT"; }
 
-  public List<String> getPlanHints() { return planHints_; }
+  public List<PlanHint> getPlanHints() { return planHints_; }
   public TableName getTargetTableName() { return targetTableName_; }
   public Table getTargetTable() { return table_; }
   public void setTargetTable(Table table) { this.table_ = table; }
@@ -788,6 +844,7 @@ public class InsertStmt extends StatementBase {
   public boolean hasNoShuffleHint() { return hasNoShuffleHint_; }
   public boolean hasClusteredHint() { return hasClusteredHint_; }
   public ArrayList<Expr> getPrimaryKeyExprs() { return primaryKeyExprs_; }
+  public List<Expr> getSortByExprs() { return sortByExprs_; }
 
   public List<String> getMentionedColumns() {
     List<String> result = Lists.newArrayList();
@@ -812,6 +869,7 @@ public class InsertStmt extends StatementBase {
     resultExprs_ = Expr.substituteList(resultExprs_, smap, analyzer, true);
     partitionKeyExprs_ = Expr.substituteList(partitionKeyExprs_, smap, analyzer, true);
     primaryKeyExprs_ = Expr.substituteList(primaryKeyExprs_, smap, analyzer, true);
+    sortByExprs_ = Expr.substituteList(sortByExprs_, smap, analyzer, true);
   }
 
   @Override
@@ -840,8 +898,8 @@ public class InsertStmt extends StatementBase {
       }
       strBuilder.append(" PARTITION (" + Joiner.on(", ").join(values) + ")");
     }
-    if (planHints_ != null) {
-      strBuilder.append(" " + ToSqlUtils.getPlanHintsSql(planHints_));
+    if (!planHints_.isEmpty()) {
+      strBuilder.append(" " + ToSqlUtils.getPlanHintsSql(getPlanHints()));
     }
     if (!needsGeneratedQueryStatement_) {
       strBuilder.append(" " + queryStmt_.toSql());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/analysis/PlanHint.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PlanHint.java b/fe/src/main/java/org/apache/impala/analysis/PlanHint.java
new file mode 100644
index 0000000..d16919f
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/PlanHint.java
@@ -0,0 +1,75 @@
+// 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.impala.analysis;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * Class to parse and store query plan hints, which can occur in various places inside SQL
+ * query statements. A hint consist of a name and an optional list of arguments.
+ */
+public class PlanHint {
+  /// The plan hint name.
+  private final String name_;
+
+  /// Optional list of arguments.
+  private final List<String> args_;
+
+  public PlanHint(String name) {
+    name_ = name;
+    args_ = Lists.newArrayList();
+  }
+
+  public PlanHint(String name, List<String> args) {
+    name_ = name;
+    args_ = args;
+  }
+
+  /// Check whether this hint equals to a given string, ignoring case.
+  public boolean is(String s) { return name_.equalsIgnoreCase(s); }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null) return false;
+    if (getClass() != o.getClass()) return false;
+    PlanHint oh = (PlanHint) o;
+    return name_.equals(oh.name_) && args_.equals(oh.args_);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(name_);
+    if (!args_.isEmpty()) {
+      sb.append("(");
+      sb.append(Joiner.on(",").join(args_));
+      sb.append(")");
+    }
+    return sb.toString();
+  }
+
+  public List<String> getArgs() { return args_; }
+  public String toSql() { return toString(); }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/analysis/SelectList.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SelectList.java b/fe/src/main/java/org/apache/impala/analysis/SelectList.java
index 4c504bf..d7f12ff 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SelectList.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SelectList.java
@@ -22,13 +22,14 @@ import java.util.List;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.rewrite.ExprRewriter;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 /**
  * Select list items plus optional distinct clause and optional plan hints.
  */
 public class SelectList {
-  private List<String> planHints_;
+  private List<PlanHint> planHints_ = Lists.newArrayList();;
   private boolean isDistinct_;
 
   /////////////////////////////////////////
@@ -50,7 +51,7 @@ public class SelectList {
   }
 
   public SelectList(List<SelectListItem> items, boolean isDistinct,
-      List<String> planHints) {
+      List<PlanHint> planHints) {
     isDistinct_ = isDistinct;
     items_ = items;
     planHints_ = planHints;
@@ -60,8 +61,7 @@ public class SelectList {
    * C'tor for cloning.
    */
   public SelectList(SelectList other) {
-    planHints_ =
-        (other.planHints_ != null) ? Lists.newArrayList(other.planHints_) : null;
+    planHints_ = Lists.newArrayList(other.planHints_);
     items_ = Lists.newArrayList();
     for (SelectListItem item: other.items_) {
       items_.add(item.clone());
@@ -70,16 +70,20 @@ public class SelectList {
   }
 
   public List<SelectListItem> getItems() { return items_; }
-  public void setPlanHints(List<String> planHints) { planHints_ = planHints; }
-  public List<String> getPlanHints() { return planHints_; }
+
+  public void setPlanHints(List<PlanHint> planHints) {
+    Preconditions.checkNotNull(planHints);
+    planHints_ = planHints;
+  }
+
+  public List<PlanHint> getPlanHints() { return planHints_; }
   public boolean isDistinct() { return isDistinct_; }
   public void setIsDistinct(boolean value) { isDistinct_ = value; }
-  public boolean hasPlanHints() { return planHints_ != null; }
+  public boolean hasPlanHints() { return !planHints_.isEmpty(); }
 
   public void analyzePlanHints(Analyzer analyzer) {
-    if (planHints_ == null) return;
-    for (String hint: planHints_) {
-      if (!hint.equalsIgnoreCase("straight_join")) {
+    for (PlanHint hint: planHints_) {
+      if (!hint.is("straight_join")) {
         analyzer.addWarning("PLAN hint not recognized: " + hint);
       }
       analyzer.setIsStraightJoin();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/analysis/TableRef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TableRef.java b/fe/src/main/java/org/apache/impala/analysis/TableRef.java
index d6bbfd2..975b70b 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TableRef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TableRef.java
@@ -82,10 +82,10 @@ public class TableRef implements ParseNode {
   protected final Privilege priv_;
 
   protected JoinOperator joinOp_;
-  protected ArrayList<String> joinHints_;
+  protected List<PlanHint> joinHints_ = Lists.newArrayList();
   protected List<String> usingColNames_;
 
-  protected ArrayList<String> tableHints_;
+  protected List<PlanHint> tableHints_ = Lists.newArrayList();
   protected TReplicaPreference replicaPreference_;
   protected boolean randomReplica_;
 
@@ -156,13 +156,11 @@ public class TableRef implements ParseNode {
     hasExplicitAlias_ = other.hasExplicitAlias_;
     priv_ = other.priv_;
     joinOp_ = other.joinOp_;
-    joinHints_ =
-        (other.joinHints_ != null) ? Lists.newArrayList(other.joinHints_) : null;
+    joinHints_ = Lists.newArrayList(other.joinHints_);
     onClause_ = (other.onClause_ != null) ? other.onClause_.clone() : null;
     usingColNames_ =
         (other.usingColNames_ != null) ? Lists.newArrayList(other.usingColNames_) : null;
-    tableHints_ =
-        (other.tableHints_ != null) ? Lists.newArrayList(other.tableHints_) : null;
+    tableHints_ = Lists.newArrayList(other.tableHints_);
     replicaPreference_ = other.replicaPreference_;
     randomReplica_ = other.randomReplica_;
     distrMode_ = other.distrMode_;
@@ -262,8 +260,8 @@ public class TableRef implements ParseNode {
     return resolvedPath_.getRootTable();
   }
   public Privilege getPrivilege() { return priv_; }
-  public ArrayList<String> getJoinHints() { return joinHints_; }
-  public ArrayList<String> getTableHints() { return tableHints_; }
+  public List<PlanHint> getJoinHints() { return joinHints_; }
+  public List<PlanHint> getTableHints() { return tableHints_; }
   public Expr getOnClause() { return onClause_; }
   public List<String> getUsingClause() { return usingColNames_; }
   public void setJoinOp(JoinOperator op) { this.joinOp_ = op; }
@@ -271,12 +269,23 @@ public class TableRef implements ParseNode {
   public void setUsingClause(List<String> colNames) { this.usingColNames_ = colNames; }
   public TableRef getLeftTblRef() { return leftTblRef_; }
   public void setLeftTblRef(TableRef leftTblRef) { this.leftTblRef_ = leftTblRef; }
-  public void setJoinHints(ArrayList<String> hints) { this.joinHints_ = hints; }
-  public void setTableHints(ArrayList<String> hints) { this.tableHints_ = hints; }
+
+  public void setJoinHints(List<PlanHint> hints) {
+    Preconditions.checkNotNull(hints);
+    joinHints_ = hints;
+  }
+
+  public void setTableHints(List<PlanHint> hints) {
+    Preconditions.checkNotNull(hints);
+    tableHints_ = hints;
+  }
+
   public boolean isBroadcastJoin() { return distrMode_ == DistributionMode.BROADCAST; }
+
   public boolean isPartitionedJoin() {
     return distrMode_ == DistributionMode.PARTITIONED;
   }
+
   public DistributionMode getDistributionMode() { return distrMode_; }
   public List<TupleId> getCorrelatedTupleIds() { return correlatedTupleIds_; }
   public boolean isAnalyzed() { return isAnalyzed_; }
@@ -336,7 +345,7 @@ public class TableRef implements ParseNode {
   }
 
   private void analyzeTableHints(Analyzer analyzer) {
-    if (tableHints_ == null) return;
+    if (tableHints_.isEmpty()) return;
     if (!(this instanceof BaseTableRef)) {
       analyzer.addWarning("Table hints not supported for inline view and collections");
       return;
@@ -347,17 +356,17 @@ public class TableRef implements ParseNode {
         !(getResolvedPath().destTable() instanceof HdfsTable)) {
       analyzer.addWarning("Table hints only supported for Hdfs tables");
     }
-    for (String hint: tableHints_) {
-      if (hint.equalsIgnoreCase("SCHEDULE_CACHE_LOCAL")) {
+    for (PlanHint hint: tableHints_) {
+      if (hint.is("SCHEDULE_CACHE_LOCAL")) {
         analyzer.setHasPlanHints();
         replicaPreference_ = TReplicaPreference.CACHE_LOCAL;
-      } else if (hint.equalsIgnoreCase("SCHEDULE_DISK_LOCAL")) {
+      } else if (hint.is("SCHEDULE_DISK_LOCAL")) {
         analyzer.setHasPlanHints();
         replicaPreference_ = TReplicaPreference.DISK_LOCAL;
-      } else if (hint.equalsIgnoreCase("SCHEDULE_REMOTE")) {
+      } else if (hint.is("SCHEDULE_REMOTE")) {
         analyzer.setHasPlanHints();
         replicaPreference_ = TReplicaPreference.REMOTE;
-      } else if (hint.equalsIgnoreCase("SCHEDULE_RANDOM_REPLICA")) {
+      } else if (hint.is("SCHEDULE_RANDOM_REPLICA")) {
         analyzer.setHasPlanHints();
         randomReplica_ = true;
       } else {
@@ -369,9 +378,9 @@ public class TableRef implements ParseNode {
   }
 
   private void analyzeJoinHints(Analyzer analyzer) throws AnalysisException {
-    if (joinHints_ == null) return;
-    for (String hint: joinHints_) {
-      if (hint.equalsIgnoreCase("BROADCAST")) {
+    if (joinHints_.isEmpty()) return;
+    for (PlanHint hint: joinHints_) {
+      if (hint.is("BROADCAST")) {
         if (joinOp_ == JoinOperator.RIGHT_OUTER_JOIN
             || joinOp_ == JoinOperator.FULL_OUTER_JOIN
             || joinOp_ == JoinOperator.RIGHT_SEMI_JOIN
@@ -384,7 +393,7 @@ public class TableRef implements ParseNode {
         }
         distrMode_ = DistributionMode.BROADCAST;
         analyzer.setHasPlanHints();
-      } else if (hint.equalsIgnoreCase("SHUFFLE")) {
+      } else if (hint.is("SHUFFLE")) {
         if (joinOp_ == JoinOperator.CROSS_JOIN) {
           throw new AnalysisException("CROSS JOIN does not support SHUFFLE.");
         }
@@ -545,7 +554,7 @@ public class TableRef implements ParseNode {
     }
 
     StringBuilder output = new StringBuilder(" " + joinOp_.toString() + " ");
-    if(joinHints_ != null) output.append(ToSqlUtils.getPlanHintsSql(joinHints_) + " ");
+    if(!joinHints_.isEmpty()) output.append(ToSqlUtils.getPlanHintsSql(joinHints_) + " ");
     output.append(tableRefToSql());
     if (usingColNames_ != null) {
       output.append(" USING (").append(Joiner.on(", ").join(usingColNames_)).append(")");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
index be4ab6f..35f7e79 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
@@ -389,8 +389,9 @@ public class ToSqlUtils {
    * commented plan hint style such that hinted views created by Impala are readable by
    * Hive (parsed as a comment by Hive).
    */
-  public static String getPlanHintsSql(List<String> hints) {
-    if (hints == null || hints.isEmpty()) return "";
+  public static String getPlanHintsSql(List<PlanHint> hints) {
+    Preconditions.checkNotNull(hints);
+    if (hints.isEmpty()) return "";
     StringBuilder sb = new StringBuilder();
     sb.append("\n-- +");
     sb.append(Joiner.on(",").join(hints));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/java/org/apache/impala/planner/Planner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index 3369686..297e9b2 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -144,7 +144,7 @@ public class Planner {
             rootFragment, insertStmt, ctx_.getRootAnalyzer(), fragments);
       }
       // Add optional sort node to the plan, based on clustered/noclustered plan hint.
-      createClusteringSort(insertStmt, rootFragment, ctx_.getRootAnalyzer());
+      createPreInsertSort(insertStmt, rootFragment, ctx_.getRootAnalyzer());
       // set up table sink for root fragment
       rootFragment.setSink(insertStmt.createDataSink());
       resultExprs = insertStmt.getResultExprs();
@@ -512,21 +512,26 @@ public class Planner {
   }
 
   /**
-   * Insert a sort node on top of the plan, depending on the clustered/noclustered plan
-   * hint. This will sort the data produced by 'inputFragment' by the clustering columns
-   * (key columns for Kudu tables), so that partitions can be written sequentially in the
-   * table sink.
+   * Insert a sort node on top of the plan, depending on the clustered/noclustered/sortby
+   * plan hint. If clustering is enabled in insertStmt, then the ordering columns will
+   * start with the clustering columns (key columns for Kudu tables), so that partitions
+   * can be written sequentially in the table sink. Any additional non-clustering columns
+   * specified by the sortby hint will be added to the ordering columns and after any
+   * clustering columns. If neither clustering nor a sortby hint are specified, then no
+   * sort node will be added to the plan.
    */
-  public void createClusteringSort(InsertStmt insertStmt, PlanFragment inputFragment,
+  public void createPreInsertSort(InsertStmt insertStmt, PlanFragment inputFragment,
        Analyzer analyzer) throws ImpalaException {
-    if (!insertStmt.hasClusteredHint()) return;
+    List<Expr> orderingExprs = Lists.newArrayList();
 
-    List<Expr> orderingExprs;
-    if (insertStmt.getTargetTable() instanceof KuduTable) {
-      orderingExprs = Lists.newArrayList(insertStmt.getPrimaryKeyExprs());
-    } else {
-      orderingExprs = Lists.newArrayList(insertStmt.getPartitionKeyExprs());
+    if (insertStmt.hasClusteredHint()) {
+      if (insertStmt.getTargetTable() instanceof KuduTable) {
+        orderingExprs.addAll(insertStmt.getPrimaryKeyExprs());
+      } else {
+        orderingExprs.addAll(insertStmt.getPartitionKeyExprs());
+      }
     }
+    orderingExprs.addAll(insertStmt.getSortByExprs());
     // Ignore constants for the sake of clustering.
     Expr.removeConstants(orderingExprs);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/main/jflex/sql-scanner.flex
----------------------------------------------------------------------
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index 982e9a2..6d1a773 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -301,7 +301,6 @@ import org.apache.impala.analysis.SqlParserSymbols;
 %}
 
 LineTerminator = \r|\n|\r\n
-NonTerminator = [^\r\n]
 Whitespace = {LineTerminator} | [ \t\f]
 
 // Order of rules to resolve ambiguity:
@@ -321,14 +320,30 @@ QuotedIdentifier = \`(\\.|[^\\\`])*\`
 SingleQuoteStringLiteral = \'(\\.|[^\\\'])*\'
 DoubleQuoteStringLiteral = \"(\\.|[^\\\"])*\"
 
+EolHintBegin = "--" " "* "+"
+CommentedHintBegin = "/*" " "* "+"
+CommentedHintEnd = "*/"
+
 // Both types of plan hints must appear within a single line.
-TraditionalCommentedPlanHints = "/*" [ ]* "+" [^\r\n*]* "*/"
-// Must end with a line terminator.
-EndOfLineCommentedPlanHints = "--" [ ]* "+" {NonTerminator}* {LineTerminator}
+HintContent = " "* "+" [^\r\n]*
 
 Comment = {TraditionalComment} | {EndOfLineComment}
-TraditionalComment = "/*" ~"*/"
-EndOfLineComment = "--" {NonTerminator}* {LineTerminator}?
+
+// Match anything that has a comment end (*/) in it.
+ContainsCommentEnd = [^]* "*/" [^]*
+// Match anything that has a line terminator in it.
+ContainsLineTerminator = [^]* {LineTerminator} [^]*
+
+// A traditional comment is anything that starts and ends like a comment and has neither a
+// plan hint inside nor a CommentEnd (*/).
+TraditionalComment = "/*" !({HintContent}|{ContainsCommentEnd}) "*/"
+// Similar for a end-of-line comment.
+EndOfLineComment = "--" !({HintContent}|{ContainsLineTerminator}) {LineTerminator}?
+
+// This additional state is needed because newlines signal the end of a end-of-line hint
+// if one has been started earlier. Hence we need to discern between newlines within and
+// outside of end-of-line hints.
+%state EOLHINT
 
 %%
 // Put '...' before '.'
@@ -412,18 +427,22 @@ EndOfLineComment = "--" {NonTerminator}* {LineTerminator}?
   return newToken(SqlParserSymbols.STRING_LITERAL, yytext().substring(1, yytext().length()-1));
 }
 
-{TraditionalCommentedPlanHints} {
-  String text = yytext();
-  // Remove everything before the first '+' as well as the trailing "*/"
-  String hintStr = text.substring(text.indexOf('+') + 1, text.length() - 2);
-  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINTS, hintStr.trim());
+{CommentedHintBegin} {
+  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINT_START, null);
 }
 
-{EndOfLineCommentedPlanHints} {
-  String text = yytext();
-  // Remove everything before the first '+'
-  String hintStr = text.substring(text.indexOf('+') + 1);
-  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINTS, hintStr.trim());
+{CommentedHintEnd} {
+  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINT_END, null);
+}
+
+{EolHintBegin} {
+  yybegin(EOLHINT);
+  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINT_START, null);
+}
+
+<EOLHINT> {LineTerminator} {
+  yybegin(YYINITIAL);
+  return newToken(SqlParserSymbols.COMMENTED_PLAN_HINT_END, null);
 }
 
 {Comment} { /* ignore */ }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
index 11f6842..9b641a0 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
@@ -1604,14 +1604,6 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
           String.format("select * from functional.alltypes a join %sbadhint%s " +
               "functional.alltypes b using (int_col)", prefix, suffix),
           "JOIN hint not recognized: badhint");
-      // Hints must be comma separated. Legacy-style hint does not parse because
-      // of space-separated identifiers.
-      if (!prefix.contains("[")) {
-        AnalyzesOk(String.format(
-            "select * from functional.alltypes a join %sbroadcast broadcast%s " +
-                "functional.alltypes b using (int_col)", prefix, suffix),
-            "JOIN hint not recognized: broadcast broadcast");
-      }
       AnalysisError(
           String.format("select * from functional.alltypes a cross join %sshuffle%s " +
           "functional.alltypes b", prefix, suffix),
@@ -1744,7 +1736,8 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
       AnalysisError(String.format(
           "insert into table functional_hbase.alltypes %sshuffle%s " +
           "select * from functional_hbase.alltypes", prefix, suffix),
-          "INSERT hints are only supported for inserting into Hdfs and Kudu tables.");
+          "INSERT hints are only supported for inserting into Hdfs and Kudu tables: " +
+          "functional_hbase.alltypes");
       // Conflicting plan hints.
       AnalysisError("insert into table functional.alltypessmall " +
           "partition (year, month) /* +shuffle,noshuffle */ " +
@@ -1766,6 +1759,46 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
           "insert into table functional.alltypessmall partition (year, month) " +
           "/* +clustered,noclustered */ select * from functional.alltypes", prefix,
           suffix), "Conflicting INSERT hints: clustered and noclustered");
+
+      // Below are tests for hints that are not supported by the legacy syntax.
+      if (prefix == "[") continue;
+
+      // Tests for sortby hint
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %ssortby(int_col)%s select * from functional.alltypes",
+          prefix, suffix));
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %sshuffle,clustered,sortby(int_col)%s select * from " +
+          "functional.alltypes", prefix, suffix));
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %ssortby(int_col, bool_col)%s select * from " +
+          "functional.alltypes", prefix, suffix));
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %sshuffle,clustered,sortby(int_col,bool_col)%s " +
+          "select * from functional.alltypes", prefix, suffix));
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %sshuffle,sortby(int_col,bool_col),clustered%s " +
+          "select * from functional.alltypes", prefix, suffix));
+      AnalyzesOk(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %ssortby(int_col,bool_col),shuffle,clustered%s " +
+          "select * from functional.alltypes", prefix, suffix));
+      // Column in sortby hint must exist.
+      AnalysisError(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %ssortby(foo)%s select * from functional.alltypes",
+          prefix, suffix), "Could not find SORTBY hint column 'foo' in table.");
+      // Column in sortby hint must not be a Hdfs partition column.
+      AnalysisError(String.format("insert into functional.alltypessmall " +
+          "partition (year, month) %ssortby(year)%s select * from " +
+          "functional.alltypes", prefix, suffix),
+          "SORTBY hint column list must not contain Hdfs partition column: 'year'");
+      // Column in sortby hint must not be a Kudu primary key column.
+      AnalysisError(String.format("insert into functional_kudu.alltypessmall " +
+          "%ssortby(id)%s select * from functional_kudu.alltypes", prefix, suffix),
+          "SORTBY hint column list must not contain Kudu primary key column: 'id'");
+      // sortby() hint is not supported in UPSERT queries
+      AnalysisError(String.format("upsert into functional_kudu.alltypessmall " +
+          "%ssortby(id)%s select * from functional_kudu.alltypes", prefix, suffix),
+          "SORTBY hint is not supported in UPSERT statements.");
     }
 
     // Multiple non-conflicting hints and case insensitivity of hints.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 180eabd..2b37bf3 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -209,6 +209,13 @@ public class ParserTest extends FrontendTestBase {
     ParsesOk("/* select 1; */ select 1");
     ParsesOk("/** select 1; */ select 1");
     ParsesOk("/* select */ select 1 /* 1 */");
+    ParsesOk("select 1 /* sortby(() */");
+    // Empty columns list in sortby hint
+    ParserError("select 1 /*+ sortby() */");
+    // Mismatching parentheses
+    ParserError("select 1 /*+ sortby(() */");
+    ParserError("select 1 /*+ sortby(a) \n");
+    ParserError("select 1 --+ sortby(a) */\n from t");
   }
 
   /**
@@ -230,6 +237,9 @@ public class ParserTest extends FrontendTestBase {
     ParserError("-- baz /*\nselect 1*/");
     ParsesOk("select -- blah\n 1");
     ParsesOk("select -- select 1\n 1");
+    ParsesOk("select 1 -- sortby(()");
+    // Mismatching parentheses
+    ParserError("select 1 -- +sortby(()\n");
   }
 
   /**
@@ -241,10 +251,10 @@ public class ParserTest extends FrontendTestBase {
     SelectStmt selectStmt = (SelectStmt) ParsesOk(stmt);
     Preconditions.checkState(selectStmt.getTableRefs().size() > 1);
     List<String> actualHints = Lists.newArrayList();
-    assertEquals(null, selectStmt.getTableRefs().get(0).getJoinHints());
+    assertTrue(selectStmt.getTableRefs().get(0).getJoinHints().isEmpty());
     for (int i = 1; i < selectStmt.getTableRefs().size(); ++i) {
-      List<String> hints = selectStmt.getTableRefs().get(i).getJoinHints();
-      if (hints != null) actualHints.addAll(hints);
+      List<PlanHint> hints = selectStmt.getTableRefs().get(i).getJoinHints();
+      for (PlanHint hint: hints) actualHints.add(hint.toString());
     }
     if (actualHints.isEmpty()) actualHints = Lists.newArrayList((String) null);
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
@@ -255,8 +265,8 @@ public class ParserTest extends FrontendTestBase {
     Preconditions.checkState(selectStmt.getTableRefs().size() > 0);
     List<String> actualHints = Lists.newArrayList();
     for (int i = 0; i < selectStmt.getTableRefs().size(); ++i) {
-      List<String> hints = selectStmt.getTableRefs().get(i).getTableHints();
-      if (hints != null) actualHints.addAll(hints);
+      List<PlanHint> hints = selectStmt.getTableRefs().get(i).getTableHints();
+      for (PlanHint hint: hints) actualHints.add(hint.toString());
     }
     if (actualHints.isEmpty()) actualHints = Lists.newArrayList((String) null);
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
@@ -267,10 +277,10 @@ public class ParserTest extends FrontendTestBase {
     Preconditions.checkState(selectStmt.getTableRefs().size() > 0);
     List<String> actualHints = Lists.newArrayList();
     for (int i = 0; i < selectStmt.getTableRefs().size(); ++i) {
-      List<String> joinHints = selectStmt.getTableRefs().get(i).getJoinHints();
-      if (joinHints != null) actualHints.addAll(joinHints);
-      List<String> tableHints = selectStmt.getTableRefs().get(i).getTableHints();
-      if (tableHints != null) actualHints.addAll(tableHints);
+      List<PlanHint> joinHints = selectStmt.getTableRefs().get(i).getJoinHints();
+      for (PlanHint hint: joinHints) actualHints.add(hint.toString());
+      List<PlanHint> tableHints = selectStmt.getTableRefs().get(i).getTableHints();
+      for (PlanHint hint: tableHints) actualHints.add(hint.toString());
     }
     if (actualHints.isEmpty()) actualHints = Lists.newArrayList((String) null);
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
@@ -282,8 +292,10 @@ public class ParserTest extends FrontendTestBase {
    */
   private void TestSelectListHints(String stmt, String... expectedHints) {
     SelectStmt selectStmt = (SelectStmt) ParsesOk(stmt);
-    List<String> actualHints = selectStmt.getSelectList().getPlanHints();
-    if (actualHints == null) actualHints = Lists.newArrayList((String) null);
+    List<String> actualHints = Lists.newArrayList();
+    List<PlanHint> hints = selectStmt.getSelectList().getPlanHints();
+    for (PlanHint hint: hints) actualHints.add(hint.toString());
+    if (actualHints.isEmpty()) actualHints = Lists.newArrayList((String) null);
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
   }
 
@@ -292,8 +304,10 @@ public class ParserTest extends FrontendTestBase {
    */
   private void TestInsertHints(String stmt, String... expectedHints) {
     InsertStmt insertStmt = (InsertStmt) ParsesOk(stmt);
-    List<String> actualHints = insertStmt.getPlanHints();
-    if (actualHints == null) actualHints = Lists.newArrayList((String) null);
+    List<String> actualHints = Lists.newArrayList();
+    List<PlanHint> hints = insertStmt.getPlanHints();
+    for (PlanHint hint: hints) actualHints.add(hint.toString());
+    if (actualHints.isEmpty()) actualHints = Lists.newArrayList((String) null);
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
   }
 
@@ -407,22 +421,26 @@ public class ParserTest extends FrontendTestBase {
           suffix), "schedule_cache_local", "schedule_random_replica", "broadcast",
           "schedule_remote");
 
+      TestSelectListHints(String.format(
+          "select %sfoo,bar,baz%s * from functional.alltypes a", prefix, suffix),
+          "foo", "bar", "baz");
+
       // Test select-list hints (e.g., straight_join). The legacy-style hint has no
       // prefix and suffix.
-      if (prefix.contains("[")) {
-        prefix = "";
-        suffix = "";
-      }
-      TestSelectListHints(String.format(
-          "select %sstraight_join%s * from functional.alltypes a", prefix, suffix),
-          "straight_join");
-      // Only the new hint-style is recognized
-      if (!prefix.equals("")) {
+      {
+        String localPrefix = prefix;
+        String localSuffix = suffix;
+        if (prefix == "[") {
+          localPrefix = "";
+          localSuffix = "";
+        }
         TestSelectListHints(String.format(
-            "select %sfoo,bar,baz%s * from functional.alltypes a", prefix, suffix),
-            "foo", "bar", "baz");
+            "select %sstraight_join%s * from functional.alltypes a", localPrefix,
+            localSuffix), "straight_join");
       }
-      if (prefix.isEmpty()) continue;
+
+      // Below are tests for hints that are not supported by the legacy syntax.
+      if (prefix == "[") continue;
 
       // Test mixing commented hints and comments.
       for (String[] commentStyle: commentStyles) {
@@ -439,6 +457,22 @@ public class ParserTest extends FrontendTestBase {
         TestSelectListHints(query, "straight_join");
         TestJoinHints(query, "shuffle");
       }
+
+      // Tests for hints with arguments.
+      TestInsertHints(String.format(
+          "insert into t %ssortby(a)%s select * from t", prefix, suffix),
+          "sortby(a)");
+      TestInsertHints(String.format(
+          "insert into t %sclustered,shuffle,sortby(a)%s select * from t", prefix,
+          suffix), "clustered", "shuffle", "sortby(a)");
+      TestInsertHints(String.format(
+          "insert into t %ssortby(a,b)%s select * from t", prefix, suffix),
+          "sortby(a,b)");
+      TestInsertHints(String.format(
+          "insert into t %ssortby(a  , b)%s select * from t", prefix, suffix),
+          "sortby(a,b)");
+      ParserError(String.format(
+          "insert into t %ssortby(  a  ,  , ,,, b  )%s select * from t", prefix, suffix));
     }
     // No "+" at the beginning so the comment is not recognized as a hint.
     TestJoinHints("select * from functional.alltypes a join /* comment */" +

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index b5cf446..9514cc6 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -463,11 +463,11 @@ public class ToSqlTest extends FrontendTestBase {
       // Insert hint.
       testToSql(String.format(
           "insert into functional.alltypes(int_col, bool_col) " +
-          "partition(year, month) %snoshuffle%s " +
+          "partition(year, month) %snoshuffle,sortby(int_col)%s " +
           "select int_col, bool_col, year, month from functional.alltypes",
           prefix, suffix),
           "INSERT INTO TABLE functional.alltypes(int_col, bool_col) " +
-              "PARTITION (year, month) \n-- +noshuffle\n " +
+              "PARTITION (year, month) \n-- +noshuffle,sortby(int_col)\n " +
           "SELECT int_col, bool_col, year, month FROM functional.alltypes");
 
       // Table hint

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
index f201d0a..8e3adb9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
@@ -705,3 +705,76 @@ WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
 00:SCAN HDFS [functional.alltypesnopart]
    partitions=1/1 files=0 size=0B
 ====
+# IMPALA-4163: sortby hint in insert statement adds sort node.
+insert into table functional.alltypes partition(year, month)
+       /*+ sortby(int_col, bool_col),shuffle */
+       select * from functional.alltypes
+---- PLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+01:SORT
+|  order by: int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+02:SORT
+|  order by: int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+01:EXCHANGE [HASH(functional.alltypes.year,functional.alltypes.month)]
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+# IMPALA-4163: sortby hint in insert statement with noshuffle adds sort node.
+insert into table functional.alltypes partition(year, month)
+       /*+ sortby(int_col, bool_col),noshuffle */
+       select * from functional.alltypes
+---- PLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+01:SORT
+|  order by: int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+01:SORT
+|  order by: int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+# IMPALA-4163: sortby hint in insert statement adds ordering columns to clustering sort.
+insert into table functional.alltypes partition(year, month)
+       /*+ sortby(int_col, bool_col),clustered */
+       select * from functional.alltypes
+---- PLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+01:SORT
+|  order by: year DESC NULLS LAST, month DESC NULLS LAST, int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(year,month)]
+|  partitions=24
+|
+02:SORT
+|  order by: year DESC NULLS LAST, month DESC NULLS LAST, int_col DESC NULLS LAST, bool_col DESC NULLS LAST
+|
+01:EXCHANGE [HASH(functional.alltypes.year,functional.alltypes.month)]
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce9b332e/testdata/workloads/functional-query/queries/QueryTest/insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/insert.test b/testdata/workloads/functional-query/queries/QueryTest/insert.test
index 5601b35..9dfaf34 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/insert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert.test
@@ -945,3 +945,29 @@ RESET alltypesnopart_insert
 ---- RESULTS
 : 100
 ====
+---- QUERY
+# IMPALA-4163: insert into table sortby() plan hint
+insert into table alltypesinsert
+partition (year, month) /*+ clustered,shuffle,sortby(int_col, bool_col) */
+select * from alltypestiny;
+---- SETUP
+DROP PARTITIONS alltypesinsert
+RESET alltypesinsert
+---- RESULTS
+year=2009/month=1/: 2
+year=2009/month=2/: 2
+year=2009/month=3/: 2
+year=2009/month=4/: 2
+====
+---- QUERY
+# IMPALA-4163: insert into table sortby() plan hint
+insert into table alltypesnopart_insert
+/*+ clustered,shuffle,sortby(int_col, bool_col) */
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col,
+double_col, date_string_col, string_col, timestamp_col from alltypestiny;
+---- SETUP
+DROP PARTITIONS alltypesinsert
+RESET alltypesinsert
+---- RESULTS
+: 8
+====