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/10/22 05:33:28 UTC

[01/14] incubator-impala git commit: IMPALA-4329: Prevent crash in scheduler when no backends are registered

Repository: incubator-impala
Updated Branches:
  refs/heads/master bf1d9677f -> 48085274f


IMPALA-4329: Prevent crash in scheduler when no backends are registered

The scheduler crashed with a segmentation fault when there were no
backends registered: After not being able to find a local backend (none
are configured at all) in ComputeScanRangeAssignment(), the previous
code would eventually try to return the top of
assignment_ctx.assignment_heap in SelectRemoteBackendHost(), but that
heap would be empty. Subsequently, when using the IP address of that
heap node, a segmentation fault would occur.

This change adds a check and aborts scheduling with an error. It also
contains a test.

Change-Id: I6d93158f34841ea66dc3682290266262c87ea7ff
Reviewed-on: http://gerrit.cloudera.org:8080/4776
Reviewed-by: Dan Hecht <dh...@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/2fa1633e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/2fa1633e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/2fa1633e

Branch: refs/heads/master
Commit: 2fa1633e4090000e9018c012df81f291a0d7566e
Parents: bf1d967
Author: Lars Volker <lv...@cloudera.com>
Authored: Thu Oct 20 14:19:56 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 03:16:30 2016 +0000

----------------------------------------------------------------------
 be/src/scheduling/simple-scheduler-test-util.cc |  6 +++---
 be/src/scheduling/simple-scheduler-test-util.h  |  5 +++--
 be/src/scheduling/simple-scheduler-test.cc      | 19 +++++++++++++++++++
 be/src/scheduling/simple-scheduler.cc           |  8 +++++++-
 common/thrift/generate_error_codes.py           |  3 +++
 5 files changed, 35 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2fa1633e/be/src/scheduling/simple-scheduler-test-util.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler-test-util.cc b/be/src/scheduling/simple-scheduler-test-util.cc
index 3e14ea7..d3f5584 100644
--- a/be/src/scheduling/simple-scheduler-test-util.cc
+++ b/be/src/scheduling/simple-scheduler-test-util.cc
@@ -449,13 +449,13 @@ SchedulerWrapper::SchedulerWrapper(const Plan& plan)
   InitializeScheduler();
 }
 
-void SchedulerWrapper::Compute(bool exec_at_coord, Result* result) {
+Status SchedulerWrapper::Compute(bool exec_at_coord, Result* result) {
   DCHECK(scheduler_ != NULL);
 
   // Compute Assignment.
   FragmentScanRangeAssignment* assignment = result->AddAssignment();
-  scheduler_->ComputeScanRangeAssignment(*scheduler_->GetBackendConfig(), 0, NULL, false,
-      plan_.scan_range_locations(), plan_.referenced_datanodes(), exec_at_coord,
+  return scheduler_->ComputeScanRangeAssignment(*scheduler_->GetBackendConfig(), 0, NULL,
+      false, plan_.scan_range_locations(), plan_.referenced_datanodes(), exec_at_coord,
       plan_.query_options(), NULL, assignment);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2fa1633e/be/src/scheduling/simple-scheduler-test-util.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler-test-util.h b/be/src/scheduling/simple-scheduler-test-util.h
index 85bb1a5..ab46e2a 100644
--- a/be/src/scheduling/simple-scheduler-test-util.h
+++ b/be/src/scheduling/simple-scheduler-test-util.h
@@ -21,6 +21,7 @@
 
 #include <boost/scoped_ptr.hpp>
 
+#include "common/status.h"
 #include "gen-cpp/ImpalaInternalService.h" // for TQueryOptions
 #include "scheduling/query-schedule.h"
 #include "util/metrics.h"
@@ -421,10 +422,10 @@ class SchedulerWrapper {
   SchedulerWrapper(const Plan& plan);
 
   /// Call ComputeScanRangeAssignment() with exec_at_coord set to false.
-  void Compute(Result* result) { Compute(false, result); }
+  Status Compute(Result* result) { return Compute(false, result); }
 
   /// Call ComputeScanRangeAssignment().
-  void Compute(bool exec_at_coord, Result* result);
+  Status Compute(bool exec_at_coord, Result* result);
 
   /// Reset the state of the scheduler by re-creating and initializing it.
   void Reset() { InitializeScheduler(); }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2fa1633e/be/src/scheduling/simple-scheduler-test.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler-test.cc b/be/src/scheduling/simple-scheduler-test.cc
index d9964af..5743ff2 100644
--- a/be/src/scheduling/simple-scheduler-test.cc
+++ b/be/src/scheduling/simple-scheduler-test.cc
@@ -361,6 +361,25 @@ TEST_F(SchedulerTest, TestSendUpdates) {
   EXPECT_EQ(0, result.NumDiskAssignedBytes(1));
 }
 
+/// IMPALA-4329: Test scheduling with no backends.
+TEST_F(SchedulerTest, TestEmptyBackendConfig) {
+  Cluster cluster;
+  cluster.AddHost(false, true);
+
+  Schema schema(cluster);
+  schema.AddMultiBlockTable("T", 1, ReplicaPlacement::REMOTE_ONLY, 1);
+
+  Plan plan(schema);
+  plan.AddTableScan("T");
+
+  Result result(plan);
+  SchedulerWrapper scheduler(plan);
+  Status status = scheduler.Compute(&result);
+  EXPECT_TRUE(!status.ok());
+  EXPECT_EQ(
+      status.GetDetail(), "Cannot schedule query: no registered backends available.\n");
+}
+
 }  // end namespace impala
 
 IMPALA_TEST_MAIN();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2fa1633e/be/src/scheduling/simple-scheduler.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler.cc b/be/src/scheduling/simple-scheduler.cc
index 9b52d5a..bc500c8 100644
--- a/be/src/scheduling/simple-scheduler.cc
+++ b/be/src/scheduling/simple-scheduler.cc
@@ -537,6 +537,10 @@ Status SimpleScheduler::ComputeScanRangeAssignment(
     const vector<TNetworkAddress>& host_list, bool exec_at_coord,
     const TQueryOptions& query_options, RuntimeProfile::Counter* timer,
     FragmentScanRangeAssignment* assignment) {
+  if (backend_config.NumBackends() == 0) {
+    return Status(TErrorCode::NO_REGISTERED_BACKENDS);
+  }
+
   SCOPED_TIMER(timer);
   // We adjust all replicas with memory distance less than base_distance to base_distance
   // and collect all replicas with equal or better distance as candidates. For a full list
@@ -917,6 +921,7 @@ SimpleScheduler::AssignmentCtx::AssignmentCtx(
   : backend_config_(backend_config), first_unused_backend_idx_(0),
     total_assignments_(total_assignments),
     total_local_assignments_(total_local_assignments) {
+  DCHECK_GT(backend_config.NumBackends(), 0);
   backend_config.GetAllBackendIps(&random_backend_order_);
   std::mt19937 g(rand());
   std::shuffle(random_backend_order_.begin(), random_backend_order_.end(), g);
@@ -965,7 +970,8 @@ const IpAddr* SimpleScheduler::AssignmentCtx::SelectRemoteBackendHost() {
   } else {
     // Pick next backend from assignment_heap. All backends must have been inserted into
     // the heap at this point.
-    DCHECK(backend_config_.NumBackends() == assignment_heap_.size());
+    DCHECK_GT(backend_config_.NumBackends(), 0);
+    DCHECK_EQ(backend_config_.NumBackends(), assignment_heap_.size());
     candidate_ip = &(assignment_heap_.top().ip);
   }
   DCHECK(candidate_ip != NULL);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2fa1633e/common/thrift/generate_error_codes.py
----------------------------------------------------------------------
diff --git a/common/thrift/generate_error_codes.py b/common/thrift/generate_error_codes.py
index 131947a..ae338d5 100755
--- a/common/thrift/generate_error_codes.py
+++ b/common/thrift/generate_error_codes.py
@@ -289,6 +289,9 @@ error_codes = (
 
   ("PARQUET_ZERO_ROWS_IN_NON_EMPTY_FILE", 93, "File '$0' is corrupt: metadata indicates "
    "a zero row count but there is at least one non-empty row group."),
+
+  ("NO_REGISTERED_BACKENDS", 94, "Cannot schedule query: no registered backends "
+   "available."),
 )
 
 import sys


[07/14] incubator-impala git commit: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
index 3acb1a3..73173cb 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
@@ -17,23 +17,12 @@
 
 package org.apache.impala.analysis;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
-import org.apache.avro.Schema;
-import org.apache.avro.SchemaParseException;
-import org.apache.hadoop.fs.permission.FsAction;
-
-import org.apache.impala.authorization.Privilege;
-import org.apache.impala.catalog.HdfsStorageDescriptor;
 import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.RowFormat;
 import org.apache.impala.common.AnalysisException;
-import org.apache.impala.common.FileSystemUtil;
-import org.apache.impala.thrift.TAccessEvent;
-import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TCreateTableParams;
 import org.apache.impala.thrift.THdfsFileFormat;
 import org.apache.impala.thrift.TTableName;
@@ -41,113 +30,80 @@ import org.apache.impala.util.AvroSchemaConverter;
 import org.apache.impala.util.AvroSchemaParser;
 import org.apache.impala.util.AvroSchemaUtils;
 import org.apache.impala.util.KuduUtil;
-import org.apache.impala.util.MetaStoreUtil;
+
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
+import com.google.common.primitives.Ints;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaParseException;
 
 /**
  * Represents a CREATE TABLE statement.
  */
 public class CreateTableStmt extends StatementBase {
-  private List<ColumnDef> columnDefs_;
-  private final String comment_;
-  private final boolean isExternal_;
-  private final boolean ifNotExists_;
-  private final THdfsFileFormat fileFormat_;
-  private final ArrayList<ColumnDef> partitionColDefs_;
-  private final RowFormat rowFormat_;
-  private TableName tableName_;
-  private final Map<String, String> tblProperties_;
-  private final Map<String, String> serdeProperties_;
-  private final HdfsCachingOp cachingOp_;
-  private HdfsUri location_;
-  private final List<DistributeParam> distributeParams_;
-
-  // Set during analysis
+
+  @VisibleForTesting
+  final static String KUDU_STORAGE_HANDLER_ERROR_MESSAGE = "Kudu tables must be"
+      + " specified using 'STORED AS KUDU' without using the storage handler table"
+      + " property.";
+
+  // Table parameters specified in a CREATE TABLE statement
+  private final TableDef tableDef_;
+
+  // Table owner. Set during analysis
   private String owner_;
 
-  /**
-   * Builds a CREATE TABLE statement
-   * @param tableName - Name of the new table
-   * @param columnDefs - List of column definitions for the table
-   * @param partitionColumnDefs - List of partition column definitions for the table
-   * @param isExternal - If true, the table's data will be preserved if dropped.
-   * @param comment - Comment to attach to the table
-   * @param rowFormat - Custom row format of the table. Use RowFormat.DEFAULT_ROW_FORMAT
-   *          to specify default row format.
-   * @param fileFormat - File format of the table
-   * @param location - The HDFS location of where the table data will stored.
-   * @param cachingOp - The HDFS caching op that should be applied to this table.
-   * @param ifNotExists - If true, no errors are thrown if the table already exists.
-   * @param tblProperties - Optional map of key/values to persist with table metadata.
-   * @param serdeProperties - Optional map of key/values to persist with table serde
-   *                          metadata.
-   */
-  public CreateTableStmt(TableName tableName, List<ColumnDef> columnDefs,
-      List<ColumnDef> partitionColumnDefs, boolean isExternal, String comment,
-      RowFormat rowFormat, THdfsFileFormat fileFormat, HdfsUri location,
-      HdfsCachingOp cachingOp, boolean ifNotExists, Map<String, String> tblProperties,
-      Map<String, String> serdeProperties, List<DistributeParam> distributeParams) {
-    Preconditions.checkNotNull(columnDefs);
-    Preconditions.checkNotNull(partitionColumnDefs);
-    Preconditions.checkNotNull(fileFormat);
-    Preconditions.checkNotNull(rowFormat);
-    Preconditions.checkNotNull(tableName);
-
-    columnDefs_ = Lists.newArrayList(columnDefs);
-    comment_ = comment;
-    isExternal_ = isExternal;
-    ifNotExists_ = ifNotExists;
-    fileFormat_ = fileFormat;
-    location_ = location;
-    cachingOp_ = cachingOp;
-    partitionColDefs_ = Lists.newArrayList(partitionColumnDefs);
-    rowFormat_ = rowFormat;
-    tableName_ = tableName;
-    tblProperties_ = tblProperties;
-    serdeProperties_ = serdeProperties;
-    unescapeProperties(tblProperties_);
-    unescapeProperties(serdeProperties_);
-    distributeParams_ = distributeParams;
+  public CreateTableStmt(TableDef tableDef) {
+    Preconditions.checkNotNull(tableDef);
+    tableDef_ = tableDef;
   }
 
   /**
    * Copy c'tor.
    */
-  public CreateTableStmt(CreateTableStmt other) {
-    columnDefs_ = Lists.newArrayList(other.columnDefs_);
-    comment_ = other.comment_;
-    isExternal_ = other.isExternal_;
-    ifNotExists_ = other.ifNotExists_;
-    fileFormat_ = other.fileFormat_;
-    location_ = other.location_;
-    cachingOp_ = other.cachingOp_;
-    partitionColDefs_ = Lists.newArrayList(other.partitionColDefs_);
-    rowFormat_ = other.rowFormat_;
-    tableName_ = other.tableName_;
-    tblProperties_ = other.tblProperties_;
-    serdeProperties_ = other.serdeProperties_;
-    distributeParams_ = other.distributeParams_;
+  CreateTableStmt(CreateTableStmt other) {
+    this(other.tableDef_);
+    owner_ = other.owner_;
   }
 
   @Override
   public CreateTableStmt clone() { return new CreateTableStmt(this); }
 
-  public String getTbl() { return tableName_.getTbl(); }
-  public TableName getTblName() { return tableName_; }
-  public List<ColumnDef> getColumnDefs() { return columnDefs_; }
-  public List<ColumnDef> getPartitionColumnDefs() { return partitionColDefs_; }
-  public String getComment() { return comment_; }
-  public boolean isExternal() { return isExternal_; }
-  public boolean getIfNotExists() { return ifNotExists_; }
-  public HdfsUri getLocation() { return location_; }
-  public void setLocation(HdfsUri location) { this.location_ = location; }
-  public THdfsFileFormat getFileFormat() { return fileFormat_; }
-  public RowFormat getRowFormat() { return rowFormat_; }
-  public Map<String, String> getTblProperties() { return tblProperties_; }
-  public Map<String, String> getSerdeProperties() { return serdeProperties_; }
+  public String getTbl() { return getTblName().getTbl(); }
+  public TableName getTblName() { return tableDef_.getTblName(); }
+  public boolean getIfNotExists() { return tableDef_.getIfNotExists(); }
+  public List<ColumnDef> getColumnDefs() { return tableDef_.getColumnDefs(); }
+  private void setColumnDefs(List<ColumnDef> colDefs) {
+    getColumnDefs().clear();
+    getColumnDefs().addAll(colDefs);
+  }
+  private List<ColumnDef> getPrimaryKeyColumnDefs() {
+    return tableDef_.getPrimaryKeyColumnDefs();
+  }
+  public boolean isExternal() { return tableDef_.isExternal(); }
+  public List<ColumnDef> getPartitionColumnDefs() {
+    return tableDef_.getPartitionColumnDefs();
+  }
+  public List<DistributeParam> getDistributeParams() {
+    return tableDef_.getDistributeParams();
+  }
+  public String getComment() { return tableDef_.getComment(); }
+  Map<String, String> getTblProperties() { return tableDef_.getTblProperties(); }
+  private HdfsCachingOp getCachingOp() { return tableDef_.getCachingOp(); }
+  public HdfsUri getLocation() { return tableDef_.getLocation(); }
+  Map<String, String> getSerdeProperties() { return tableDef_.getSerdeProperties(); }
+  public THdfsFileFormat getFileFormat() { return tableDef_.getFileFormat(); }
+  RowFormat getRowFormat() { return tableDef_.getRowFormat(); }
+
+  // Only exposed for ToSqlUtils. Returns the list of primary keys declared by the user
+  // at the table level. Note that primary keys may also be declared in column
+  // definitions, those are not included here (they are stored in the ColumnDefs).
+  List<String> getTblPrimaryKeyColumnNames() {
+    return tableDef_.getPrimaryKeyColumnNames();
+  }
 
   /**
    * Can only be called after analysis, returns the owner of this table (the user from
@@ -164,7 +120,7 @@ public class CreateTableStmt extends StatementBase {
    */
   public String getDb() {
     Preconditions.checkState(isAnalyzed());
-    return tableName_.getDb();
+    return getTblName().getDb();
   }
 
   @Override
@@ -173,240 +129,246 @@ public class CreateTableStmt extends StatementBase {
   public TCreateTableParams toThrift() {
     TCreateTableParams params = new TCreateTableParams();
     params.setTable_name(new TTableName(getDb(), getTbl()));
-    for (ColumnDef col: getColumnDefs()) {
-      params.addToColumns(col.toThrift());
-    }
+    List<org.apache.impala.thrift.TColumn> tColumns = Lists.newArrayList();
+    for (ColumnDef col: getColumnDefs()) tColumns.add(col.toThrift());
+    params.setColumns(tColumns);
     for (ColumnDef col: getPartitionColumnDefs()) {
       params.addToPartition_columns(col.toThrift());
     }
     params.setOwner(getOwner());
     params.setIs_external(isExternal());
-    params.setComment(comment_);
-    params.setLocation(location_ == null ? null : location_.toString());
-    if (cachingOp_ != null) params.setCache_op(cachingOp_.toThrift());
-    params.setRow_format(rowFormat_.toThrift());
-    params.setFile_format(fileFormat_);
+    params.setComment(getComment());
+    params.setLocation(getLocation() == null ? null : getLocation().toString());
+    if (getCachingOp() != null) params.setCache_op(getCachingOp().toThrift());
+    if (getRowFormat() != null) params.setRow_format(getRowFormat().toThrift());
+    params.setFile_format(getFileFormat());
     params.setIf_not_exists(getIfNotExists());
-    if (tblProperties_ != null) params.setTable_properties(tblProperties_);
-    if (serdeProperties_ != null) params.setSerde_properties(serdeProperties_);
-    if (distributeParams_ != null) {
-      for (DistributeParam d : distributeParams_) {
-        params.addToDistribute_by(d.toThrift());
-      }
+    params.setTable_properties(getTblProperties());
+    params.setSerde_properties(getSerdeProperties());
+    for (DistributeParam d: getDistributeParams()) {
+      params.addToDistribute_by(d.toThrift());
+    }
+    for (ColumnDef pkColDef: getPrimaryKeyColumnDefs()) {
+      params.addToPrimary_key_column_names(pkColDef.getColName());
     }
+
     return params;
   }
 
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
     super.analyze(analyzer);
-    Preconditions.checkState(tableName_ != null && !tableName_.isEmpty());
-    tableName_ = analyzer.getFqTableName(tableName_);
-    tableName_.analyze();
     owner_ = analyzer.getUser().getName();
-
-    MetaStoreUtil.checkShortPropertyMap("Property", tblProperties_);
-    MetaStoreUtil.checkShortPropertyMap("Serde property", serdeProperties_);
-
-    if (analyzer.dbContainsTable(tableName_.getDb(), tableName_.getTbl(),
-        Privilege.CREATE) && !ifNotExists_) {
-      throw new AnalysisException(Analyzer.TBL_ALREADY_EXISTS_ERROR_MSG + tableName_);
-    }
-
-    analyzer.addAccessEvent(new TAccessEvent(tableName_.toString(),
-        TCatalogObjectType.TABLE, Privilege.CREATE.toString()));
-
-    // Only Avro tables can have empty column defs because they can infer them from
-    // the Avro schema.
-    if (columnDefs_.isEmpty() && fileFormat_ != THdfsFileFormat.AVRO) {
+    tableDef_.analyze(analyzer);
+    analyzeKuduFormat(analyzer);
+    // Avro tables can have empty column defs because they can infer them from the Avro
+    // schema. Likewise for external Kudu tables, the schema can be read from Kudu.
+    if (getColumnDefs().isEmpty() && getFileFormat() != THdfsFileFormat.AVRO
+        && getFileFormat() != THdfsFileFormat.KUDU) {
       throw new AnalysisException("Table requires at least 1 column");
     }
-
-    if (location_ != null) {
-      location_.analyze(analyzer, Privilege.ALL, FsAction.READ_WRITE);
-    }
-
-    analyzeRowFormat(analyzer);
-
-    // Check that all the column names are valid and unique.
-    analyzeColumnDefs(analyzer);
-
-    if (getTblProperties() != null && KuduTable.KUDU_STORAGE_HANDLER.equals(
-        getTblProperties().get(KuduTable.KEY_STORAGE_HANDLER))) {
-      analyzeKuduTable(analyzer);
-    } else if (distributeParams_ != null) {
-      throw new AnalysisException("Only Kudu tables can use DISTRIBUTE BY clause.");
-    }
-
-    if (fileFormat_ == THdfsFileFormat.AVRO) {
-      columnDefs_ = analyzeAvroSchema(analyzer);
-      if (columnDefs_.isEmpty()) {
+    if (getFileFormat() == THdfsFileFormat.AVRO) {
+      setColumnDefs(analyzeAvroSchema(analyzer));
+      if (getColumnDefs().isEmpty()) {
         throw new AnalysisException(
             "An Avro table requires column definitions or an Avro schema.");
       }
-      AvroSchemaUtils.setFromSerdeComment(columnDefs_);
-      analyzeColumnDefs(analyzer);
+      AvroSchemaUtils.setFromSerdeComment(getColumnDefs());
     }
+  }
 
-    if (cachingOp_ != null) {
-      cachingOp_.analyze(analyzer);
-      if (cachingOp_.shouldCache() && location_ != null &&
-          !FileSystemUtil.isPathCacheable(location_.getPath())) {
-        throw new AnalysisException(String.format("Location '%s' cannot be cached. " +
-            "Please retry without caching: CREATE TABLE %s ... UNCACHED",
-            location_.toString(), tableName_));
+  /**
+   * Analyzes the parameters of a CREATE TABLE ... STORED AS KUDU statement. Also checks
+   * if Kudu specific properties and parameters are specified for non-Kudu tables.
+   */
+  private void analyzeKuduFormat(Analyzer analyzer) throws AnalysisException {
+    if (getFileFormat() != THdfsFileFormat.KUDU) {
+      if (KuduTable.KUDU_STORAGE_HANDLER.equals(
+          getTblProperties().get(KuduTable.KEY_STORAGE_HANDLER))) {
+        throw new AnalysisException(KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+      }
+      AnalysisUtils.throwIfNotEmpty(getDistributeParams(),
+          "Only Kudu tables can use the DISTRIBUTE BY clause.");
+      if (hasPrimaryKey()) {
+        throw new AnalysisException("Only Kudu tables can specify a PRIMARY KEY.");
       }
+      return;
     }
 
-    // Analyze 'skip.header.line.format' property.
-    if (tblProperties_ != null) {
-      AlterTableSetTblProperties.analyzeSkipHeaderLineCount(tblProperties_);
+    analyzeKuduTableProperties(analyzer);
+    if (isExternal()) {
+      analyzeExternalKuduTableParams();
+    } else {
+      analyzeManagedKuduTableParams(analyzer);
     }
   }
 
-  private void analyzeRowFormat(Analyzer analyzer) throws AnalysisException {
-    Byte fieldDelim = analyzeRowFormatValue(rowFormat_.getFieldDelimiter());
-    Byte lineDelim = analyzeRowFormatValue(rowFormat_.getLineDelimiter());
-    Byte escapeChar = analyzeRowFormatValue(rowFormat_.getEscapeChar());
-    if (fileFormat_ == THdfsFileFormat.TEXT) {
-      if (fieldDelim == null) fieldDelim = HdfsStorageDescriptor.DEFAULT_FIELD_DELIM;
-      if (lineDelim == null) lineDelim = HdfsStorageDescriptor.DEFAULT_LINE_DELIM;
-      if (escapeChar == null) escapeChar = HdfsStorageDescriptor.DEFAULT_ESCAPE_CHAR;
-      if (fieldDelim != null && lineDelim != null && fieldDelim.equals(lineDelim)) {
-        throw new AnalysisException("Field delimiter and line delimiter have same " +
-            "value: byte " + fieldDelim);
-      }
-      if (fieldDelim != null && escapeChar != null && fieldDelim.equals(escapeChar)) {
-        analyzer.addWarning("Field delimiter and escape character have same value: " +
-            "byte " + fieldDelim + ". Escape character will be ignored");
-      }
-      if (lineDelim != null && escapeChar != null && lineDelim.equals(escapeChar)) {
-        analyzer.addWarning("Line delimiter and escape character have same value: " +
-            "byte " + lineDelim + ". Escape character will be ignored");
+  /**
+   * Analyzes and checks table properties which are common to both managed and external
+   * Kudu tables.
+   */
+  private void analyzeKuduTableProperties(Analyzer analyzer) throws AnalysisException {
+    if (getTblProperties().containsKey(KuduTable.KEY_STORAGE_HANDLER)) {
+      throw new AnalysisException(KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    }
+    getTblProperties().put(KuduTable.KEY_STORAGE_HANDLER, KuduTable.KUDU_STORAGE_HANDLER);
+
+    String masterHosts = getTblProperties().get(KuduTable.KEY_MASTER_HOSTS);
+    if (Strings.isNullOrEmpty(masterHosts)) {
+      masterHosts = analyzer.getCatalog().getDefaultKuduMasterHosts();
+      if (masterHosts.isEmpty()) {
+        throw new AnalysisException(String.format(
+            "Table property '%s' is required when the impalad startup flag " +
+            "-kudu_master_hosts is not used.", KuduTable.KEY_MASTER_HOSTS));
       }
+      getTblProperties().put(KuduTable.KEY_MASTER_HOSTS, masterHosts);
     }
+
+    // TODO: Find out what is creating a directory in HDFS and stop doing that. Kudu
+    //       tables shouldn't have HDFS dirs.
+    //       https://issues.cloudera.org/browse/IMPALA-3570
+    AnalysisUtils.throwIfNotNull(getCachingOp(),
+        "A Kudu table cannot be cached in HDFS.");
+    AnalysisUtils.throwIfNotNull(getLocation(), "LOCATION cannot be specified for a " +
+        "Kudu table.");
+    AnalysisUtils.throwIfNotEmpty(tableDef_.getPartitionColumnDefs(),
+        "PARTITIONED BY cannot be used in Kudu tables.");
   }
 
   /**
-   * Analyzes columnDefs_ and partitionColDefs_ checking whether all column
-   * names are unique.
+   * Analyzes and checks parameters specified for external Kudu tables.
    */
-  private void analyzeColumnDefs(Analyzer analyzer) throws AnalysisException {
-    Set<String> colNames = Sets.newHashSet();
-    for (ColumnDef colDef: columnDefs_) {
-      colDef.analyze();
-      if (!colNames.add(colDef.getColName().toLowerCase())) {
-        throw new AnalysisException("Duplicate column name: " + colDef.getColName());
-      }
+  private void analyzeExternalKuduTableParams() throws AnalysisException {
+    AnalysisUtils.throwIfNull(getTblProperties().get(KuduTable.KEY_TABLE_NAME),
+        String.format("Table property %s must be specified when creating " +
+            "an external Kudu table.", KuduTable.KEY_TABLE_NAME));
+    if (hasPrimaryKey()
+        || getTblProperties().containsKey(KuduTable.KEY_KEY_COLUMNS)) {
+      throw new AnalysisException("Primary keys cannot be specified for an external " +
+          "Kudu table");
     }
-    for (ColumnDef colDef: partitionColDefs_) {
-      colDef.analyze();
-      if (!colDef.getType().supportsTablePartitioning()) {
-        throw new AnalysisException(
-            String.format("Type '%s' is not supported as partition-column type " +
-                "in column: %s", colDef.getType().toSql(), colDef.getColName()));
+    AnalysisUtils.throwIfNotNull(getTblProperties().get(KuduTable.KEY_TABLET_REPLICAS),
+        String.format("Table property '%s' cannot be used with an external Kudu table.",
+            KuduTable.KEY_TABLET_REPLICAS));
+    AnalysisUtils.throwIfNotEmpty(getColumnDefs(),
+        "Columns cannot be specified with an external Kudu table.");
+    AnalysisUtils.throwIfNotEmpty(getDistributeParams(),
+        "DISTRIBUTE BY cannot be used with an external Kudu table.");
+  }
+
+  /**
+   * Analyzes and checks parameters specified for managed Kudu tables.
+   */
+  private void analyzeManagedKuduTableParams(Analyzer analyzer) throws AnalysisException {
+    // If no Kudu table name is specified in tblproperties, generate one using the
+    // current database as a prefix to avoid conflicts in Kudu.
+    if (!getTblProperties().containsKey(KuduTable.KEY_TABLE_NAME)) {
+      getTblProperties().put(KuduTable.KEY_TABLE_NAME,
+          KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));
+    }
+    AnalysisUtils.throwIfNotNull(getTblProperties().get(KuduTable.KEY_KEY_COLUMNS),
+        String.format("PRIMARY KEY must be used instead of the table property '%s'.",
+            KuduTable.KEY_KEY_COLUMNS));
+    if (!hasPrimaryKey()) {
+      throw new AnalysisException("A primary key is required for a Kudu table.");
+    }
+    String tabletReplicas = getTblProperties().get(KuduTable.KEY_TABLET_REPLICAS);
+    if (tabletReplicas != null) {
+      Integer r = Ints.tryParse(tabletReplicas);
+      if (r == null) {
+        throw new AnalysisException(String.format(
+            "Table property '%s' must be an integer.", KuduTable.KEY_TABLET_REPLICAS));
+      }
+      if (r <= 0) {
+        throw new AnalysisException("Number of tablet replicas must be greater than " +
+            "zero. Given number of replicas is: " + r.toString());
       }
-      if (!colNames.add(colDef.getColName().toLowerCase())) {
-        throw new AnalysisException("Duplicate column name: " + colDef.getColName());
+    }
+
+    if (!getDistributeParams().isEmpty()) {
+      analyzeDistributeParams(analyzer);
+    } else {
+      throw new AnalysisException("Table distribution must be specified for " +
+          "managed Kudu tables.");
+    }
+  }
+
+  /**
+   * Analyzes the distribution schemes specified in the CREATE TABLE statement.
+   */
+  private void analyzeDistributeParams(Analyzer analyzer) throws AnalysisException {
+    Preconditions.checkState(getFileFormat() == THdfsFileFormat.KUDU);
+    Map<String, ColumnDef> pkColDefsByName =
+        ColumnDef.mapByColumnNames(getPrimaryKeyColumnDefs());
+    for (DistributeParam distributeParam: getDistributeParams()) {
+      // If no column names were specified in this distribution scheme, use all the
+      // primary key columns.
+      if (!distributeParam.hasColumnNames()) {
+        distributeParam.setColumnNames(pkColDefsByName.keySet());
       }
+      distributeParam.setPkColumnDefMap(pkColDefsByName);
+      distributeParam.analyze(analyzer);
     }
   }
 
   /**
-   * Analyzes the Avro schema and compares it with the columnDefs_ to detect
+   * Checks if a primary key is specified in a CREATE TABLE stmt. Should only be called
+   * after tableDef_ has been analyzed.
+   */
+  private boolean hasPrimaryKey() {
+    Preconditions.checkState(tableDef_.isAnalyzed());
+    return !tableDef_.getPrimaryKeyColumnDefs().isEmpty();
+  }
+
+  /**
+   * Analyzes the Avro schema and compares it with the getColumnDefs() to detect
    * inconsistencies. Returns a list of column descriptors that should be
-   * used for creating the table (possibly identical to columnDefs_).
+   * used for creating the table (possibly identical to getColumnDefs()).
    */
-  private List<ColumnDef> analyzeAvroSchema(Analyzer analyzer)
-      throws AnalysisException {
-    Preconditions.checkState(fileFormat_ == THdfsFileFormat.AVRO);
+  private List<ColumnDef> analyzeAvroSchema(Analyzer analyzer) throws AnalysisException {
+    Preconditions.checkState(getFileFormat() == THdfsFileFormat.AVRO);
     // Look for the schema in TBLPROPERTIES and in SERDEPROPERTIES, with latter
     // taking precedence.
     List<Map<String, String>> schemaSearchLocations = Lists.newArrayList();
-    schemaSearchLocations.add(serdeProperties_);
-    schemaSearchLocations.add(tblProperties_);
-    String avroSchema = null;
-    List<ColumnDef> avroCols = null; // parsed from avroSchema
+    schemaSearchLocations.add(getSerdeProperties());
+    schemaSearchLocations.add(getTblProperties());
+    String avroSchema;
+    List<ColumnDef> avroCols; // parsed from avroSchema
     try {
       avroSchema = AvroSchemaUtils.getAvroSchema(schemaSearchLocations);
       if (avroSchema == null) {
         // No Avro schema was explicitly set in the serde or table properties, so infer
         // the Avro schema from the column definitions.
         Schema inferredSchema = AvroSchemaConverter.convertColumnDefs(
-            columnDefs_, tableName_.toString());
+            getColumnDefs(), getTblName().toString());
         avroSchema = inferredSchema.toString();
       }
       if (Strings.isNullOrEmpty(avroSchema)) {
         throw new AnalysisException("Avro schema is null or empty: " +
-            tableName_.toString());
+            getTblName().toString());
       }
       avroCols = AvroSchemaParser.parse(avroSchema);
     } catch (SchemaParseException e) {
       throw new AnalysisException(String.format(
-          "Error parsing Avro schema for table '%s': %s", tableName_.toString(),
+          "Error parsing Avro schema for table '%s': %s", getTblName().toString(),
           e.getMessage()));
     }
     Preconditions.checkNotNull(avroCols);
 
-    // Analyze the Avro schema to detect inconsistencies with the columnDefs_.
+    // Analyze the Avro schema to detect inconsistencies with the getColumnDefs().
     // In case of inconsistencies, the column defs are ignored in favor of the Avro
     // schema for simplicity and, in particular, to enable COMPUTE STATS (IMPALA-1104).
     StringBuilder warning = new StringBuilder();
     List<ColumnDef> reconciledColDefs =
-        AvroSchemaUtils.reconcileSchemas(columnDefs_, avroCols, warning);
+        AvroSchemaUtils.reconcileSchemas(getColumnDefs(), avroCols, warning);
     if (warning.length() > 0) analyzer.addWarning(warning.toString());
     return reconciledColDefs;
   }
 
-  private void analyzeKuduTable(Analyzer analyzer) throws AnalysisException {
-    // Validate that Kudu table is correctly specified.
-    if (!KuduTable.tableParamsAreValid(getTblProperties())) {
-      throw new AnalysisException("Kudu table is missing parameters " +
-          String.format("in table properties. Please verify if %s, %s, and %s are "
-                  + "present and have valid values.",
-              KuduTable.KEY_TABLE_NAME, KuduTable.KEY_MASTER_ADDRESSES,
-              KuduTable.KEY_KEY_COLUMNS));
-    }
-
-    // Kudu table cannot be a cached table
-    if (cachingOp_ != null) {
-      throw new AnalysisException("A Kudu table cannot be cached in HDFS.");
-    }
-
-    if (distributeParams_ != null) {
-      if (isExternal_) {
-        throw new AnalysisException(
-            "The DISTRIBUTE BY clause may not be specified for external tables.");
-      }
-
-      List<String> keyColumns = KuduUtil.parseKeyColumnsAsList(
-          getTblProperties().get(KuduTable.KEY_KEY_COLUMNS));
-      for (DistributeParam d : distributeParams_) {
-        // If the columns are not set, default to all key columns
-        if (d.getColumns() == null) d.setColumns(keyColumns);
-        d.analyze(analyzer);
-      }
-    } else if (!isExternal_) {
-      throw new AnalysisException(
-          "A data distribution must be specified using the DISTRIBUTE BY clause.");
-    }
-  }
-
-  private Byte analyzeRowFormatValue(String value) throws AnalysisException {
-    if (value == null) return null;
-    Byte byteVal = HdfsStorageDescriptor.parseDelim(value);
-    if (byteVal == null) {
-      throw new AnalysisException("ESCAPED BY values and LINE/FIELD " +
-          "terminators must be specified as a single character or as a decimal " +
-          "value in the range [-128:127]: " + value);
-    }
-    return byteVal;
-  }
-
   /**
    * Unescapes all values in the property map.
    */
-  public static void unescapeProperties(Map<String, String> propertyMap) {
+  static void unescapeProperties(Map<String, String> propertyMap) {
     if (propertyMap == null) return;
     for (Map.Entry<String, String> kv : propertyMap.entrySet()) {
       propertyMap.put(kv.getKey(),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java b/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
index 319fe50..34bed86 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
@@ -17,19 +17,20 @@
 
 package org.apache.impala.analysis;
 
-import java.math.BigDecimal;
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TDistributeByHashParam;
 import org.apache.impala.thrift.TDistributeByRangeParam;
 import org.apache.impala.thrift.TDistributeParam;
-import org.apache.impala.thrift.TDistributeType;
 import org.apache.impala.thrift.TRangeLiteral;
 import org.apache.impala.thrift.TRangeLiteralList;
+import org.apache.impala.util.KuduUtil;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 
 /**
@@ -50,16 +51,16 @@ public class DistributeParam implements ParseNode {
   /**
    * Creates a DistributeParam partitioned by hash.
    */
-  public static DistributeParam createHashParam(List<String> cols, BigDecimal buckets) {
-    return new DistributeParam(Type.HASH, cols, buckets);
+  public static DistributeParam createHashParam(List<String> cols, int buckets) {
+    return new DistributeParam(Type.HASH, cols, buckets, null);
   }
 
   /**
    * Creates a DistributeParam partitioned by range.
    */
   public static DistributeParam createRangeParam(List<String> cols,
-      ArrayList<ArrayList<LiteralExpr>> splitRows) {
-    return new DistributeParam(Type.RANGE, cols, splitRows);
+      List<List<LiteralExpr>> splitRows) {
+    return new DistributeParam(Type.RANGE, cols, NO_BUCKETS, splitRows);
   }
 
   private static final int NO_BUCKETS = -1;
@@ -69,131 +70,159 @@ public class DistributeParam implements ParseNode {
    */
   public enum Type {
     HASH, RANGE
-  };
+  }
+
+  // May be empty indicating that all keys in the table should be used.
+  private final List<String> colNames_ = Lists.newArrayList();
 
-  private List<String> columns_;
+  // Map of primary key column names to the associated column definitions. Must be set
+  // before the call to analyze().
+  private Map<String, ColumnDef> pkColumnDefByName_;
 
+  // Distribution type
   private final Type type_;
 
   // Only relevant for hash partitioning, -1 otherwise
-  private final int num_buckets_;
+  private final int numBuckets_;
 
   // Only relevant for range partitioning, null otherwise
-  private final ArrayList<ArrayList<LiteralExpr>> splitRows_;
-
-  // Set in analyze()
-  private TDistributeByRangeParam rangeParam_;
-
-  private DistributeParam(Type t, List<String> cols, BigDecimal buckets) {
-    type_ = t;
-    columns_ = cols;
-    num_buckets_ = buckets.intValue();
-    splitRows_ = null;
-  }
+  private final List<List<LiteralExpr>> splitRows_;
 
-  private DistributeParam(Type t, List<String> cols,
-      ArrayList<ArrayList<LiteralExpr>> splitRows) {
+  private DistributeParam(Type t, List<String> colNames, int buckets,
+      List<List<LiteralExpr>> splitRows) {
     type_ = t;
-    columns_ = cols;
+    for (String name: colNames) colNames_.add(name.toLowerCase());
+    numBuckets_ = buckets;
     splitRows_ = splitRows;
-    num_buckets_ = NO_BUCKETS;
   }
 
-  /**
-   * TODO Refactor the logic below to analyze 'columns_'. This analysis should output
-   * a vector of column types that would then be used during the analysis of the split
-   * rows.
-   */
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
-    if (type_ == Type.HASH && num_buckets_ <= 1) {
-      throw new AnalysisException(String.format(
-          "Number of buckets in DISTRIBUTE BY clause '%s' must be larger than 1.",
-          toSql()));
-    } else if (type_ == Type.RANGE) {
-      // Creating the thrift structure simultaneously checks for semantic errors
-      rangeParam_ = new TDistributeByRangeParam();
-      rangeParam_.setColumns(columns_);
-
-      for (ArrayList<LiteralExpr> splitRow : splitRows_) {
-        TRangeLiteralList list = new TRangeLiteralList();
-        if (splitRow.size() != columns_.size()) {
+    Preconditions.checkState(!colNames_.isEmpty());
+    Preconditions.checkNotNull(pkColumnDefByName_);
+    Preconditions.checkState(!pkColumnDefByName_.isEmpty());
+    // Validate the columns specified in the DISTRIBUTE BY clause
+    for (String colName: colNames_) {
+      if (!pkColumnDefByName_.containsKey(colName)) {
+        throw new AnalysisException(String.format("Column '%s' in '%s' is not a key " +
+            "column. Only key columns can be used in DISTRIBUTE BY.", colName, toSql()));
+      }
+    }
+
+    if (type_ == Type.RANGE) {
+      for (List<LiteralExpr> splitRow : splitRows_) {
+        if (splitRow.size() != colNames_.size()) {
           throw new AnalysisException(String.format(
               "SPLIT ROWS has different size than number of projected key columns: %d. "
-                  + "Split row: %s", columns_.size(), splitRowToString(splitRow)));
+                  + "Split row: %s", colNames_.size(), splitRowToString(splitRow)));
         }
-        for (LiteralExpr expr : splitRow) {
+        for (int i = 0; i < splitRow.size(); ++i) {
+          LiteralExpr expr = splitRow.get(i);
+          ColumnDef colDef = pkColumnDefByName_.get(colNames_.get(i));
+          org.apache.impala.catalog.Type colType = colDef.getType();
+          Preconditions.checkState(KuduUtil.isSupportedKeyType(colType));
           expr.analyze(analyzer);
-          TRangeLiteral literal = new TRangeLiteral();
-          if (expr instanceof NumericLiteral) {
-            NumericLiteral num = (NumericLiteral) expr;
-            if (num.getType().isDecimal() || num.getType().isFloatingPointType()) {
-              throw new AnalysisException("Only integral and string values allowed for" +
-                  " split rows.");
-            } else {
-              literal.setInt_literal(num.getIntValue());
-            }
-          } else if (expr instanceof StringLiteral) {
-            StringLiteral string = (StringLiteral) expr;
-            literal.setString_literal(string.getStringValue());
-          } else if (expr instanceof BoolLiteral) {
-            BoolLiteral bool = (BoolLiteral) expr;
-            literal.setBool_literal(bool.getValue());
-          } else {
-            throw new AnalysisException(String.format("Split row value is not supported: "
-                + "%s (Type: %s).", expr.getStringValue(), expr.getType().toSql()));
+          org.apache.impala.catalog.Type exprType = expr.getType();
+          if (exprType.isNull()) {
+            throw new AnalysisException("Split values cannot be NULL. Split row: " +
+                splitRowToString(splitRow));
+          }
+          if (!org.apache.impala.catalog.Type.isImplicitlyCastable(exprType, colType,
+              true)) {
+            throw new AnalysisException(String.format("Split value %s (type: %s) is " +
+                "not type compatible with column '%s' (type: %s).", expr.toSql(),
+                exprType, colDef.getColName(), colType.toSql()));
           }
-          list.addToValues(literal);
         }
-        rangeParam_.addToSplit_rows(list);
       }
     }
   }
 
   @Override
   public String toSql() {
-    if (num_buckets_ == NO_BUCKETS) {
-      List<String> splitRowStrings = Lists.newArrayList();
-      for (ArrayList<LiteralExpr> splitRow : splitRows_) {
-        splitRowStrings.add(splitRowToString(splitRow));
-      }
-      return String.format("RANGE(%s) INTO RANGES(%s)", Joiner.on(", ").join(columns_),
-          Joiner.on(", ").join(splitRowStrings));
+    StringBuilder builder = new StringBuilder(type_.toString());
+    if (!colNames_.isEmpty()) {
+      builder.append(" (");
+      Joiner.on(", ").appendTo(builder, colNames_).append(")");
+    }
+    if (type_ == Type.HASH) {
+      builder.append(" INTO ");
+      Preconditions.checkState(numBuckets_ != NO_BUCKETS);
+      builder.append(numBuckets_).append(" BUCKETS");
     } else {
-      return String.format("HASH(%s) INTO %d BUCKETS", Joiner.on(", ").join(columns_),
-          num_buckets_);
+      builder.append(" SPLIT ROWS (");
+      if (splitRows_ == null) {
+        builder.append("...");
+      } else {
+        for (List<LiteralExpr> splitRow: splitRows_) {
+          builder.append(splitRowToString(splitRow));
+        }
+      }
+      builder.append(")");
     }
+    return builder.toString();
   }
 
-  private String splitRowToString(ArrayList<LiteralExpr> splitRow) {
-    StringBuilder builder = new StringBuilder();
-    builder.append("(");
-    List<String> rangeElementStrings = Lists.newArrayList();
-    for (LiteralExpr rangeElement : splitRow) {
-      rangeElementStrings.add(rangeElement.toSql());
+  @Override
+  public String toString() { return toSql(); }
+
+  private String splitRowToString(List<LiteralExpr> splitRow) {
+    StringBuilder builder = new StringBuilder("(");
+    for (LiteralExpr expr: splitRow) {
+      if (builder.length() > 1) builder.append(", ");
+      builder.append(expr.toSql());
     }
-    builder.append(Joiner.on(", ").join(rangeElementStrings));
-    builder.append(")");
-    return builder.toString();
+    return builder.append(")").toString();
   }
 
-  TDistributeParam toThrift() {
+  public TDistributeParam toThrift() {
     TDistributeParam result = new TDistributeParam();
+    // TODO: Add a validate() function to ensure the validity of distribute params.
     if (type_ == Type.HASH) {
       TDistributeByHashParam hash = new TDistributeByHashParam();
-      hash.setNum_buckets(num_buckets_);
-      hash.setColumns(columns_);
+      Preconditions.checkState(numBuckets_ != NO_BUCKETS);
+      hash.setNum_buckets(numBuckets_);
+      hash.setColumns(colNames_);
       result.setBy_hash_param(hash);
     } else {
       Preconditions.checkState(type_ == Type.RANGE);
-
-      result.setBy_range_param(rangeParam_);
+      TDistributeByRangeParam rangeParam = new TDistributeByRangeParam();
+      rangeParam.setColumns(colNames_);
+      if (splitRows_ == null) {
+        result.setBy_range_param(rangeParam);
+        return result;
+      }
+      for (List<LiteralExpr> splitRow : splitRows_) {
+        TRangeLiteralList list = new TRangeLiteralList();
+        for (int i = 0; i < splitRow.size(); ++i) {
+          LiteralExpr expr = splitRow.get(i);
+          TRangeLiteral literal = new TRangeLiteral();
+          if (expr instanceof NumericLiteral) {
+            literal.setInt_literal(((NumericLiteral)expr).getIntValue());
+          } else {
+            String exprValue = expr.getStringValue();
+            Preconditions.checkState(!Strings.isNullOrEmpty(exprValue));
+            literal.setString_literal(exprValue);
+          }
+          list.addToValues(literal);
+        }
+        rangeParam.addToSplit_rows(list);
+      }
+      result.setBy_range_param(rangeParam);
     }
     return result;
   }
 
-  public List<String> getColumns() { return columns_; }
-  public void setColumns(List<String> cols) { columns_ = cols; }
-  public Type getType_() { return type_; }
-  public int getNumBuckets() { return num_buckets_; }
+  void setPkColumnDefMap(Map<String, ColumnDef> pkColumnDefByName) {
+    pkColumnDefByName_ = pkColumnDefByName;
+  }
+
+  boolean hasColumnNames() { return !colNames_.isEmpty(); }
+
+  void setColumnNames(Collection<String> colNames) {
+    Preconditions.checkState(colNames_.isEmpty());
+    colNames_.addAll(colNames);
+  }
+
+  public Type getType() { return type_; }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
index 24b8417..28de1a8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
@@ -190,7 +190,7 @@ public abstract class ModifyStmt extends StatementBase {
 
     // cast result expressions to the correct type of the referenced slot of the
     // target table
-    int keyColumnsOffset = table_.getKuduKeyColumnNames().size();
+    int keyColumnsOffset = table_.getPrimaryKeyColumnNames().size();
     for (int i = keyColumnsOffset; i < sourceStmt_.resultExprs_.size(); ++i) {
       sourceStmt_.resultExprs_.set(i, sourceStmt_.resultExprs_.get(i).castTo(
           assignments_.get(i - keyColumnsOffset).first.getType()));
@@ -225,7 +225,7 @@ public abstract class ModifyStmt extends StatementBase {
     }
 
     // Add the key columns as slot refs
-    for (String k : table_.getKuduKeyColumnNames()) {
+    for (String k : table_.getPrimaryKeyColumnNames()) {
       ArrayList<String> path = Path.createRawPath(targetTableRef_.getUniqueAlias(), k);
       SlotRef ref = new SlotRef(path);
       ref.analyze(analyzer);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java b/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
new file mode 100644
index 0000000..4d3ed80
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
@@ -0,0 +1,56 @@
+// 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 com.google.common.collect.Lists;
+
+import java.util.List;
+
+/**
+ * Represents the PARTITION BY and DISTRIBUTED BY clauses of a DDL statement.
+ * TODO: Reconsider this class when we add support for new range partitioning syntax (see
+ * IMPALA-3724).
+ */
+class TableDataLayout {
+
+  private final List<ColumnDef> partitionColDefs_;
+  private final List<DistributeParam> distributeParams_;
+
+  private TableDataLayout(List<ColumnDef> partitionColumnDefs,
+      List<DistributeParam> distributeParams) {
+    partitionColDefs_ = partitionColumnDefs;
+    distributeParams_ = distributeParams;
+  }
+
+  static TableDataLayout createPartitionedLayout(List<ColumnDef> partitionColumnDefs) {
+    return new TableDataLayout(partitionColumnDefs,
+        Lists.<DistributeParam>newArrayList());
+  }
+
+  static TableDataLayout createDistributedLayout(List<DistributeParam> distributeParams) {
+    return new TableDataLayout(Lists.<ColumnDef>newArrayList(), distributeParams);
+  }
+
+  static TableDataLayout createEmptyLayout() {
+    return new TableDataLayout(Lists.<ColumnDef>newArrayList(),
+        Lists.<DistributeParam>newArrayList());
+  }
+
+  List<ColumnDef> getPartitionColumnDefs() { return partitionColDefs_; }
+  List<DistributeParam> getDistributeParams() { return distributeParams_; }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/TableDef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TableDef.java b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
new file mode 100644
index 0000000..ce08e36
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
@@ -0,0 +1,316 @@
+// 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.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.impala.authorization.Privilege;
+import org.apache.impala.catalog.HdfsStorageDescriptor;
+import org.apache.impala.catalog.RowFormat;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.common.FileSystemUtil;
+import org.apache.impala.thrift.TAccessEvent;
+import org.apache.impala.thrift.TCatalogObjectType;
+import org.apache.impala.thrift.THdfsFileFormat;
+import org.apache.impala.util.MetaStoreUtil;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.fs.permission.FsAction;
+
+/**
+ * Represents the table parameters in a CREATE TABLE statement. These parameters
+ * correspond to the following clauses in a CREATE TABLE statement:
+ * - EXTERNAL
+ * - IF NOT EXISTS
+ * - PARTITIONED BY
+ * - DISTRIBUTE BY
+ * - ROWFORMAT
+ * - FILEFORMAT
+ * - COMMENT
+ * - SERDEPROPERTIES
+ * - TBLPROPERTIES
+ * - LOCATION
+ * - CACHED IN
+ */
+class TableDef {
+
+  // Name of the new table
+  private final TableName tableName_;
+
+  // List of column definitions
+  private final List<ColumnDef> columnDefs_ = Lists.newArrayList();
+
+  // Names of primary key columns. Populated by the parser. An empty value doesn't
+  // mean no primary keys were specified as the columnDefs_ could contain primary keys.
+  private final List<String> primaryKeyColNames_ = Lists.newArrayList();
+
+  // Authoritative list of primary key column definitions populated during analysis.
+  private final List<ColumnDef> primaryKeyColDefs_ = Lists.newArrayList();
+
+  // If true, the table's data will be preserved if dropped.
+  private final boolean isExternal_;
+
+  // If true, no errors are thrown if the table already exists.
+  private final boolean ifNotExists_;
+
+  // Partitioned/distribute by parameters.
+  private final TableDataLayout dataLayout_;
+
+  // True if analyze() has been called.
+  private boolean isAnalyzed_ = false;
+
+  /**
+   * Set of table options. These options are grouped together for convenience while
+   * parsing CREATE TABLE statements. They are typically found at the end of CREATE
+   * TABLE statements.
+   */
+  static class Options {
+    // Comment to attach to the table
+    final String comment;
+
+    // Custom row format of the table. Leave null to specify default row format.
+    final RowFormat rowFormat;
+
+    // Key/values to persist with table serde metadata.
+    final Map<String, String> serdeProperties;
+
+    // File format of the table
+    final THdfsFileFormat fileFormat;
+
+    // The HDFS location of where the table data will stored.
+    final HdfsUri location;
+
+    // The HDFS caching op that should be applied to this table.
+    final HdfsCachingOp cachingOp;
+
+    // Key/values to persist with table metadata.
+    final Map<String, String> tblProperties;
+
+    Options(String comment, RowFormat rowFormat,
+        Map<String, String> serdeProperties, THdfsFileFormat fileFormat, HdfsUri location,
+        HdfsCachingOp cachingOp, Map<String, String> tblProperties) {
+      this.comment = comment;
+      this.rowFormat = rowFormat;
+      Preconditions.checkNotNull(serdeProperties);
+      this.serdeProperties = serdeProperties;
+      this.fileFormat = fileFormat == null ? THdfsFileFormat.TEXT : fileFormat;
+      this.location = location;
+      this.cachingOp = cachingOp;
+      Preconditions.checkNotNull(tblProperties);
+      this.tblProperties = tblProperties;
+    }
+
+    public Options(String comment) {
+      this(comment, RowFormat.DEFAULT_ROW_FORMAT, Maps.<String, String>newHashMap(),
+          THdfsFileFormat.TEXT, null, null, Maps.<String, String>newHashMap());
+    }
+  }
+
+  private Options options_;
+
+  // Result of analysis.
+  private TableName fqTableName_;
+
+  TableDef(TableName tableName, boolean isExternal, boolean ifNotExists) {
+    tableName_ = tableName;
+    isExternal_ = isExternal;
+    ifNotExists_ = ifNotExists;
+    dataLayout_ = TableDataLayout.createEmptyLayout();
+  }
+
+  public TableName getTblName() {
+    return fqTableName_ != null ? fqTableName_ : tableName_;
+  }
+  public String getTbl() { return tableName_.getTbl(); }
+  public boolean isAnalyzed() { return isAnalyzed_; }
+  List<ColumnDef> getColumnDefs() { return columnDefs_; }
+  List<ColumnDef> getPartitionColumnDefs() {
+    return dataLayout_.getPartitionColumnDefs();
+  }
+  List<String> getPrimaryKeyColumnNames() { return primaryKeyColNames_; }
+  List<ColumnDef> getPrimaryKeyColumnDefs() { return primaryKeyColDefs_; }
+  boolean isExternal() { return isExternal_; }
+  boolean getIfNotExists() { return ifNotExists_; }
+  List<DistributeParam> getDistributeParams() {
+    return dataLayout_.getDistributeParams();
+  }
+  void setOptions(Options options) {
+    Preconditions.checkNotNull(options);
+    options_ = options;
+  }
+  String getComment() { return options_.comment; }
+  Map<String, String> getTblProperties() { return options_.tblProperties; }
+  HdfsCachingOp getCachingOp() { return options_.cachingOp; }
+  HdfsUri getLocation() { return options_.location; }
+  Map<String, String> getSerdeProperties() { return options_.serdeProperties; }
+  THdfsFileFormat getFileFormat() { return options_.fileFormat; }
+  RowFormat getRowFormat() { return options_.rowFormat; }
+
+  /**
+   * Analyzes the parameters of a CREATE TABLE statement.
+   */
+  void analyze(Analyzer analyzer) throws AnalysisException {
+    Preconditions.checkState(tableName_ != null && !tableName_.isEmpty());
+    fqTableName_ = analyzer.getFqTableName(getTblName());
+    fqTableName_.analyze();
+    analyzeColumnDefs();
+    analyzePrimaryKeys();
+
+    if (analyzer.dbContainsTable(getTblName().getDb(), getTbl(), Privilege.CREATE)
+        && !getIfNotExists()) {
+      throw new AnalysisException(Analyzer.TBL_ALREADY_EXISTS_ERROR_MSG + getTblName());
+    }
+
+    analyzer.addAccessEvent(new TAccessEvent(getTblName().toString(),
+        TCatalogObjectType.TABLE, Privilege.CREATE.toString()));
+
+    Preconditions.checkNotNull(options_);
+    analyzeOptions(analyzer);
+    isAnalyzed_ = true;
+  }
+
+  /**
+   * Analyzes table and partition column definitions, checking whether all column
+   * names are unique.
+   */
+  private void analyzeColumnDefs() throws AnalysisException {
+    Set<String> colNames = Sets.newHashSet();
+    for (ColumnDef colDef: columnDefs_) {
+      colDef.analyze();
+      if (!colNames.add(colDef.getColName().toLowerCase())) {
+        throw new AnalysisException("Duplicate column name: " + colDef.getColName());
+      }
+    }
+    for (ColumnDef colDef: getPartitionColumnDefs()) {
+      colDef.analyze();
+      if (!colDef.getType().supportsTablePartitioning()) {
+        throw new AnalysisException(
+            String.format("Type '%s' is not supported as partition-column type " +
+                "in column: %s", colDef.getType().toSql(), colDef.getColName()));
+      }
+      if (!colNames.add(colDef.getColName().toLowerCase())) {
+        throw new AnalysisException("Duplicate column name: " + colDef.getColName());
+      }
+    }
+  }
+
+  /**
+   * Analyzes the primary key columns. Checks if the specified primary key columns exist
+   * in the table column definitions and if composite primary keys are properly defined
+   * using the PRIMARY KEY (col,..col) clause.
+   */
+  private void analyzePrimaryKeys() throws AnalysisException {
+    for (ColumnDef colDef: columnDefs_) {
+      if (colDef.isPrimaryKey()) primaryKeyColDefs_.add(colDef);
+    }
+    if (primaryKeyColDefs_.size() > 1) {
+      throw new AnalysisException("Multiple primary keys specified. " +
+          "Composite primary keys can be specified using the " +
+          "PRIMARY KEY (col1, col2, ...) syntax at the end of the column definition.");
+    }
+    if (primaryKeyColNames_.isEmpty()) return;
+    if (!primaryKeyColDefs_.isEmpty()) {
+      throw new AnalysisException("Multiple primary keys specified. " +
+          "Composite primary keys can be specified using the " +
+          "PRIMARY KEY (col1, col2, ...) syntax at the end of the column definition.");
+    }
+    Map<String, ColumnDef> colDefsByColName = ColumnDef.mapByColumnNames(columnDefs_);
+    for (String colName: primaryKeyColNames_) {
+      colName = colName.toLowerCase();
+      ColumnDef colDef = colDefsByColName.remove(colName);
+      if (colDef == null) {
+        if (ColumnDef.toColumnNames(primaryKeyColDefs_).contains(colName)) {
+          throw new AnalysisException(String.format("Column '%s' is listed multiple " +
+              "times as a PRIMARY KEY.", colName));
+        }
+        throw new AnalysisException(String.format(
+            "PRIMARY KEY column '%s' does not exist in the table", colName));
+      }
+      primaryKeyColDefs_.add(colDef);
+    }
+  }
+
+  private void analyzeOptions(Analyzer analyzer) throws AnalysisException {
+    MetaStoreUtil.checkShortPropertyMap("Property", options_.tblProperties);
+    MetaStoreUtil.checkShortPropertyMap("Serde property", options_.serdeProperties);
+
+    if (options_.location != null) {
+      options_.location.analyze(analyzer, Privilege.ALL, FsAction.READ_WRITE);
+    }
+
+    if (options_.cachingOp != null) {
+      options_.cachingOp.analyze(analyzer);
+      if (options_.cachingOp.shouldCache() && options_.location != null &&
+          !FileSystemUtil.isPathCacheable(options_.location.getPath())) {
+        throw new AnalysisException(String.format("Location '%s' cannot be cached. " +
+            "Please retry without caching: CREATE TABLE ... UNCACHED",
+            options_.location));
+      }
+    }
+
+    // Analyze 'skip.header.line.format' property.
+    AlterTableSetTblProperties.analyzeSkipHeaderLineCount(options_.tblProperties);
+    analyzeRowFormat(analyzer);
+  }
+
+  private void analyzeRowFormat(Analyzer analyzer) throws AnalysisException {
+    if (options_.rowFormat == null) return;
+    if (options_.fileFormat == THdfsFileFormat.KUDU) {
+      throw new AnalysisException(String.format(
+          "ROW FORMAT cannot be specified for file format %s.", options_.fileFormat));
+    }
+
+    Byte fieldDelim = analyzeRowFormatValue(options_.rowFormat.getFieldDelimiter());
+    Byte lineDelim = analyzeRowFormatValue(options_.rowFormat.getLineDelimiter());
+    Byte escapeChar = analyzeRowFormatValue(options_.rowFormat.getEscapeChar());
+    if (options_.fileFormat == THdfsFileFormat.TEXT) {
+      if (fieldDelim == null) fieldDelim = HdfsStorageDescriptor.DEFAULT_FIELD_DELIM;
+      if (lineDelim == null) lineDelim = HdfsStorageDescriptor.DEFAULT_LINE_DELIM;
+      if (escapeChar == null) escapeChar = HdfsStorageDescriptor.DEFAULT_ESCAPE_CHAR;
+      if (fieldDelim.equals(lineDelim)) {
+        throw new AnalysisException("Field delimiter and line delimiter have same " +
+            "value: byte " + fieldDelim);
+      }
+      if (fieldDelim.equals(escapeChar)) {
+        analyzer.addWarning("Field delimiter and escape character have same value: " +
+            "byte " + fieldDelim + ". Escape character will be ignored");
+      }
+      if (lineDelim.equals(escapeChar)) {
+        analyzer.addWarning("Line delimiter and escape character have same value: " +
+            "byte " + lineDelim + ". Escape character will be ignored");
+      }
+    }
+  }
+
+  private Byte analyzeRowFormatValue(String value) throws AnalysisException {
+    if (value == null) return null;
+    Byte byteVal = HdfsStorageDescriptor.parseDelim(value);
+    if (byteVal == null) {
+      throw new AnalysisException("ESCAPED BY values and LINE/FIELD " +
+          "terminators must be specified as a single character or as a decimal " +
+          "value in the range [-128:127]: " + value);
+    }
+    return byteVal;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/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 b125987..aa24336 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
@@ -22,10 +22,16 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.impala.catalog.KuduTable;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.Token;
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.parse.HiveLexer;
 
@@ -35,16 +41,11 @@ import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.HBaseTable;
 import org.apache.impala.catalog.HdfsCompression;
 import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.RowFormat;
 import org.apache.impala.catalog.Table;
 import org.apache.impala.catalog.View;
-import org.apache.impala.common.PrintUtils;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import org.apache.impala.util.KuduUtil;
 
 /**
  * Contains utility methods for creating SQL strings, for example,
@@ -132,8 +133,9 @@ public class ToSqlUtils {
     }
     // TODO: Pass the correct compression, if applicable.
     return getCreateTableSql(stmt.getDb(), stmt.getTbl(), stmt.getComment(), colsSql,
-        partitionColsSql, stmt.getTblProperties(), stmt.getSerdeProperties(),
-        stmt.isExternal(), stmt.getIfNotExists(), stmt.getRowFormat(),
+        partitionColsSql, stmt.getTblPrimaryKeyColumnNames(), null,
+        stmt.getTblProperties(), stmt.getSerdeProperties(), stmt.isExternal(),
+        stmt.getIfNotExists(), stmt.getRowFormat(),
         HdfsFileFormat.fromThrift(stmt.getFileFormat()), HdfsCompression.NONE, null,
         stmt.getLocation());
   }
@@ -152,7 +154,8 @@ public class ToSqlUtils {
     }
     // TODO: Pass the correct compression, if applicable.
     String createTableSql = getCreateTableSql(innerStmt.getDb(), innerStmt.getTbl(),
-        innerStmt.getComment(), null, partitionColsSql, innerStmt.getTblProperties(),
+        innerStmt.getComment(), null, partitionColsSql,
+        innerStmt.getTblPrimaryKeyColumnNames(), null, innerStmt.getTblProperties(),
         innerStmt.getSerdeProperties(), innerStmt.isExternal(),
         innerStmt.getIfNotExists(), innerStmt.getRowFormat(),
         HdfsFileFormat.fromThrift(innerStmt.getFileFormat()), HdfsCompression.NONE, null,
@@ -169,6 +172,9 @@ public class ToSqlUtils {
     if (table instanceof View) return getCreateViewSql((View)table);
     org.apache.hadoop.hive.metastore.api.Table msTable = table.getMetaStoreTable();
     HashMap<String, String> properties = Maps.newHashMap(msTable.getParameters());
+    if (properties.containsKey("transient_lastDdlTime")) {
+      properties.remove("transient_lastDdlTime");
+    }
     boolean isExternal = msTable.getTableType() != null &&
         msTable.getTableType().equals(TableType.EXTERNAL_TABLE.toString());
     String comment = properties.get("comment");
@@ -194,17 +200,40 @@ public class ToSqlUtils {
     Map<String, String> serdeParameters = msTable.getSd().getSerdeInfo().getParameters();
 
     String storageHandlerClassName = table.getStorageHandlerClassName();
+    List<String> primaryKeySql = Lists.newArrayList();
+    String kuduDistributeByParams = null;
     if (table instanceof KuduTable) {
+      KuduTable kuduTable = (KuduTable) table;
       // Kudu tables don't use LOCATION syntax
       location = null;
-      format = null;
+      format = HdfsFileFormat.KUDU;
       // Kudu tables cannot use the Hive DDL syntax for the storage handler
       storageHandlerClassName = null;
+      properties.remove(KuduTable.KEY_STORAGE_HANDLER);
+      String kuduTableName = properties.get(KuduTable.KEY_TABLE_NAME);
+      Preconditions.checkNotNull(kuduTableName);
+      if (kuduTableName.equals(KuduUtil.getDefaultCreateKuduTableName(
+          table.getDb().getName(), table.getName()))) {
+        properties.remove(KuduTable.KEY_TABLE_NAME);
+      }
+      // Internal property, should not be exposed to the user.
+      properties.remove(StatsSetupConst.DO_NOT_UPDATE_STATS);
+
+      if (!isExternal) {
+        primaryKeySql.addAll(kuduTable.getPrimaryKeyColumnNames());
+
+        List<String> paramsSql = Lists.newArrayList();
+        for (DistributeParam param: kuduTable.getDistributeBy()) {
+          paramsSql.add(param.toSql());
+        }
+        kuduDistributeByParams = Joiner.on(", ").join(paramsSql);
+      }
     }
     HdfsUri tableLocation = location == null ? null : new HdfsUri(location);
     return getCreateTableSql(table.getDb().getName(), table.getName(), comment, colsSql,
-        partitionColsSql, properties, serdeParameters, isExternal, false, rowFormat,
-        format, compression, storageHandlerClassName, tableLocation);
+        partitionColsSql, primaryKeySql, kuduDistributeByParams, properties,
+        serdeParameters, isExternal, false, rowFormat, format, compression,
+        storageHandlerClassName, tableLocation);
   }
 
   /**
@@ -214,6 +243,7 @@ public class ToSqlUtils {
    */
   public static String getCreateTableSql(String dbName, String tableName,
       String tableComment, List<String> columnsSql, List<String> partitionColumnsSql,
+      List<String> primaryKeysSql, String kuduDistributeByParams,
       Map<String, String> tblProperties, Map<String, String> serdeParameters,
       boolean isExternal, boolean ifNotExists, RowFormat rowFormat,
       HdfsFileFormat fileFormat, HdfsCompression compression, String storageHandlerClass,
@@ -227,7 +257,11 @@ public class ToSqlUtils {
     sb.append(tableName);
     if (columnsSql != null) {
       sb.append(" (\n  ");
-      sb.append(Joiner.on(", \n  ").join(columnsSql));
+      sb.append(Joiner.on(",\n  ").join(columnsSql));
+      if (!primaryKeysSql.isEmpty()) {
+        sb.append(",\n  PRIMARY KEY (");
+        Joiner.on(", ").appendTo(sb, primaryKeysSql).append(")");
+      }
       sb.append("\n)");
     }
     sb.append("\n");
@@ -238,6 +272,10 @@ public class ToSqlUtils {
           Joiner.on(", \n  ").join(partitionColumnsSql)));
     }
 
+    if (kuduDistributeByParams != null) {
+      sb.append("DISTRIBUTE BY " + kuduDistributeByParams + "\n");
+    }
+
     if (rowFormat != null && !rowFormat.isDefault()) {
       sb.append("ROW FORMAT DELIMITED");
       if (rowFormat.getFieldDelimiter() != null) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/Catalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Catalog.java b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
index 41573ed..733b2f2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -519,4 +519,8 @@ public abstract class Catalog {
     }
     return result;
   }
+
+  public static boolean isDefaultDb(String dbName) {
+    return DEFAULT_DB.equals(dbName.toLowerCase());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 41c8d62..149b00b 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -23,7 +23,6 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -41,7 +40,6 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
@@ -52,7 +50,6 @@ import org.apache.log4j.Logger;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.apache.thrift.TException;
 
-import org.apache.impala.analysis.TableName;
 import org.apache.impala.authorization.SentryConfig;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.common.FileSystemUtil;
@@ -65,7 +62,6 @@ import org.apache.impala.thrift.TCatalog;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TFunction;
-import org.apache.impala.thrift.TFunctionBinaryType;
 import org.apache.impala.thrift.TGetAllCatalogObjectsResponse;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TPrivilege;
@@ -79,7 +75,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.google.common.io.Files;
 
 /**
  * Specialized Catalog that implements the CatalogService specific Catalog
@@ -693,7 +688,7 @@ public class CatalogServiceCatalog extends Catalog {
    * Adds a table with the given name to the catalog and returns the new table,
    * loading the metadata if needed.
    */
-  public Table addTable(String dbName, String tblName) throws TableNotFoundException {
+  public Table addTable(String dbName, String tblName) {
     Db db = getDb(dbName);
     if (db == null) return null;
     Table incompleteTable =

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/Db.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Db.java b/fe/src/main/java/org/apache/impala/catalog/Db.java
index d6fb185..0ed67c6 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Db.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Db.java
@@ -152,6 +152,11 @@ public class Db implements CatalogObject {
     return Lists.newArrayList(tableCache_.keySet());
   }
 
+  /**
+   * Returns the tables in the cache.
+   */
+  public List<Table> getTables() { return tableCache_.getValues(); }
+
   public boolean containsTable(String tableName) {
     return tableCache_.contains(tableName.toLowerCase());
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
index 86a65bd..e4fce60 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
@@ -31,8 +31,12 @@ import com.google.common.collect.Lists;
  * 2) the output format class
  * 3) the serialization library class
  * 4) whether scanning complex types from it is supported
+ * 5) whether the file format can skip complex columns in scans and just materialize
+ *    scalar typed columns
  *
  * Important note: Always keep consistent with the classes used in Hive.
+ * TODO: Kudu doesn't belong in this list. Either rename this enum or create a separate
+ * list of storage engines (see IMPALA-4178).
  */
 public enum HdfsFileFormat {
   RC_FILE("org.apache.hadoop.hive.ql.io.RCFileInputFormat",
@@ -57,7 +61,10 @@ public enum HdfsFileFormat {
   PARQUET("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
       "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
       "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
-      true, true);
+      true, true),
+  KUDU("org.apache.kudu.mapreduce.KuduTableInputFormat",
+      "org.apache.kudu.mapreduce.KuduTableOutputFormat",
+      "", false, false);
 
   private final String inputFormat_;
   private final String outputFormat_;
@@ -103,6 +110,7 @@ public enum HdfsFileFormat {
           .put(PARQUET_LEGACY_INPUT_FORMATS[0], PARQUET)
           .put(PARQUET_LEGACY_INPUT_FORMATS[1], PARQUET)
           .put(PARQUET_LEGACY_INPUT_FORMATS[2], PARQUET)
+          .put(KUDU.inputFormat(), KUDU)
           .build();
 
   /**
@@ -138,6 +146,7 @@ public enum HdfsFileFormat {
       case SEQUENCE_FILE: return HdfsFileFormat.SEQUENCE_FILE;
       case AVRO: return HdfsFileFormat.AVRO;
       case PARQUET: return HdfsFileFormat.PARQUET;
+      case KUDU: return HdfsFileFormat.KUDU;
       default:
         throw new RuntimeException("Unknown THdfsFileFormat: "
             + thriftFormat + " - should never happen!");
@@ -151,6 +160,7 @@ public enum HdfsFileFormat {
       case SEQUENCE_FILE: return THdfsFileFormat.SEQUENCE_FILE;
       case AVRO: return THdfsFileFormat.AVRO;
       case PARQUET: return THdfsFileFormat.PARQUET;
+      case KUDU: return THdfsFileFormat.KUDU;
       default:
         throw new RuntimeException("Unknown HdfsFormat: "
             + this + " - should never happen!");
@@ -173,6 +183,7 @@ public enum HdfsFileFormat {
       case SEQUENCE_FILE: return "SEQUENCEFILE";
       case AVRO: return "AVRO";
       case PARQUET: return "PARQUET";
+      case KUDU: return "KUDU";
       default:
         throw new RuntimeException("Unknown HdfsFormat: "
             + this + " - should never happen!");
@@ -230,6 +241,8 @@ public enum HdfsFileFormat {
       case AVRO:
       case PARQUET:
         return true;
+      case KUDU:
+        return false;
       default:
         throw new RuntimeException("Unknown HdfsFormat: "
             + this + " - should never happen!");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
index c416bee..3647256 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
@@ -87,12 +87,17 @@ public class ImpaladCatalog extends Catalog {
   // Object that is used to synchronize on and signal when a catalog update is received.
   private final Object catalogUpdateEventNotifier_ = new Object();
 
+  // The addresses of the Kudu masters to use if no Kudu masters were explicitly provided.
+  // Used during table creation.
+  private final String defaultKuduMasterHosts_;
+
   /**
    * C'tor used by tests that need to validate the ImpaladCatalog outside of the
    * CatalogServer.
    */
-  public ImpaladCatalog() {
+  public ImpaladCatalog(String defaultKuduMasterHosts) {
     super(false);
+    defaultKuduMasterHosts_ = defaultKuduMasterHosts;
   }
 
   /**
@@ -445,4 +450,5 @@ public class ImpaladCatalog extends Catalog {
   // Only used for testing.
   public void setIsReady(boolean isReady) { isReady_.set(isReady); }
   public AuthorizationPolicy getAuthPolicy() { return authPolicy_; }
+  public String getDefaultKuduMasterHosts() { return defaultKuduMasterHosts_; }
 }


[04/14] incubator-impala git commit: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/common/kudu_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/kudu_test_suite.py b/tests/common/kudu_test_suite.py
new file mode 100644
index 0000000..7a93c12
--- /dev/null
+++ b/tests/common/kudu_test_suite.py
@@ -0,0 +1,148 @@
+# 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.
+
+import os
+import string
+import pytest
+from contextlib import contextmanager
+from kudu.schema import (
+    BOOL,
+    DOUBLE,
+    FLOAT,
+    INT16,
+    INT32,
+    INT64,
+    INT8,
+    SchemaBuilder,
+    STRING)
+from kudu.client import Partitioning
+from random import choice, sample
+from string import ascii_lowercase, digits
+
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.skip import SkipIf
+from tests.common.test_dimensions import create_uncompressed_text_dimension
+
+class KuduTestSuite(ImpalaTestSuite):
+
+  # Lazily set.
+  __DB_NAME = None
+
+  @classmethod
+  def setup_class(cls):
+    if os.environ["KUDU_IS_SUPPORTED"] == "false":
+      pytest.skip("Kudu is not supported")
+
+    super(KuduTestSuite, cls).setup_class()
+
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(KuduTestSuite, cls).add_test_dimensions()
+    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+
+  @classmethod
+  def auto_create_db(cls):
+    return True
+
+  @classmethod
+  def get_db_name(cls):
+    # When py.test runs with the xdist plugin, several processes are started and each
+    # process runs some partition of the tests. It's possible that multiple processes
+    # will call this method. A random value is generated so the processes won't try
+    # to use the same database at the same time. The value is cached so within a single
+    # process the same database name is always used for the class. This doesn't need to
+    # be thread-safe since multi-threading is never used.
+    if not cls.__DB_NAME:
+      cls.__DB_NAME = \
+          choice(ascii_lowercase) + "".join(sample(ascii_lowercase + digits, 5))
+    return cls.__DB_NAME
+
+  @classmethod
+  def random_table_name(cls):
+    return "".join(choice(string.lowercase) for _ in xrange(10))
+
+  @classmethod
+  def get_kudu_table_base_name(cls, name):
+    return name.split(".")[-1]
+
+  @contextmanager
+  def temp_kudu_table(self, kudu, col_types, name=None, num_key_cols=1, col_names=None,
+      prepend_db_name=True, db_name=None):
+    """Create and return a table. This function should be used in a "with" context.
+       'kudu' must be a kudu.client.Client. If a table name is not provided, a random
+       name will be used. If 'prepend_db_name' is True, the table name will be prepended
+       with (get_db_name() + "."). If column names are not provided, the letters
+       "a", "b", "c", ... will be used.
+
+       Example:
+         with self.temp_kudu_table(kudu, [INT32]) as kudu_table:
+            assert kudu.table_exists(kudu_table.name)
+         assert not kudu.table_exists(kudu_table.name)
+    """
+    if not col_names:
+      if len(col_types) > 26:
+        raise Exception("Too many columns for default naming")
+      col_names = [chr(97 + i) for i in xrange(len(col_types))]
+    schema_builder = SchemaBuilder()
+    for i, t in enumerate(col_types):
+      column_spec = schema_builder.add_column(col_names[i], type_=t)
+      if i < num_key_cols:
+        column_spec.nullable(False)
+    schema_builder.set_primary_keys(col_names[:num_key_cols])
+    schema = schema_builder.build()
+    name = name or self.random_table_name()
+    if prepend_db_name:
+      name = (db_name or self.get_db_name().lower()) + "." + name
+    kudu.create_table(name, schema,
+        partitioning=Partitioning().add_hash_partitions(col_names[:num_key_cols], 2))
+    try:
+      yield kudu.table(name)
+    finally:
+      if kudu.table_exists(name):
+        kudu.delete_table(name)
+
+  @contextmanager
+  def drop_impala_table_after_context(self, cursor, table_name):
+    """For use in a "with" block: The named table will be dropped using the provided
+       cursor when the block exits.
+
+       cursor.execute("CREATE TABLE foo ...")
+       with drop_impala_table_after_context(cursor, "foo"):
+         ...
+       # Now table foo no longer exists.
+    """
+    try:
+      yield
+    finally:
+      cursor.execute("DROP TABLE %s" % table_name)
+
+  def kudu_col_type_to_impala_col_type(self, col_type):
+    mapping = {BOOL: "BOOLEAN",
+        DOUBLE: "DOUBLE",
+        FLOAT: "FLOAT",
+        INT16: "SMALLINT",
+        INT32: "INT",
+        INT64: "BIGINT",
+        INT8: "TINYINT",
+        STRING: "STRING"}
+    if col_type not in mapping:
+      raise Exception("Unexpected column type: %s" % col_type)
+    return mapping[col_type]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/conftest.py
----------------------------------------------------------------------
diff --git a/tests/conftest.py b/tests/conftest.py
index 3193c9e..c22de39 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -27,7 +27,7 @@ import logging
 import os
 import pytest
 
-from common import KUDU_MASTER_HOST, KUDU_MASTER_PORT
+from common import KUDU_MASTER_HOSTS
 from common.test_result_verifier import QueryTestResult
 from tests.common.patterns import is_valid_impala_identifier
 from tests.util.filesystem_utils import FILESYSTEM, ISILON_WEBHDFS_PORT
@@ -288,7 +288,13 @@ def kudu_client():
   """Provides a new Kudu client as a pytest fixture. The client only exists for the
      duration of the method it is used in.
   """
-  kudu_client = kudu_connect(KUDU_MASTER_HOST, KUDU_MASTER_PORT)
+  if "," in KUDU_MASTER_HOSTS:
+    raise Exception("Multi-master not supported yet")
+  if ":" in KUDU_MASTER_HOSTS:
+    host, port = KUDU_MASTER_HOSTS.split(":")
+  else:
+    host, port = KUDU_MASTER_HOSTS, 7051
+  kudu_client = kudu_connect(host, port)
   try:
     yield kudu_client
   finally:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/custom_cluster/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_kudu.py b/tests/custom_cluster/test_kudu.py
new file mode 100644
index 0000000..898a29e
--- /dev/null
+++ b/tests/custom_cluster/test_kudu.py
@@ -0,0 +1,53 @@
+# 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.
+
+import logging
+import pytest
+from kudu.schema import INT32
+
+from tests.common import KUDU_MASTER_HOSTS
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+from tests.common.kudu_test_suite import KuduTestSuite
+
+LOG = logging.getLogger(__name__)
+
+class TestKuduOperations(CustomClusterTestSuite, KuduTestSuite):
+
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(impalad_args="-kudu_master_hosts=")
+  def test_kudu_master_hosts(self, cursor, kudu_client):
+    """Check behavior when -kudu_master_hosts is not provided to catalogd."""
+    with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
+      table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % (kudu_table.name)
+      try:
+        cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (table_name,
+            props))
+        assert False
+      except Exception as e:
+        assert "Table property 'kudu.master_addresses' is required" in str(e)
+
+      cursor.execute("""
+          CREATE EXTERNAL TABLE %s STORED AS KUDU
+          TBLPROPERTIES ('kudu.master_addresses' = '%s',
+          'kudu.table_name'='%s')
+          """ % (table_name, KUDU_MASTER_HOSTS, kudu_table.name))
+      cursor.execute("DROP TABLE %s" % table_name)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/metadata/test_ddl.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 18ed1af..8079855 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -220,9 +220,8 @@ class TestDdlStatements(TestDdlBase):
   @SkipIf.kudu_not_supported
   @UniqueDatabase.parametrize(sync_ddl=True)
   def test_create_kudu(self, vector, unique_database):
-    self.expected_exceptions = 2
     vector.get_value('exec_option')['abort_on_error'] = False
-    self.run_test_case('QueryTest/create_kudu', vector, use_db=unique_database,
+    self.run_test_case('QueryTest/kudu_create', vector, use_db=unique_database,
         multiple_impalad=self._use_multiple_impalad(vector))
 
   @UniqueDatabase.parametrize(sync_ddl=True)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/metadata/test_show_create_table.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_show_create_table.py b/tests/metadata/test_show_create_table.py
index 717223a..aae9f0c 100644
--- a/tests/metadata/test_show_create_table.py
+++ b/tests/metadata/test_show_create_table.py
@@ -54,11 +54,6 @@ class TestShowCreateTable(ImpalaTestSuite):
     self.__run_show_create_table_test_case('QueryTest/show-create-table', vector,
                                            unique_database)
 
-  @SkipIf.kudu_not_supported
-  def test_kudu_show_create_table(self, vector, unique_database):
-    self.__run_show_create_table_test_case('QueryTest/kudu-show-create', vector,
-                                           unique_database)
-
   def __run_show_create_table_test_case(self, test_file_name, vector, unique_db_name):
     """
     Runs a show-create-table test file, containing the following sections:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index d791608..c22de3e 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -15,104 +15,427 @@
 # specific language governing permissions and limitations
 # under the License.
 
+from kudu.schema import (
+    BOOL,
+    DOUBLE,
+    FLOAT,
+    INT16,
+    INT32,
+    INT64,
+    INT8,
+    STRING)
+import logging
 import pytest
-from copy import copy
+import textwrap
 
-from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
-from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.skip import SkipIf
-from tests.common.test_dimensions import create_uncompressed_text_dimension
-from tests.common.test_vector import TestDimension
+from tests.common import KUDU_MASTER_HOSTS
+from tests.common.kudu_test_suite import KuduTestSuite
 
+LOG = logging.getLogger(__name__)
 
-@SkipIf.kudu_not_supported
-class TestKuduOperations(ImpalaTestSuite):
+class TestKuduOperations(KuduTestSuite):
   """
   This suite tests the different modification operations when using a kudu table.
   """
 
-  @classmethod
-  def file_format_constraint(cls, v):
-    return v.get_value('table_format').file_format in ["parquet"]
+  def test_kudu_scan_node(self, vector, unique_database):
+    self.run_test_case('QueryTest/kudu-scan-node', vector, use_db=unique_database,
+        wait_secs_between_stmts=1)
 
-  @classmethod
-  def get_workload(cls):
-    return 'functional-query'
+  def test_kudu_crud(self, vector, unique_database):
+    self.run_test_case('QueryTest/kudu_crud', vector, use_db=unique_database,
+        wait_secs_between_stmts=1)
 
-  @classmethod
-  def add_test_dimensions(cls):
-    super(TestKuduOperations, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(cls.file_format_constraint)
-
-  # TODO(kudu-merge) IMPALA-3178 DROP DATABASE ... CASCADE is broken in Kudu so we need
-  # to clean up table-by-table. Once solved, delete this and rely on the overriden method.
-  def cleanup_db(self, db_name):
-    self.client.execute("use default")
-    self.client.set_configuration({'sync_ddl': True})
-    if db_name + "\t" in self.client.execute("show databases", ).data:
-      # We use quoted identifiers to avoid name clashes with keywords
-      for tbl_name in self.client.execute("show tables in `" + db_name + "`").data:
-        full_tbl_name = '`%s`.`%s`' % (db_name, tbl_name)
-        result = self.client.execute("describe formatted " + full_tbl_name)
-        if 'VIRTUAL_VIEW' in '\n'.join(result.data):
-          self.client.execute("drop view " + full_tbl_name)
-        else:
-          self.client.execute("drop table " + full_tbl_name)
-      for fn_result in self.client.execute("show functions in `" + db_name + "`").data:
-        # First column is the return type, second is the function signature
-        fn_name = fn_result.split('\t')[1]
-        self.client.execute("drop function `%s`.%s" % (db_name, fn_name))
-      for fn_result in self.client.execute(\
-        "show aggregate functions in `" + db_name + "`").data:
-        fn_name = fn_result.split('\t')[1]
-        self.client.execute("drop function `%s`.%s" % (db_name, fn_name))
-      self.client.execute("drop database `" + db_name + "`")
-
-  def setup_method(self, method):
-    self.cleanup_db("kududb_test")
-    self.client.execute("create database kududb_test")
-
-  def teardown_method(self, method):
-    self.cleanup_db("kududb_test")
+  def test_kudu_partition_ddl(self, vector, unique_database):
+    self.run_test_case('QueryTest/kudu_partition_ddl', vector, use_db=unique_database)
 
-  @pytest.mark.execute_serially
-  def test_kudu_scan_node(self, vector):
-    self.run_test_case('QueryTest/kudu-scan-node', vector, use_db="functional_kudu",
-        wait_secs_between_stmts=1)
+  def test_kudu_alter_table(self, vector, unique_database):
+    self.run_test_case('QueryTest/kudu_alter', vector, use_db=unique_database)
 
-  @pytest.mark.execute_serially
-  def test_insert_update_delete(self, vector):
-    self.run_test_case('QueryTest/kudu_crud', vector, use_db="kududb_test",
-        wait_secs_between_stmts=1)
+  def test_kudu_stats(self, vector, unique_database):
+    self.run_test_case('QueryTest/kudu_stats', vector, use_db=unique_database)
 
-  @pytest.mark.execute_serially
-  def test_kudu_partition_ddl(self, vector):
-    self.run_test_case('QueryTest/kudu_partition_ddl', vector, use_db="kududb_test")
 
-  @pytest.mark.execute_serially
-  def test_kudu_alter_table(self, vector):
-    self.run_test_case('QueryTest/kudu_alter', vector, use_db="kududb_test")
+class TestCreateExternalTable(KuduTestSuite):
 
-  @pytest.mark.execute_serially
-  def test_kudu_stats(self, vector):
-    self.run_test_case('QueryTest/kudu_stats', vector, use_db="kududb_test")
+  def test_implicit_table_props(self, cursor, kudu_client):
+    """Check that table properties added internally during table creation are as
+       expected.
+    """
+    with self.temp_kudu_table(kudu_client, [STRING, INT8, BOOL], num_key_cols=2) \
+        as kudu_table:
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (impala_table_name,
+          props))
+      with self.drop_impala_table_after_context(cursor, impala_table_name):
+        cursor.execute("DESCRIBE FORMATTED %s" % impala_table_name)
+        table_desc = [[col.strip() if col else col for col in row] for row in cursor]
+        LOG.info(table_desc)
+        # Pytest shows truncated output on failure, so print the details just in case.
+        assert ["", "EXTERNAL", "TRUE"] in table_desc
+        assert ["", "kudu.master_addresses", KUDU_MASTER_HOSTS] in table_desc
+        assert ["", "kudu.table_name", kudu_table.name] in table_desc
+        assert ["", "storage_handler", "com.cloudera.kudu.hive.KuduStorageHandler"] \
+            in table_desc
+
+  def test_col_types(self, cursor, kudu_client):
+    """Check that a table can be created using all available column types."""
+    # TODO: The python Kudu client doesn't yet support TIMESTAMP or BYTE[], those should
+    #       be tested for graceful failure.
+    kudu_types = [STRING, BOOL, DOUBLE, FLOAT, INT16, INT32, INT64, INT8]
+    with self.temp_kudu_table(kudu_client, kudu_types) as kudu_table:
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (impala_table_name,
+          props))
+      with self.drop_impala_table_after_context(cursor, impala_table_name):
+        cursor.execute("DESCRIBE %s" % impala_table_name)
+        kudu_schema = kudu_table.schema
+        for i, (col_name, col_type, _) in enumerate(cursor):
+          kudu_col = kudu_schema[i]
+          assert col_name == kudu_col.name
+          assert col_type.upper() == \
+              self.kudu_col_type_to_impala_col_type(kudu_col.type.type)
+
+  def test_drop_external_table(self, cursor, kudu_client):
+    """Check that dropping an external table only affects the catalog and does not delete
+       the table in Kudu.
+    """
+    with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (impala_table_name,
+          props))
+      with self.drop_impala_table_after_context(cursor, impala_table_name):
+        cursor.execute("SELECT COUNT(*) FROM %s" % impala_table_name)
+        assert cursor.fetchall() == [(0, )]
+      try:
+        cursor.execute("SELECT COUNT(*) FROM %s" % impala_table_name)
+        assert False
+      except Exception as e:
+        assert "Could not resolve table reference" in str(e)
+      assert kudu_client.table_exists(kudu_table.name)
+
+  def test_explicit_name(self, cursor, kudu_client):
+    """Check that a Kudu table can be specified using a table property."""
+    with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
+      table_name = self.random_table_name()
+      cursor.execute("""
+          CREATE EXTERNAL TABLE %s
+          STORED AS KUDU
+          TBLPROPERTIES('kudu.table_name' = '%s')""" % (table_name, kudu_table.name))
+      with self.drop_impala_table_after_context(cursor, table_name):
+        cursor.execute("SELECT * FROM %s" % table_name)
+        assert len(cursor.fetchall()) == 0
+
+  def test_explicit_name_preference(self, cursor, kudu_client):
+    """Check that the table name from a table property is used when a table of the
+       implied name also exists.
+    """
+    with self.temp_kudu_table(kudu_client, [INT64]) as preferred_kudu_table:
+      with self.temp_kudu_table(kudu_client, [INT8]) as other_kudu_table:
+        impala_table_name = self.get_kudu_table_base_name(other_kudu_table.name)
+        cursor.execute("""
+            CREATE EXTERNAL TABLE %s
+            STORED AS KUDU
+            TBLPROPERTIES('kudu.table_name' = '%s')""" % (
+                impala_table_name, preferred_kudu_table.name))
+        with self.drop_impala_table_after_context(cursor, impala_table_name):
+          cursor.execute("DESCRIBE %s" % impala_table_name)
+          assert cursor.fetchall() == [("a", "bigint", "")]
+
+  def test_explicit_name_doesnt_exist(self, cursor, kudu_client):
+    kudu_table_name = self.random_table_name()
+    try:
+      cursor.execute("""
+          CREATE EXTERNAL TABLE %s
+          STORED AS KUDU
+          TBLPROPERTIES('kudu.table_name' = '%s')""" % (
+              self.random_table_name(), kudu_table_name))
+    except Exception as e:
+      assert "Table does not exist in Kudu: '%s'" % kudu_table_name in str(e)
+
+  def test_explicit_name_doesnt_exist_but_implicit_does(self, cursor, kudu_client):
+    """Check that when an explicit table name is given but that table doesn't exist,
+       there is no fall-through to an existing implicit table.
+    """
+    with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
+      table_name = self.random_table_name()
+      try:
+        cursor.execute("""
+            CREATE EXTERNAL TABLE %s
+            STORED AS KUDU
+            TBLPROPERTIES('kudu.table_name' = '%s')""" % (
+              self.get_kudu_table_base_name(kudu_table.name), table_name))
+      except Exception as e:
+        assert "Table does not exist in Kudu: '%s'" % table_name in str(e)
+
+
+class TestShowCreateTable(KuduTestSuite):
+
+  def assert_show_create_equals(self, cursor, create_sql, show_create_sql):
+    """Executes 'create_sql' to create a table, then runs "SHOW CREATE TABLE" and checks
+       that the output is the same as 'show_create_sql'. 'create_sql' and
+       'show_create_sql' can be templates that can be used with str.format(). format()
+       will be called with 'table' and 'db' as keyword args.
+    """
+    format_args = {"table": self.random_table_name(), "db": cursor.conn.db_name}
+    cursor.execute(create_sql.format(**format_args))
+    cursor.execute("SHOW CREATE TABLE {table}".format(**format_args))
+    assert cursor.fetchall()[0][0] == \
+        textwrap.dedent(show_create_sql.format(**format_args)).strip()
+
+  def test_primary_key_and_distribution(self, cursor):
+    # TODO: Add test cases with column comments once KUDU-1711 is fixed.
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {table} (c INT PRIMARY KEY)
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS STORED AS KUDU""",
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          PRIMARY KEY (c)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {table} (c INT PRIMARY KEY, d STRING)
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, RANGE (c) SPLIT ROWS ((1), (2))
+        STORED AS KUDU""",
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          d STRING,
+          PRIMARY KEY (c)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, RANGE (c) SPLIT ROWS (...)
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {table} (c INT, PRIMARY KEY (c))
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS STORED AS KUDU""",
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          PRIMARY KEY (c)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {table} (c INT, d STRING, PRIMARY KEY(c, d))
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, HASH (d) INTO 3 BUCKETS,
+        RANGE (c, d) SPLIT ROWS ((1, 'aaa'), (2, 'bbb')) STORED AS KUDU""",
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          d STRING,
+          PRIMARY KEY (c, d)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, HASH (d) INTO 3 BUCKETS, RANGE (c, d) SPLIT ROWS (...)
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {table} (c INT, d STRING, e INT, PRIMARY KEY(c, d))
+        DISTRIBUTE BY RANGE (c) SPLIT ROWS ((1), (2), (3)) STORED AS KUDU""",
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          d STRING,
+          e INT,
+          PRIMARY KEY (c, d)
+        )
+        DISTRIBUTE BY RANGE (c) SPLIT ROWS (...)
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
+
+  def test_properties(self, cursor):
+    # If an explicit table name is used for the Kudu table and it differs from what
+    # would be the default Kudu table name, the name should be shown as a table property.
+    kudu_table = self.random_table_name()
+    props = "'kudu.table_name'='%s'" % kudu_table
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {{table}} (c INT PRIMARY KEY)
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ({props})""".format(props=props),
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          PRIMARY KEY (c)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}', {props})""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS, props=props))
 
+    # If the name is explicitly given (or not given at all) so that the name is the same
+    # as the default name, the table name is not shown.
+    props = "'kudu.table_name'='impala::{db}.{table}'"
+    self.assert_show_create_equals(cursor,
+        """
+        CREATE TABLE {{table}} (c INT PRIMARY KEY)
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ({props})""".format(props=props),
+        """
+        CREATE TABLE {db}.{{table}} (
+          c INT,
+          PRIMARY KEY (c)
+        )
+        DISTRIBUTE BY HASH (c) INTO 3 BUCKETS
+        STORED AS KUDU
+        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
+            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
 
-@SkipIf.kudu_not_supported
-class TestKuduMemLimits(ImpalaTestSuite):
-  QUERIES = ["select * from kudu_mem_limit.lineitem where l_orderkey = -1",
-             "select * from kudu_mem_limit.lineitem where l_commitdate like '%cheese'",
-             "select * from kudu_mem_limit.lineitem limit 90"]
+
+class TestDropDb(KuduTestSuite):
+
+  def test_drop_non_empty_db(self, unique_cursor, kudu_client):
+    """Check that an attempt to drop a database will fail if Kudu tables are present
+       and that the tables remain.
+    """
+    db_name = unique_cursor.conn.db_name
+    with self.temp_kudu_table(kudu_client, [INT32], db_name=db_name) as kudu_table:
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      unique_cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (
+          impala_table_name, props))
+      unique_cursor.execute("USE DEFAULT")
+      try:
+        unique_cursor.execute("DROP DATABASE %s" % db_name)
+        assert False
+      except Exception as e:
+        assert "One or more tables exist" in str(e)
+      unique_cursor.execute("SELECT COUNT(*) FROM %s.%s" % (db_name, impala_table_name))
+      assert unique_cursor.fetchall() == [(0, )]
+
+  def test_drop_db_cascade(self, unique_cursor, kudu_client):
+    """Check that an attempt to drop a database will succeed even if Kudu tables are
+       present and that the managed tables are removed.
+    """
+    db_name = unique_cursor.conn.db_name
+    with self.temp_kudu_table(kudu_client, [INT32], db_name=db_name) as kudu_table:
+      # Create an external Kudu table
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      unique_cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (
+          impala_table_name, props))
+
+      # Create a managed Kudu table
+      managed_table_name = self.random_table_name()
+      unique_cursor.execute("""
+          CREATE TABLE %s (a INT PRIMARY KEY) DISTRIBUTE BY HASH (a) INTO 3 BUCKETS
+          STORED AS KUDU TBLPROPERTIES ('kudu.table_name' = '%s')"""
+          % (managed_table_name, managed_table_name))
+      assert kudu_client.table_exists(managed_table_name)
+
+      # Create a table in HDFS
+      hdfs_table_name = self.random_table_name()
+      unique_cursor.execute("""
+          CREATE TABLE %s (a INT) PARTITIONED BY (x INT)""" % (hdfs_table_name))
+
+      unique_cursor.execute("USE DEFAULT")
+      unique_cursor.execute("DROP DATABASE %s CASCADE" % db_name)
+      unique_cursor.execute("SHOW DATABASES")
+      assert db_name not in unique_cursor.fetchall()
+      assert kudu_client.table_exists(kudu_table.name)
+      assert not kudu_client.table_exists(managed_table_name)
+
+class TestImpalaKuduIntegration(KuduTestSuite):
+  def test_replace_kudu_table(self, cursor, kudu_client):
+    """Check that an external Kudu table is accessible if the underlying Kudu table is
+        modified using the Kudu client.
+    """
+    # Create an external Kudu table
+    col_names = ['a']
+    with self.temp_kudu_table(kudu_client, [INT32], col_names=col_names) as kudu_table:
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (
+          impala_table_name, props))
+      cursor.execute("DESCRIBE %s" % (impala_table_name))
+      assert cursor.fetchall() == [("a", "int", "")]
+
+      # Drop the underlying Kudu table and replace it with another Kudu table that has
+      # the same name but different schema
+      kudu_client.delete_table(kudu_table.name)
+      assert not kudu_client.table_exists(kudu_table.name)
+      new_col_names = ['b', 'c']
+      name_parts = kudu_table.name.split(".")
+      assert len(name_parts) == 2
+      with self.temp_kudu_table(kudu_client, [STRING, STRING], col_names=new_col_names,
+          db_name=name_parts[0], name= name_parts[1]) as new_kudu_table:
+        assert kudu_client.table_exists(new_kudu_table.name)
+        # Refresh the external table and verify that the new schema is loaded from
+        # Kudu.
+        cursor.execute("REFRESH %s" % (impala_table_name))
+        cursor.execute("DESCRIBE %s" % (impala_table_name))
+        assert cursor.fetchall() == [("b", "string", ""), ("c", "string", "")]
+
+  def test_delete_external_kudu_table(self, cursor, kudu_client):
+    """Check that Impala can recover from the case where the underlying Kudu table of
+        an external table is dropped using the Kudu client. The external table can be
+        dropped using DROP TABLE IF EXISTS statement.
+    """
+    with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
+      # Create an external Kudu table
+      impala_table_name = self.get_kudu_table_base_name(kudu_table.name)
+      props = "TBLPROPERTIES('kudu.table_name'='%s')" % kudu_table.name
+      cursor.execute("CREATE EXTERNAL TABLE %s STORED AS KUDU %s" % (
+          impala_table_name, props))
+      cursor.execute("DESCRIBE %s" % (impala_table_name))
+      assert cursor.fetchall() == [("a", "int", "")]
+      # Drop the underlying Kudu table
+      kudu_client.delete_table(kudu_table.name)
+      assert not kudu_client.table_exists(kudu_table.name)
+      err_msg = 'The table does not exist: table_name: "%s"' % (kudu_table.name)
+      try:
+        cursor.execute("REFRESH %s" % (impala_table_name))
+      except Exception as e:
+        assert err_msg in str(e)
+      cursor.execute("DROP TABLE IF EXISTS %s" % (impala_table_name))
+      cursor.execute("SHOW TABLES")
+      assert impala_table_name not in cursor.fetchall()
+
+  def test_delete_managed_kudu_table(self, cursor, kudu_client, unique_database):
+    """Check that dropping a managed Kudu table works even if the underlying Kudu table
+        has been dropped externally."""
+    impala_tbl_name = "foo"
+    cursor.execute("""CREATE TABLE %s.%s (a INT PRIMARY KEY) DISTRIBUTE BY HASH (a)
+        INTO 3 BUCKETS STORED AS KUDU""" % (unique_database, impala_tbl_name))
+    kudu_tbl_name = "impala::%s.%s" % (unique_database, impala_tbl_name)
+    assert kudu_client.table_exists(kudu_tbl_name)
+    kudu_client.delete_table(kudu_tbl_name)
+    assert not kudu_client.table_exists(kudu_tbl_name)
+    cursor.execute("DROP TABLE IF EXISTS %s" % (impala_tbl_name))
+    cursor.execute("SHOW TABLES")
+    assert impala_tbl_name not in cursor.fetchall()
+
+class TestKuduMemLimits(KuduTestSuite):
+
+  QUERIES = ["select * from lineitem where l_orderkey = -1",
+             "select * from lineitem where l_commitdate like '%cheese'",
+             "select * from lineitem limit 90"]
 
   # The value indicates the minimum memory requirements for the queries above, the first
   # memory limit corresponds to the first query
   QUERY_MEM_LIMITS = [1, 1, 10]
 
-  # The values from this array are used as a mem_limit test dimension
-  TEST_LIMITS = [1, 10, 0]
-
   CREATE = """
-    CREATE TABLE kudu_mem_limit.lineitem (
+    CREATE TABLE lineitem (
     l_orderkey BIGINT,
     l_linenumber INT,
     l_partkey BIGINT,
@@ -128,88 +451,41 @@ class TestKuduMemLimits(ImpalaTestSuite):
     l_receiptdate STRING,
     l_shipinstruct STRING,
     l_shipmode STRING,
-    l_comment STRING
-  )
-  DISTRIBUTE BY HASH (l_orderkey) INTO 9 BUCKETS
-  TBLPROPERTIES(
-    'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-    'kudu.table_name' = 'tpch_lineitem',
-    'kudu.master_addresses' = '127.0.0.1',
-    'kudu.key_columns' = 'l_orderkey,l_linenumber'
-  )
-  """
+    l_comment STRING,
+    PRIMARY KEY (l_orderkey, l_linenumber))
+  DISTRIBUTE BY HASH (l_orderkey, l_linenumber) INTO 3 BUCKETS
+  STORED AS KUDU"""
 
   LOAD = """
-  insert into kudu_mem_limit.lineitem
+  insert into lineitem
   select l_orderkey, l_linenumber, l_partkey, l_suppkey, cast(l_quantity as double),
   cast(l_extendedprice as double), cast(l_discount as double), cast(l_tax as double),
   l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct,
-  l_shipmode, l_comment from tpch_parquet.lineitem;
-  """
-
-  @classmethod
-  def get_workload(cls):
-    return 'functional-query'
+  l_shipmode, l_comment from tpch_parquet.lineitem"""
 
   @classmethod
-  def add_test_dimensions(cls):
-    super(TestKuduMemLimits, cls).add_test_dimensions()
+  def auto_create_db(cls):
+    return True
 
-    # add mem_limit as a test dimension.
-    new_dimension = TestDimension('mem_limit', *TestKuduMemLimits.TEST_LIMITS)
-    cls.TestMatrix.add_dimension(new_dimension)
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
-
-  @classmethod
-  def setup_class(cls):
-    super(TestKuduMemLimits, cls).setup_class()
-    cls.cleanup_db("kudu_mem_limit")
-    cls.client.execute("create database kudu_mem_limit")
-    cls.client.execute(cls.CREATE)
-    cls.client.execute(cls.LOAD)
-
-  @classmethod
-  def teardown_class(cls):
-    cls.cleanup_db("kudu_mem_limit")
-    super(TestKuduMemLimits, cls).teardown_class()
-
-  # TODO(kudu-merge) IMPALA-3178 DROP DATABASE ... CASCADE is broken in Kudu so we need
-  # to clean up table-by-table. Once solved, delete this and rely on the overriden method.
-  @classmethod
-  def cleanup_db(cls, db_name):
-    cls.client.execute("use default")
-    cls.client.set_configuration({'sync_ddl': True})
-    if db_name + "\t" in cls.client.execute("show databases", ).data:
-      # We use quoted identifiers to avoid name clashes with keywords
-      for tbl_name in cls.client.execute("show tables in `" + db_name + "`").data:
-        full_tbl_name = '`%s`.`%s`' % (db_name, tbl_name)
-        result = cls.client.execute("describe formatted " + full_tbl_name)
-        if 'VIRTUAL_VIEW' in '\n'.join(result.data):
-          cls.client.execute("drop view " + full_tbl_name)
-        else:
-          cls.client.execute("drop table " + full_tbl_name)
-      for fn_result in cls.client.execute("show functions in `" + db_name + "`").data:
-        # First column is the return type, second is the function signature
-        fn_name = fn_result.split('\t')[1]
-        cls.client.execute("drop function `%s`.%s" % (db_name, fn_name))
-      for fn_result in cls.client.execute(\
-        "show aggregate functions in `" + db_name + "`").data:
-        fn_name = fn_result.split('\t')[1]
-        cls.client.execute("drop function `%s`.%s" % (db_name, fn_name))
-      cls.client.execute("drop database `" + db_name + "`")
+  @pytest.fixture(scope='class')
+  def test_data(cls, cls_cursor):
+    cls_cursor.execute(cls.CREATE)
+    cls_cursor.execute(cls.LOAD)
 
   @pytest.mark.execute_serially
-  def test_low_mem_limit_low_selectivity_scan(self, vector):
+  @pytest.mark.usefixtures("test_data")
+  @pytest.mark.parametrize("mem_limit", [1, 10, 0])
+  def test_low_mem_limit_low_selectivity_scan(self, cursor, mem_limit, vector):
     """Tests that the queries specified in this test suite run under the given
     memory limits."""
-    mem_limit = copy(vector.get_value('mem_limit'))
-    exec_options = copy(vector.get_value('exec_option'))
+    exec_options = dict((k, str(v)) for k, v
+                        in vector.get_value('exec_option').iteritems())
     exec_options['mem_limit'] = "{0}m".format(mem_limit)
     for i, q in enumerate(self.QUERIES):
       try:
-        self.execute_query(q, exec_options)
-        pass
-      except ImpalaBeeswaxException as e:
+        cursor.execute(q, configuration=exec_options)
+        cursor.fetchall()
+      except Exception as e:
         if (mem_limit > self.QUERY_MEM_LIMITS[i]):
           raise
         assert "Memory limit exceeded" in str(e)



[05/14] incubator-impala git commit: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/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 a0a0122..def0be2 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -145,6 +145,7 @@ import org.apache.impala.analysis.SqlParserSymbols;
     keywordMap.put("iregexp", new Integer(SqlParserSymbols.KW_IREGEXP));
     keywordMap.put("is", new Integer(SqlParserSymbols.KW_IS));
     keywordMap.put("join", new Integer(SqlParserSymbols.KW_JOIN));
+    keywordMap.put("kudu", new Integer(SqlParserSymbols.KW_KUDU));
     keywordMap.put("last", new Integer(SqlParserSymbols.KW_LAST));
     keywordMap.put("left", new Integer(SqlParserSymbols.KW_LEFT));
     keywordMap.put("like", new Integer(SqlParserSymbols.KW_LIKE));
@@ -173,6 +174,7 @@ import org.apache.impala.analysis.SqlParserSymbols;
     keywordMap.put("partitions", new Integer(SqlParserSymbols.KW_PARTITIONS));
     keywordMap.put("preceding", new Integer(SqlParserSymbols.KW_PRECEDING));
     keywordMap.put("prepare_fn", new Integer(SqlParserSymbols.KW_PREPARE_FN));
+    keywordMap.put("primary", new Integer(SqlParserSymbols.KW_PRIMARY));
     keywordMap.put("produced", new Integer(SqlParserSymbols.KW_PRODUCED));
     keywordMap.put("purge", new Integer(SqlParserSymbols.KW_PURGE));
     keywordMap.put("range", new Integer(SqlParserSymbols.KW_RANGE));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index c47135e..e79c5ab 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -25,20 +25,15 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.junit.Assert;
-import org.junit.Test;
+import junit.framework.Assert;
 
+import org.apache.impala.analysis.CreateTableStmt;
 import org.apache.impala.catalog.ArrayType;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.DataSource;
 import org.apache.impala.catalog.DataSourceTable;
+import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.StructField;
@@ -50,10 +45,21 @@ import org.apache.impala.common.FrontendTestBase;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.testutil.TestUtils;
 import org.apache.impala.util.MetaStoreUtil;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import org.junit.Test;
+
 public class AnalyzeDDLTest extends FrontendTestBase {
 
   @Test
@@ -1233,6 +1239,11 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create table if not exists functional.zipcode_incomes like parquet "
         + "'/test-warehouse/schemas/malformed_decimal_tiny.parquet'",
         "Unsupported parquet type FIXED_LEN_BYTE_ARRAY for field c1");
+
+    // Invalid file format
+    AnalysisError("create table newtbl_kudu like parquet " +
+        "'/test-warehouse/schemas/alltypestiny.parquet' stored as kudu",
+        "CREATE TABLE LIKE FILE statement is not supported for Kudu tables.");
   }
 
   @Test
@@ -1278,11 +1289,11 @@ public class AnalyzeDDLTest extends FrontendTestBase {
 
     // Unsupported file formats
     AnalysisError("create table foo stored as sequencefile as select 1",
-        "CREATE TABLE AS SELECT does not support (SEQUENCEFILE) file format. " +
-         "Supported formats are: (PARQUET, TEXTFILE)");
+        "CREATE TABLE AS SELECT does not support the (SEQUENCEFILE) file format. " +
+         "Supported formats are: (PARQUET, TEXTFILE, KUDU)");
     AnalysisError("create table foo stored as RCFILE as select 1",
-        "CREATE TABLE AS SELECT does not support (RCFILE) file format. " +
-         "Supported formats are: (PARQUET, TEXTFILE)");
+        "CREATE TABLE AS SELECT does not support the (RCFILE) file format. " +
+         "Supported formats are: (PARQUET, TEXTFILE, KUDU)");
 
     // CTAS with a WITH clause and inline view (IMPALA-1100)
     AnalyzesOk("create table test_with as with with_1 as (select 1 as int_col from " +
@@ -1330,6 +1341,17 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create table p partitioned by (tinyint_col, int_col) as " +
         "select double_col, int_col, tinyint_col from functional.alltypes",
         "Partition column name mismatch: tinyint_col != int_col");
+
+    // CTAS into managed Kudu tables
+    AnalyzesOk("create table t primary key (id) distribute by hash (id) into 3 buckets" +
+        " stored as kudu as select id, bool_col, tinyint_col, smallint_col, int_col, " +
+        "bigint_col, float_col, double_col, date_string_col, string_col " +
+        "from functional.alltypestiny");
+    // CTAS in an external Kudu table
+    AnalysisError("create external table t stored as kudu " +
+        "tblproperties('kudu.table_name'='t') as select id, int_col from " +
+        "functional.alltypestiny", "CREATE TABLE AS SELECT is not supported for " +
+        "external Kudu tables.");
   }
 
   @Test
@@ -1376,6 +1398,12 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "No FileSystem for scheme: foofs");
     AnalysisError("create table functional.baz like functional.alltypes location '  '",
         "URI path cannot be empty.");
+
+    // CREATE TABLE LIKE is not currently supported for Kudu tables (see IMPALA-4052)
+    AnalysisError("create table kudu_tbl like functional.alltypestiny stored as kudu",
+        "CREATE TABLE LIKE is not supported for Kudu tables");
+    AnalysisError("create table tbl like functional_kudu.dimtbl", "Cloning a Kudu " +
+        "table using CREATE TABLE LIKE is not supported.");
   }
 
   @Test
@@ -1458,12 +1486,18 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     String [] fileFormats =
         {"TEXTFILE", "SEQUENCEFILE", "PARQUET", "PARQUETFILE", "RCFILE"};
     for (String format: fileFormats) {
-      AnalyzesOk(String.format("create table new_table (i int) " +
-          "partitioned by (d decimal) comment 'c' stored as %s", format));
-      // No column definitions.
-      AnalysisError(String.format("create table new_table " +
-          "partitioned by (d decimal) comment 'c' stored as %s", format),
-          "Table requires at least 1 column");
+      for (String create: ImmutableList.of("create table", "create external table")) {
+        AnalyzesOk(String.format("%s new_table (i int) " +
+            "partitioned by (d decimal) comment 'c' stored as %s", create, format));
+        // No column definitions.
+        AnalysisError(String.format("%s new_table " +
+            "partitioned by (d decimal) comment 'c' stored as %s", create, format),
+            "Table requires at least 1 column");
+      }
+      AnalysisError(String.format("create table t (i int primary key) stored as %s",
+          format), "Only Kudu tables can specify a PRIMARY KEY");
+      AnalysisError(String.format("create table t (i int, primary key(i)) stored as %s",
+          format), "Only Kudu tables can specify a PRIMARY KEY");
     }
 
     // Note: Backslashes need to be escaped twice - once for Java and once for Impala.
@@ -1541,7 +1575,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create table cached_tbl(i int) location " +
         "'file:///test-warehouse/cache_tbl' cached in 'testPool'",
         "Location 'file:/test-warehouse/cache_tbl' cannot be cached. " +
-        "Please retry without caching: CREATE TABLE default.cached_tbl ... UNCACHED");
+        "Please retry without caching: CREATE TABLE ... UNCACHED");
 
     // Invalid database name.
     AnalysisError("create table `???`.new_table (x int) PARTITIONED BY (y int)",
@@ -1668,175 +1702,179 @@ public class AnalyzeDDLTest extends FrontendTestBase {
   }
 
   @Test
-  public void TestCreateKuduTable() {
+  public void TestCreateManagedKuduTable() {
     TestUtils.assumeKuduIsSupported();
-    // Create Kudu Table with all required properties
-    AnalyzesOk("create table tab (x int) " +
-        "distribute by hash into 2 buckets tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080, 127.0.0.1:8081', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")");
-
-    // Check that all properties are present
-    AnalysisError("create table tab (x int) " +
-        "distribute by hash into 2 buckets tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")",
-        "Kudu table is missing parameters in table properties. Please verify " +
-        "if kudu.table_name, kudu.master_addresses, and kudu.key_columns are " +
-        "present and have valid values.");
-
-    AnalysisError("create table tab (x int) " +
-            "distribute by hash into 2 buckets tblproperties (" +
-            "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-            "'kudu.table_name'='tab'," +
-            "'kudu.key_columns' = 'a,b,c'"
-            + ")",
-        "Kudu table is missing parameters in table properties. Please verify " +
-            "if kudu.table_name, kudu.master_addresses, and kudu.key_columns are " +
-            "present and have valid values.");
-
-    AnalysisError("create table tab (x int) " +
-        "distribute by hash into 2 buckets tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080'" +
-        ")",
-        "Kudu table is missing parameters in table properties. Please verify " +
-        "if kudu.table_name, kudu.master_addresses, and kudu.key_columns are " +
-        "present and have valid values.");
-
-    // Check that properties are not empty
-    AnalysisError("create table tab (x int) " +
-            "distribute by hash into 2 buckets tblproperties (" +
-            "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-            "'kudu.table_name'=''," +
-            "'kudu.master_addresses' = '127.0.0.1:8080', " +
-            "'kudu.key_columns' = 'a,b,c'" +
-            ")",
-        "Kudu table is missing parameters in table properties. Please verify " +
-            "if kudu.table_name, kudu.master_addresses, and kudu.key_columns are " +
-            "present and have valid values.");
-
-    AnalysisError("create table tab (x int) " +
-            "distribute by hash into 2 buckets tblproperties (" +
-            "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-            "'kudu.table_name'='asd'," +
-            "'kudu.master_addresses' = '', " +
-            "'kudu.key_columns' = 'a,b,c'" +
-            ")",
-        "Kudu table is missing parameters in table properties. Please verify " +
-            "if kudu.table_name, kudu.master_addresses, and kudu.key_columns are " +
-            "present and have valid values.");
-
-    // Don't allow caching
-    AnalysisError("create table tab (x int) cached in 'testPool' " +
-        "distribute by hash into 2 buckets tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")", "A Kudu table cannot be cached in HDFS.");
-
+    // Test primary keys and distribute by clauses
+    AnalyzesOk("create table tab (x int primary key) distribute by hash(x) " +
+        "into 8 buckets stored as kudu");
+    AnalyzesOk("create table tab (x int, primary key(x)) distribute by hash(x) " +
+        "into 8 buckets stored as kudu");
+    AnalyzesOk("create table tab (x int, y int, primary key (x, y)) " +
+        "distribute by hash(x, y) into 8 buckets stored as kudu");
+    AnalyzesOk("create table tab (x int, y int, primary key (x)) " +
+        "distribute by hash(x) into 8 buckets stored as kudu");
+    AnalyzesOk("create table tab (x int, y int, primary key(x, y)) " +
+        "distribute by hash(y) into 8 buckets stored as kudu");
+    // Multilevel partitioning. Data is split into 3 buckets based on 'x' and each
+    // bucket is partitioned into 4 tablets based on the split points of 'y'.
+    AnalyzesOk("create table tab (x int, y string, primary key(x, y)) " +
+        "distribute by hash(x) into 3 buckets, range(y) split rows " +
+        "(('aa'), ('bb'), ('cc')) stored as kudu");
+    // Key column in upper case
+    AnalyzesOk("create table tab (x int, y int, primary key (X)) " +
+        "distribute by hash (x) into 8 buckets stored as kudu");
     // Flexible Partitioning
-    AnalyzesOk("create table tab (a int, b int, c int, d int) " +
-        "distribute by hash(a,b) into 8 buckets, hash(c) into 2 buckets " +
-        "tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")");
-
-    AnalyzesOk("create table tab (a int, b int, c int, d int) " +
-        "distribute by hash into 8 buckets " +
-        "tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")");
-
-    // DISTRIBUTE BY is required for managed tables.
-    AnalysisError("create table tab (a int) tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a'" +
-        ")",
-        "A data distribution must be specified using the DISTRIBUTE BY clause.");
-
-    // DISTRIBUTE BY is not allowed for external tables.
-    AnalysisError("create external table tab (a int) " +
-        "distribute by hash into 3 buckets tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a'" +
-        ")",
-        "The DISTRIBUTE BY clause may not be specified for external tables.");
-
-    // Number of buckets must be larger 1
-    AnalysisError("create table tab (a int, b int, c int, d int) " +
-        "distribute by hash(a,b) into 8 buckets, hash(c) into 1 buckets " +
-        "tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")",
-        "Number of buckets in DISTRIBUTE BY clause 'HASH(c) INTO 1 BUCKETS' must " +
-            "be larger than 1");
-
-    // Key ranges must match the column types.
-    // TODO(kudu-merge) uncomment this when IMPALA-3156 is addressed.
-    //AnalysisError("create table tab (a int, b int, c int, d int) " +
-    //    "distribute by hash(a,b,c) into 8 buckets, " +
-    //    "range(a) split rows ((1),('abc'),(3)) " +
-    //    "tblproperties (" +
-    //    "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-    //    "'kudu.table_name'='tab'," +
-    //    "'kudu.master_addresses' = '127.0.0.1:8080', " +
-    //    "'kudu.key_columns' = 'a,b,c')");
-
-    // Distribute range data types are picked up during analysis and forwarded to Kudu
-    AnalyzesOk("create table tab (a int, b int, c int, d int) " +
-        "distribute by hash(a,b,c) into 8 buckets, " +
-        "range(a) split rows ((1),(2),(3)) " +
-        "tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")");
-
+    AnalyzesOk("create table tab (a int, b int, c int, d int, primary key (a, b, c))" +
+        "distribute by hash (a, b) into 8 buckets, hash(c) into 2 buckets stored as " +
+        "kudu");
+    // No columns specified in the DISTRIBUTE BY HASH clause
+    AnalyzesOk("create table tab (a int primary key, b int, c int, d int) " +
+        "distribute by hash into 8 buckets stored as kudu");
+    // Distribute range data types are picked up during analysis and forwarded to Kudu.
+    // Column names in distribute params should also be case-insensitive.
+    AnalyzesOk("create table tab (a int, b int, c int, d int, primary key(a, b, c, d))" +
+        "distribute by hash (a, B, c) into 8 buckets, " +
+        "range (A) split rows ((1),(2),(3)) stored as kudu");
+    // Allowing range distribution on a subset of the primary keys
+    AnalyzesOk("create table tab (id int, name string, valf float, vali bigint, " +
+        "primary key (id, name)) distribute by range (name) split rows (('abc')) " +
+        "stored as kudu");
+    // Null values in SPLIT ROWS
+    AnalysisError("create table tab (id int, name string, primary key(id, name)) " +
+        "distribute by hash (id) into 3 buckets, range (name) split rows ((null),(1)) " +
+        "stored as kudu", "Split values cannot be NULL. Split row: (NULL)");
+    // Primary key specified in tblproperties
+    AnalysisError(String.format("create table tab (x int) distribute by hash (x) " +
+        "into 8 buckets stored as kudu tblproperties ('%s' = 'x')",
+        KuduTable.KEY_KEY_COLUMNS), "PRIMARY KEY must be used instead of the table " +
+        "property");
+    // Primary key column that doesn't exist
+    AnalysisError("create table tab (x int, y int, primary key (z)) " +
+        "distribute by hash (x) into 8 buckets stored as kudu",
+        "PRIMARY KEY column 'z' does not exist in the table");
+    // Invalid composite primary key
+    AnalysisError("create table tab (x int primary key, primary key(x)) stored " +
+        "as kudu", "Multiple primary keys specified. Composite primary keys can " +
+        "be specified using the PRIMARY KEY (col1, col2, ...) syntax at the end " +
+        "of the column definition.");
+    AnalysisError("create table tab (x int primary key, y int primary key) stored " +
+        "as kudu", "Multiple primary keys specified. Composite primary keys can " +
+        "be specified using the PRIMARY KEY (col1, col2, ...) syntax at the end " +
+        "of the column definition.");
+    // Specifying the same primary key column multiple times
+    AnalysisError("create table tab (x int, primary key (x, x)) distribute by hash (x) " +
+        "into 8 buckets stored as kudu",
+        "Column 'x' is listed multiple times as a PRIMARY KEY.");
     // Each split row size should equals to the number of range columns.
-    AnalysisError("create table tab (a int, b int, c int, d int) " +
-        "distribute by range(a) split rows ((1,'extra_val'),(2),(3)) " +
-        "tblproperties (" +
-        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-        "'kudu.table_name'='tab'," +
-        "'kudu.master_addresses' = '127.0.0.1:8080', " +
-        "'kudu.key_columns' = 'a,b,c'" +
-        ")",
+    AnalysisError("create table tab (a int, b int, c int, d int, primary key(a, b, c)) " +
+        "distribute by range(a) split rows ((1,'extra_val'),(2),(3)) stored as kudu",
         "SPLIT ROWS has different size than number of projected key columns: 1. " +
         "Split row: (1, 'extra_val')");
-
+    // Key ranges must match the column types.
+    AnalysisError("create table tab (a int, b int, c int, d int, primary key(a, b, c)) " +
+        "distribute by hash (a, b, c) into 8 buckets, " +
+        "range (a) split rows ((1), ('abc'), (3)) stored as kudu",
+        "Split value 'abc' (type: STRING) is not type compatible with column 'a'" +
+        " (type: INT).");
+    // Non-key column used in DISTRIBUTE BY
+    AnalysisError("create table tab (a int, b string, c bigint, primary key (a)) " +
+        "distribute by range (b) split rows (('abc')) stored as kudu",
+        "Column 'b' in 'RANGE (b) SPLIT ROWS (('abc'))' is not a key column. " +
+        "Only key columns can be used in DISTRIBUTE BY.");
     // No float split keys
-    AnalysisError("create table tab (a int, b int, c int, d int) " +
-            "distribute by hash(a,b,c) into 8 buckets, " +
-            "range(a) split rows ((1.2),('abc'),(3)) " +
-            "tblproperties (" +
-            "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler', " +
-            "'kudu.table_name'='tab'," +
-            "'kudu.master_addresses' = '127.0.0.1:8080', " +
-            "'kudu.key_columns' = 'a,b,c'" +
-            ")",
-        "Only integral and string values allowed for split rows.");
+    AnalysisError("create table tab (a int, b int, c int, d int, primary key (a, b, c))" +
+        "distribute by hash (a, b, c) into 8 buckets, " +
+        "range (a) split rows ((1.2), ('abc'), (3)) stored as kudu",
+        "Split value 1.2 (type: DECIMAL(2,1)) is not type compatible with column 'a' " +
+        "(type: INT).");
+    // Non-existing column used in DISTRIBUTE BY
+    AnalysisError("create table tab (a int, b int, primary key (a, b)) " +
+        "distribute by range(unknown_column) split rows (('abc')) stored as kudu",
+        "Column 'unknown_column' in 'RANGE (unknown_column) SPLIT ROWS (('abc'))' " +
+        "is not a key column. Only key columns can be used in DISTRIBUTE BY");
+    // Kudu table name is specified in tblproperties
+    AnalyzesOk("create table tab (x int primary key) distribute by hash (x) " +
+        "into 8 buckets stored as kudu tblproperties ('kudu.table_name'='tab_1'," +
+        "'kudu.num_tablet_replicas'='1'," +
+        "'kudu.master_addresses' = '127.0.0.1:8080, 127.0.0.1:8081')");
+    // No port is specified in kudu master address
+    AnalyzesOk("create table tdata_no_port (id int primary key, name string, " +
+        "valf float, vali bigint) DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) " +
+        "STORED AS KUDU tblproperties('kudu.master_addresses'='127.0.0.1')");
+    // Not using the STORED AS KUDU syntax to specify a Kudu table
+    AnalysisError("create table tab (x int primary key) tblproperties (" +
+        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler')",
+        CreateTableStmt.KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    AnalysisError("create table tab (x int primary key) stored as kudu tblproperties (" +
+        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler')",
+        CreateTableStmt.KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    // Invalid value for number of replicas
+    AnalysisError("create table t (x int primary key) stored as kudu tblproperties (" +
+        "'kudu.num_tablet_replicas'='1.1')",
+        "Table property 'kudu.num_tablet_replicas' must be an integer.");
+    // Don't allow caching
+    AnalysisError("create table tab (x int primary key) stored as kudu cached in " +
+        "'testPool'", "A Kudu table cannot be cached in HDFS.");
+    // LOCATION cannot be used with Kudu tables
+    AnalysisError("create table tab (a int primary key) distribute by hash (a) " +
+        "into 3 buckets stored as kudu location '/test-warehouse/'",
+        "LOCATION cannot be specified for a Kudu table.");
+    // DISTRIBUTE BY is required for managed tables.
+    AnalysisError("create table tab (a int, primary key (a)) stored as kudu",
+        "Table distribution must be specified for managed Kudu tables.");
+    AnalysisError("create table tab (a int) stored as kudu",
+        "A primary key is required for a Kudu table.");
+    // Using ROW FORMAT with a Kudu table
+    AnalysisError("create table tab (x int primary key) " +
+        "row format delimited escaped by 'X' stored as kudu",
+        "ROW FORMAT cannot be specified for file format KUDU.");
+    // Using PARTITIONED BY with a Kudu table
+    AnalysisError("create table tab (x int primary key) " +
+        "partitioned by (y int) stored as kudu", "PARTITIONED BY cannot be used " +
+        "in Kudu tables.");
+  }
+
+  @Test
+  public void TestCreateExternalKuduTable() {
+    AnalyzesOk("create external table t stored as kudu " +
+        "tblproperties('kudu.table_name'='t')");
+    // Use all allowed optional table props.
+    AnalyzesOk("create external table t stored as kudu tblproperties (" +
+        "'kudu.table_name'='tab'," +
+        "'kudu.master_addresses' = '127.0.0.1:8080, 127.0.0.1:8081')");
+    // Kudu table should be specified using the STORED AS KUDU syntax.
+    AnalysisError("create external table t tblproperties (" +
+        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler'," +
+        "'kudu.table_name'='t')",
+        CreateTableStmt.KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    // Columns should not be specified in an external Kudu table
+    AnalysisError("create external table t (x int) stored as kudu " +
+        "tblproperties('kudu.table_name'='t')",
+        "Columns cannot be specified with an external Kudu table.");
+    // Primary keys cannot be specified in an external Kudu table
+    AnalysisError("create external table t (x int primary key) stored as kudu " +
+        "tblproperties('kudu.table_name'='t')", "Primary keys cannot be specified " +
+        "for an external Kudu table");
+    // Invalid syntax for specifying a Kudu table
+    AnalysisError("create external table t (x int) stored as parquet tblproperties (" +
+        "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler'," +
+        "'kudu.table_name'='t')", CreateTableStmt.KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    AnalysisError("create external table t stored as kudu tblproperties (" +
+        "'storage_handler'='foo', 'kudu.table_name'='t')",
+        CreateTableStmt.KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
+    // Cannot specify the number of replicas for external Kudu tables
+    AnalysisError("create external table tab (x int) stored as kudu " +
+        "tblproperties ('kudu.num_tablet_replicas' = '1', " +
+        "'kudu.table_name'='tab')",
+        "Table property 'kudu.num_tablet_replicas' cannot be used with an external " +
+        "Kudu table.");
+    // Don't allow caching
+    AnalysisError("create external table t stored as kudu cached in 'testPool' " +
+        "tblproperties('kudu.table_name'='t')", "A Kudu table cannot be cached in HDFS.");
+    // LOCATION cannot be used for a Kudu table
+    AnalysisError("create external table t stored as kudu " +
+        "location '/test-warehouse' tblproperties('kudu.table_name'='t')",
+        "LOCATION cannot be specified for a Kudu table.");
   }
 
   @Test
@@ -2036,6 +2074,13 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create table functional.new_table (i int) " +
         "partitioned by (x struct<f1:int>)",
         "Type 'STRUCT<f1:INT>' is not supported as partition-column type in column: x");
+
+    // Kudu specific clauses used in an Avro table.
+    AnalysisError("create table functional.new_table (i int primary key) " +
+        "distribute by hash(i) into 3 buckets stored as avro",
+        "Only Kudu tables can use the DISTRIBUTE BY clause.");
+    AnalysisError("create table functional.new_table (i int primary key) " +
+        "stored as avro", "Only Kudu tables can specify a PRIMARY KEY.");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/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 8b8ea42..6308293 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -2225,8 +2225,21 @@ public class ParserTest {
       // No column definitions.
       ParsesOk(String.format(
           "CREATE EXTERNAL TABLE Foo COMMENT 'c' STORED AS %s LOCATION '/b'", format));
+      ParserError(String.format("CREATE EXTERNAL TABLE t PRIMARY KEYS (i) STORED AS " +
+          "%s", format));
     }
 
+    ParsesOk("CREATE TABLE foo (i INT) STORED AS KUDU");
+    ParsesOk("CREATE TABLE foo (i INT PRIMARY KEY) STORED AS KUDU");
+    ParsesOk("CREATE TABLE foo (i INT, j INT, PRIMARY KEY (i, j)) STORED AS KUDU");
+    ParsesOk("CREATE TABLE foo (i INT, j INT, PRIMARY KEY (j, i)) STORED AS KUDU");
+    ParsesOk("CREATE TABLE foo (i INT PRIMARY KEY, PRIMARY KEY(i)) STORED AS KUDU");
+    ParsesOk("CREATE TABLE foo (i INT PRIMARY KEY, j INT PRIMARY KEY) STORED AS KUDU");
+    ParserError("CREATE TABLE foo (i INT) PRIMARY KEY (i) STORED AS KUDU");
+    ParserError("CREATE TABLE foo (i INT, PRIMARY KEY) STORED AS KUDU");
+    ParserError("CREATE TABLE foo (PRIMARY KEY(a), a INT) STORED AS KUDU");
+    ParserError("CREATE TABLE foo (i INT) PRIMARY KEY (i) STORED AS KUDU");
+
     // Table Properties
     String[] tblPropTypes = {"TBLPROPERTIES", "WITH SERDEPROPERTIES"};
     for (String propType: tblPropTypes) {
@@ -2383,6 +2396,7 @@ public class ParserTest {
     ParserError("CREATE TABLE Foo (i int) DISTRIBUTE BY RANGE(i) " +
         "SPLIT ROWS ()");
     ParserError("CREATE TABLE Foo (i int) DISTRIBUTE BY RANGE(i)");
+    ParserError("CREATE EXTERNAL TABLE Foo DISTRIBUTE BY HASH INTO 4 BUCKETS");
 
     // Combine both
     ParsesOk("CREATE TABLE Foo (i int) DISTRIBUTE BY HASH(i) INTO 4 BUCKETS, RANGE(i) " +
@@ -2391,6 +2405,9 @@ public class ParserTest {
     // Can only have one range clause
     ParserError("CREATE TABLE Foo (i int) DISTRIBUTE BY HASH(i) INTO 4 BUCKETS, RANGE(i) " +
         "SPLIT ROWS ((1, 2.0, 'asdas')), RANGE(i) SPLIT ROWS ((1, 2.0, 'asdas'))");
+    // Range needs to be the last DISTRIBUTE BY clause
+    ParserError("CREATE TABLE Foo (i int) DISTRIBUTE BY RANGE(i) SPLIT ROWS ((1),(2)), " +
+        "HASH (i) INTO 3 BUCKETS");
   }
 
   @Test
@@ -2527,6 +2544,15 @@ public class ParserTest {
     ParsesOk("CREATE TABLE Foo STORED AS PARQUET AS SELECT 1");
     ParsesOk("CREATE TABLE Foo ROW FORMAT DELIMITED STORED AS PARQUETFILE AS SELECT 1");
     ParsesOk("CREATE TABLE Foo TBLPROPERTIES ('a'='b', 'c'='d') AS SELECT * from bar");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a, b) AS SELECT * from bar");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a, b) DISTRIBUTE BY HASH INTO 2 BUCKETS " +
+        "AS SELECT * from bar");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a, b) DISTRIBUTE BY HASH (b) INTO 2 " +
+        "BUCKETS AS SELECT * from bar");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a, b) DISTRIBUTE BY RANGE (b) SPLIT ROWS " +
+        "(('foo'), ('bar')) STORED AS KUDU AS SELECT * from bar");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a, b) DISTRIBUTE BY RANGE SPLIT ROWS " +
+        "(('foo'), ('bar')) STORED AS KUDU AS SELECT * from bar");
 
     // With clause works
     ParsesOk("CREATE TABLE Foo AS with t1 as (select 1) select * from t1");
@@ -2554,8 +2580,10 @@ public class ParserTest {
     ParserError("CREATE TABLE Foo PARTITIONED BY (a, b=2) AS SELECT * from Bar");
 
     // Flexible partitioning
-    ParsesOk("CREATE TABLE Foo DISTRIBUTE BY HASH(i) INTO 4 BUCKETS AS SELECT 1");
-    ParsesOk("CREATE TABLE Foo DISTRIBUTE BY HASH(a) INTO 4 BUCKETS " +
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (i) DISTRIBUTE BY HASH(i) INTO 4 BUCKETS AS " +
+        "SELECT 1");
+    ParserError("CREATE TABLE Foo DISTRIBUTE BY HASH(i) INTO 4 BUCKETS AS SELECT 1");
+    ParsesOk("CREATE TABLE Foo PRIMARY KEY (a) DISTRIBUTE BY HASH(a) INTO 4 BUCKETS " +
         "TBLPROPERTIES ('a'='b', 'c'='d') AS SELECT * from bar");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/test/java/org/apache/impala/service/JdbcTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/service/JdbcTest.java b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
index a1f4a29..6309e8e 100644
--- a/fe/src/test/java/org/apache/impala/service/JdbcTest.java
+++ b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
@@ -49,7 +49,7 @@ import com.google.common.collect.Lists;
  * JdbcTest
  *
  * Basic JDBC metadata test. It exercises getTables, getCatalogs, getSchemas,
- * getTableTypes, getColumns.
+ * getTableTypes, getColumnNames.
  *
  */
 public class JdbcTest {
@@ -204,7 +204,7 @@ public class JdbcTest {
     ResultSet rs = con_.getMetaData().getColumns(null,
         "functional", "alltypessmall", "s%rin%");
 
-    // validate the metadata for the getColumns result set
+    // validate the metadata for the getColumnNames result set
     ResultSetMetaData rsmd = rs.getMetaData();
     assertEquals("TABLE_CAT", rsmd.getColumnName(1));
     assertTrue(rs.next());
@@ -477,7 +477,7 @@ public class JdbcTest {
   }
 
   /**
-   * Validate the Metadata for the result set of a metadata getColumns call.
+   * Validate the Metadata for the result set of a metadata getColumnNames call.
    */
   @Test
   public void testMetaDataGetColumnsMetaData() throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java b/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
index 1dfed8a..b3167ce 100644
--- a/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
+++ b/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
@@ -43,7 +43,7 @@ public class ImpaladTestCatalog extends ImpaladCatalog {
    * Takes an AuthorizationConfig to bootstrap the backing CatalogServiceCatalog.
    */
   public ImpaladTestCatalog(AuthorizationConfig authzConfig) {
-    super();
+    super("127.0.0.1");
     CatalogServiceCatalog catalogServerCatalog =
         CatalogServiceTestCatalog.createWithAuth(authzConfig.getSentryConfig());
     // Bootstrap the catalog by adding all dbs, tables, and functions.
@@ -51,7 +51,7 @@ public class ImpaladTestCatalog extends ImpaladCatalog {
       // Adding DB should include all tables/fns in that database.
       addDb(db);
     }
-    authPolicy_ = ((CatalogServiceTestCatalog) catalogServerCatalog).getAuthPolicy();
+    authPolicy_ = catalogServerCatalog.getAuthPolicy();
     srcCatalog_ = catalogServerCatalog;
     setIsReady(true);
   }
@@ -88,4 +88,4 @@ public class ImpaladTestCatalog extends ImpaladCatalog {
     db.addTable(newTbl);
     return super.getTable(dbName, tableName);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/infra/python/deps/download_requirements
----------------------------------------------------------------------
diff --git a/infra/python/deps/download_requirements b/infra/python/deps/download_requirements
index daa5025..d586104 100755
--- a/infra/python/deps/download_requirements
+++ b/infra/python/deps/download_requirements
@@ -29,5 +29,5 @@ PY26="$(./find_py26.py)"
 "$PY26" pip_download.py virtualenv 13.1.0
 # kudu-python is downloaded separately because pip install attempts to execute a
 # setup.py subcommand for kudu-python that can fail even if the download succeeds.
-"$PY26" pip_download.py kudu-python 0.1.1
+"$PY26" pip_download.py kudu-python 0.2.0
 popd

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/infra/python/deps/requirements.txt
----------------------------------------------------------------------
diff --git a/infra/python/deps/requirements.txt b/infra/python/deps/requirements.txt
index b433713..2ba0853 100644
--- a/infra/python/deps/requirements.txt
+++ b/infra/python/deps/requirements.txt
@@ -82,13 +82,9 @@ texttable == 0.8.3
 # functional and determines the expected kudu-python version. The version must be listed
 # in the format below including # and spacing. Keep this formatting! The kudu-python
 # version in download_requirements must be kept in sync with this version.
-# kudu-python==0.1.1
+# kudu-python==0.2.0
   Cython == 0.23.4
   numpy == 1.10.4
-  # These should eventually be removed  https://issues.apache.org/jira/browse/KUDU-1456
-  unittest2 == 1.1.0
-    linecache2 == 1.0.0
-    traceback2 == 1.4.0
 
 # For dev purposes, not used in scripting. Version 1.2.1 is the latest that supports 2.6.
 ipython == 1.2.1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/bin/generate-schema-statements.py
----------------------------------------------------------------------
diff --git a/testdata/bin/generate-schema-statements.py b/testdata/bin/generate-schema-statements.py
index ca3441b..4c8ef02 100755
--- a/testdata/bin/generate-schema-statements.py
+++ b/testdata/bin/generate-schema-statements.py
@@ -134,7 +134,7 @@ FILE_FORMAT_MAP = {
     "\nOUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'",
   'avro': 'AVRO',
   'hbase': "'org.apache.hadoop.hive.hbase.HBaseStorageHandler'",
-  'kudu': "'com.cloudera.kudu.hive.KuduStorageHandler'",
+  'kudu': "KUDU",
   }
 
 HIVE_TO_AVRO_TYPE_MAP = {
@@ -193,17 +193,17 @@ def build_table_template(file_format, columns, partition_columns, row_format,
   if file_format == 'hbase':
     return build_hbase_create_stmt_in_hive(columns, partition_columns, table_name)
 
+  primary_keys_clause = ""
+
   partitioned_by = str()
   if partition_columns:
     partitioned_by = 'PARTITIONED BY (%s)' % ', '.join(partition_columns.split('\n'))
 
   row_format_stmt = str()
-  if row_format:
+  if row_format and file_format != 'kudu':
     row_format_stmt = 'ROW FORMAT ' + row_format
 
-  file_format_string = str()
-  if file_format != 'kudu':
-    file_format_string = "STORED AS {file_format}"
+  file_format_string = "STORED AS {file_format}"
 
   tblproperties_clause = "TBLPROPERTIES (\n{0}\n)"
   tblproperties = {}
@@ -218,7 +218,7 @@ def build_table_template(file_format, columns, partition_columns, row_format,
     else:
       tblproperties["avro.schema.url"] = "hdfs://%s/%s/%s/{table_name}.json" \
         % (options.hdfs_namenode, options.hive_warehouse_dir, avro_schema_dir)
-  elif file_format == 'parquet':
+  elif file_format in 'parquet':
     row_format_stmt = str()
   elif file_format == 'kudu':
     # Use partitioned_by to set a trivial hash distribution
@@ -229,11 +229,9 @@ def build_table_template(file_format, columns, partition_columns, row_format,
     kudu_master = os.getenv("KUDU_MASTER_ADDRESS", "127.0.0.1")
     kudu_master_port = os.getenv("KUDU_MASTER_PORT", "7051")
     row_format_stmt = str()
-    tblproperties["storage_handler"] = "com.cloudera.kudu.hive.KuduStorageHandler"
     tblproperties["kudu.master_addresses"] = \
       "{0}:{1}".format(kudu_master, kudu_master_port)
-    tblproperties["kudu.table_name"] = table_name
-    tblproperties["kudu.key_columns"] = columns.split("\n")[0].split(" ")[0]
+    primary_keys_clause = ", PRIMARY KEY (%s)" % columns.split("\n")[0].split(" ")[0]
     # Kudu's test tables are managed.
     external = ""
 
@@ -261,7 +259,8 @@ def build_table_template(file_format, columns, partition_columns, row_format,
   # (e.g. Avro)
   stmt = """
 CREATE {external} TABLE IF NOT EXISTS {{db_name}}{{db_suffix}}.{{table_name}} (
-{columns})
+{columns}
+{primary_keys})
 {partitioned_by}
 {row_format}
 {file_format_string}
@@ -271,6 +270,7 @@ LOCATION '{{hdfs_location}}'
     external=external,
     row_format=row_format_stmt,
     columns=',\n'.join(columns.split('\n')),
+    primary_keys=primary_keys_clause,
     partitioned_by=partitioned_by,
     tblproperties=tblproperties_clause,
     file_format_string=file_format_string

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/datasets/functional/functional_schema_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index aeeba9b..76e1427 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -78,7 +78,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypes/101101.txt' OVERW
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypes/101201.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=12);
 ---- CREATE_KUDU
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
-  id INT,
+  id INT PRIMARY KEY,
   bool_col BOOLEAN,
   tinyint_col TINYINT,
   smallint_col SMALLINT,
@@ -92,13 +92,7 @@ CREATE TABLE {db_name}{db_suffix}.{table_name} (
   year INT,
   month INT
 )
-DISTRIBUTE BY HASH (id) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'id'
-);
+DISTRIBUTE BY HASH (id) INTO 3 BUCKETS STORED AS KUDU;
 ---- DEPENDENT_LOAD_KUDU
 INSERT into TABLE {db_name}{db_suffix}.{table_name}
 SELECT id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col,
@@ -161,7 +155,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesSmall/090301.txt'
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesSmall/090401.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=4);
 ---- CREATE_KUDU
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
-  id INT,
+  id INT PRIMARY KEY,
   bool_col BOOLEAN,
   tinyint_col TINYINT,
   smallint_col SMALLINT,
@@ -175,13 +169,7 @@ CREATE TABLE {db_name}{db_suffix}.{table_name} (
   year INT,
   month INT
 )
-DISTRIBUTE BY HASH (id) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'id'
-);
+DISTRIBUTE BY HASH (id) INTO 3 BUCKETS STORED AS KUDU;
 ---- DEPENDENT_LOAD_KUDU
 INSERT into TABLE {db_name}{db_suffix}.{table_name}
 SELECT id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col,
@@ -225,7 +213,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesTiny/090301.txt' O
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesTiny/090401.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=4);
 ---- CREATE_KUDU
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
-  id INT,
+  id INT PRIMARY KEY,
   bool_col BOOLEAN,
   tinyint_col TINYINT,
   smallint_col SMALLINT,
@@ -239,13 +227,7 @@ CREATE TABLE {db_name}{db_suffix}.{table_name} (
   year INT,
   month INT
 )
-DISTRIBUTE BY HASH (id) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'id'
-);
+DISTRIBUTE BY HASH (id) INTO 3 BUCKETS STORED AS KUDU;
 ---- DEPENDENT_LOAD_KUDU
 INSERT INTO TABLE {db_name}{db_suffix}.{table_name}
 SELECT id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col,
@@ -565,7 +547,7 @@ DROP VIEW IF EXISTS {db_name}{db_suffix}.{table_name};
 DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name}_idx;
 
 CREATE TABLE {db_name}{db_suffix}.{table_name}_idx (
-  kudu_idx BIGINT,
+  kudu_idx BIGINT PRIMARY KEY,
   id INT,
   bool_col BOOLEAN,
   tinyint_col TINYINT,
@@ -581,14 +563,7 @@ CREATE TABLE {db_name}{db_suffix}.{table_name}_idx (
   month INT,
   day INT
 )
-DISTRIBUTE BY HASH (kudu_idx) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'kudu_idx'
-);
-
+DISTRIBUTE BY HASH (kudu_idx) INTO 3 BUCKETS STORED AS KUDU;
 CREATE VIEW {db_name}{db_suffix}.{table_name} AS
 SELECT id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col,
        double_col, date_string_col, string_col, timestamp_col, year, month, day
@@ -651,7 +626,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesAggNoNulls/100109.
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesAggNoNulls/100110.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=1, day=10);
 ---- CREATE_KUDU
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
-  id INT,
+  id INT PRIMARY KEY,
   bool_col BOOLEAN,
   tinyint_col TINYINT,
   smallint_col SMALLINT,
@@ -666,13 +641,7 @@ CREATE TABLE {db_name}{db_suffix}.{table_name} (
   month INT,
   day INT
 )
-DISTRIBUTE BY HASH (id) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'id'
-);
+DISTRIBUTE BY HASH (id) INTO 3 BUCKETS STORED AS KUDU;
 ---- DEPENDENT_LOAD_KUDU
 INSERT into TABLE {db_name}{db_suffix}.{table_name}
 SELECT id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col,
@@ -788,17 +757,11 @@ zip int
 delimited fields terminated by ','  escaped by '\\'
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  id bigint,
+  id bigint primary key,
   name string,
   zip int
 )
-distribute by range(id) split rows ((1003), (1007))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'id'
-);
+distribute by range(id) split rows ((1003), (1007)) stored as kudu;
 ====
 ---- DATASET
 functional
@@ -816,17 +779,11 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/DimTbl/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  id bigint,
+  id bigint primary key,
   name string,
   zip int
 )
-distribute by range(id) split rows ((1003), (1007))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'id'
-);
+distribute by range(id) split rows ((1003), (1007)) stored as kudu;
 ====
 ---- DATASET
 functional
@@ -848,15 +805,10 @@ create table {db_name}{db_suffix}.{table_name} (
   test_id bigint,
   test_name string,
   test_zip int,
-  alltypes_id int
+  alltypes_id int,
+  primary key (test_id, test_name, test_zip, alltypes_id)
 )
-distribute by range(test_id) split rows ((1003), (1007))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'test_id, test_name, test_zip, alltypes_id'
-);
+distribute by range(test_id) split rows ((1003), (1007)) stored as kudu;
 ====
 ---- DATASET
 functional
@@ -1191,16 +1143,10 @@ f2 int
 field string
 ---- CREATE_KUDU
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
-  field STRING,
+  field STRING PRIMARY KEY,
   f2 INT
 )
-DISTRIBUTE BY HASH (field) INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = '{table_name}',
-'kudu.master_addresses' = '127.0.0.1:7051',
-'kudu.key_columns' = 'field'
-);
+DISTRIBUTE BY HASH (field) INTO 3 BUCKETS STORED AS KUDU;
 ====
 ---- DATASET
 functional
@@ -1303,16 +1249,10 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/TinyTable/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  a string,
+  a string primary key,
   b string
 )
-distribute by range(a) split rows (('b'), ('d'))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'a'
-);
+distribute by range(a) split rows (('b'), ('d')) stored as kudu;
 ====
 ---- DATASET
 functional
@@ -1328,15 +1268,9 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/TinyIntTable/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  int_col int
+  int_col int primary key
 )
-distribute by range(int_col) split rows ((2), (4), (6), (8))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'int_col'
-);
+distribute by range(int_col) split rows ((2), (4), (6), (8)) stored as kudu;
 ====
 ---- DATASET
 functional
@@ -1359,15 +1293,9 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/NullTable/data.csv'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  a string, b string, c string, d int, e double, f string, g string
+  a string primary key, b string, c string, d int, e double, f string, g string
 )
-distribute by hash(a) into 3 buckets
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'a'
-);
+distribute by hash(a) into 3 buckets stored as kudu;
 ====
 ---- DATASET
 functional
@@ -1390,15 +1318,9 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/NullTable/data.csv'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
 create table {db_name}{db_suffix}.{table_name} (
-  a string, b string, c string, d int, e double, f string, g string
+  a string primary key, b string, c string, d int, e double, f string, g string
 )
-distribute by hash(a) into 3 buckets
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'a'
-);
+distribute by hash(a) into 3 buckets stored as kudu;
 ====
 ---- DATASET
 functional
@@ -1474,15 +1396,10 @@ create table {db_name}{db_suffix}.{table_name} (
   zip string,
   description1 string,
   description2 string,
-  income int
-)
+  income int,
+  primary key (id, zip))
 distribute by range(id, zip) split rows (('8600000US01475', '01475'), ('8600000US63121', '63121'), ('8600000US84712', '84712'))
-tblproperties (
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'id, zip'
-);
+stored as kudu;
 ====
 ---- DATASET
 functional

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/datasets/tpch/tpch_schema_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/tpch/tpch_schema_template.sql b/testdata/datasets/tpch/tpch_schema_template.sql
index dc2af15..0512f6a 100644
--- a/testdata/datasets/tpch/tpch_schema_template.sql
+++ b/testdata/datasets/tpch/tpch_schema_template.sql
@@ -57,15 +57,10 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   L_RECEIPTDATE STRING,
   L_SHIPINSTRUCT STRING,
   L_SHIPMODE STRING,
-  L_COMMENT STRING
+  L_COMMENT STRING,
+  PRIMARY KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)
 )
-distribute by hash (l_orderkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'l_orderkey, l_partkey, l_suppkey, l_linenumber'
-);
+distribute by hash (l_orderkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -90,7 +85,7 @@ P_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  P_PARTKEY BIGINT,
+  P_PARTKEY BIGINT PRIMARY KEY,
   P_NAME STRING,
   P_MFGR STRING,
   P_BRAND STRING,
@@ -100,13 +95,7 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   P_RETAILPRICE DOUBLE,
   P_COMMENT STRING
 )
-distribute by hash (p_partkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'p_partkey'
-);
+distribute by hash (p_partkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -131,15 +120,10 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   PS_SUPPKEY BIGINT,
   PS_AVAILQTY BIGINT,
   PS_SUPPLYCOST DOUBLE,
-  PS_COMMENT STRING
+  PS_COMMENT STRING,
+  PRIMARY KEY(PS_PARTKEY, PS_SUPPKEY)
 )
-distribute by hash (ps_partkey, ps_suppkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'ps_partkey, ps_suppkey'
-);
+distribute by hash (ps_partkey, ps_suppkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -162,7 +146,7 @@ S_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  S_SUPPKEY BIGINT,
+  S_SUPPKEY BIGINT PRIMARY KEY,
   S_NAME STRING,
   S_ADDRESS STRING,
   S_NATIONKEY SMALLINT,
@@ -170,13 +154,7 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   S_ACCTBAL DOUBLE,
   S_COMMENT STRING
 )
-distribute by hash (s_suppkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 's_suppkey'
-);
+distribute by hash (s_suppkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -196,18 +174,12 @@ N_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  N_NATIONKEY SMALLINT,
+  N_NATIONKEY SMALLINT PRIMARY KEY,
   N_NAME STRING,
   N_REGIONKEY SMALLINT,
   N_COMMENT STRING
 )
-distribute by hash (n_nationkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'n_nationkey'
-);
+distribute by hash (n_nationkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -226,17 +198,11 @@ R_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  R_REGIONKEY SMALLINT,
+  R_REGIONKEY SMALLINT PRIMARY KEY,
   R_NAME STRING,
   R_COMMENT STRING
 )
-distribute by hash (r_regionkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'r_regionkey'
-);
+distribute by hash (r_regionkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -261,7 +227,7 @@ O_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  O_ORDERKEY BIGINT,
+  O_ORDERKEY BIGINT PRIMARY KEY,
   O_CUSTKEY BIGINT,
   O_ORDERSTATUS STRING,
   O_TOTALPRICE DOUBLE,
@@ -271,13 +237,7 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   O_SHIPPRIORITY INT,
   O_COMMENT STRING
 )
-distribute by hash (o_orderkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'o_orderkey'
-);
+distribute by hash (o_orderkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
@@ -301,7 +261,7 @@ C_COMMENT STRING
 DELIMITED FIELDS TERMINATED BY '|'
 ---- CREATE_KUDU
 create table if not exists {db_name}{db_suffix}.{table_name} (
-  C_CUSTKEY BIGINT,
+  C_CUSTKEY BIGINT PRIMARY KEY,
   C_NAME STRING,
   C_ADDRESS STRING,
   C_NATIONKEY SMALLINT,
@@ -310,54 +270,10 @@ create table if not exists {db_name}{db_suffix}.{table_name} (
   C_MKTSEGMENT STRING,
   C_COMMENT STRING
 )
-distribute by hash (c_custkey) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'c_custkey'
-);
+distribute by hash (c_custkey) into 9 buckets stored as kudu;
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}.{table_name};
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/{table_name}'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ====
----- DATASET
-tpch
----- BASE_TABLE_NAME
-revenue
----- COLUMNS
-supplier_no bigint
-total_revenue Decimal(38,4)
----- CREATE_KUDU
-create table if not exists {db_name}{db_suffix}.{table_name} (
-  supplier_no bigint,
-  total_revevue double
-)
-distribute by hash (supplier_no) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'supplier_no'
-);
-====
----- DATASET
-tpch
----- BASE_TABLE_NAME
-max_revenue
----- COLUMNS
-max_revenue Decimal(38, 4)
----- CREATE_KUDU
-create table if not exists {db_name}{db_suffix}.{table_name} (
-  max_revenue bigint
-)
-distribute by hash (max_revenue) into 9 buckets
-tblproperties(
-  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-  'kudu.master_addresses' = '127.0.0.1:7051',
-  'kudu.table_name' = '{table_name}',
-  'kudu.key_columns' = 'max_revenue'
-);
-====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
index 06ce157..d3022a8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
@@ -5,9 +5,9 @@ PLAN-ROOT SINK
 00:SCAN KUDU [functional_kudu.testtbl]
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
-  ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1007), <end>)}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -23,9 +23,9 @@ PLAN-ROOT SINK
    kudu predicates: name = '10'
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
-  ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1007), <end>)}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -111,7 +111,7 @@ PLAN-ROOT SINK
    kudu predicates: id <= 20, zip <= 30, id >= 10, zip < 50, zip <= 5, zip > 1, zip >= 0, name = 'foo'
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -131,7 +131,7 @@ PLAN-ROOT SINK
    kudu predicates: id <= 60, id < 40, id < 103
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -152,9 +152,9 @@ PLAN-ROOT SINK
    kudu predicates: name = 'a'
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
-  ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1007), <end>)}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -175,9 +175,9 @@ PLAN-ROOT SINK
    predicates: name IS NULL, CAST(sin(id) AS BOOLEAN) = TRUE
 ---- SCANRANGELOCATIONS
 NODE 0:
-  ScanToken{table=testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
-  ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
-  ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1003), (int64 id=1007))}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [(int64 id=1007), <end>)}
+  ScanToken{table=impala::functional_kudu.testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test b/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
deleted file mode 100644
index 835e273..0000000
--- a/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
+++ /dev/null
@@ -1,90 +0,0 @@
-====
----- QUERY
-# Create managed Kudu table
-create table managed_kudu
-( id int, f float, d double, s string, v varchar(10), t tinyint, m smallint )
-distribute by hash into 3 buckets
-tblproperties
-(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'managed_kudu',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'id'
-)
----- RESULTS
-====
----- QUERY
-describe managed_kudu
----- RESULTS
-'id','int',''
-'f','float',''
-'d','double',''
-'s','string',''
-'v','varchar(10)',''
-'t','tinyint',''
-'m','smallint',''
----- TYPES
-STRING,STRING,STRING
-====
----- QUERY
-# Create external kudu table with non-matching schema (name)
-create external table external_kudu
-( id int, f float, do double, s string, v varchar(10), t tinyint, m smallint )
-tblproperties
-(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'managed_kudu',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'id'
-)
----- CATCH
-ImpalaRuntimeException: Table external_kudu (managed_kudu) has a different schema in Kudu than in Hive.
-====
----- QUERY
-# Create external kudu table with non-matching schema (type)
-create external table external_kudu
-( id bigint, f float, d double, s string, v varchar(10), t tinyint, m smallint )
-tblproperties
-(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'managed_kudu',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'id'
-)
----- CATCH
-ImpalaRuntimeException: Table external_kudu (managed_kudu) has a different schema in Kudu than in Hive.
-====
----- QUERY
-# Create external kudu table with matching schema
-create external table external_kudu
-( id int, f float, d double, s string, v varchar(10), t tinyint, m smallint )
-tblproperties
-(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'managed_kudu',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'id'
-)
----- RESULTS
-====
----- QUERY
-describe external_kudu
----- RESULTS
-'id','int',''
-'f','float',''
-'d','double',''
-'s','string',''
-'v','varchar(10)',''
-'t','tinyint',''
-'m','smallint',''
----- TYPES
-STRING,STRING,STRING
-====
----- QUERY
-drop table external_kudu
----- RESULTS
-=====
----- QUERY
-drop table managed_kudu
----- RESULTS
-=====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test b/testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
index 8aa1457..494bb58 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
@@ -1,7 +1,7 @@
 ====
 ---- QUERY
 # Make sure LIMIT is enforced.
-select * from dimtbl order by id limit 1;
+select * from functional_kudu.dimtbl order by id limit 1;
 ---- RESULTS
 1001,'Name1',94611
 ---- TYPES
@@ -10,8 +10,8 @@ BIGINT, STRING, INT
 ---- QUERY
 # Make sure that we can list the columns to be scanned in any order, that predicates
 # work and that we can have predicates on columns not referenced elsewhere.
-select zip, id from dimtbl where id >= 1000 and 1002 >= id and 94611 = zip and
-'Name1' = name order by id;
+select zip, id from functional_kudu.dimtbl where id >= 1000 and 1002 >= id and
+94611 = zip and 'Name1' = name order by id;
 ---- RESULTS
 94611,1001
 ---- TYPES
@@ -20,14 +20,8 @@ INT, BIGINT
 ---- QUERY
 # Regression test for IMPALA-2740, a NULL value from a previously filtered row would
 # carry over into the next unfiltered row (the result below would incorrectly be 2,NULL).
-USE kududb_test;
-CREATE TABLE impala_2740 (key INT, value INT)
-    DISTRIBUTE BY HASH INTO 3 BUCKETS
-    TBLPROPERTIES(
-        'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-        'kudu.table_name' = 'impala_2740',
-        'kudu.master_addresses' = '127.0.0.1',
-        'kudu.key_columns' = 'key');
+CREATE TABLE impala_2740 (key INT PRIMARY KEY, value INT)
+  DISTRIBUTE BY HASH (key) INTO 3 BUCKETS STORED AS KUDU;
 INSERT INTO impala_2740 VALUES (1, NULL), (2, -2);
 SELECT * FROM impala_2740 WHERE key != 1;
 ---- RESULTS
@@ -40,20 +34,10 @@ INT, INT
 # threads that are never started. The union and both scans land in the same fragment which
 # is run on all impalads. However, for the t1 table there is only as single scan range,
 # so two of the scan instances get empty scan ranges.
-CREATE TABLE impala_2635_t1 (id BIGINT, name STRING)
-    DISTRIBUTE BY RANGE SPLIT ROWS ((0))
-    TBLPROPERTIES(
-        'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-        'kudu.table_name' = 'impala_2635_t1',
-        'kudu.master_addresses' = '127.0.0.1',
-        'kudu.key_columns' = 'id');
-CREATE TABLE impala_2635_t2 (id BIGINT, name STRING)
-    DISTRIBUTE BY HASH(id) INTO 16 BUCKETS
-    TBLPROPERTIES(
-        'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-        'kudu.table_name' = 'impala_2635_t2',
-        'kudu.master_addresses' = '127.0.0.1',
-        'kudu.key_columns' = 'id');
+CREATE TABLE impala_2635_t1 (id BIGINT PRIMARY KEY, name STRING)
+  DISTRIBUTE BY HASH (id) INTO 3 BUCKETS STORED AS KUDU;
+CREATE TABLE impala_2635_t2 (id BIGINT PRIMARY KEY, name STRING)
+  DISTRIBUTE BY HASH(id) INTO 16 BUCKETS STORED AS KUDU;
 INSERT INTO impala_2635_t1 VALUES (0, 'Foo');
 INSERT INTO impala_2635_t2 VALUES (1, 'Blah');
 SELECT * FROM impala_2635_t1 UNION ALL SELECT * FROM impala_2635_t2;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test b/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
deleted file mode 100644
index b07efb7..0000000
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
+++ /dev/null
@@ -1,16 +0,0 @@
-====
----- QUERY
-SHOW CREATE TABLE functional_kudu.dimtbl
----- RESULTS
-CREATE TABLE functional_kudu.dimtbl (
-  id BIGINT,
-  name STRING,
-  zip INT
-)
-TBLPROPERTIES (
-  'kudu.master_addresses'='127.0.0.1:7051',
-  'kudu.key_columns'='id',
-  'kudu.table_name'='dimtbl',
-  'transient_lastDdlTime'='1441325601',
-  'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler')
-====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
index b32e0d0..e6814e1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
@@ -1,20 +1,13 @@
 ====
 ---- QUERY
-create table simple (id int, name string, valf float, vali bigint)
-distribute by hash into 3 buckets
-TBLPROPERTIES(
- 'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
- 'kudu.table_name' = 'query_test_simple',
- 'kudu.split_keys' = '[[10], [30]]',
- 'kudu.master_addresses' = '127.0.0.1',
- 'kudu.key_columns' = 'id'
-)
+create table simple (id int primary key, name string, valf float, vali bigint)
+  distribute by hash (id) into 3 buckets stored as kudu
 ---- RESULTS
 ====
 ---- QUERY
 -- Alter master address to a different location
 alter table simple set tblproperties (
-  'kudu.master_addresses' = '192.168.0.1'
+  'kudu.master_addresses' = 'localhost'
 )
 ---- RESULTS
 ====
@@ -22,7 +15,7 @@ alter table simple set tblproperties (
 -- Show that new address is picked up
 describe formatted simple
 ---- RESULTS: VERIFY_IS_SUBSET
-'','kudu.master_addresses','192.168.0.1         '
+'','kudu.master_addresses','localhost           '
 ---- TYPES
 STRING,STRING,STRING
 ====
@@ -31,6 +24,12 @@ alter table simple set tblproperties ('kudu.master_addresses' = '127.0.0.1')
 ---- RESULTS
 ====
 ---- QUERY
+-- Try to use an invalid master address
+alter table simple set tblproperties ('kudu.master_addresses' = 'invalid_host')
+---- CATCH
+ImpalaRuntimeException: Kudu table 'impala::$DATABASE.simple' does not exist on master 'invalid_host'
+====
+---- QUERY
 alter table simple rename to simple_new;
 ---- RESULTS
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
new file mode 100644
index 0000000..6950fb6
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
@@ -0,0 +1,105 @@
+====
+---- QUERY
+# This test file contains several cases for what basically amount to analysis errors,
+# but they only show up at runtime. These cases correspond to constraints enforced by
+# the Kudu storage engine.
+#
+# Incompatible column types in CTAS.
+create table t primary key (id) distribute by hash (id) into 3 buckets
+stored as kudu
+as select * from functional.alltypestiny
+---- CATCH
+ImpalaRuntimeException: Type TIMESTAMP is not supported in Kudu
+====
+---- QUERY
+create table t primary key (id) distribute by hash (id) into 3 buckets
+stored as kudu
+as select c1 as id from functional.decimal_tiny
+---- CATCH
+ImpalaRuntimeException: Type DECIMAL(10,4) is not supported in Kudu
+====
+---- QUERY
+create table t (a int, b array<string>, primary key(a)) distribute by hash (a)
+into 3 buckets stored as kudu
+---- CATCH
+ImpalaRuntimeException: Non-scalar type ARRAY<STRING> is not supported in Kudu
+====
+---- QUERY
+create table t primary key (id) distribute by hash (id) into 3 buckets
+stored as kudu
+as select id, int_col from functional.alltypestiny;
+select * from t;
+---- RESULTS
+0,0
+1,1
+2,0
+3,1
+4,0
+5,1
+6,0
+7,1
+---- TYPES
+INT,INT
+====
+---- QUERY
+# Boolean primary key column
+create table tab (x int, y boolean, primary key(x, y))
+  distribute by hash (x) into 3 buckets stored as kudu
+---- CATCH
+NonRecoverableException: Key column may not have type of BOOL, FLOAT, or DOUBLE
+====
+---- QUERY
+# Float primary key column
+create table tab (x int, y float, primary key(x, y))
+  distribute by hash (x) into 3 buckets stored as kudu
+---- CATCH
+NonRecoverableException: Key column may not have type of BOOL, FLOAT, or DOUBLE
+====
+---- QUERY
+# Primary keys should be declared first
+# TODO: See KUDU-1709 for improving Kudu error messages.
+create table tab (x int, y int, primary key(y))
+  distribute by hash (y) into 3 buckets stored as kudu
+---- CATCH
+NonRecoverableException: Got out-of-order key column: name: "y" type: INT32 is_key: true is_nullable: false cfile_block_size: 0
+====
+---- QUERY
+# Small number of hash buckets
+create table tab (a int, b int, c int, d int, primary key(a, b, c))
+  distribute by hash(a,b) into 8 buckets, hash(c) into 1 buckets stored as kudu
+---- CATCH
+NonRecoverableException: must have at least two hash buckets
+====
+---- QUERY
+# Same column in multiple hash based distributions
+create table tab (a int, b int, primary key (a))
+  distribute by hash (a) into 3 buckets, hash (a) into 2 buckets stored as kudu
+---- CATCH
+NonRecoverableException: hash bucket schema components must not contain columns in common
+====
+---- QUERY
+# Same column referenced multiple times in the same hash-based distribution
+create table tab (a int primary key) distribute by hash (a, a, a) into 3 buckets
+stored as kudu
+---- CATCH
+NonRecoverableException: hash bucket schema components must not contain columns in common
+====
+---- QUERY
+# Kudu table that uses Impala keywords as table name and column names
+create table `add`(`analytic` int, `function` int, primary key(`analytic`, `function`))
+distribute by hash (`analytic`) into 4 buckets, range (`function`) split rows ((1), (10))
+stored as kudu;
+insert into `add` select id, int_col from functional.alltypestiny;
+select * from `add`
+---- RESULTS
+0,0
+1,1
+2,0
+3,1
+4,0
+5,1
+6,0
+7,1
+---- TYPES
+INT,INT
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
index 8a58375..a06d203 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
@@ -1,52 +1,25 @@
 ====
 ---- QUERY
--- Test KuduClient will automatically set the default port if no port is given
-create table tdata_no_port ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'tdata_no_port',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id'
- )
----- RESULTS
-====
----- QUERY
 -- Invalid hostname
-create table tdata_bogus_host ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'tdata_no_port',
-'kudu.master_addresses' = 'bogus host name',
-'kudu.key_columns' = 'id'
- )
+create table tdata_bogus_host (id int primary key, name string, valf float, vali bigint)
+  DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
+  TBLPROPERTIES('kudu.master_addresses' = 'bogus host name')
 ---- CATCH
 Couldn't resolve this master's address bogus host name:7051
 ====
 ---- QUERY
 -- Non-existing host
-create table tdata_non_existing_host ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'tdata_no_port',
-'kudu.master_addresses' = 'bogus.host.name',
-'kudu.key_columns' = 'id'
- )
+create table tdata_non_existing_host
+(id int primary key, name string, valf float, vali bigint)
+  DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
+  TBLPROPERTIES('kudu.master_addresses' = 'bogus.host.name')
 ---- CATCH
 Couldn't resolve this master's address bogus.host.name:7051
 ====
 ---- QUERY
 create table tdata
-( id int, name string, valf float, vali bigint, valv varchar(20), valb boolean)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'tdata',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'id'
- )
+  (id int primary key, name string, valf float, vali bigint, valv varchar(20), valb boolean)
+  DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
@@ -189,14 +162,9 @@ INT,STRING,STRING,BOOLEAN
 ====
 ---- QUERY
 -- Test that string case is ignored
-create table ignore_column_case ( Id int, NAME string, vAlf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10, 'b'), (30, 'a'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'ignore_column_case',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'Id,NAME'
- )
+create table ignore_column_case (Id int, NAME string, vAlf float, vali bigint,
+  primary key (Id, NAME)) DISTRIBUTE BY RANGE SPLIT ROWS ((10, 'b'), (30, 'a'))
+  STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
@@ -259,15 +227,8 @@ delete ignore a from tdata a, tdata b where a.id = 666
 ---- QUERY
 # IMPALA-3454: A delete that requires a rewrite may not get the Kudu column order correct
 # if the Kudu columns are of different types.
-create table impala_3454
-(key_1 tinyint, key_2 bigint)
-DISTRIBUTE BY HASH INTO 3 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'impala_3454',
-'kudu.master_addresses' = '0.0.0.0:7051',
-'kudu.key_columns' = 'key_1,key_2'
- )
+create table impala_3454 (key_1 tinyint, key_2 bigint, PRIMARY KEY (key_1, key_2))
+  DISTRIBUTE BY HASH INTO 3 BUCKETS STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
index e5b2a95..bd61407 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
@@ -1,65 +1,9 @@
 ====
 ---- QUERY
--- Test allowing range distribution on a subset of the primary keys
-create table simple_range_with_key_projection ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE(name) SPLIT ROWS (('abc'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_range_with_key_projection',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
----- RESULTS
-====
----- QUERY
--- Test error handling for creating split rows on a non-key column
-create table simple_range_non_key_col ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE(name) SPLIT ROWS (('abc'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_range_non_key_col',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id'
- )
----- CATCH
-must specify only primary key columns for range partition component
-====
----- QUERY
--- Test error handling for split rows and wrong type
-create table simple_hash ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE(id) SPLIT ROWS (('abc'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
----- CATCH
-Expected int32 literal for column 'id' got 'abc'
-====
----- QUERY
--- Test error handling for distribute clauses
-create table simple_hash ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE(col_does_not_exist) SPLIT ROWS ((1))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
----- CATCH
-Unknown column: col_does_not_exist
-====
----- QUERY
 -- Test HASH partitioning
-create table simple_hash ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY HASH(id) INTO 4 BUCKETS, HASH(name) INTO 2 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
+create table simple_hash (id int, name string, valf float, vali bigint,
+  PRIMARY KEY (id, name)) DISTRIBUTE BY HASH(id) INTO 4 BUCKETS,
+  HASH(name) INTO 2 BUCKETS STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
@@ -80,14 +24,9 @@ INT,STRING,STRING,STRING,INT
 ====
 ---- QUERY
 -- Test HASH and RANGE partitioning
-create table simple_hash_range ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY HASH(id) INTO 4 BUCKETS, RANGE(id, name) SPLIT ROWS ((10, 'martin'), (20, 'Peter'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash_range',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
+create table simple_hash_range (id int, name string, valf float, vali bigint,
+  PRIMARY KEY (id, name)) DISTRIBUTE BY HASH(id) INTO 4 BUCKETS,
+  RANGE(id, name) SPLIT ROWS ((10, 'martin'), (20, 'Peter')) STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
@@ -112,13 +51,8 @@ INT,STRING,STRING,STRING,INT
 ====
 ---- QUERY
 create table simple_hash_range_ctas
-DISTRIBUTE BY HASH(id) INTO 4 BUCKETS, RANGE(id, name) SPLIT ROWS ((10, 'martin'), (20, 'Peter'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash_range_ctas',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
+  PRIMARY KEY (id, name) DISTRIBUTE BY HASH(id) INTO 4 BUCKETS,
+  RANGE(id, name) SPLIT ROWS ((10, 'martin'), (20, 'Peter')) STORED AS KUDU
 as select * from simple_hash
 ---- RESULTS
 'Inserted 0 row(s)'
@@ -145,14 +79,8 @@ INT,STRING,STRING,STRING,INT
 ====
 ---- QUERY
 -- Test HASH defaults to all columns
-create table simple_hash_all_columns ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY HASH INTO 4 BUCKETS
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_hash_all_columns',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
+create table simple_hash_all_columns (id int, name string, valf float, vali bigint,
+  PRIMARY KEY (id, name)) DISTRIBUTE BY HASH INTO 4 BUCKETS STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
@@ -169,14 +97,9 @@ INT,STRING,STRING,STRING,INT
 ====
 ---- QUERY
 -- Test RANGE defaults to all columns
-create table simple_range_all_columns ( id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((1, 'a'), (2, 'b'))
-TBLPROPERTIES(
-'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
-'kudu.table_name' = 'simple_range_all_columns',
-'kudu.master_addresses' = '127.0.0.1',
-'kudu.key_columns' = 'id, name'
- )
+create table simple_range_all_columns (id int, name string, valf float, vali bigint,
+  PRIMARY KEY (id, name)) DISTRIBUTE BY RANGE SPLIT ROWS ((1, 'a'), (2, 'b'))
+  STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
index 828d430..589bbf0 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
@@ -1,14 +1,8 @@
 ====
 ---- QUERY
-create table simple (id int, name string, valf float, vali bigint)
-DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30))
-TBLPROPERTIES(
- 'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
- 'kudu.table_name' = 'simple',
- 'kudu.master_addresses' = '127.0.0.1',
- 'kudu.key_columns' = 'id',
- 'kudu.num_tablet_replicas' = '2'
-)
+create table simple (id int primary key, name string, valf float, vali bigint)
+  DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
+  TBLPROPERTIES('kudu.num_tablet_replicas' = '2')
 ---- RESULTS
 ====
 ---- QUERY

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/tests/common/__init__.py
----------------------------------------------------------------------
diff --git a/tests/common/__init__.py b/tests/common/__init__.py
index 010f973..665bfe2 100644
--- a/tests/common/__init__.py
+++ b/tests/common/__init__.py
@@ -1,2 +1,2 @@
-KUDU_MASTER_HOST = "127.0.0.1"
-KUDU_MASTER_PORT = 7051
+# These values should match the impalad startup flag -kudu_master_hosts
+KUDU_MASTER_HOSTS = "127.0.0.1"



[14/14] incubator-impala git commit: IMPALA-4335: Don't send 0-row batches to clients

Posted by ta...@apache.org.
IMPALA-4335: Don't send 0-row batches to clients

This patch restores some behaviour from pre-IMPALA-2905 where we would
not send 0-row batches to the client. Although 0-row batches are legal,
they're not very useful for clients to receive (and clients may not
correctly process them).

No query was found which reliably produced 0-row batches, so no test is
added.

Change-Id: I7d339c1f9a55d9d75fb0e97d16b3176cc34f2171
Reviewed-on: http://gerrit.cloudera.org:8080/4787
Reviewed-by: Henry Robinson <he...@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/48085274
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/48085274
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/48085274

Branch: refs/heads/master
Commit: 48085274fa8ae57453477db21dae0e53eae6b766
Parents: e39f167
Author: Henry Robinson <he...@cloudera.com>
Authored: Fri Oct 21 15:38:34 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Oct 22 04:49:39 2016 +0000

----------------------------------------------------------------------
 be/src/exec/plan-root-sink.cc | 8 ++++++--
 be/src/runtime/coordinator.h  | 1 -
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/48085274/be/src/exec/plan-root-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/plan-root-sink.cc b/be/src/exec/plan-root-sink.cc
index bd73953..c728f4a 100644
--- a/be/src/exec/plan-root-sink.cc
+++ b/be/src/exec/plan-root-sink.cc
@@ -84,7 +84,11 @@ void ValidateCollectionSlots(const RowDescriptor& row_desc, RowBatch* batch) {
 Status PlanRootSink::Send(RuntimeState* state, RowBatch* batch) {
   ValidateCollectionSlots(row_desc_, batch);
   int current_batch_row = 0;
-  do {
+
+  // Don't enter the loop if batch->num_rows() == 0; no point triggering the consumer with
+  // 0 rows to return. Be wary of ever returning 0-row batches to the client; some poorly
+  // written clients may not cope correctly with them. See IMPALA-4335.
+  while (current_batch_row < batch->num_rows()) {
     unique_lock<mutex> l(lock_);
     while (results_ == nullptr && !consumer_done_) sender_cv_.wait(l);
     if (consumer_done_ || batch == nullptr) {
@@ -114,7 +118,7 @@ Status PlanRootSink::Send(RuntimeState* state, RowBatch* batch) {
     results_ = nullptr;
     ExprContext::FreeLocalAllocations(output_expr_ctxs_);
     consumer_cv_.notify_all();
-  } while (current_batch_row < batch->num_rows());
+  }
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/48085274/be/src/runtime/coordinator.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index f73cf42..9904def 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
 #ifndef IMPALA_RUNTIME_COORDINATOR_H
 #define IMPALA_RUNTIME_COORDINATOR_H
 


[13/14] incubator-impala git commit: IMPALA-4295: XFAIL wildcard SSL test

Posted by ta...@apache.org.
IMPALA-4295: XFAIL wildcard SSL test

commit 9f61397fc4d638aa78b37db2cd5b9c35b6deed94 exposed a bug (one
that was latent before the commit). I am XFAILing this now just to
green the build; IMPALA-4295 can be resolved when this issue is fixed
and not just XFAILed.

Change-Id: Ie809c6c6c967447d527927ebbc6b110095e7320a
Reviewed-on: http://gerrit.cloudera.org:8080/4784
Reviewed-by: Jim Apple <jb...@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/e39f1676
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/e39f1676
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/e39f1676

Branch: refs/heads/master
Commit: e39f1676e1c9269fb6549e8c319adf1a7ea9d445
Parents: d1d88aa
Author: Jim Apple <jb...@cloudera.com>
Authored: Fri Oct 21 14:56:10 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Oct 22 02:51:25 2016 +0000

----------------------------------------------------------------------
 tests/custom_cluster/test_client_ssl.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e39f1676/tests/custom_cluster/test_client_ssl.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_client_ssl.py b/tests/custom_cluster/test_client_ssl.py
index 4bc7bea..9d19234 100644
--- a/tests/custom_cluster/test_client_ssl.py
+++ b/tests/custom_cluster/test_client_ssl.py
@@ -109,6 +109,7 @@ class TestClientSsl(CustomClusterTestSuite):
   @CustomClusterTestSuite.with_args(impalad_args=SSL_WILDCARD_SAN_ARGS,
                                     statestored_args=SSL_WILDCARD_SAN_ARGS,
                                     catalogd_args=SSL_WILDCARD_SAN_ARGS)
+  @pytest.mark.xfail(run=True, reason="IMPALA-4295")
   def test_wildcard_san_ssl(self, vector):
     """ Test for IMPALA-3159: Test with a certificate which has a wildcard as a SAN. """
 


[02/14] incubator-impala git commit: IMPALA-4277: remove references for unsupported s3/s3n connectors

Posted by ta...@apache.org.
IMPALA-4277: remove references for unsupported s3/s3n connectors

We only support s3a://.

Support will be removed for s3:// in Hadoop 3.0 by HADOOP-12709

Change-Id: Ibfadd2bc91c7dbcb6f2bc962c404caea30f9b776
Reviewed-on: http://gerrit.cloudera.org:8080/4748
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>
(cherry picked from commit 3cb3f34d6d65bf52b2b8ba57a02d9ac785c8a937)
Reviewed-on: http://gerrit.cloudera.org:8080/4778
Reviewed-by: Alex Behm <al...@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/9ef9512e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/9ef9512e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/9ef9512e

Branch: refs/heads/master
Commit: 9ef9512e5b58a75e075538b3b94ac551363609e5
Parents: 2fa1633
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Oct 18 13:26:37 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 05:45:18 2016 +0000

----------------------------------------------------------------------
 fe/src/main/java/org/apache/impala/common/FileSystemUtil.java | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9ef9512e/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java b/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java
index fbf8478..d9fd6e8 100644
--- a/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java
+++ b/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java
@@ -30,9 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.S3FileSystem;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3native.NativeS3FileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
@@ -280,8 +278,7 @@ public class FileSystemUtil {
     // Common case.
     if (isDistributedFileSystem(fs)) return true;
     // Blacklist FileSystems that are known to not implement getFileBlockLocations().
-    return !(fs instanceof S3AFileSystem || fs instanceof NativeS3FileSystem ||
-        fs instanceof S3FileSystem || fs instanceof LocalFileSystem);
+    return !(fs instanceof S3AFileSystem || fs instanceof LocalFileSystem);
   }
 
   /**


[06/14] incubator-impala git commit: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/KuduTable.java b/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
index d55f8da..d0185b7 100644
--- a/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
@@ -19,21 +19,19 @@ package org.apache.impala.catalog;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import javax.xml.bind.DatatypeConverter;
 
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.log4j.Logger;
-import org.apache.kudu.client.KuduClient;
-import org.apache.kudu.client.LocatedTablet;
-
+import org.apache.impala.analysis.ColumnDef;
+import org.apache.impala.analysis.DistributeParam;
+import org.apache.impala.analysis.ToSqlUtils;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TColumn;
+import org.apache.impala.thrift.TDistributeByHashParam;
+import org.apache.impala.thrift.TDistributeByRangeParam;
+import org.apache.impala.thrift.TDistributeParam;
 import org.apache.impala.thrift.TKuduTable;
 import org.apache.impala.thrift.TResultSet;
 import org.apache.impala.thrift.TResultSetMetadata;
@@ -42,76 +40,86 @@ import org.apache.impala.thrift.TTableDescriptor;
 import org.apache.impala.thrift.TTableType;
 import org.apache.impala.util.KuduUtil;
 import org.apache.impala.util.TResultRowBuilder;
+import org.apache.impala.service.CatalogOpExecutor;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
+
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.client.KuduClient;
+import org.apache.kudu.client.KuduException;
+import org.apache.kudu.client.LocatedTablet;
+import org.apache.kudu.client.PartitionSchema.HashBucketSchema;
+import org.apache.kudu.client.PartitionSchema.RangeSchema;
+import org.apache.kudu.client.PartitionSchema;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
 
 /**
- * Impala representation of a Kudu table.
- *
- * The Kudu-related metadata is stored in the Metastore table's table properties.
+ * Representation of a Kudu table in the catalog cache.
  */
 public class KuduTable extends Table {
-  private static final Logger LOG = Logger.getLogger(Table.class);
+
+  private static final Logger LOG = Logger.getLogger(KuduTable.class);
 
   // Alias to the string key that identifies the storage handler for Kudu tables.
   public static final String KEY_STORAGE_HANDLER =
       hive_metastoreConstants.META_TABLE_STORAGE;
 
-  // Key to access the table name from the table properties
+  // Key to access the table name from the table properties.
   public static final String KEY_TABLE_NAME = "kudu.table_name";
 
   // Key to access the columns used to build the (composite) key of the table.
-  // The order of the keys is important.
+  // Deprecated - Used only for error checking.
   public static final String KEY_KEY_COLUMNS = "kudu.key_columns";
 
-  // Key to access the master address from the table properties. Error handling for
+  // Key to access the master host from the table properties. Error handling for
   // this string is done in the KuduClient library.
-  // TODO we should have something like KuduConfig.getDefaultConfig()
-  public static final String KEY_MASTER_ADDRESSES = "kudu.master_addresses";
+  // TODO: Rename kudu.master_addresses to kudu.master_host will break compatibility
+  // with older versions.
+  public static final String KEY_MASTER_HOSTS = "kudu.master_addresses";
 
   // Kudu specific value for the storage handler table property keyed by
   // KEY_STORAGE_HANDLER.
+  // TODO: Fix the storage handler name (see IMPALA-4271).
   public static final String KUDU_STORAGE_HANDLER =
       "com.cloudera.kudu.hive.KuduStorageHandler";
 
   // Key to specify the number of tablet replicas.
-  // TODO(KUDU): Allow modification in alter table.
   public static final String KEY_TABLET_REPLICAS = "kudu.num_tablet_replicas";
 
   public static final long KUDU_RPC_TIMEOUT_MS = 50000;
 
-  // The name of the table in Kudu.
+  // Table name in the Kudu storage engine. It may not neccessarily be the same as the
+  // table name specified in the CREATE TABLE statement; the latter
+  // is stored in Table.name_. Reasons why KuduTable.kuduTableName_ and Table.name_ may
+  // differ:
+  // 1. For managed tables, 'kuduTableName_' is prefixed with 'impala::<db_name>' to
+  // avoid conficts. TODO: Remove this when Kudu supports databases.
+  // 2. The user may specify a table name using the 'kudu.table_name' table property.
   private String kuduTableName_;
 
   // Comma separated list of Kudu master hosts with optional ports.
   private String kuduMasters_;
 
-  // The set of columns that are key columns in Kudu.
-  private ImmutableList<String> kuduKeyColumnNames_;
+  // Primary key column names.
+  private final List<String> primaryKeyColumnNames_ = Lists.newArrayList();
+
+  // Distribution schemes of this Kudu table. Both range and hash-based distributions are
+  // supported.
+  private final List<DistributeParam> distributeBy_ = Lists.newArrayList();
 
   protected KuduTable(TableId id, org.apache.hadoop.hive.metastore.api.Table msTable,
       Db db, String name, String owner) {
     super(id, msTable, db, name, owner);
-  }
-
-  public TKuduTable getKuduTable() {
-    TKuduTable tbl = new TKuduTable();
-    tbl.setKey_columns(Preconditions.checkNotNull(kuduKeyColumnNames_));
-    tbl.setMaster_addresses(Lists.newArrayList(kuduMasters_.split(",")));
-    tbl.setTable_name(kuduTableName_);
-    return tbl;
-  }
-
-  @Override
-  public TTableDescriptor toThriftDescriptor(Set<Long> referencedPartitions) {
-    TTableDescriptor desc = new TTableDescriptor(id_.asInt(), TTableType.KUDU_TABLE,
-        getTColumnDescriptors(), numClusteringCols_, kuduTableName_, db_.getName());
-    desc.setKuduTable(getKuduTable());
-    return desc;
+    kuduTableName_ = msTable.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    kuduMasters_ = msTable.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
   }
 
   @Override
@@ -126,78 +134,149 @@ public class KuduTable extends Table {
   @Override
   public ArrayList<Column> getColumnsInHiveOrder() { return getColumns(); }
 
-  public static boolean isKuduTable(org.apache.hadoop.hive.metastore.api.Table mstbl) {
-    return KUDU_STORAGE_HANDLER.equals(mstbl.getParameters().get(KEY_STORAGE_HANDLER));
+  public static boolean isKuduTable(org.apache.hadoop.hive.metastore.api.Table msTbl) {
+    return KUDU_STORAGE_HANDLER.equals(msTbl.getParameters().get(KEY_STORAGE_HANDLER));
+  }
+
+  public String getKuduTableName() { return kuduTableName_; }
+  public String getKuduMasterHosts() { return kuduMasters_; }
+
+  public List<String> getPrimaryKeyColumnNames() {
+    return ImmutableList.copyOf(primaryKeyColumnNames_);
+  }
+
+  public List<DistributeParam> getDistributeBy() {
+    return ImmutableList.copyOf(distributeBy_);
   }
 
   /**
-   * Load the columns from the schema list
+   * Loads the metadata of a Kudu table.
+   *
+   * Schema and distribution schemes are loaded directly from Kudu whereas column stats
+   * are loaded from HMS. The function also updates the table schema in HMS in order to
+   * propagate alterations made to the Kudu table to HMS.
    */
-  private void loadColumns(List<FieldSchema> schema, IMetaStoreClient client,
-      Set<String> keyColumns) throws TableLoadingException {
+  @Override
+  public void load(boolean dummy /* not used */, IMetaStoreClient msClient,
+      org.apache.hadoop.hive.metastore.api.Table msTbl) throws TableLoadingException {
+    msTable_ = msTbl;
+    // This is set to 0 for Kudu tables.
+    // TODO: Change this to reflect the number of pk columns and modify all the
+    // places (e.g. insert stmt) that currently make use of this parameter.
+    numClusteringCols_ = 0;
+    kuduTableName_ = msTable_.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    Preconditions.checkNotNull(kuduTableName_);
+    kuduMasters_ = msTable_.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
+    Preconditions.checkNotNull(kuduMasters_);
+    org.apache.kudu.client.KuduTable kuduTable = null;
+    numRows_ = getRowCount(msTable_.getParameters());
+
+    // Connect to Kudu to retrieve table metadata
+    try (KuduClient kuduClient = new KuduClient.KuduClientBuilder(
+        getKuduMasterHosts()).build()) {
+      kuduTable = kuduClient.openTable(kuduTableName_);
+    } catch (KuduException e) {
+      LOG.error("Error accessing Kudu table " + kuduTableName_);
+      throw new TableLoadingException(e.getMessage());
+    }
+    Preconditions.checkNotNull(kuduTable);
+
+    // Load metadata from Kudu and HMS
+    try {
+      loadSchema(kuduTable);
+      loadDistributeByParams(kuduTable);
+      loadAllColumnStats(msClient);
+    } catch (ImpalaRuntimeException e) {
+      LOG.error("Error loading metadata for Kudu table: " + kuduTableName_);
+      throw new TableLoadingException("Error loading metadata for Kudu table " +
+          kuduTableName_, e);
+    }
 
-    if (keyColumns.size() == 0 || keyColumns.size() > schema.size()) {
-      throw new TableLoadingException(String.format("Kudu tables must have at least one"
-          + "key column (had %d), and no more key columns than there are table columns "
-          + "(had %d).", keyColumns.size(), schema.size()));
+    // Update the table schema in HMS.
+    try {
+      long lastDdlTime = CatalogOpExecutor.calculateDdlTime(msTable_);
+      msTable_.putToParameters("transient_lastDdlTime", Long.toString(lastDdlTime));
+      msTable_.putToParameters(StatsSetupConst.DO_NOT_UPDATE_STATS,
+          StatsSetupConst.TRUE);
+      msClient.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
+    } catch (TException e) {
+      throw new TableLoadingException(e.getMessage());
     }
+  }
 
+  /**
+   * Loads the schema from the Kudu table including column definitions and primary key
+   * columns. Replaces the columns in the HMS table with the columns from the Kudu table.
+   * Throws an ImpalaRuntimeException if Kudu column data types cannot be mapped to
+   * Impala data types.
+   */
+  private void loadSchema(org.apache.kudu.client.KuduTable kuduTable)
+      throws ImpalaRuntimeException {
+    Preconditions.checkNotNull(kuduTable);
     clearColumns();
-    Set<String> columnNames = Sets.newHashSet();
+    primaryKeyColumnNames_.clear();
+    List<FieldSchema> cols = msTable_.getSd().getCols();
+    cols.clear();
     int pos = 0;
-    for (FieldSchema field: schema) {
-      org.apache.impala.catalog.Type type = parseColumnType(field);
-      // TODO(kudu-merge): Check for decimal types?
-      boolean isKey = keyColumns.contains(field.getName());
-      KuduColumn col = new KuduColumn(field.getName(), isKey, !isKey, type,
-          field.getComment(), pos);
-      columnNames.add(col.getName());
-      addColumn(col);
+    for (ColumnSchema colSchema: kuduTable.getSchema().getColumns()) {
+      Type type = KuduUtil.toImpalaType(colSchema.getType());
+      String colName = colSchema.getName();
+      cols.add(new FieldSchema(colName, type.toSql().toLowerCase(), null));
+      boolean isKey = colSchema.isKey();
+      if (isKey) primaryKeyColumnNames_.add(colName);
+      addColumn(new KuduColumn(colName, isKey, !isKey, type, null, pos));
       ++pos;
     }
+  }
 
-    if (!columnNames.containsAll(keyColumns)) {
-      throw new TableLoadingException(String.format("Some key columns were not found in"
-              + " the set of columns. List of column names: %s, List of key column names:"
-              + " %s", Iterables.toString(columnNames), Iterables.toString(keyColumns)));
+  private void loadDistributeByParams(org.apache.kudu.client.KuduTable kuduTable) {
+    Preconditions.checkNotNull(kuduTable);
+    PartitionSchema partitionSchema = kuduTable.getPartitionSchema();
+    Preconditions.checkState(!colsByPos_.isEmpty());
+    distributeBy_.clear();
+    for (HashBucketSchema hashBucketSchema: partitionSchema.getHashBucketSchemas()) {
+      List<String> columnNames = Lists.newArrayList();
+      for (int colPos: hashBucketSchema.getColumnIds()) {
+        columnNames.add(colsByPos_.get(colPos).getName());
+      }
+      distributeBy_.add(
+          DistributeParam.createHashParam(columnNames, hashBucketSchema.getNumBuckets()));
     }
-
-    kuduKeyColumnNames_ = ImmutableList.copyOf(keyColumns);
-
-    loadAllColumnStats(client);
+    RangeSchema rangeSchema = partitionSchema.getRangeSchema();
+    List<Integer> columnIds = rangeSchema.getColumns();
+    if (columnIds.isEmpty()) return;
+    List<String> columnNames = Lists.newArrayList();
+    for (int colPos: columnIds) columnNames.add(colsByPos_.get(colPos).getName());
+    // We don't populate the split values because Kudu's API doesn't currently support
+    // retrieving the split values for range partitions.
+    // TODO: File a Kudu JIRA.
+    distributeBy_.add(DistributeParam.createRangeParam(columnNames, null));
   }
 
-  @Override
-  public void load(boolean reuseMetadata, IMetaStoreClient client,
-      org.apache.hadoop.hive.metastore.api.Table msTbl) throws TableLoadingException {
-    // TODO handle 'reuseMetadata'
-    if (getMetaStoreTable() == null || !tableParamsAreValid(msTbl.getParameters())) {
-      throw new TableLoadingException(String.format(
-          "Cannot load Kudu table %s, table is corrupt.", name_));
+  /**
+   * Creates a temporary KuduTable object populated with the specified properties but has
+   * an invalid TableId and is not added to the Kudu storage engine or the
+   * HMS. This is used for CTAS statements.
+   */
+  public static KuduTable createCtasTarget(Db db,
+      org.apache.hadoop.hive.metastore.api.Table msTbl, List<ColumnDef> columnDefs,
+      List<String> primaryKeyColumnNames, List<DistributeParam> distributeParams) {
+    KuduTable tmpTable = new KuduTable(TableId.createInvalidId(), msTbl, db,
+        msTbl.getTableName(), msTbl.getOwner());
+    int pos = 0;
+    for (ColumnDef colDef: columnDefs) {
+      tmpTable.addColumn(new Column(colDef.getColName(), colDef.getType(), pos++));
     }
-
-    msTable_ = msTbl;
-    kuduTableName_ = msTbl.getParameters().get(KEY_TABLE_NAME);
-    kuduMasters_ = msTbl.getParameters().get(KEY_MASTER_ADDRESSES);
-
-    String keyColumnsProp = Preconditions.checkNotNull(msTbl.getParameters()
-        .get(KEY_KEY_COLUMNS).toLowerCase(), "'kudu.key_columns' cannot be null.");
-    Set<String> keyColumns = KuduUtil.parseKeyColumns(keyColumnsProp);
-
-    // Load the rest of the data from the table parameters directly
-    loadColumns(msTbl.getSd().getCols(), client, keyColumns);
-
-    numClusteringCols_ = 0;
-
-    // Get row count from stats
-    numRows_ = getRowCount(getMetaStoreTable().getParameters());
+    tmpTable.primaryKeyColumnNames_.addAll(primaryKeyColumnNames);
+    tmpTable.distributeBy_.addAll(distributeParams);
+    return tmpTable;
   }
 
   @Override
   public TTable toThrift() {
     TTable table = super.toThrift();
     table.setTable_type(TTableType.KUDU_TABLE);
-    table.setKudu_table(getKuduTable());
+    table.setKudu_table(getTKuduTable());
     return table;
   }
 
@@ -207,33 +286,46 @@ public class KuduTable extends Table {
     TKuduTable tkudu = thriftTable.getKudu_table();
     kuduTableName_ = tkudu.getTable_name();
     kuduMasters_ = Joiner.on(',').join(tkudu.getMaster_addresses());
-    kuduKeyColumnNames_ = ImmutableList.copyOf(tkudu.getKey_columns());
+    primaryKeyColumnNames_.clear();
+    primaryKeyColumnNames_.addAll(tkudu.getKey_columns());
+    loadDistributeByParamsFromThrift(tkudu.getDistribute_by());
   }
 
-  public String getKuduTableName() { return kuduTableName_; }
-  public String getKuduMasterAddresses() { return kuduMasters_; }
-  public int getNumKeyColumns() { return kuduKeyColumnNames_.size(); }
-
-  /**
-   * Returns true if all required parameters are present in the given table properties
-   * map.
-   * TODO(kudu-merge) Return a more specific error string.
-   */
-  public static boolean tableParamsAreValid(Map<String, String> params) {
-    return params.get(KEY_TABLE_NAME) != null && params.get(KEY_TABLE_NAME).length() > 0
-        && params.get(KEY_MASTER_ADDRESSES) != null
-        && params.get(KEY_MASTER_ADDRESSES).length() > 0
-        && params.get(KEY_KEY_COLUMNS) != null
-        && params.get(KEY_KEY_COLUMNS).length() > 0;
-   }
+  private void loadDistributeByParamsFromThrift(List<TDistributeParam> params) {
+    distributeBy_.clear();
+    for (TDistributeParam param: params) {
+      if (param.isSetBy_hash_param()) {
+        TDistributeByHashParam hashParam = param.getBy_hash_param();
+        distributeBy_.add(DistributeParam.createHashParam(
+            hashParam.getColumns(), hashParam.getNum_buckets()));
+      } else {
+        Preconditions.checkState(param.isSetBy_range_param());
+        TDistributeByRangeParam rangeParam = param.getBy_range_param();
+        distributeBy_.add(DistributeParam.createRangeParam(rangeParam.getColumns(),
+            null));
+      }
+    }
+  }
 
-  /**
-   * The number of nodes is not know ahead of time and will be updated during computeStats
-   * in the scan node.
-   */
-  public int getNumNodes() { return -1; }
+  @Override
+  public TTableDescriptor toThriftDescriptor(Set<Long> referencedPartitions) {
+    TTableDescriptor desc = new TTableDescriptor(id_.asInt(), TTableType.KUDU_TABLE,
+        getTColumnDescriptors(), numClusteringCols_, kuduTableName_, db_.getName());
+    desc.setKuduTable(getTKuduTable());
+    return desc;
+  }
 
-  public List<String> getKuduKeyColumnNames() { return kuduKeyColumnNames_; }
+  private TKuduTable getTKuduTable() {
+    TKuduTable tbl = new TKuduTable();
+    tbl.setKey_columns(Preconditions.checkNotNull(primaryKeyColumnNames_));
+    tbl.setMaster_addresses(Lists.newArrayList(kuduMasters_.split(",")));
+    tbl.setTable_name(kuduTableName_);
+    Preconditions.checkNotNull(distributeBy_);
+    for (DistributeParam distributeParam: distributeBy_) {
+      tbl.addToDistribute_by(distributeParam.toThrift());
+    }
+    return tbl;
+  }
 
   public TResultSet getTableStats() throws ImpalaRuntimeException {
     TResultSet result = new TResultSet();
@@ -247,7 +339,7 @@ public class KuduTable extends Table {
     resultSchema.addToColumns(new TColumn("# Replicas", Type.INT.toThrift()));
 
     try (KuduClient client = new KuduClient.KuduClientBuilder(
-        getKuduMasterAddresses()).build()) {
+        getKuduMasterHosts()).build()) {
       org.apache.kudu.client.KuduTable kuduTable = client.openTable(kuduTableName_);
       List<LocatedTablet> tablets =
           kuduTable.getTabletsLocations(KUDU_RPC_TIMEOUT_MS);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/Table.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index 6145cc5..4b40b44 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -455,6 +455,11 @@ public abstract class Table implements CatalogObject {
   @Override
   public boolean isLoaded() { return true; }
 
+  public static boolean isExternalTable(
+      org.apache.hadoop.hive.metastore.api.Table msTbl) {
+    return msTbl.getTableType().equalsIgnoreCase(TableType.EXTERNAL_TABLE.toString());
+  }
+
   /**
    * If the table is cached, it returns a <cache pool name, replication factor> pair
    * and adds the table cached directive ID to 'cacheDirIds'. Otherwise, it

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
index 764abe0..8541a3a 100644
--- a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
+++ b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
@@ -18,7 +18,6 @@
 package org.apache.impala.catalog;
 
 import java.util.EnumSet;
-import java.util.Set;
 
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/Type.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Type.java b/fe/src/main/java/org/apache/impala/catalog/Type.java
index 91fc2e3..05c71c7 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Type.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Type.java
@@ -317,6 +317,15 @@ public abstract class Type {
   }
 
   /**
+   * Checks if types t1 and t2 are assignment compatible, i.e. if both t1 and t2 can be
+   * assigned to a type t without an explicit cast and without any conversions that would
+   * result in loss of precision.
+   */
+  public static boolean areAssignmentCompatibleTypes(Type t1, Type t2) {
+    return getAssignmentCompatibleType(t1, t2, true) != ScalarType.INVALID;
+  }
+
+  /**
    * Returns true if this type exceeds the MAX_NESTING_DEPTH, false otherwise.
    */
   public boolean exceedsMaxNestingDepth() { return exceedsMaxNestingDepth(0); }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java b/fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
deleted file mode 100644
index 6c3ba8e..0000000
--- a/fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
+++ /dev/null
@@ -1,75 +0,0 @@
-// 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.catalog.delegates;
-
-import java.util.List;
-
-import org.apache.impala.thrift.TDistributeParam;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-import org.apache.impala.common.ImpalaRuntimeException;
-import org.apache.impala.thrift.TAlterTableParams;
-
-/**
- * Abstract class to implement the storage specific portion of DDL requests.
- *
- * During catalog DDL operations the CatalogOpExecutor will instantiate the correct
- * subclass of this class to handle the DDL operation to the storage backend. See,
- * CatalogOpExecutor::createDDLDelegate() for details.
- *
- */
-public abstract class DdlDelegate {
-
-  protected Table msTbl_;
-  protected TAlterTableParams tAlterTableParams_;
-  protected List<TDistributeParam> distributeParams_;
-
-  /**
-   * Creates a new delegate to modify Table 'msTbl'.
-   */
-  public DdlDelegate setMsTbl(Table msTbl) {
-    msTbl_ = msTbl;
-    return this;
-  }
-
-  public DdlDelegate setAlterTableParams(TAlterTableParams p) {
-    tAlterTableParams_ = p;
-    return this;
-  }
-
-  public DdlDelegate setDistributeParams(List<TDistributeParam> p) {
-    distributeParams_ = p;
-    return this;
-  }
-
-  /**
-   * Creates the table.
-   */
-  public abstract void createTable() throws ImpalaRuntimeException;
-
-  /**
-   * Drops the table.
-   */
-  public abstract void dropTable() throws ImpalaRuntimeException;
-
-  /**
-   * Performs an alter table with the parameters set with setAlterTableParams().
-   */
-  public abstract boolean alterTable() throws ImpalaRuntimeException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java b/fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
deleted file mode 100644
index 8410868..0000000
--- a/fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
+++ /dev/null
@@ -1,190 +0,0 @@
-// 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.catalog.delegates;
-
-import static org.apache.impala.util.KuduUtil.compareSchema;
-import static org.apache.impala.util.KuduUtil.fromImpalaType;
-import static org.apache.impala.util.KuduUtil.parseKeyColumns;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.kudu.ColumnSchema;
-import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder;
-import org.apache.kudu.Schema;
-import org.apache.kudu.Type;
-import org.apache.kudu.client.CreateTableOptions;
-import org.apache.kudu.client.KuduClient;
-import org.apache.kudu.client.PartialRow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.impala.catalog.KuduTable;
-import org.apache.impala.common.ImpalaRuntimeException;
-import org.apache.impala.thrift.TDistributeParam;
-import org.apache.impala.util.KuduUtil;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-
-
-/**
- * Implementation of the Kudu DDL Delegate. Propagates create and drop table statements to
- * Kudu.
- */
-public class KuduDdlDelegate extends DdlDelegate {
-
-  private static final Logger LOG = LoggerFactory.getLogger(KuduDdlDelegate.class);
-
-  public KuduDdlDelegate(Table msTbl) {
-    setMsTbl(msTbl);
-  }
-
-  /**
-   * Creates the Kudu table if it does not exist and returns true. If the table exists and
-   * the table is not a managed table ignore and return false, otherwise throw an
-   * exception.
-   */
-  @Override
-  public void createTable()
-      throws ImpalaRuntimeException {
-
-    String kuduTableName = msTbl_.getParameters().get(KuduTable.KEY_TABLE_NAME);
-    String kuduMasters = msTbl_.getParameters().get(KuduTable.KEY_MASTER_ADDRESSES);
-
-    // Can be optional for un-managed tables
-    String kuduKeyCols = msTbl_.getParameters().get(KuduTable.KEY_KEY_COLUMNS);
-
-    String replication = msTbl_.getParameters().get(KuduTable.KEY_TABLET_REPLICAS);
-
-    try (KuduClient client = new KuduClient.KuduClientBuilder(kuduMasters).build()) {
-      // TODO should we throw if the table does not exist when its an external table?
-      if (client.tableExists(kuduTableName)) {
-        if (msTbl_.getTableType().equals(TableType.MANAGED_TABLE.toString())) {
-          throw new ImpalaRuntimeException(String.format(
-              "Table %s already exists in Kudu master %s.", kuduTableName, kuduMasters));
-        }
-
-        // Check if the external table matches the schema
-        org.apache.kudu.client.KuduTable kuduTable = client.openTable(kuduTableName);
-        if (!compareSchema(msTbl_, kuduTable)) {
-          throw new ImpalaRuntimeException(String.format(
-              "Table %s (%s) has a different schema in Kudu than in Hive.",
-              msTbl_.getTableName(), kuduTableName));
-        }
-        return;
-      }
-
-      HashSet<String> keyColNames = parseKeyColumns(kuduKeyCols);
-      List<ColumnSchema> keyColSchemas = new ArrayList<>();
-
-      // Create a new Schema and map the types accordingly
-      ArrayList<ColumnSchema> columns = Lists.newArrayList();
-      for (FieldSchema fieldSchema: msTbl_.getSd().getCols()) {
-        org.apache.impala.catalog.Type catalogType = org.apache.impala.catalog.Type
-            .parseColumnType(fieldSchema.getType());
-        if (catalogType == null) {
-          throw new ImpalaRuntimeException(String.format(
-              "Could not parse column type %s.", fieldSchema.getType()));
-        }
-        Type t = fromImpalaType(catalogType);
-        // Create the actual column and check if the column is a key column
-        ColumnSchemaBuilder csb = new ColumnSchemaBuilder(
-            fieldSchema.getName(), t);
-        boolean isKeyColumn = keyColNames.contains(fieldSchema.getName());
-        csb.key(isKeyColumn);
-        csb.nullable(!isKeyColumn);
-        ColumnSchema cs = csb.build();
-        columns.add(cs);
-        if (isKeyColumn) keyColSchemas.add(cs);
-      }
-
-      Schema schema = new Schema(columns);
-      CreateTableOptions cto = new CreateTableOptions();
-
-      // Handle auto-partitioning of the Kudu table
-      if (distributeParams_ != null) {
-        for (TDistributeParam param : distributeParams_) {
-          if (param.isSetBy_hash_param()) {
-            Preconditions.checkState(!param.isSetBy_range_param());
-            cto.addHashPartitions(param.getBy_hash_param().getColumns(),
-                param.getBy_hash_param().getNum_buckets());
-          } else {
-            Preconditions.checkState(param.isSetBy_range_param());
-            cto.setRangePartitionColumns(param.getBy_range_param().getColumns());
-            for (PartialRow p : KuduUtil.parseSplits(schema, param.getBy_range_param())) {
-              cto.addSplitRow(p);
-            }
-          }
-        }
-      }
-
-      if (!Strings.isNullOrEmpty(replication)) {
-        int r = Integer.parseInt(replication);
-        if (r <= 0) {
-          throw new ImpalaRuntimeException(
-              "Number of tablet replicas must be greater than zero. " +
-              "Given number of replicas is: " + Integer.toString(r));
-        }
-        cto.setNumReplicas(r);
-      }
-
-      client.createTable(kuduTableName, schema, cto);
-    } catch (ImpalaRuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new ImpalaRuntimeException("Error creating Kudu table", e);
-    }
-  }
-
-  @Override
-  public void dropTable() throws ImpalaRuntimeException {
-    // If table is an external table, do not delete the data
-    if (msTbl_.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) return;
-
-    String kuduTableName = msTbl_.getParameters().get(KuduTable.KEY_TABLE_NAME);
-    String kuduMasters = msTbl_.getParameters().get(KuduTable.KEY_MASTER_ADDRESSES);
-
-    try (KuduClient client = new KuduClient.KuduClientBuilder(kuduMasters).build()) {
-      if (!client.tableExists(kuduTableName)) {
-        LOG.warn("Table: %s is in inconsistent state. It does not exist in Kudu master(s)"
-            + " %s, but it exists in Hive metastore. Deleting from metastore only.",
-            kuduTableName, kuduMasters);
-        return;
-      }
-      client.deleteTable(kuduTableName);
-      return;
-    } catch (Exception e) {
-      throw new ImpalaRuntimeException("Error dropping Kudu table", e);
-    }
-  }
-
-  public static boolean canHandle(org.apache.hadoop.hive.metastore.api.Table msTbl) {
-    return KuduTable.isKuduTable(msTbl);
-  }
-
-  @Override
-  public boolean alterTable() throws ImpalaRuntimeException {
-    throw new ImpalaRuntimeException(
-        "Alter table operations are not supported for Kudu tables.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java b/fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
deleted file mode 100644
index 8aabaa4..0000000
--- a/fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
+++ /dev/null
@@ -1,35 +0,0 @@
-// 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.catalog.delegates;
-
-import org.apache.impala.common.ImpalaRuntimeException;
-
-/**
- * Empty implementation for the DdlDelegate interface that does nothing.
- */
-public class UnsupportedOpDelegate extends DdlDelegate {
-
-  @Override
-  public void createTable() throws ImpalaRuntimeException { }
-
-  @Override
-  public void dropTable() throws ImpalaRuntimeException { }
-
-  @Override
-  public boolean alterTable() throws ImpalaRuntimeException { return true; }
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
index 3345c1b..8d15425 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
@@ -52,7 +52,7 @@ public class HdfsPartitionFilter {
 
   // lhs exprs of smap used in isMatch()
   private final ArrayList<SlotRef> lhsSlotRefs_ = Lists.newArrayList();
-  // indices into Table.getColumns()
+  // indices into Table.getColumnNames()
   private final ArrayList<Integer> refdKeys_ = Lists.newArrayList();
 
   public HdfsPartitionFilter(Expr predicate, HdfsTable tbl, Analyzer analyzer) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index 64ef822..9434801 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -107,7 +107,7 @@ public class KuduScanNode extends ScanNode {
     conjuncts_ = orderConjunctsByCost(conjuncts_);
 
     try (KuduClient client =
-         new KuduClientBuilder(kuduTable_.getKuduMasterAddresses()).build()) {
+         new KuduClientBuilder(kuduTable_.getKuduMasterHosts()).build()) {
       org.apache.kudu.client.KuduTable rpcTable =
           client.openTable(kuduTable_.getKuduTableName());
       validateSchema(rpcTable);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 5743a59..54493d1 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -51,6 +51,11 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 import org.apache.impala.analysis.FunctionName;
 import org.apache.impala.analysis.TableName;
@@ -61,7 +66,6 @@ import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.Column;
 import org.apache.impala.catalog.ColumnNotFoundException;
 import org.apache.impala.catalog.DataSource;
-import org.apache.impala.catalog.DatabaseNotFoundException;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.HBaseTable;
@@ -70,6 +74,7 @@ import org.apache.impala.catalog.HdfsPartition;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.HiveStorageDescriptorFactory;
 import org.apache.impala.catalog.IncompleteTable;
+import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.PartitionNotFoundException;
 import org.apache.impala.catalog.PartitionStatsUtil;
@@ -82,9 +87,6 @@ import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.catalog.TableNotFoundException;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.View;
-import org.apache.impala.catalog.delegates.DdlDelegate;
-import org.apache.impala.catalog.delegates.KuduDdlDelegate;
-import org.apache.impala.catalog.delegates.UnsupportedOpDelegate;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
@@ -121,7 +123,6 @@ import org.apache.impala.thrift.TCreateTableParams;
 import org.apache.impala.thrift.TDatabase;
 import org.apache.impala.thrift.TDdlExecRequest;
 import org.apache.impala.thrift.TDdlExecResponse;
-import org.apache.impala.thrift.TDistributeParam;
 import org.apache.impala.thrift.TDropDataSourceParams;
 import org.apache.impala.thrift.TDropDbParams;
 import org.apache.impala.thrift.TDropFunctionParams;
@@ -149,11 +150,6 @@ import org.apache.impala.thrift.TTruncateParams;
 import org.apache.impala.thrift.TUpdateCatalogRequest;
 import org.apache.impala.thrift.TUpdateCatalogResponse;
 import org.apache.impala.util.HdfsCachingUtil;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 /**
  * Class used to execute Catalog Operations, including DDL and refresh/invalidate
@@ -1103,8 +1099,7 @@ public class CatalogOpExecutor {
 
   /**
    * Drops a database from the metastore and removes the database's metadata from the
-   * internal cache. Re-throws any Hive Meta Store exceptions encountered during
-   * the drop.
+   * internal cache. Re-throws any HMS exceptions encountered during the drop.
    */
   private void dropDatabase(TDropDbParams params, TDdlExecResponse resp)
       throws ImpalaException {
@@ -1120,6 +1115,9 @@ public class CatalogOpExecutor {
 
     TCatalogObject removedObject = new TCatalogObject();
     synchronized (metastoreDdlLock_) {
+      // Remove all the Kudu tables of 'db' from the Kudu storage engine.
+      if (db != null && params.cascade) dropTablesFromKudu(db);
+
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().dropDatabase(
             params.getDb(), true, params.if_exists, params.cascade);
@@ -1144,6 +1142,44 @@ public class CatalogOpExecutor {
   }
 
   /**
+   * Drops all the Kudu tables of database 'db' from the Kudu storage engine. Retrieves
+   * the Kudu table name of each table in 'db' from HMS. Throws an ImpalaException if
+   * metadata for Kudu tables cannot be loaded from HMS or if an error occurs while
+   * trying to drop a table from Kudu.
+   */
+  private void dropTablesFromKudu(Db db) throws ImpalaException {
+    // If the table format isn't available, because the table hasn't been loaded yet,
+    // the metadata must be fetched from the Hive Metastore.
+    List<String> incompleteTableNames = Lists.newArrayList();
+    List<org.apache.hadoop.hive.metastore.api.Table> msTables = Lists.newArrayList();
+    for (Table table: db.getTables()) {
+      org.apache.hadoop.hive.metastore.api.Table msTable = table.getMetaStoreTable();
+      if (msTable == null) {
+        incompleteTableNames.add(table.getName());
+      } else {
+        msTables.add(msTable);
+      }
+    }
+    if (!incompleteTableNames.isEmpty()) {
+      try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+        msTables.addAll(msClient.getHiveClient().getTableObjectsByName(
+            db.getName(), incompleteTableNames));
+      } catch (TException e) {
+        LOG.error(String.format(HMS_RPC_ERROR_FORMAT_STR, "getTableObjectsByName") +
+            e.getMessage());
+      }
+    }
+    for (org.apache.hadoop.hive.metastore.api.Table msTable: msTables) {
+      if (!KuduTable.isKuduTable(msTable) || Table.isExternalTable(msTable)) continue;
+      // The operation will be aborted if the Kudu table cannot be dropped. If for
+      // some reason Kudu is permanently stuck in a non-functional state, the user is
+      // expected to ALTER TABLE to either set the table to UNMANAGED or set the format
+      // to something else.
+      KuduCatalogOpExecutor.dropTable(msTable, /*if exists*/ true);
+    }
+  }
+
+  /**
    * Drops a table or view from the metastore and removes it from the catalog.
    * Also drops all associated caching requests on the table and/or table's partitions,
    * uncaching all table data. If params.purge is true, table data is permanently
@@ -1157,17 +1193,6 @@ public class CatalogOpExecutor {
 
     TCatalogObject removedObject = new TCatalogObject();
     synchronized (metastoreDdlLock_) {
-
-      // Forward the DDL operation to the specified storage backend.
-      try {
-        org.apache.hadoop.hive.metastore.api.Table msTbl = getExistingTable(
-            tableName.getDb(), tableName.getTbl()).getMetaStoreTable();
-        DdlDelegate handler = createDdlDelegate(msTbl);
-        handler.dropTable();
-      } catch (TableNotFoundException | DatabaseNotFoundException e) {
-        // Do nothing
-      }
-
       Db db = catalog_.getDb(params.getTable_name().db_name);
       if (db == null) {
         if (params.if_exists) return;
@@ -1179,6 +1204,23 @@ public class CatalogOpExecutor {
         if (params.if_exists) return;
         throw new CatalogException("Table/View does not exist: " + tableName);
       }
+
+      // Retrieve the HMS table to determine if this is a Kudu table.
+      org.apache.hadoop.hive.metastore.api.Table msTbl = existingTbl.getMetaStoreTable();
+      if (msTbl == null) {
+        Preconditions.checkState(existingTbl instanceof IncompleteTable);
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+          msTbl = msClient.getHiveClient().getTable(tableName.getDb(),
+              tableName.getTbl());
+        } catch (TException e) {
+          LOG.error(String.format(HMS_RPC_ERROR_FORMAT_STR, "getTable") + e.getMessage());
+        }
+      }
+      if (msTbl != null && KuduTable.isKuduTable(msTbl)
+          && !Table.isExternalTable(msTbl)) {
+        KuduCatalogOpExecutor.dropTable(msTbl, /* if exists */ true);
+      }
+
       // Check to make sure we don't drop a view with "drop table" statement and
       // vice versa. is_table field is marked optional in TDropTableOrViewParams to
       // maintain catalog api compatibility.
@@ -1343,7 +1385,8 @@ public class CatalogOpExecutor {
 
   /**
    * Creates a new table in the metastore and adds an entry to the metadata cache to
-   * lazily load the new metadata on the next access. Re-throws any Hive Meta Store
+   * lazily load the new metadata on the next access. If this is a managed Kudu table,
+   * the table is also created in the Kudu storage engine. Re-throws any HMS or Kudu
    * exceptions encountered during the create.
    */
   private boolean createTable(TCreateTableParams params, TDdlExecResponse response)
@@ -1351,9 +1394,8 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(params);
     TableName tableName = TableName.fromThrift(params.getTable_name());
     Preconditions.checkState(tableName != null && tableName.isFullyQualified());
-    Preconditions.checkState(params.getColumns() != null &&
-        params.getColumns().size() > 0,
-        "Null or empty column list given as argument to Catalog.createTable");
+    Preconditions.checkState(params.getColumns() != null,
+        "Null column list given as argument to Catalog.createTable");
 
     if (params.if_not_exists &&
         catalog_.containsTable(tableName.getDb(), tableName.getTbl())) {
@@ -1362,11 +1404,161 @@ public class CatalogOpExecutor {
       response.getResult().setVersion(catalog_.getCatalogVersion());
       return false;
     }
-    org.apache.hadoop.hive.metastore.api.Table tbl =
-        createMetaStoreTable(params);
+    org.apache.hadoop.hive.metastore.api.Table tbl = createMetaStoreTable(params);
     LOG.debug(String.format("Creating table %s", tableName));
-    return createTable(tbl, params.if_not_exists, params.getCache_op(),
-        params.getDistribute_by(), response);
+    if (KuduTable.isKuduTable(tbl)) return createKuduTable(tbl, params, response);
+    Preconditions.checkState(params.getColumns().size() > 0,
+        "Empty column list given as argument to Catalog.createTable");
+    return createTable(tbl, params.if_not_exists, params.getCache_op(), response);
+  }
+
+  /**
+   * Utility function that creates a hive.metastore.api.Table object based on the given
+   * TCreateTableParams.
+   * TODO: Extract metastore object creation utility functions into a separate
+   * helper/factory class.
+   */
+  public static org.apache.hadoop.hive.metastore.api.Table createMetaStoreTable(
+      TCreateTableParams params) {
+    Preconditions.checkNotNull(params);
+    TableName tableName = TableName.fromThrift(params.getTable_name());
+    org.apache.hadoop.hive.metastore.api.Table tbl =
+        new org.apache.hadoop.hive.metastore.api.Table();
+    tbl.setDbName(tableName.getDb());
+    tbl.setTableName(tableName.getTbl());
+    tbl.setOwner(params.getOwner());
+    if (params.isSetTable_properties()) {
+      tbl.setParameters(params.getTable_properties());
+    } else {
+      tbl.setParameters(new HashMap<String, String>());
+    }
+
+    if (params.getComment() != null) {
+      tbl.getParameters().put("comment", params.getComment());
+    }
+    if (params.is_external) {
+      tbl.setTableType(TableType.EXTERNAL_TABLE.toString());
+      tbl.putToParameters("EXTERNAL", "TRUE");
+    } else {
+      tbl.setTableType(TableType.MANAGED_TABLE.toString());
+    }
+
+    tbl.setSd(createSd(params));
+    if (params.getPartition_columns() != null) {
+      // Add in any partition keys that were specified
+      tbl.setPartitionKeys(buildFieldSchemaList(params.getPartition_columns()));
+    } else {
+      tbl.setPartitionKeys(new ArrayList<FieldSchema>());
+    }
+    return tbl;
+  }
+
+  private static StorageDescriptor createSd(TCreateTableParams params) {
+    StorageDescriptor sd = HiveStorageDescriptorFactory.createSd(
+        params.getFile_format(), RowFormat.fromThrift(params.getRow_format()));
+    if (params.isSetSerde_properties()) {
+      if (sd.getSerdeInfo().getParameters() == null) {
+        sd.getSerdeInfo().setParameters(params.getSerde_properties());
+      } else {
+        sd.getSerdeInfo().getParameters().putAll(params.getSerde_properties());
+      }
+    }
+
+    if (params.getLocation() != null) sd.setLocation(params.getLocation());
+
+    // Add in all the columns
+    sd.setCols(buildFieldSchemaList(params.getColumns()));
+    return sd;
+  }
+
+  /**
+   * Creates a new Kudu table. The Kudu table is first created in the Kudu storage engine
+   * (only applicable to managed tables), then in HMS and finally in the catalog cache.
+   * Failure to add the table in HMS results in the table being dropped from Kudu.
+   * 'response' is populated with the results of this operation. Returns true if a new
+   * table was created as part of this call, false otherwise.
+   */
+  private boolean createKuduTable(org.apache.hadoop.hive.metastore.api.Table newTable,
+      TCreateTableParams params, TDdlExecResponse response) throws ImpalaException {
+    Preconditions.checkState(KuduTable.isKuduTable(newTable));
+    if (Table.isExternalTable(newTable)) {
+      KuduCatalogOpExecutor.populateColumnsFromKudu(newTable);
+    } else {
+      KuduCatalogOpExecutor.createManagedTable(newTable, params);
+    }
+    try {
+      // Add the table to the HMS and the catalog cache. Aquire metastoreDdlLock_ to
+      // ensure the atomicity of these operations.
+      synchronized (metastoreDdlLock_) {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+          msClient.getHiveClient().createTable(newTable);
+        }
+        // Add the table to the catalog cache
+        Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
+        addTableToCatalogUpdate(newTbl, response.result);
+      }
+    } catch (Exception e) {
+      try {
+        // Error creating the table in HMS, drop the managed table from Kudu.
+        if (!Table.isExternalTable(newTable)) {
+          KuduCatalogOpExecutor.dropTable(newTable, false);
+        }
+      } catch (Exception logged) {
+        String kuduTableName = newTable.getParameters().get(KuduTable.KEY_TABLE_NAME);
+        LOG.error(String.format("Failed to drop Kudu table '%s'", kuduTableName),
+            logged);
+        throw new RuntimeException(String.format("Failed to create the table '%s' in " +
+            " the Metastore and the newly created Kudu table '%s' could not be " +
+            " dropped. The log contains more information.", newTable.getTableName(),
+            kuduTableName), e);
+      }
+      if (e instanceof AlreadyExistsException && params.if_not_exists) return false;
+      throw new ImpalaRuntimeException(
+          String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
+    }
+    return true;
+  }
+
+  /**
+   * Creates a new table. The table is initially created in HMS and, if that operation
+   * succeeds, it is then added in the catalog cache. It also sets HDFS caching if
+   * 'cacheOp' is not null. 'response' is populated with the results of this operation.
+   * Returns true if a new table was created as part of this call, false otherwise.
+   */
+  private boolean createTable(org.apache.hadoop.hive.metastore.api.Table newTable,
+      boolean if_not_exists, THdfsCachingOp cacheOp, TDdlExecResponse response)
+      throws ImpalaException {
+    Preconditions.checkState(!KuduTable.isKuduTable(newTable));
+    synchronized (metastoreDdlLock_) {
+      try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+        msClient.getHiveClient().createTable(newTable);
+        // If this table should be cached, and the table location was not specified by
+        // the user, an extra step is needed to read the table to find the location.
+        if (cacheOp != null && cacheOp.isSet_cached() &&
+            newTable.getSd().getLocation() == null) {
+          newTable = msClient.getHiveClient().getTable(
+              newTable.getDbName(), newTable.getTableName());
+        }
+      } catch (Exception e) {
+        if (e instanceof AlreadyExistsException && if_not_exists) return false;
+        throw new ImpalaRuntimeException(
+            String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
+      }
+
+      // Submit the cache request and update the table metadata.
+      if (cacheOp != null && cacheOp.isSet_cached()) {
+        short replication = cacheOp.isSetReplication() ? cacheOp.getReplication() :
+            JniCatalogConstants.HDFS_DEFAULT_CACHE_REPLICATION_FACTOR;
+        long id = HdfsCachingUtil.submitCacheTblDirective(newTable,
+            cacheOp.getCache_pool_name(), replication);
+        catalog_.watchCacheDirs(Lists.<Long>newArrayList(id),
+            new TTableName(newTable.getDbName(), newTable.getTableName()));
+        applyAlterTable(newTable);
+      }
+      Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
+      addTableToCatalogUpdate(newTbl, response.result);
+    }
+    return true;
   }
 
   /**
@@ -1392,7 +1584,7 @@ public class CatalogOpExecutor {
         new org.apache.hadoop.hive.metastore.api.Table();
     setViewAttributes(params, view);
     LOG.debug(String.format("Creating view %s", tableName));
-    createTable(view, params.if_not_exists, null, null, response);
+    createTable(view, params.if_not_exists, null, response);
   }
 
   /**
@@ -1423,6 +1615,8 @@ public class CatalogOpExecutor {
     Table srcTable = getExistingTable(srcTblName.getDb(), srcTblName.getTbl());
     org.apache.hadoop.hive.metastore.api.Table tbl =
         srcTable.getMetaStoreTable().deepCopy();
+    Preconditions.checkState(!KuduTable.isKuduTable(tbl),
+        "CREATE TABLE LIKE is not supported for Kudu tables.");
     tbl.setDbName(tblName.getDb());
     tbl.setTableName(tblName.getTbl());
     tbl.setOwner(params.getOwner());
@@ -1460,7 +1654,7 @@ public class CatalogOpExecutor {
     tbl.getSd().setLocation(params.getLocation());
     if (fileFormat != null) {
       setStorageDescriptorFileFormat(tbl.getSd(), fileFormat);
-    } else if (fileFormat == null && srcTable instanceof View) {
+    } else if (srcTable instanceof View) {
       // Here, source table is a view which has no input format. So to be
       // consistent with CREATE TABLE, default input format is assumed to be
       // TEXT unless otherwise specified.
@@ -1469,85 +1663,7 @@ public class CatalogOpExecutor {
     // Set the row count of this table to unknown.
     tbl.putToParameters(StatsSetupConst.ROW_COUNT, "-1");
     LOG.debug(String.format("Creating table %s LIKE %s", tblName, srcTblName));
-    createTable(tbl, params.if_not_exists, null, null, response);
-  }
-
-  /**
-   * Creates a new table in the HMS. If ifNotExists=true, no error will be thrown if
-   * the table already exists, otherwise an exception will be thrown.
-   * Accepts an optional 'cacheOp' param, which if specified will cache the table's
-   * HDFS location according to the 'cacheOp' spec after creation.
-   * Stores details of the operations (such as the resulting catalog version) in
-   * 'response' output parameter.
-   * Returns true if a new table was created as part of this call, false otherwise.
-   */
-  private boolean createTable(org.apache.hadoop.hive.metastore.api.Table newTable,
-      boolean ifNotExists, THdfsCachingOp cacheOp, List<TDistributeParam> distribute_by,
-      TDdlExecResponse response)
-      throws ImpalaException {
-    synchronized (metastoreDdlLock_) {
-
-      try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-        msClient.getHiveClient().createTable(newTable);
-        // If this table should be cached, and the table location was not specified by
-        // the user, an extra step is needed to read the table to find the location.
-        if (cacheOp != null && cacheOp.isSet_cached() &&
-            newTable.getSd().getLocation() == null) {
-          newTable = msClient.getHiveClient().getTable(newTable.getDbName(),
-              newTable.getTableName());
-        }
-      } catch (AlreadyExistsException e) {
-        if (!ifNotExists) {
-          throw new ImpalaRuntimeException(
-              String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
-        }
-        LOG.debug(String.format("Ignoring '%s' when creating table %s.%s because " +
-            "IF NOT EXISTS was specified.", e,
-            newTable.getDbName(), newTable.getTableName()));
-        return false;
-      } catch (TException e) {
-        throw new ImpalaRuntimeException(
-            String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
-      }
-
-      // Forward the operation to a specific storage backend. If the operation fails,
-      // delete the just created hive table to avoid inconsistencies.
-      try {
-        createDdlDelegate(newTable).setDistributeParams(distribute_by).createTable();
-      } catch (ImpalaRuntimeException e) {
-        try (MetaStoreClient c = catalog_.getMetaStoreClient()) {
-          c.getHiveClient().dropTable(newTable.getDbName(), newTable.getTableName(),
-              false, ifNotExists);
-        } catch (Exception hE) {
-          throw new ImpalaRuntimeException(String.format(HMS_RPC_ERROR_FORMAT_STR,
-              "dropTable"), hE);
-        }
-        throw e;
-      }
-
-      // Submit the cache request and update the table metadata.
-      if (cacheOp != null && cacheOp.isSet_cached()) {
-        short replication = cacheOp.isSetReplication() ? cacheOp.getReplication() :
-            JniCatalogConstants.HDFS_DEFAULT_CACHE_REPLICATION_FACTOR;
-        long id = HdfsCachingUtil.submitCacheTblDirective(newTable,
-            cacheOp.getCache_pool_name(), replication);
-        catalog_.watchCacheDirs(Lists.<Long>newArrayList(id),
-            new TTableName(newTable.getDbName(), newTable.getTableName()));
-        applyAlterTable(newTable);
-      }
-      Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
-      addTableToCatalogUpdate(newTbl, response.result);
-    }
-    return true;
-  }
-
-  /**
-   * Instantiate the appropriate DDL delegate for the table. If no known delegate is
-   * available for the table, returns a UnsupportedOpDelegate instance.
-   */
-  private DdlDelegate createDdlDelegate(org.apache.hadoop.hive.metastore.api.Table tab) {
-    if (KuduDdlDelegate.canHandle(tab)) return new KuduDdlDelegate(tab);
-    return new UnsupportedOpDelegate();
+    createTable(tbl, params.if_not_exists, null, response);
   }
 
   /**
@@ -1967,6 +2083,9 @@ public class CatalogOpExecutor {
       switch (params.getTarget()) {
         case TBL_PROPERTY:
           msTbl.getParameters().putAll(properties);
+          if (KuduTable.isKuduTable(msTbl)) {
+            KuduCatalogOpExecutor.validateKuduTblExists(msTbl);
+          }
           break;
         case SERDE_PROPERTY:
           msTbl.getSd().getSerdeInfo().getParameters().putAll(properties);
@@ -2120,7 +2239,6 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(cacheOp);
     Preconditions.checkNotNull(params.getPartition_spec());
     // Alter partition params.
-    final String RUNTIME_FILTER_FORMAT = "apply %s on %s";
     TableName tableName = tbl.getTableName();
     HdfsPartition partition = catalog_.getHdfsPartition(
         tableName.getDb(), tableName.getTbl(), params.getPartition_spec());
@@ -2535,16 +2653,6 @@ public class CatalogOpExecutor {
   }
 
   /**
-   * Returns a deep copy of the metastore.api.Table object for the given TableName.
-   */
-  private org.apache.hadoop.hive.metastore.api.Table getMetaStoreTable(
-      TableName tableName) throws CatalogException {
-    Preconditions.checkState(tableName != null && tableName.isFullyQualified());
-    return getExistingTable(tableName.getDb(), tableName.getTbl())
-        .getMetaStoreTable().deepCopy();
-  }
-
-  /**
    * Returns the metastore.api.Table object from the Hive Metastore for an existing
    * fully loaded table.
    */
@@ -2608,7 +2716,7 @@ public class CatalogOpExecutor {
   /**
    * Calculates the next transient_lastDdlTime value.
    */
-  private static long calculateDdlTime(
+  public static long calculateDdlTime(
       org.apache.hadoop.hive.metastore.api.Table msTbl) {
     long existingLastDdlTime = CatalogServiceCatalog.getLastDdlTime(msTbl);
     long currentTime = System.currentTimeMillis() / 1000;
@@ -2617,63 +2725,6 @@ public class CatalogOpExecutor {
   }
 
   /**
-   * Utility function that creates a hive.metastore.api.Table object based on the given
-   * TCreateTableParams.
-   * TODO: Extract metastore object creation utility functions into a separate
-   * helper/factory class.
-   */
-  public static org.apache.hadoop.hive.metastore.api.Table
-      createMetaStoreTable(TCreateTableParams params) {
-    Preconditions.checkNotNull(params);
-    TableName tableName = TableName.fromThrift(params.getTable_name());
-    org.apache.hadoop.hive.metastore.api.Table tbl =
-        new org.apache.hadoop.hive.metastore.api.Table();
-    tbl.setDbName(tableName.getDb());
-    tbl.setTableName(tableName.getTbl());
-    tbl.setOwner(params.getOwner());
-    if (params.isSetTable_properties()) {
-      tbl.setParameters(params.getTable_properties());
-    } else {
-      tbl.setParameters(new HashMap<String, String>());
-    }
-
-    if (params.getComment() != null) {
-      tbl.getParameters().put("comment", params.getComment());
-    }
-    if (params.is_external) {
-      tbl.setTableType(TableType.EXTERNAL_TABLE.toString());
-      tbl.putToParameters("EXTERNAL", "TRUE");
-    } else {
-      tbl.setTableType(TableType.MANAGED_TABLE.toString());
-    }
-
-    StorageDescriptor sd = HiveStorageDescriptorFactory.createSd(
-        params.getFile_format(), RowFormat.fromThrift(params.getRow_format()));
-
-    if (params.isSetSerde_properties()) {
-      if (sd.getSerdeInfo().getParameters() == null) {
-        sd.getSerdeInfo().setParameters(params.getSerde_properties());
-      } else {
-        sd.getSerdeInfo().getParameters().putAll(params.getSerde_properties());
-      }
-    }
-
-    if (params.getLocation() != null) {
-      sd.setLocation(params.getLocation());
-    }
-    // Add in all the columns
-    sd.setCols(buildFieldSchemaList(params.getColumns()));
-    tbl.setSd(sd);
-    if (params.getPartition_columns() != null) {
-      // Add in any partition keys that were specified
-      tbl.setPartitionKeys(buildFieldSchemaList(params.getPartition_columns()));
-    } else {
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>());
-    }
-    return tbl;
-  }
-
-  /**
    * Executes a TResetMetadataRequest and returns the result as a
    * TResetMetadataResponse. Based on the request parameters, this operation
    * may do one of three things:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 00a3d93..6d535fd 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -170,9 +170,11 @@ public class Frontend {
   private final AtomicReference<AuthorizationChecker> authzChecker_;
   private final ScheduledExecutorService policyReader_ =
       Executors.newScheduledThreadPool(1);
+  private final String defaultKuduMasterHosts_;
 
-  public Frontend(AuthorizationConfig authorizationConfig) {
-    this(authorizationConfig, new ImpaladCatalog());
+  public Frontend(AuthorizationConfig authorizationConfig,
+      String defaultKuduMasterHosts) {
+    this(authorizationConfig, new ImpaladCatalog(defaultKuduMasterHosts));
   }
 
   /**
@@ -181,6 +183,7 @@ public class Frontend {
   public Frontend(AuthorizationConfig authorizationConfig, ImpaladCatalog catalog) {
     authzConfig_ = authorizationConfig;
     impaladCatalog_ = catalog;
+    defaultKuduMasterHosts_ = catalog.getDefaultKuduMasterHosts();
     authzChecker_ = new AtomicReference<AuthorizationChecker>(
         new AuthorizationChecker(authzConfig_, impaladCatalog_.getAuthPolicy()));
     // If authorization is enabled, reload the policy on a regular basis.
@@ -226,7 +229,7 @@ public class Frontend {
 
     // If this is not a delta, this update should replace the current
     // Catalog contents so create a new catalog and populate it.
-    if (!req.is_delta) catalog = new ImpaladCatalog();
+    if (!req.is_delta) catalog = new ImpaladCatalog(defaultKuduMasterHosts_);
 
     TUpdateCatalogCacheResponse response = catalog.updateCatalog(req);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/service/JniCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index fc8deaf..7d0af54 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -102,7 +102,7 @@ public class JniCatalog {
     try {
       catalog_.reset();
     } catch (CatalogException e) {
-      LOG.error("Error initialializing Catalog. Please run 'invalidate metadata'", e);
+      LOG.error("Error initializing Catalog. Please run 'invalidate metadata'", e);
     }
     catalogOpExecutor_ = new CatalogOpExecutor(catalog_);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 07d6ec6..0d502e5 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -117,7 +117,8 @@ public class JniFrontend {
    */
   public JniFrontend(boolean lazy, String serverName, String authorizationPolicyFile,
       String sentryConfigFile, String authPolicyProviderClass, int impalaLogLevel,
-      int otherLogLevel, boolean allowAuthToLocal) throws InternalException {
+      int otherLogLevel, boolean allowAuthToLocal, String defaultKuduMasterHosts)
+      throws InternalException {
     BackendConfig.setAuthToLocal(allowAuthToLocal);
     GlogAppender.Install(TLogLevel.values()[impalaLogLevel],
         TLogLevel.values()[otherLogLevel]);
@@ -136,7 +137,7 @@ public class JniFrontend {
     }
     LOG.info(JniUtil.getJavaVersion());
 
-    frontend_ = new Frontend(authConfig);
+    frontend_ = new Frontend(authConfig, defaultKuduMasterHosts);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
new file mode 100644
index 0000000..bd6d0fe
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
@@ -0,0 +1,240 @@
+// 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.service;
+
+import java.lang.NumberFormatException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.impala.analysis.ToSqlUtils;
+import org.apache.impala.catalog.KuduTable;
+import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.TableNotFoundException;
+import org.apache.impala.catalog.Type;
+import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.thrift.TCreateTableParams;
+import org.apache.impala.thrift.TDistributeParam;
+import org.apache.impala.util.KuduUtil;
+import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder;
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.Schema;
+import org.apache.kudu.client.CreateTableOptions;
+import org.apache.kudu.client.KuduClient;
+import org.apache.kudu.client.PartialRow;
+import org.apache.log4j.Logger;
+
+/**
+ * This is a helper for the CatalogOpExecutor to provide Kudu related DDL functionality
+ * such as creating and dropping tables from Kudu.
+ */
+public class KuduCatalogOpExecutor {
+  public static final Logger LOG = Logger.getLogger(KuduCatalogOpExecutor.class);
+
+  /**
+   * Create a table in Kudu with a schema equivalent to the schema stored in 'msTbl'.
+   * Throws an exception if 'msTbl' represents an external table or if the table couldn't
+   * be created in Kudu.
+   */
+  static void createManagedTable(org.apache.hadoop.hive.metastore.api.Table msTbl,
+      TCreateTableParams params) throws ImpalaRuntimeException {
+    Preconditions.checkState(!Table.isExternalTable(msTbl));
+    String kuduTableName = msTbl.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    String masterHosts = msTbl.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
+    LOG.debug(String.format("Creating table '%s' in master '%s'", kuduTableName,
+        masterHosts));
+    try (KuduClient kudu = new KuduClient.KuduClientBuilder(masterHosts).build()) {
+      // TODO: The IF NOT EXISTS case should be handled by Kudu to ensure atomicity.
+      // (see KUDU-1710).
+      if (kudu.tableExists(kuduTableName)) {
+        if (params.if_not_exists) return;
+        throw new ImpalaRuntimeException(String.format(
+            "Table '%s' already exists in Kudu.", kuduTableName));
+      }
+      Schema schema = createTableSchema(msTbl, params);
+      CreateTableOptions tableOpts = buildTableOptions(msTbl, params, schema);
+      kudu.createTable(kuduTableName, schema, tableOpts);
+    } catch (Exception e) {
+      throw new ImpalaRuntimeException(String.format("Error creating table '%s'",
+          kuduTableName), e);
+    }
+  }
+
+  /**
+   * Creates the schema of a new Kudu table.
+   */
+  private static Schema createTableSchema(
+      org.apache.hadoop.hive.metastore.api.Table msTbl, TCreateTableParams params)
+      throws ImpalaRuntimeException {
+    Set<String> keyColNames = new HashSet<>(params.getPrimary_key_column_names());
+    List<FieldSchema> fieldSchemas = msTbl.getSd().getCols();
+    List<ColumnSchema> colSchemas = new ArrayList<>(fieldSchemas.size());
+    for (FieldSchema fieldSchema : fieldSchemas) {
+      Type type = Type.parseColumnType(fieldSchema.getType());
+      Preconditions.checkState(type != null);
+      org.apache.kudu.Type kuduType = KuduUtil.fromImpalaType(type);
+      // Create the actual column and check if the column is a key column
+      ColumnSchemaBuilder csb =
+          new ColumnSchemaBuilder(fieldSchema.getName(), kuduType);
+      boolean isKeyCol = keyColNames.contains(fieldSchema.getName());
+      csb.key(isKeyCol);
+      csb.nullable(!isKeyCol);
+      colSchemas.add(csb.build());
+    }
+    return new Schema(colSchemas);
+  }
+
+  /**
+   * Builds the table options of a new Kudu table.
+   */
+  private static CreateTableOptions buildTableOptions(
+      org.apache.hadoop.hive.metastore.api.Table msTbl,
+      TCreateTableParams params, Schema schema) throws ImpalaRuntimeException {
+    CreateTableOptions tableOpts = new CreateTableOptions();
+    // Set the distribution schemes
+    List<TDistributeParam> distributeParams = params.getDistribute_by();
+    if (distributeParams != null) {
+      boolean hasRangePartitioning = false;
+      for (TDistributeParam distParam : distributeParams) {
+        if (distParam.isSetBy_hash_param()) {
+          Preconditions.checkState(!distParam.isSetBy_range_param());
+          tableOpts.addHashPartitions(distParam.getBy_hash_param().getColumns(),
+              distParam.getBy_hash_param().getNum_buckets());
+        } else {
+          Preconditions.checkState(distParam.isSetBy_range_param());
+          hasRangePartitioning = true;
+          tableOpts.setRangePartitionColumns(
+              distParam.getBy_range_param().getColumns());
+          for (PartialRow partialRow :
+              KuduUtil.parseSplits(schema, distParam.getBy_range_param())) {
+            tableOpts.addSplitRow(partialRow);
+          }
+        }
+      }
+      // If no range-based distribution is specified in a CREATE TABLE statement, Kudu
+      // generates one by default that includes all the primary key columns. To prevent
+      // this from happening, explicitly set the range partition columns to be
+      // an empty list.
+      if (!hasRangePartitioning) {
+        tableOpts.setRangePartitionColumns(Collections.<String>emptyList());
+      }
+    }
+
+    // Set the number of table replicas, if specified.
+    String replication = msTbl.getParameters().get(KuduTable.KEY_TABLET_REPLICAS);
+    if (!Strings.isNullOrEmpty(replication)) {
+      try {
+        int r = Integer.parseInt(replication);
+        Preconditions.checkState(r > 0);
+        tableOpts.setNumReplicas(r);
+      } catch (NumberFormatException e) {
+        throw new ImpalaRuntimeException(String.format("Invalid number of table " +
+            "replicas specified: '%s'", replication), e);
+      }
+    }
+    return tableOpts;
+  }
+
+  /**
+   * Drops the table in Kudu. If the table does not exist and 'ifExists' is false, a
+   * TableNotFoundException is thrown. If the table exists and could not be dropped,
+   * an ImpalaRuntimeException is thrown.
+   */
+  static void dropTable(org.apache.hadoop.hive.metastore.api.Table msTbl,
+      boolean ifExists) throws ImpalaRuntimeException, TableNotFoundException {
+    Preconditions.checkState(!Table.isExternalTable(msTbl));
+    String tableName = msTbl.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    String masterHosts = msTbl.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
+    LOG.debug(String.format("Dropping table '%s' from master '%s'", tableName,
+        masterHosts));
+    try (KuduClient kudu = new KuduClient.KuduClientBuilder(masterHosts).build()) {
+      Preconditions.checkState(!Strings.isNullOrEmpty(tableName));
+      // TODO: The IF EXISTS case should be handled by Kudu to ensure atomicity.
+      // (see KUDU-1710).
+      if (kudu.tableExists(tableName)) {
+        kudu.deleteTable(tableName);
+      } else if (!ifExists) {
+        throw new TableNotFoundException(String.format(
+            "Table '%s' does not exist in Kudu master(s) '%s'.", tableName, masterHosts));
+      }
+    } catch (Exception e) {
+      throw new ImpalaRuntimeException(String.format("Error dropping table '%s'",
+          tableName), e);
+    }
+  }
+
+  /**
+   * Reads the column definitions from a Kudu table and populates 'msTbl' with
+   * an equivalent schema. Throws an exception if any errors are encountered.
+   */
+  public static void populateColumnsFromKudu(
+      org.apache.hadoop.hive.metastore.api.Table msTbl) throws ImpalaRuntimeException {
+    org.apache.hadoop.hive.metastore.api.Table msTblCopy = msTbl.deepCopy();
+    List<FieldSchema> cols = msTblCopy.getSd().getCols();
+    String kuduTableName = msTblCopy.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    Preconditions.checkState(!Strings.isNullOrEmpty(kuduTableName));
+    String masterHosts = msTblCopy.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
+    LOG.debug(String.format("Loading schema of table '%s' from master '%s'",
+        kuduTableName, masterHosts));
+    try (KuduClient kudu = new KuduClient.KuduClientBuilder(masterHosts).build()) {
+      if (!kudu.tableExists(kuduTableName)) {
+        throw new ImpalaRuntimeException(String.format("Table does not exist in Kudu: " +
+            "'%s'", kuduTableName));
+      }
+      org.apache.kudu.client.KuduTable kuduTable = kudu.openTable(kuduTableName);
+      // Replace the columns in the Metastore table with the columns from the recently
+      // accessed Kudu schema.
+      cols.clear();
+      for (ColumnSchema colSchema : kuduTable.getSchema().getColumns()) {
+        Type type = KuduUtil.toImpalaType(colSchema.getType());
+        cols.add(new FieldSchema(colSchema.getName(), type.toSql().toLowerCase(), null));
+      }
+    } catch (Exception e) {
+      throw new ImpalaRuntimeException(String.format("Error loading schema of table " +
+          "'%s'", kuduTableName), e);
+    }
+    List<FieldSchema> newCols = msTbl.getSd().getCols();
+    newCols.clear();
+    newCols.addAll(cols);
+  }
+
+  /**
+   * Validates the table properties of a Kudu table. It checks that the specified master
+   * addresses point to valid Kudu masters and that the table exists.
+   * Throws an ImpalaRuntimeException if this is not the case.
+   */
+  public static void validateKuduTblExists(
+      org.apache.hadoop.hive.metastore.api.Table msTbl) throws ImpalaRuntimeException {
+    String masterHosts = msTbl.getParameters().get(KuduTable.KEY_MASTER_HOSTS);
+    Preconditions.checkState(!Strings.isNullOrEmpty(masterHosts));
+    String kuduTableName = msTbl.getParameters().get(KuduTable.KEY_TABLE_NAME);
+    Preconditions.checkState(!Strings.isNullOrEmpty(kuduTableName));
+    try (KuduClient kudu = new KuduClient.KuduClientBuilder(masterHosts).build()) {
+      kudu.tableExists(kuduTableName);
+    } catch (Exception e) {
+      throw new ImpalaRuntimeException(String.format("Kudu table '%s' does not exist " +
+          "on master '%s'", kuduTableName, masterHosts), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/util/KuduUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/KuduUtil.java b/fe/src/main/java/org/apache/impala/util/KuduUtil.java
index b9f8653..a679032 100644
--- a/fe/src/main/java/org/apache/impala/util/KuduUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/KuduUtil.java
@@ -18,14 +18,15 @@
 package org.apache.impala.util;
 
 import java.io.StringReader;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import javax.json.Json;
 import javax.json.JsonArray;
 import javax.json.JsonReader;
 
+import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.catalog.Type;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.thrift.TDistributeByRangeParam;
 import org.apache.impala.thrift.TRangeLiteral;
@@ -33,48 +34,17 @@ import org.apache.impala.thrift.TRangeLiteralList;
 import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Schema;
-import org.apache.kudu.Type;
-import org.apache.kudu.client.KuduTable;
 import org.apache.kudu.client.PartialRow;
 
-import static org.apache.impala.catalog.Type.parseColumnType;
 import static java.lang.String.format;
 
 public class KuduUtil {
 
   private static final String SPLIT_KEYS_ERROR_MESSAGE = "Error parsing splits keys.";
-
-  /**
-   * Compare the schema of a HMS table and a Kudu table. Returns true if both tables have
-   * a matching schema.
-   */
-  public static boolean compareSchema(Table msTable, KuduTable kuduTable)
-      throws ImpalaRuntimeException {
-    List<FieldSchema> msFields = msTable.getSd().getCols();
-    List<ColumnSchema> kuduFields = kuduTable.getSchema().getColumns();
-    if (msFields.size() != kuduFields.size()) return false;
-
-    HashMap<String, ColumnSchema> kuduFieldMap = Maps.newHashMap();
-    for (ColumnSchema kuduField : kuduFields) {
-      kuduFieldMap.put(kuduField.getName().toUpperCase(), kuduField);
-    }
-
-    for (FieldSchema msField : msFields) {
-      ColumnSchema kuduField = kuduFieldMap.get(msField.getName().toUpperCase());
-      if (kuduField == null
-          || fromImpalaType(parseColumnType(msField.getType())) != kuduField.getType()) {
-        return false;
-      }
-    }
-
-    return true;
-  }
+  private static final String KUDU_TABLE_NAME_PREFIX = "impala::";
 
   /**
    * Parses split keys from statements.
@@ -145,10 +115,9 @@ public class KuduUtil {
   /**
    * Sets the value in 'key' at 'pos', given the json representation.
    */
-  private static void setKey(Type type, JsonArray array, int pos, PartialRow key)
-      throws ImpalaRuntimeException {
+  private static void setKey(org.apache.kudu.Type type, JsonArray array, int pos,
+      PartialRow key) throws ImpalaRuntimeException {
     switch (type) {
-      case BOOL: key.addBoolean(pos, array.getBoolean(pos)); break;
       case INT8: key.addByte(pos, (byte) array.getInt(pos)); break;
       case INT16: key.addShort(pos, (short) array.getInt(pos)); break;
       case INT32: key.addInt(pos, array.getInt(pos)); break;
@@ -163,13 +132,9 @@ public class KuduUtil {
   /**
    * Sets the value in 'key' at 'pos', given the range literal.
    */
-  private static void setKey(Type type, TRangeLiteral literal, int pos, String colName,
-      PartialRow key) throws ImpalaRuntimeException {
+  private static void setKey(org.apache.kudu.Type type, TRangeLiteral literal, int pos,
+      String colName, PartialRow key) throws ImpalaRuntimeException {
     switch (type) {
-      case BOOL:
-        checkCorrectType(literal.isSetBool_literal(), type, colName, literal);
-        key.addBoolean(pos, literal.isBool_literal());
-        break;
       case INT8:
         checkCorrectType(literal.isSetInt_literal(), type, colName, literal);
         key.addByte(pos, (byte) literal.getInt_literal());
@@ -200,8 +165,8 @@ public class KuduUtil {
    * If correctType is true, returns. Otherwise throws a formatted error message
    * indicating problems with the type of the literal of the range literal.
    */
-  private static void checkCorrectType(boolean correctType, Type t, String colName,
-      TRangeLiteral literal) throws ImpalaRuntimeException {
+  private static void checkCorrectType(boolean correctType, org.apache.kudu.Type t,
+      String colName, TRangeLiteral literal) throws ImpalaRuntimeException {
     if (correctType) return;
     throw new ImpalaRuntimeException(
         format("Expected %s literal for column '%s' got '%s'", t.getName(), colName,
@@ -220,11 +185,24 @@ public class KuduUtil {
     return Lists.newArrayList(Splitter.on(",").trimResults().split(cols.toLowerCase()));
   }
 
+  public static boolean isSupportedKeyType(org.apache.impala.catalog.Type type) {
+    return type.isIntegerType() || type.isStringType();
+  }
+
+  /**
+   * Return the name that should be used in Kudu when creating a table, assuming a custom
+   * name was not provided.
+   */
+  public static String getDefaultCreateKuduTableName(String metastoreDbName,
+      String metastoreTableName) {
+    return KUDU_TABLE_NAME_PREFIX + metastoreDbName + "." + metastoreTableName;
+  }
+
   /**
    * Converts a given Impala catalog type to the Kudu type. Throws an exception if the
    * type cannot be converted.
    */
-  public static Type fromImpalaType(org.apache.impala.catalog.Type t)
+  public static org.apache.kudu.Type fromImpalaType(Type t)
       throws ImpalaRuntimeException {
     if (!t.isScalarType()) {
       throw new ImpalaRuntimeException(format(
@@ -232,16 +210,16 @@ public class KuduUtil {
     }
     ScalarType s = (ScalarType) t;
     switch (s.getPrimitiveType()) {
-      case TINYINT: return Type.INT8;
-      case SMALLINT: return Type.INT16;
-      case INT: return Type.INT32;
-      case BIGINT: return Type.INT64;
-      case BOOLEAN: return Type.BOOL;
-      case CHAR: return Type.STRING;
-      case STRING: return Type.STRING;
-      case VARCHAR: return Type.STRING;
-      case DOUBLE: return Type.DOUBLE;
-      case FLOAT: return Type.FLOAT;
+      case TINYINT: return org.apache.kudu.Type.INT8;
+      case SMALLINT: return org.apache.kudu.Type.INT16;
+      case INT: return org.apache.kudu.Type.INT32;
+      case BIGINT: return org.apache.kudu.Type.INT64;
+      case BOOLEAN: return org.apache.kudu.Type.BOOL;
+      case CHAR: return org.apache.kudu.Type.STRING;
+      case STRING: return org.apache.kudu.Type.STRING;
+      case VARCHAR: return org.apache.kudu.Type.STRING;
+      case DOUBLE: return org.apache.kudu.Type.DOUBLE;
+      case FLOAT: return org.apache.kudu.Type.FLOAT;
         /* Fall through below */
       case INVALID_TYPE:
       case NULL_TYPE:
@@ -256,11 +234,27 @@ public class KuduUtil {
     }
   }
 
+  public static Type toImpalaType(org.apache.kudu.Type t)
+      throws ImpalaRuntimeException {
+    switch (t) {
+      case BOOL: return Type.BOOLEAN;
+      case DOUBLE: return Type.DOUBLE;
+      case FLOAT: return Type.FLOAT;
+      case INT8: return Type.TINYINT;
+      case INT16: return Type.SMALLINT;
+      case INT32: return Type.INT;
+      case INT64: return Type.BIGINT;
+      case STRING: return Type.STRING;
+      default:
+        throw new ImpalaRuntimeException(String.format(
+            "Kudu type %s is not supported in Impala", t));
+    }
+  }
+
   /**
    * Returns the string value of the RANGE literal.
    */
   static String toString(TRangeLiteral l) throws ImpalaRuntimeException {
-    if (l.isSetBool_literal()) return String.valueOf(l.bool_literal);
     if (l.isSetString_literal()) return String.valueOf(l.string_literal);
     if (l.isSetInt_literal()) return String.valueOf(l.int_literal);
     throw new ImpalaRuntimeException("Unsupported type for RANGE literal.");



[11/14] incubator-impala git commit: IMPALA-3718: Add test_cancellation tests for Kudu

Posted by ta...@apache.org.
IMPALA-3718: Add test_cancellation tests for Kudu

Additional functional tests for Kudu.

Change-Id: Icf3d3853e7075991f6d12f125407ebdbe6a287e2
Reviewed-on: http://gerrit.cloudera.org:8080/4700
Reviewed-by: Matthew Jacobs <mj...@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/8d7b01fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/8d7b01fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/8d7b01fa

Branch: refs/heads/master
Commit: 8d7b01faea6362af675a2a335b462fad3e0caa03
Parents: 8a49cea
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Wed Sep 21 15:05:54 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 23:32:58 2016 +0000

----------------------------------------------------------------------
 tests/query_test/test_cancellation.py | 49 +++++++++++++++++++++---------
 1 file changed, 35 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8d7b01fa/tests/query_test/test_cancellation.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_cancellation.py b/tests/query_test/test_cancellation.py
index 265c781..91e81dc 100644
--- a/tests/query_test/test_cancellation.py
+++ b/tests/query_test/test_cancellation.py
@@ -27,13 +27,17 @@ from tests.common.test_vector import TestDimension
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.verifiers.metric_verifier import MetricVerifier
 
-# Queries to execute. Use the TPC-H dataset because tables are large so queries take some
-# time to execute.
-QUERIES = ['select l_returnflag from lineitem',
-           'select count(l_returnflag) from lineitem',
-           'select * from lineitem limit 50',
-           'compute stats lineitem',
-           'select * from lineitem order by l_orderkey']
+# PRIMARY KEY for lineitem
+LINEITEM_PK = 'l_orderkey, l_partkey, l_suppkey, l_linenumber'
+
+# Queries to execute, mapped to a unique PRIMARY KEY for use in CTAS with Kudu. If None
+# is specified for the PRIMARY KEY, it will not be used in a CTAS statement on Kudu.
+# Use the TPC-H dataset because tables are large so queries take some time to execute.
+QUERIES = {'select l_returnflag from lineitem' : None,
+           'select count(l_returnflag) pk from lineitem' : 'pk',
+           'select * from lineitem limit 50' : LINEITEM_PK,
+           'compute stats lineitem' : None,
+           'select * from lineitem order by l_orderkey' : LINEITEM_PK}
 
 QUERY_TYPE = ["SELECT", "CTAS"]
 
@@ -59,19 +63,25 @@ class TestCancellation(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestCancellation, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('query', *QUERIES))
+    cls.TestMatrix.add_dimension(TestDimension('query', *QUERIES.keys()))
     cls.TestMatrix.add_dimension(TestDimension('query_type', *QUERY_TYPE))
     cls.TestMatrix.add_dimension(TestDimension('cancel_delay', *CANCEL_DELAY_IN_SECONDS))
     cls.TestMatrix.add_dimension(TestDimension('action', *DEBUG_ACTIONS))
     cls.TestMatrix.add_dimension(TestDimension('max_block_mgr_memory', 0))
 
     cls.TestMatrix.add_constraint(lambda v: v.get_value('query_type') != 'CTAS' or (\
-        v.get_value('table_format').file_format in ['text', 'parquet'] and\
+        v.get_value('table_format').file_format in ['text', 'parquet', 'kudu'] and\
         v.get_value('table_format').compression_codec == 'none'))
     cls.TestMatrix.add_constraint(lambda v: v.get_value('exec_option')['batch_size'] == 0)
     # Ignore 'compute stats' queries for the CTAS query type.
     cls.TestMatrix.add_constraint(lambda v: not (v.get_value('query_type') == 'CTAS' and
          v.get_value('query').startswith('compute stats')))
+
+    # Ignore CTAS on Kudu if there is no PRIMARY KEY specified.
+    cls.TestMatrix.add_constraint(lambda v: not (v.get_value('query_type') == 'CTAS' and
+         v.get_value('table_format').file_format == 'kudu' and
+         QUERIES[v.get_value('query')] is None))
+
     # tpch tables are not generated for hbase as the data loading takes a very long time.
     # TODO: Add cancellation tests for hbase.
     cls.TestMatrix.add_constraint(lambda v:\
@@ -87,15 +97,24 @@ class TestCancellation(ImpalaTestSuite):
     query_type = vector.get_value('query_type')
     if query_type == "CTAS":
       self.cleanup_test_table(vector.get_value('table_format'))
-      query = "create table ctas_cancel stored as %sfile as %s" %\
-          (vector.get_value('table_format').file_format, query)
+      file_format = vector.get_value('table_format').file_format
+      if file_format == 'kudu':
+        assert QUERIES.has_key(query) and QUERIES[query] is not None,\
+            "PRIMARY KEY for query %s not specified" % query
+        query = "create table ctas_cancel primary key (%s) "\
+            "distribute by hash into 3 buckets stored as kudu as %s" %\
+            (QUERIES[query], query)
+      else:
+        query = "create table ctas_cancel stored as %sfile as %s" %\
+            (file_format, query)
 
     action = vector.get_value('action')
     # node ID 0 is the scan node
     debug_action = '0:GETNEXT:' + action if action != None else ''
     vector.get_value('exec_option')['debug_action'] = debug_action
 
-    vector.get_value('exec_option')['max_block_mgr_memory'] = vector.get_value('max_block_mgr_memory')
+    vector.get_value('exec_option')['max_block_mgr_memory'] =\
+        vector.get_value('max_block_mgr_memory')
 
     # Execute the query multiple times, cancelling it each time.
     for i in xrange(NUM_CANCELATION_ITERATIONS):
@@ -167,7 +186,8 @@ class TestCancellationSerial(TestCancellation):
     # Don't run across all cancel delay options unless running in exhaustive mode
     if cls.exploration_strategy() != 'exhaustive':
       cls.TestMatrix.add_constraint(lambda v: v.get_value('cancel_delay') in [3])
-      cls.TestMatrix.add_constraint(lambda v: v.get_value('query') == choice(QUERIES))
+      cls.TestMatrix.add_constraint(lambda v: v.get_value('query') ==\
+          choice(QUERIES.keys()))
 
   @pytest.mark.execute_serially
   def test_cancel_insert(self, vector):
@@ -186,7 +206,8 @@ class TestCancellationFullSort(TestCancellation):
     cls.TestMatrix.add_dimension(TestDimension('query', SORT_QUERY))
     cls.TestMatrix.add_dimension(TestDimension('query_type', 'SELECT'))
     cls.TestMatrix.add_dimension(TestDimension('cancel_delay', *SORT_CANCEL_DELAY))
-    cls.TestMatrix.add_dimension(TestDimension('max_block_mgr_memory', *SORT_BLOCK_MGR_LIMIT))
+    cls.TestMatrix.add_dimension(TestDimension('max_block_mgr_memory',\
+        *SORT_BLOCK_MGR_LIMIT))
     cls.TestMatrix.add_dimension(TestDimension('action', None))
     cls.TestMatrix.add_constraint(lambda v:\
        v.get_value('table_format').file_format =='parquet' and\


[08/14] incubator-impala git commit: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by ta...@apache.org.
IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Reviewed-on: http://gerrit.cloudera.org:8080/4414
Reviewed-by: Alex Behm <al...@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/041fa6d9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/041fa6d9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/041fa6d9

Branch: refs/heads/master
Commit: 041fa6d946e1cbe309593c4a5515bef88e06cdb4
Parents: f8d48b8
Author: Dimitris Tsirogiannis <dt...@cloudera.com>
Authored: Wed Aug 31 16:56:47 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 10:52:25 2016 +0000

----------------------------------------------------------------------
 be/src/service/frontend.cc                      |   9 +-
 bin/start-catalogd.sh                           |  30 +-
 bin/start-impala-cluster.py                     |  11 +-
 common/thrift/CatalogObjects.thrift             |  39 +-
 common/thrift/JniCatalog.thrift                 |  44 +-
 fe/src/main/cup/sql-parser.cup                  | 407 +++++++-------
 .../apache/impala/analysis/AnalysisUtils.java   |  43 ++
 .../org/apache/impala/analysis/ColumnDef.java   |  66 ++-
 .../analysis/CreateTableAsSelectStmt.java       |  41 +-
 .../impala/analysis/CreateTableDataSrcStmt.java |  30 +-
 .../analysis/CreateTableLikeFileStmt.java       |  29 +-
 .../impala/analysis/CreateTableLikeStmt.java    |  19 +-
 .../apache/impala/analysis/CreateTableStmt.java | 480 ++++++++--------
 .../apache/impala/analysis/DistributeParam.java | 211 ++++---
 .../org/apache/impala/analysis/ModifyStmt.java  |   4 +-
 .../apache/impala/analysis/TableDataLayout.java |  56 ++
 .../org/apache/impala/analysis/TableDef.java    | 316 +++++++++++
 .../org/apache/impala/analysis/ToSqlUtils.java  |  68 ++-
 .../java/org/apache/impala/catalog/Catalog.java |   4 +
 .../impala/catalog/CatalogServiceCatalog.java   |   7 +-
 .../main/java/org/apache/impala/catalog/Db.java |   5 +
 .../apache/impala/catalog/HdfsFileFormat.java   |  15 +-
 .../apache/impala/catalog/ImpaladCatalog.java   |   8 +-
 .../org/apache/impala/catalog/KuduTable.java    | 318 +++++++----
 .../java/org/apache/impala/catalog/Table.java   |   5 +
 .../org/apache/impala/catalog/TableLoader.java  |   1 -
 .../java/org/apache/impala/catalog/Type.java    |   9 +
 .../impala/catalog/delegates/DdlDelegate.java   |  75 ---
 .../catalog/delegates/KuduDdlDelegate.java      | 190 -------
 .../delegates/UnsupportedOpDelegate.java        |  35 --
 .../impala/planner/HdfsPartitionFilter.java     |   2 +-
 .../org/apache/impala/planner/KuduScanNode.java |   2 +-
 .../impala/service/CatalogOpExecutor.java       | 413 ++++++++------
 .../org/apache/impala/service/Frontend.java     |   9 +-
 .../org/apache/impala/service/JniCatalog.java   |   2 +-
 .../org/apache/impala/service/JniFrontend.java  |   5 +-
 .../impala/service/KuduCatalogOpExecutor.java   | 240 ++++++++
 .../java/org/apache/impala/util/KuduUtil.java   | 106 ++--
 fe/src/main/jflex/sql-scanner.flex              |   2 +
 .../apache/impala/analysis/AnalyzeDDLTest.java  | 409 ++++++++------
 .../org/apache/impala/analysis/ParserTest.java  |  32 +-
 .../org/apache/impala/service/JdbcTest.java     |   6 +-
 .../impala/testutil/ImpaladTestCatalog.java     |   6 +-
 infra/python/deps/download_requirements         |   2 +-
 infra/python/deps/requirements.txt              |   6 +-
 testdata/bin/generate-schema-statements.py      |  20 +-
 .../functional/functional_schema_template.sql   | 143 +----
 testdata/datasets/tpch/tpch_schema_template.sql | 120 +---
 .../queries/PlannerTest/kudu.test               |  28 +-
 .../queries/QueryTest/create_kudu.test          |  90 ---
 .../queries/QueryTest/kudu-scan-node.test       |  34 +-
 .../queries/QueryTest/kudu-show-create.test     |  16 -
 .../queries/QueryTest/kudu_alter.test           |  21 +-
 .../queries/QueryTest/kudu_create.test          | 105 ++++
 .../queries/QueryTest/kudu_crud.test            |  67 +--
 .../queries/QueryTest/kudu_partition_ddl.test   | 103 +---
 .../queries/QueryTest/kudu_stats.test           |  12 +-
 tests/common/__init__.py                        |   4 +-
 tests/common/kudu_test_suite.py                 | 148 +++++
 tests/conftest.py                               |  10 +-
 tests/custom_cluster/test_kudu.py               |  53 ++
 tests/metadata/test_ddl.py                      |   3 +-
 tests/metadata/test_show_create_table.py        |   5 -
 tests/query_test/test_kudu.py                   | 560 ++++++++++++++-----
 64 files changed, 3121 insertions(+), 2238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/be/src/service/frontend.cc
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index 855924f..ca8ecbb 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -58,11 +58,13 @@ DEFINE_string(authorized_proxy_user_config, "",
     "users. For example: hue=user1,user2;admin=*");
 DEFINE_string(authorized_proxy_user_config_delimiter, ",",
     "Specifies the delimiter used in authorized_proxy_user_config. ");
-
+DEFINE_string(kudu_master_hosts, "", "Specifies the default Kudu master(s). The given "
+    "value should be a comma separated list of hostnames or IP addresses; ports are "
+    "optional.");
 Frontend::Frontend() {
   JniMethodDescriptor methods[] = {
     {"<init>", "(ZLjava/lang/String;Ljava/lang/String;Ljava/lang/String;"
-        "Ljava/lang/String;IIZ)V", &fe_ctor_},
+        "Ljava/lang/String;IIZLjava/lang/String;)V", &fe_ctor_},
     {"createExecRequest", "([B)[B", &create_exec_request_id_},
     {"getExplainPlan", "([B)Ljava/lang/String;", &get_explain_plan_id_},
     {"getHadoopConfig", "([B)[B", &get_hadoop_config_id_},
@@ -111,9 +113,10 @@ Frontend::Frontend() {
   // auth_to_local rules are read if --load_auth_to_local_rules is set to true
   // and impala is kerberized.
   jboolean auth_to_local = FLAGS_load_auth_to_local_rules && !FLAGS_principal.empty();
+  jstring kudu_master_hosts = jni_env->NewStringUTF(FLAGS_kudu_master_hosts.c_str());
   jobject fe = jni_env->NewObject(fe_class_, fe_ctor_, lazy, server_name,
       policy_file_path, sentry_config, auth_provider_class, FlagToTLogLevel(FLAGS_v),
-      FlagToTLogLevel(FLAGS_non_impala_java_vlog), auth_to_local);
+      FlagToTLogLevel(FLAGS_non_impala_java_vlog), auth_to_local, kudu_master_hosts);
   EXIT_IF_EXC(jni_env);
   ABORT_IF_ERROR(JniUtil::LocalToGlobalRef(jni_env, fe, &fe_));
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/bin/start-catalogd.sh
----------------------------------------------------------------------
diff --git a/bin/start-catalogd.sh b/bin/start-catalogd.sh
index 05eb8bd..1046c82 100755
--- a/bin/start-catalogd.sh
+++ b/bin/start-catalogd.sh
@@ -34,29 +34,17 @@ JVM_ARGS=""
 for ARG in $*
 do
   case "$ARG" in
-    -build_type=debug)
-      BUILD_TYPE=debug
-      ;;
-    -build_type=release)
-      BUILD_TYPE=release
-      ;;
-    -build_type=latest)
-      ;;
+    -build_type=debug) BUILD_TYPE=debug;;
+    -build_type=release) BUILD_TYPE=release;;
+    -build_type=latest) ;;
     -build_type=*)
       echo "Invalid build type. Valid values are: debug, release"
-      exit 1
-      ;;
-    -jvm_debug_port=*)
-      JVM_DEBUG_PORT="${ARG#*=}"
-      ;;
-    -jvm_suspend)
-      JVM_SUSPEND="y"
-      ;;
-    -jvm_args=*)
-      JVM_ARGS="${ARG#*=}"
-      ;;
-    *)
-      CATALOGD_ARGS="${CATALOGD_ARGS} ${ARG}"
+      exit 1;;
+    -jvm_debug_port=*) JVM_DEBUG_PORT="${ARG#*=}";;
+    -jvm_suspend) JVM_SUSPEND="y";;
+    -jvm_args=*) JVM_ARGS="${ARG#*=}";;
+    -kudu_masters=*) CATALOGD_ARGS+=" ${ARG#*=}";;
+    *) CATALOGD_ARGS="${CATALOGD_ARGS} ${ARG}";;
   esac
 done
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/bin/start-impala-cluster.py
----------------------------------------------------------------------
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index 3ea338a..df7dea6 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -27,6 +27,7 @@ from getpass import getuser
 from time import sleep, time
 from optparse import OptionParser
 from testdata.common import cgroups
+from tests.common import KUDU_MASTER_HOSTS
 
 
 DEFAULT_IMPALA_MAX_LOG_FILES = os.environ.get('IMPALA_MAX_LOG_FILES', 10)
@@ -57,7 +58,8 @@ parser.add_option("-r", "--restart_impalad_only", dest="restart_impalad_only",
                   help="Restarts only the impalad processes")
 parser.add_option("--in-process", dest="inprocess", action="store_true", default=False,
                   help="Start all Impala backends and state store in a single process.")
-parser.add_option("--log_dir", dest="log_dir", default=os.environ['IMPALA_CLUSTER_LOGS_DIR'],
+parser.add_option("--log_dir", dest="log_dir",
+                  default=os.environ['IMPALA_CLUSTER_LOGS_DIR'],
                   help="Directory to store output logs to.")
 parser.add_option('--max_log_files', default=DEFAULT_IMPALA_MAX_LOG_FILES,
                   help='Max number of log files before rotation occurs.')
@@ -70,6 +72,9 @@ parser.add_option("--log_level", type="int", dest="log_level", default=1,
                    help="Set the impalad backend logging level")
 parser.add_option("--jvm_args", dest="jvm_args", default="",
                   help="Additional arguments to pass to the JVM(s) during startup.")
+parser.add_option("--kudu_masters", default=KUDU_MASTER_HOSTS,
+                  help="The host name or address of the Kudu master. Multiple masters "
+                      "can be specified using a comma separated list.")
 
 options, args = parser.parse_args()
 
@@ -193,7 +198,6 @@ def build_impalad_port_args(instance_num):
                           BASE_WEBSERVER_PORT + instance_num)
 
 def build_impalad_logging_args(instance_num, service_name):
-  log_file_path = os.path.join(options.log_dir, "%s.INFO" % service_name)
   return BE_LOGGING_ARGS % (service_name, options.log_dir, options.log_level,
                             options.max_log_files)
 
@@ -233,6 +237,9 @@ def start_impalad_instances(cluster_size):
           (mem_limit,  # Goes first so --impalad_args will override it.
            build_impalad_logging_args(i, service_name), build_jvm_args(i),
            build_impalad_port_args(i), param_args)
+    if options.kudu_masters:
+      # Must be prepended, otherwise the java options interfere.
+      args = "-kudu_master_hosts %s %s" % (options.kudu_masters, args)
     stderr_log_file_path = os.path.join(options.log_dir, '%s-error.log' % service_name)
     exec_impala_process(IMPALAD_PATH, args, stderr_log_file_path)
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/common/thrift/CatalogObjects.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index 5378988..78aa19d 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -49,14 +49,20 @@ enum TTableType {
   KUDU_TABLE,
 }
 
+// TODO: Separate the storage engines (e.g. Kudu) from the file formats.
+// TODO: Make the names consistent with the file format keywords specified in
+// the parser.
 enum THdfsFileFormat {
   TEXT,
   RC_FILE,
   SEQUENCE_FILE,
   AVRO,
-  PARQUET
+  PARQUET,
+  KUDU
 }
 
+// TODO: Since compression is also enabled for Kudu columns, we should
+// rename this enum to not be Hdfs specific.
 enum THdfsCompression {
   NONE,
   DEFAULT,
@@ -337,6 +343,34 @@ struct TDataSourceTable {
   2: required string init_string
 }
 
+// Parameters needed for hash distribution
+struct TDistributeByHashParam {
+  1: required list<string> columns
+  2: required i32 num_buckets
+}
+
+struct TRangeLiteral {
+  1: optional i64 int_literal
+  2: optional string string_literal
+}
+
+struct TRangeLiteralList {
+  // TODO: Replace TRangeLiteral with Exprs.TExpr.
+  1: required list<TRangeLiteral> values
+}
+
+// A range distribution is identified by a list of columns and a series of split rows.
+struct TDistributeByRangeParam {
+  1: required list<string> columns
+  2: optional list<TRangeLiteralList> split_rows;
+}
+
+// Parameters for the DISTRIBUTE BY clause.
+struct TDistributeParam {
+  1: optional TDistributeByHashParam by_hash_param;
+  2: optional TDistributeByRangeParam by_range_param;
+}
+
 // Represents a Kudu table
 struct TKuduTable {
   1: required string table_name
@@ -346,6 +380,9 @@ struct TKuduTable {
 
   // Name of the key columns
   3: required list<string> key_columns
+
+  // Distribution schemes
+  4: required list<TDistributeParam> distribute_by
 }
 
 // Represents a table or view.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/common/thrift/JniCatalog.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
index b97e458..8ed9fe3 100644
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -155,45 +155,6 @@ struct THdfsCachingOp {
   3: optional i16 replication
 }
 
-// Enum listing all possible DISTRIBUTE BY types
-enum TDistributeType {
-  HASH,
-  RANGE,
-}
-
-// Parameters needed for hash distribution
-struct TDistributeByHashParam {
-  1: required list<string> columns
-  2: required i32 num_buckets
-}
-
-struct TRangeLiteral {
-  1: optional i64 int_literal
-  2: optional double float_literal
-  3: optional string string_literal
-  4: optional bool bool_literal
-}
-
-struct TRangeLiteralList {
-  1: required list<TRangeLiteral> values
-}
-
-// A range distribution is identified by a list of columns and a series of split rows.
-struct TDistributeByRangeParam {
-  1: required list<string> columns
-  2: required list<TRangeLiteralList> split_rows;
-}
-
-// Parameters for the DISTRIBUTE BY clause. The actual distribution is identified by
-// the type parameter.
-struct TDistributeParam {
-  // Set if type is set to HASH
-  1: optional TDistributeByHashParam by_hash_param;
-
-  // Set if type is set to RANGE
-  2: optional TDistributeByRangeParam by_range_param;
-}
-
 // Parameters for ALTER TABLE rename commands
 struct TAlterTableOrViewRenameParams {
   // The new table name
@@ -434,7 +395,10 @@ struct TCreateTableParams {
 
   // If set, the table is automatically distributed according to this parameter.
   // Kudu-only.
-  14: optional list<TDistributeParam> distribute_by;
+  14: optional list<CatalogObjects.TDistributeParam> distribute_by
+
+  // Primary key column names (Kudu-only)
+  15: optional list<string> primary_key_column_names;
 }
 
 // Parameters of a CREATE VIEW or ALTER VIEW AS SELECT command

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/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 6fc76f9..7554b5a 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -17,32 +17,33 @@
 
 package org.apache.impala.analysis;
 
-import org.apache.impala.catalog.Type;
-import org.apache.impala.catalog.ScalarType;
+import com.google.common.collect.Lists;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java_cup.runtime.Symbol;
+
+import org.apache.impala.analysis.ColumnDef;
+import org.apache.impala.analysis.UnionStmt.Qualifier;
+import org.apache.impala.analysis.UnionStmt.UnionOperand;
 import org.apache.impala.catalog.ArrayType;
 import org.apache.impala.catalog.MapType;
-import org.apache.impala.catalog.StructType;
-import org.apache.impala.catalog.StructField;
 import org.apache.impala.catalog.RowFormat;
+import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.catalog.StructField;
+import org.apache.impala.catalog.StructType;
+import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.View;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.Pair;
-import org.apache.impala.analysis.ColumnDef;
-import org.apache.impala.analysis.UnionStmt.UnionOperand;
-import org.apache.impala.analysis.UnionStmt.Qualifier;
 import org.apache.impala.thrift.TCatalogObjectType;
-import org.apache.impala.thrift.TFunctionCategory;
 import org.apache.impala.thrift.TDescribeOutputStyle;
+import org.apache.impala.thrift.TFunctionCategory;
 import org.apache.impala.thrift.THdfsFileFormat;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TTablePropertyType;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java_cup.runtime.Symbol;
-import com.google.common.collect.Lists;
 
 parser code {:
   private Symbol errorToken_;
@@ -248,11 +249,11 @@ terminal
   KW_FUNCTION, KW_FUNCTIONS, KW_GRANT, KW_GROUP, KW_HASH, KW_IGNORE, KW_HAVING, KW_IF,
   KW_ILIKE, KW_IN, KW_INCREMENTAL, KW_INIT_FN, KW_INNER, KW_INPATH, KW_INSERT, KW_INT,
   KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP, KW_IS, KW_JOIN,
-  KW_LAST, KW_LEFT, KW_LIKE, KW_LIMIT, KW_LINES, KW_LOAD, KW_LOCATION, KW_MAP,
+  KW_KUDU, KW_LAST, KW_LEFT, KW_LIKE, KW_LIMIT, KW_LINES, KW_LOAD, KW_LOCATION, KW_MAP,
   KW_MERGE_FN, KW_METADATA, KW_NOT, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORDER,
   KW_OUTER, KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION,
-  KW_PARTITIONED, KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRODUCED, KW_PURGE,
-  KW_RANGE, KW_RCFILE, KW_RECOVER, KW_REFRESH, KW_REGEXP, KW_RENAME, KW_REPLACE,
+  KW_PARTITIONED, KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED,
+  KW_PURGE, KW_RANGE, KW_RCFILE, KW_RECOVER, KW_REFRESH, KW_REGEXP, KW_RENAME, KW_REPLACE,
   KW_REPLICATION, KW_RESTRICT, KW_RETURNS, KW_REVOKE, KW_RIGHT, KW_RLIKE, KW_ROLE,
   KW_ROLES, KW_ROW, KW_ROWS, KW_SCHEMA, KW_SCHEMAS, KW_SELECT, KW_SEMI, KW_SEQUENCEFILE,
   KW_SERDEPROPERTIES, KW_SERIALIZE_FN, KW_SET, KW_SHOW, KW_SMALLINT, KW_SPLIT, KW_STORED,
@@ -318,7 +319,7 @@ nonterminal SelectListItem star_expr;
 nonterminal Expr expr, non_pred_expr, arithmetic_expr, timestamp_arithmetic_expr;
 nonterminal ArrayList<Expr> expr_list;
 nonterminal String alias_clause;
-nonterminal ArrayList<String> ident_list;
+nonterminal ArrayList<String> ident_list, primary_keys, opt_primary_keys;
 nonterminal ArrayList<String> opt_ident_list;
 nonterminal TableName table_name;
 nonterminal FunctionName function_name;
@@ -383,8 +384,10 @@ nonterminal DropTableOrViewStmt drop_tbl_or_view_stmt;
 nonterminal CreateDbStmt create_db_stmt;
 nonterminal CreateTableAsSelectStmt create_tbl_as_select_stmt;
 nonterminal CreateTableLikeStmt create_tbl_like_stmt;
-nonterminal CreateTableLikeFileStmt create_tbl_like_file_stmt;
-nonterminal CreateTableStmt create_unpartitioned_tbl_stmt, create_partitioned_tbl_stmt;
+nonterminal CreateTableStmt create_tbl_stmt;
+nonterminal TableDef tbl_def_without_col_defs, tbl_def_with_col_defs;
+nonterminal TableDataLayout opt_tbl_data_layout, distributed_data_layout;
+nonterminal TableDef.Options tbl_options;
 nonterminal CreateViewStmt create_view_stmt;
 nonterminal CreateDataSrcStmt create_data_src_stmt;
 nonterminal DropDataSrcStmt drop_data_src_stmt;
@@ -393,14 +396,13 @@ nonterminal StructField struct_field_def;
 nonterminal String ident_or_keyword;
 nonterminal DistributeParam distribute_hash_param;
 nonterminal ArrayList<DistributeParam> distribute_hash_param_list;
-nonterminal ArrayList<DistributeParam> opt_distribute_param_list;
 nonterminal ArrayList<DistributeParam> distribute_param_list;
 nonterminal DistributeParam distribute_range_param;
-nonterminal ArrayList<ArrayList<LiteralExpr>> split_row_list;
-nonterminal ArrayList<LiteralExpr> literal_list;
+nonterminal List<List<LiteralExpr>> split_row_list;
+nonterminal List<LiteralExpr> literal_list;
 nonterminal ColumnDef column_def, view_column_def;
-nonterminal ArrayList<ColumnDef> column_def_list, view_column_def_list;
-nonterminal ArrayList<ColumnDef> partition_column_defs, view_column_defs;
+nonterminal ArrayList<ColumnDef> column_def_list, partition_column_defs,
+  view_column_def_list, view_column_defs;
 nonterminal ArrayList<StructField> struct_field_def_list;
 // Options for DDL commands - CREATE/DROP/ALTER
 nonterminal HdfsCachingOp cache_op_val;
@@ -413,6 +415,7 @@ nonterminal THdfsFileFormat file_format_val;
 nonterminal THdfsFileFormat file_format_create_table_val;
 nonterminal Boolean if_exists_val;
 nonterminal Boolean if_not_exists_val;
+nonterminal Boolean is_primary_key_val;
 nonterminal Boolean replace_existing_cols_val;
 nonterminal HdfsUri location_val;
 nonterminal RowFormat row_format_val;
@@ -451,11 +454,12 @@ nonterminal Boolean opt_kw_role;
 // To avoid creating common keywords such as 'SERVER' or 'SOURCES' we treat them as
 // identifiers rather than keywords. Throws a parse exception if the identifier does not
 // match the expected string.
+nonterminal key_ident;
+nonterminal Boolean option_ident;
+nonterminal Boolean server_ident;
 nonterminal Boolean source_ident;
 nonterminal Boolean sources_ident;
-nonterminal Boolean server_ident;
 nonterminal Boolean uri_ident;
-nonterminal Boolean option_ident;
 
 // For Create/Drop/Show function ddl
 nonterminal FunctionArgs function_def_args;
@@ -550,11 +554,7 @@ stmt ::=
   {: RESULT = create_tbl_as_select; :}
   | create_tbl_like_stmt:create_tbl_like
   {: RESULT = create_tbl_like; :}
-  | create_tbl_like_file_stmt:create_tbl_like_file
-  {: RESULT = create_tbl_like_file; :}
-  | create_unpartitioned_tbl_stmt:create_tbl
-  {: RESULT = create_tbl; :}
-  | create_partitioned_tbl_stmt:create_tbl
+  | create_tbl_stmt:create_tbl
   {: RESULT = create_tbl; :}
   | create_view_stmt:create_view
   {: RESULT = create_view; :}
@@ -940,134 +940,177 @@ create_db_stmt ::=
   {: RESULT = new CreateDbStmt(db_name, comment, location, if_not_exists); :}
   ;
 
-create_tbl_like_stmt ::=
-  KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table KW_LIKE table_name:other_table comment_val:comment
-  KW_STORED KW_AS file_format_val:file_format location_val:location
+create_tbl_as_select_stmt ::=
+  tbl_def_without_col_defs:tbl_def
+  tbl_options:options
+  KW_AS query_stmt:select_stmt
+  {:
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def), select_stmt, null);
+  :}
+  | tbl_def_without_col_defs:tbl_def
+    // An optional clause cannot be used directly below because it would conflict with
+    // the first rule in "create_tbl_stmt".
+    primary_keys:primary_keys
+    distributed_data_layout:distribute_params
+    tbl_options:options
+    KW_AS query_stmt:select_stmt
+  {:
+    tbl_def.getPrimaryKeyColumnNames().addAll(primary_keys);
+    tbl_def.getDistributeParams().addAll(distribute_params.getDistributeParams());
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def), select_stmt, null);
+  :}
+  | tbl_def_without_col_defs:tbl_def
+    KW_PARTITIONED KW_BY LPAREN ident_list:partition_cols RPAREN
+    tbl_options:options
+    KW_AS query_stmt:select_stmt
+  {:
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def),
+        select_stmt, partition_cols);
+  :}
+  ;
+
+create_tbl_stmt ::=
+  tbl_def_without_col_defs:tbl_def
+  tbl_options:options
+  {:
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableStmt(tbl_def);
+  :}
+  | tbl_def_without_col_defs:tbl_def
+    // If "opt_tbl_data_layout" were used instead so that this rule could be combined with
+    // the rule above, there would be a conflict with the first rule in
+    // "create_tbl_as_select_stmt".
+    partition_column_defs:partition_column_defs
+    tbl_options:options
+  {:
+    tbl_def.setOptions(options);
+    CreateTableStmt create_tbl_stmt = new CreateTableStmt(tbl_def);
+    create_tbl_stmt.getPartitionColumnDefs().addAll(partition_column_defs);
+    RESULT = create_tbl_stmt;
+  :}
+  | tbl_def_with_col_defs:tbl_def
+    opt_tbl_data_layout:data_layout
+    tbl_options:options
+  {:
+    tbl_def.getPartitionColumnDefs().addAll(data_layout.getPartitionColumnDefs());
+    tbl_def.getDistributeParams().addAll(data_layout.getDistributeParams());
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableStmt(tbl_def);
+  :}
+  | tbl_def_with_col_defs:tbl_def
+    KW_PRODUCED KW_BY KW_DATA source_ident:is_source_id IDENT:data_src_name
+    opt_init_string_val:init_string
+    comment_val:comment
   {:
-    RESULT = new CreateTableLikeStmt(table, other_table, external, comment,
-        file_format, location, if_not_exists);
+    // Need external_val in the grammar to avoid shift/reduce conflict with other
+    // CREATE TABLE statements.
+    if (tbl_def.isExternal()) {
+      parser.parseError("external", SqlParserSymbols.KW_EXTERNAL);
+    }
+    tbl_def.setOptions(new TableDef.Options(comment));
+    RESULT = new CreateTableDataSrcStmt(new CreateTableStmt(tbl_def),
+        data_src_name, init_string);
   :}
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-    table_name:table KW_LIKE table_name:other_table comment_val:comment
-    location_val:location
+  | tbl_def_without_col_defs:tbl_def
+    KW_LIKE file_format_val:schema_file_format
+    STRING_LITERAL:schema_location
+    opt_tbl_data_layout:data_layout
+    tbl_options:options
   {:
-    RESULT = new CreateTableLikeStmt(table, other_table, external, comment,
-        null, location, if_not_exists);
+    tbl_def.getPartitionColumnDefs().addAll(data_layout.getPartitionColumnDefs());
+    tbl_def.getDistributeParams().addAll(data_layout.getDistributeParams());
+    tbl_def.setOptions(options);
+    RESULT = new CreateTableLikeFileStmt(new CreateTableStmt(tbl_def),
+        schema_file_format, new HdfsUri(schema_location));
   :}
   ;
 
-create_tbl_like_file_stmt ::=
-  KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table KW_LIKE file_format_val:schema_file_format
-  STRING_LITERAL:schema_location partition_column_defs:partition_col_defs
-  comment_val:comment row_format_val:row_format serde_properties:serde_props
-  file_format_create_table_val:file_format location_val:location cache_op_val:cache_op
-  tbl_properties:tbl_props
+// The form of CREATE TABLE below should logically be grouped with the forms above but
+// 'create_tbl_stmt' must return a CreateTableStmt instance and CreateTableLikeFileStmt
+// class doesn't inherit from CreateTableStmt.
+// TODO: Refactor the CREATE TABLE statements to improve the grammar and the way we
+// handle table options.
+create_tbl_like_stmt ::=
+  tbl_def_without_col_defs:tbl_def
+  KW_LIKE table_name:other_table
+  comment_val:comment
+  file_format_create_table_val:file_format location_val:location
   {:
-    RESULT = new CreateTableLikeFileStmt(table, schema_file_format,
-        new HdfsUri(schema_location), partition_col_defs, external, comment, row_format,
-        file_format, location, cache_op, if_not_exists, tbl_props, serde_props);
+    RESULT = new CreateTableLikeStmt(tbl_def.getTblName(), other_table,
+        tbl_def.isExternal(), comment, file_format, location, tbl_def.getIfNotExists());
   :}
   ;
 
-create_tbl_as_select_stmt ::=
-  KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table comment_val:comment row_format_val:row_format
-  serde_properties:serde_props file_format_create_table_val:file_format
-  location_val:location cache_op_val:cache_op distribute_param_list:distribute
-  tbl_properties:tbl_props
-  KW_AS query_stmt:query
-  {:
-    // Initialize with empty List of columns and partition columns. The
-    // columns will be added from the query statement during analysis
-    CreateTableStmt create_stmt = new CreateTableStmt(table, new ArrayList<ColumnDef>(),
-        new ArrayList<ColumnDef>(), external, comment, row_format, file_format, location,
-        cache_op, if_not_exists, tbl_props, serde_props, distribute);
-    RESULT = new CreateTableAsSelectStmt(create_stmt, query, null);
-  :}
-  // Create partitioned tables with CTAS statement. We need a separate production
-  // here, combining both into one causes an unresolvable reduce/reduce
-  // conflicts due to the optional clauses.
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table KW_PARTITIONED KW_BY LPAREN ident_list:partition_cols RPAREN
-  comment_val:comment row_format_val:row_format
-  serde_properties:serde_props file_format_create_table_val:file_format
-  location_val:location cache_op_val:cache_op tbl_properties:tbl_props
-  KW_AS query_stmt:query
-  {:
-    // Initialize with empty list of columns. The columns will be added by the query
-    // statement during analysis.
-    CreateTableStmt create_stmt = new CreateTableStmt(table,
-        new ArrayList<ColumnDef>(), new ArrayList<ColumnDef>(), external,
-        comment, row_format, file_format, location, cache_op, if_not_exists, tbl_props,
-        serde_props, null);
-    RESULT = new CreateTableAsSelectStmt(create_stmt, query, partition_cols);
-  :}
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-    table_name:table comment_val:comment row_format_val:row_format
-    serde_properties:serde_props file_format_create_table_val:file_format
-    location_val:location cache_op_val:cache_op tbl_properties:tbl_props
-    KW_AS query_stmt:query
-  {:
-    // Initialize with empty list of columns and partition columns. The
-    // columns will be added from the query statement during analysis
-    CreateTableStmt create_stmt = new CreateTableStmt(table, new ArrayList<ColumnDef>(),
-      new ArrayList<ColumnDef>(), external, comment, row_format, file_format, location,
-      cache_op, if_not_exists, tbl_props, serde_props,null);
-    RESULT = new CreateTableAsSelectStmt(create_stmt, query, null);
-  :}
-  ;
-
-// Create unpartitioned tables with and without column definitions.
-// We cannot coalesce this production with create_partitioned_tbl_stmt because
-// that results in an unresolvable reduce/reduce conflict due to the many
-// optional clauses (not clear which rule to reduce on 'empty').
-// TODO: Clean up by consolidating everything after the column defs and
-// partition clause into a CreateTableParams.
-create_unpartitioned_tbl_stmt ::=
+// Used for creating tables where the schema is inferred externally, e.g., from an Avro
+// schema, Kudu table or query statement.
+tbl_def_without_col_defs ::=
   KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table LPAREN column_def_list:col_defs RPAREN comment_val:comment
-  row_format_val:row_format serde_properties:serde_props
-  file_format_create_table_val:file_format location_val:location cache_op_val:cache_op
-  opt_distribute_param_list:distribute
-  tbl_properties:tbl_props
+  table_name:table
+  {: RESULT = new TableDef(table, external, if_not_exists); :}
+  ;
+
+tbl_def_with_col_defs ::=
+  tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list RPAREN
   {:
-    RESULT = new CreateTableStmt(table, col_defs, new ArrayList<ColumnDef>(), external,
-        comment, row_format, file_format, location, cache_op, if_not_exists, tbl_props,
-        serde_props, distribute);
+    tbl_def.getColumnDefs().addAll(list);
+    RESULT = tbl_def;
   :}
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-    table_name:table comment_val:comment row_format_val:row_format
-    serde_properties:serde_props file_format_create_table_val:file_format
-    location_val:location cache_op_val:cache_op tbl_properties:tbl_props
+  | tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA opt_primary_keys:primary_keys RPAREN
   {:
-    RESULT = new CreateTableStmt(table, new ArrayList<ColumnDef>(),
-        new ArrayList<ColumnDef>(), external, comment, row_format, file_format,
-        location, cache_op, if_not_exists, tbl_props, serde_props, null);
+    tbl_def.getColumnDefs().addAll(list);
+    tbl_def.getPrimaryKeyColumnNames().addAll(primary_keys);
+    RESULT = tbl_def;
   :}
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-    table_name:table LPAREN column_def_list:col_defs RPAREN
-    KW_PRODUCED KW_BY KW_DATA source_ident:is_source_id IDENT:data_src_name
-    opt_init_string_val:init_string comment_val:comment
+  ;
+
+opt_primary_keys ::=
+  primary_keys:col_names
+  {: RESULT = col_names; :}
+  | /* empty */
+  {: RESULT = null; :}
+  ;
+
+primary_keys ::=
+  KW_PRIMARY key_ident LPAREN ident_list:col_names RPAREN
+  {: RESULT = col_names; :}
+  ;
+
+tbl_options ::=
+  comment_val:comment row_format_val:row_format serde_properties:serde_props
+  file_format_create_table_val:file_format location_val:location cache_op_val:cache_op
+  tbl_properties:tbl_props
   {:
-    // Need external_val in the grammar to avoid shift/reduce conflict with other
-    // CREATE TABLE statements.
-    if (external) parser.parseError("external", SqlParserSymbols.KW_EXTERNAL);
-    RESULT = new CreateTableDataSrcStmt(table, col_defs, data_src_name, init_string,
-        comment, if_not_exists);
+    CreateTableStmt.unescapeProperties(serde_props);
+    CreateTableStmt.unescapeProperties(tbl_props);
+    RESULT = new TableDef.Options(comment, row_format, serde_props, file_format,
+        location, cache_op, tbl_props);
   :}
   ;
 
-// The DISTRIBUTE clause contains any number of HASH() clauses followed by exactly zero
-// or one RANGE clause
-opt_distribute_param_list ::=
-  distribute_param_list:list
-  {: RESULT = list; :}
+opt_tbl_data_layout ::=
+  partition_column_defs:partition_column_defs
+  {: RESULT = TableDataLayout.createPartitionedLayout(partition_column_defs); :}
+  | distributed_data_layout:data_layout
+  {: RESULT = data_layout; :}
+  ;
+
+distributed_data_layout ::=
+  distribute_param_list:distribute_params
+  {: RESULT = TableDataLayout.createDistributedLayout(distribute_params); :}
   | /* empty */
-  {: RESULT = null; :}
+  {: RESULT = TableDataLayout.createEmptyLayout(); :}
   ;
 
+partition_column_defs ::=
+  KW_PARTITIONED KW_BY LPAREN column_def_list:col_defs RPAREN
+  {: RESULT = col_defs; :}
+  ;
+
+// The DISTRIBUTE clause contains any number of HASH() clauses followed by exactly zero
+// or one RANGE clauses
 distribute_param_list ::=
   KW_DISTRIBUTE KW_BY distribute_hash_param_list:list
   {: RESULT = list; :}
@@ -1095,9 +1138,12 @@ distribute_hash_param_list ::=
 distribute_hash_param ::=
   KW_HASH LPAREN ident_list:cols RPAREN KW_INTO
     INTEGER_LITERAL:buckets KW_BUCKETS
-  {: RESULT = DistributeParam.createHashParam(cols, buckets); :}
+  {: RESULT = DistributeParam.createHashParam(cols, buckets.intValue()); :}
   | KW_HASH KW_INTO INTEGER_LITERAL:buckets KW_BUCKETS
-  {: RESULT = DistributeParam.createHashParam(null, buckets); :}
+  {:
+    RESULT = DistributeParam.createHashParam(Lists.<String>newArrayList(),
+        buckets.intValue());
+  :}
   ;
 
 // The column list for a RANGE clause is optional.
@@ -1106,12 +1152,12 @@ distribute_range_param ::=
   LPAREN split_row_list:list RPAREN
   {: RESULT = DistributeParam.createRangeParam(cols, list); :}
   | KW_RANGE KW_SPLIT KW_ROWS LPAREN split_row_list:list RPAREN
-  {: RESULT = DistributeParam.createRangeParam(null, list); :}
+  {: RESULT = DistributeParam.createRangeParam(Lists.<String>newArrayList(), list); :}
   ;
 
 split_row_list ::=
   LPAREN literal_list:l RPAREN
-  {: RESULT = Lists.<ArrayList<LiteralExpr>>newArrayList(l); :}
+  {: RESULT = Lists.<List<LiteralExpr>>newArrayList(l); :}
   | split_row_list:list COMMA LPAREN literal_list:l RPAREN
   {:
     list.add(l);
@@ -1129,34 +1175,6 @@ literal_list ::=
   :}
   ;
 
-// Create partitioned tables with and without column definitions.
-// TODO: Clean up by consolidating everything after the column defs and
-// partition clause into a CreateTableParams.
-create_partitioned_tbl_stmt ::=
-  KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table LPAREN column_def_list:col_defs RPAREN KW_PARTITIONED KW_BY
-  LPAREN column_def_list:partition_col_defs RPAREN comment_val:comment
-  row_format_val:row_format serde_properties:serde_props
-  file_format_create_table_val:file_format location_val:location cache_op_val:cache_op
-  tbl_properties:tbl_props
-  {:
-    RESULT = new CreateTableStmt(table, col_defs, partition_col_defs, external, comment,
-        row_format, file_format, location, cache_op, if_not_exists, tbl_props,
-        serde_props, null);
-  :}
-  | KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-    table_name:table KW_PARTITIONED KW_BY
-    LPAREN column_def_list:partition_col_defs RPAREN
-    comment_val:comment row_format_val:row_format serde_properties:serde_props
-    file_format_create_table_val:file_format location_val:location cache_op_val:cache_op
-    tbl_properties:tbl_props
-  {:
-    RESULT = new CreateTableStmt(table, new ArrayList<ColumnDef>(), partition_col_defs,
-        external, comment, row_format, file_format, location, cache_op, if_not_exists,
-        tbl_props, serde_props, null);
-  :}
-  ;
-
 create_udf_stmt ::=
   KW_CREATE KW_FUNCTION if_not_exists_val:if_not_exists
   function_name:fn_name function_def_args:fn_args
@@ -1252,7 +1270,7 @@ row_format_val ::=
   escaped_by_val:escaped_by line_terminator_val:line_terminator
   {: RESULT = new RowFormat(field_terminator, line_terminator, escaped_by); :}
   |/* empty */
-  {: RESULT = RowFormat.DEFAULT_ROW_FORMAT; :}
+  {: RESULT = null; :}
   ;
 
 escaped_by_val ::=
@@ -1284,12 +1302,14 @@ terminator_val ::=
 file_format_create_table_val ::=
   KW_STORED KW_AS file_format_val:file_format
   {: RESULT = file_format; :}
-  | /* empty - default to TEXT */
-  {: RESULT = THdfsFileFormat.TEXT; :}
+  |
+  {: RESULT = null; :}
   ;
 
 file_format_val ::=
-  KW_PARQUET
+  KW_KUDU
+  {: RESULT = THdfsFileFormat.KUDU; :}
+  | KW_PARQUET
   {: RESULT = THdfsFileFormat.PARQUET; :}
   | KW_PARQUETFILE
   {: RESULT = THdfsFileFormat.PARQUET; :}
@@ -1307,14 +1327,14 @@ tbl_properties ::=
   KW_TBLPROPERTIES LPAREN properties_map:map RPAREN
   {: RESULT = map; :}
   | /* empty */
-  {: RESULT = null; :}
+  {: RESULT = new HashMap<String, String>(); :}
   ;
 
 serde_properties ::=
   KW_WITH KW_SERDEPROPERTIES LPAREN properties_map:map RPAREN
   {: RESULT = map; :}
   | /* empty */
-  {: RESULT = null; :}
+  {: RESULT = new HashMap<String, String>(); :}
   ;
 
 properties_map ::=
@@ -1331,17 +1351,10 @@ properties_map ::=
   :}
   ;
 
-partition_column_defs ::=
-  KW_PARTITIONED KW_BY LPAREN column_def_list:col_defs RPAREN
-  {: RESULT = col_defs; :}
-  | /* Empty - not a partitioned table */
-  {: RESULT = new ArrayList<ColumnDef>(); :}
-  ;
-
 column_def_list ::=
   column_def:col_def
   {:
-    ArrayList<ColumnDef> list = new ArrayList<ColumnDef>();
+    ArrayList<ColumnDef> list = Lists.newArrayList();
     list.add(col_def);
     RESULT = list;
   :}
@@ -1353,8 +1366,15 @@ column_def_list ::=
   ;
 
 column_def ::=
-  IDENT:col_name type_def:type comment_val:comment
-  {: RESULT = new ColumnDef(col_name, type, comment); :}
+  IDENT:col_name type_def:type is_primary_key_val:primary_key comment_val:comment
+  {: RESULT = new ColumnDef(col_name, type, primary_key, comment); :}
+  ;
+
+is_primary_key_val ::=
+  KW_PRIMARY key_ident
+  {: RESULT = true; :}
+  | /* empty */
+  {: RESULT = false; :}
   ;
 
 create_view_stmt ::=
@@ -1377,6 +1397,15 @@ create_data_src_stmt ::=
   :}
   ;
 
+key_ident ::=
+  IDENT:ident
+  {:
+    if (!ident.toUpperCase().equals("KEY")) {
+      parser.parseError("identifier", SqlParserSymbols.IDENT, "KEY");
+    }
+  :}
+  ;
+
 source_ident ::=
   IDENT:ident
   {:
@@ -2996,6 +3025,8 @@ ident_or_keyword ::=
   {: RESULT = r.toString(); :}
   | KW_JOIN:r
   {: RESULT = r.toString(); :}
+  | KW_KUDU:r
+  {: RESULT = r.toString(); :}
   | KW_LAST:r
   {: RESULT = r.toString(); :}
   | KW_LEFT:r
@@ -3050,6 +3081,8 @@ ident_or_keyword ::=
   {: RESULT = r.toString(); :}
   | KW_PREPARE_FN:r
   {: RESULT = r.toString(); :}
+  | KW_PRIMARY:r
+  {: RESULT = r.toString(); :}
   | KW_PRODUCED:r
   {: RESULT = r.toString(); :}
   | KW_PURGE:r

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
new file mode 100644
index 0000000..2a3294e
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
@@ -0,0 +1,43 @@
+// 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 org.apache.impala.common.AnalysisException;
+
+import java.util.Collection;
+
+class AnalysisUtils {
+
+  static <T> void throwIfNotNull(T o, String message) throws AnalysisException {
+    if (o != null) throw new AnalysisException(message);
+  }
+
+  static void throwIfNotEmpty(Collection<?> c, String message)
+      throws AnalysisException {
+    if (c != null && !c.isEmpty()) throw new AnalysisException(message);
+  }
+
+  static <T> void throwIfNull(T o, String message) throws AnalysisException {
+    if (o == null) throw new AnalysisException(message);
+  }
+
+  static void throwIfNullOrEmpty(Collection<?> c, String message)
+      throws AnalysisException {
+    if (c == null || c.isEmpty()) throw new AnalysisException(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java b/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
index 6b2a1d2..1b634f7 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
@@ -17,8 +17,15 @@
 
 package org.apache.impala.analysis;
 
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
@@ -26,9 +33,6 @@ import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TColumn;
 import org.apache.impala.util.MetaStoreUtil;
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 /**
  * Represents a column definition in a CREATE/ALTER TABLE/VIEW statement.
@@ -47,9 +51,19 @@ public class ColumnDef {
   private final TypeDef typeDef_;
   private Type type_;
 
+  // Set to true if the user specified "PRIMARY KEY" in the column definition. Kudu table
+  // definitions may use this.
+  private boolean isPrimaryKey_;
+
   public ColumnDef(String colName, TypeDef typeDef, String comment) {
+    this(colName, typeDef, false, comment);
+  }
+
+  public ColumnDef(String colName, TypeDef typeDef, boolean isPrimaryKey,
+      String comment) {
     colName_ = colName.toLowerCase();
     typeDef_ = typeDef;
+    isPrimaryKey_ = isPrimaryKey;
     comment_ = comment;
   }
 
@@ -67,13 +81,15 @@ public class ColumnDef {
     colName_ = fs.getName();
     typeDef_ = new TypeDef(type);
     comment_ = fs.getComment();
+    isPrimaryKey_ = false;
     analyze();
   }
 
+  public String getColName() { return colName_; }
   public void setType(Type type) { type_ = type; }
   public Type getType() { return type_; }
   public TypeDef getTypeDef() { return typeDef_; }
-  public String getColName() { return colName_; }
+  boolean isPrimaryKey() { return isPrimaryKey_; }
   public void setComment(String comment) { comment_ = comment; }
   public String getComment() { return comment_; }
 
@@ -107,16 +123,32 @@ public class ColumnDef {
 
   @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder(colName_);
+    StringBuilder sb = new StringBuilder(colName_).append(" ");
     if (type_ != null) {
-      sb.append(" " + type_.toString());
+      sb.append(type_);
     } else {
-      sb.append(" " + typeDef_.toString());
+      sb.append(typeDef_);
     }
+    if (isPrimaryKey_) sb.append(" PRIMARY KEY");
     if (comment_ != null) sb.append(String.format(" COMMENT '%s'", comment_));
     return sb.toString();
   }
 
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) return false;
+    if (obj == this) return true;
+    if (obj.getClass() != getClass()) return false;
+    ColumnDef rhs = (ColumnDef) obj;
+    return new EqualsBuilder()
+        .append(colName_, rhs.colName_)
+        .append(comment_, rhs.comment_)
+        .append(isPrimaryKey_, rhs.isPrimaryKey_)
+        .append(typeDef_, rhs.typeDef_)
+        .append(type_, rhs.type_)
+        .isEquals();
+  }
+
   public TColumn toThrift() {
     TColumn col = new TColumn(new TColumn(getColName(), type_.toThrift()));
     col.setComment(getComment());
@@ -140,4 +172,24 @@ public class ColumnDef {
     });
   }
 
+  static List<String> toColumnNames(Collection<ColumnDef> colDefs) {
+    List<String> colNames = Lists.newArrayList();
+    for (ColumnDef colDef: colDefs) {
+      colNames.add(colDef.getColName());
+    }
+    return colNames;
+  }
+
+  /**
+   * Generates and returns a map of column names to column definitions. Assumes that
+   * the column names are unique.
+   */
+  static Map<String, ColumnDef> mapByColumnNames(Collection<ColumnDef> colDefs) {
+    Map<String, ColumnDef> colDefsByColName = Maps.newHashMap();
+    for (ColumnDef colDef: colDefs) {
+      ColumnDef def = colDefsByColName.put(colDef.getColName(), colDef);
+      Preconditions.checkState(def == null);
+    }
+    return colDefsByColName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
index b2a95c4..816af80 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
@@ -27,7 +27,6 @@ import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.Table;
 import org.apache.impala.catalog.TableId;
-import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.service.CatalogOpExecutor;
 import org.apache.impala.thrift.THdfsFileFormat;
@@ -62,7 +61,7 @@ public class CreateTableAsSelectStmt extends StatementBase {
   /////////////////////////////////////////
 
   private final static EnumSet<THdfsFileFormat> SUPPORTED_INSERT_FORMATS =
-      EnumSet.of(THdfsFileFormat.PARQUET, THdfsFileFormat.TEXT);
+      EnumSet.of(THdfsFileFormat.PARQUET, THdfsFileFormat.TEXT, THdfsFileFormat.KUDU);
 
   /**
    * Builds a CREATE TABLE AS SELECT statement
@@ -95,6 +94,18 @@ public class CreateTableAsSelectStmt extends StatementBase {
     if (isAnalyzed()) return;
     super.analyze(analyzer);
 
+    if (!SUPPORTED_INSERT_FORMATS.contains(createStmt_.getFileFormat())) {
+      throw new AnalysisException(String.format("CREATE TABLE AS SELECT " +
+          "does not support the (%s) file format. Supported formats are: (%s)",
+          createStmt_.getFileFormat().toString().replace("_", ""),
+          "PARQUET, TEXTFILE, KUDU"));
+    }
+    if (createStmt_.getFileFormat() == THdfsFileFormat.KUDU && createStmt_.isExternal()) {
+      // TODO: Add support for CTAS on external Kudu tables (see IMPALA-4318)
+      throw new AnalysisException(String.format("CREATE TABLE AS SELECT is not " +
+          "supported for external Kudu tables."));
+    }
+
     // The analysis for CTAS happens in two phases - the first phase happens before
     // the target table exists and we want to validate the CREATE statement and the
     // query portion of the insert statement. If this passes, analysis will be run
@@ -154,12 +165,6 @@ public class CreateTableAsSelectStmt extends StatementBase {
     }
     createStmt_.analyze(analyzer);
 
-    if (!SUPPORTED_INSERT_FORMATS.contains(createStmt_.getFileFormat())) {
-      throw new AnalysisException(String.format("CREATE TABLE AS SELECT " +
-          "does not support (%s) file format. Supported formats are: (%s)",
-          createStmt_.getFileFormat().toString().replace("_", ""),
-          "PARQUET, TEXTFILE"));
-    }
 
     // The full privilege check for the database will be done as part of the INSERT
     // analysis.
@@ -188,14 +193,20 @@ public class CreateTableAsSelectStmt extends StatementBase {
       // SelectStmt (or the BE will be very confused). To ensure the ID is unique within
       // this query, just assign it the invalid table ID. The CatalogServer will assign
       // this table a proper ID once it is created there as part of the CTAS execution.
-      Table table = Table.fromMetastoreTable(TableId.createInvalidId(), db, msTbl);
-      Preconditions.checkState(table != null &&
-          (table instanceof HdfsTable || table instanceof KuduTable));
+      Table tmpTable = null;
+      if (KuduTable.isKuduTable(msTbl)) {
+        tmpTable = KuduTable.createCtasTarget(db, msTbl, createStmt_.getColumnDefs(),
+            createStmt_.getTblPrimaryKeyColumnNames(), createStmt_.getDistributeParams());
+      } else {
+        // TODO: Creating a tmp table using load() is confusing.
+        // Refactor it to use a 'createCtasTarget()' function similar to Kudu table.
+        tmpTable = Table.fromMetastoreTable(TableId.createInvalidId(), db, msTbl);
+        tmpTable.load(true, client.getHiveClient(), msTbl);
+      }
+      Preconditions.checkState(tmpTable != null &&
+          (tmpTable instanceof HdfsTable || tmpTable instanceof KuduTable));
 
-      table.load(true, client.getHiveClient(), msTbl);
-      insertStmt_.setTargetTable(table);
-    } catch (TableLoadingException e) {
-      throw new AnalysisException(e.getMessage(), e);
+      insertStmt_.setTargetTable(tmpTable);
     } catch (Exception e) {
       throw new AnalysisException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
index 3c54dfd..1df8280 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
@@ -23,19 +23,12 @@ import static org.apache.impala.catalog.DataSourceTable.TBL_PROP_DATA_SRC_NAME;
 import static org.apache.impala.catalog.DataSourceTable.TBL_PROP_INIT_STRING;
 import static org.apache.impala.catalog.DataSourceTable.TBL_PROP_LOCATION;
 
-import java.util.List;
-import java.util.Map;
-
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.catalog.DataSource;
 import org.apache.impala.catalog.DataSourceTable;
-import org.apache.impala.catalog.RowFormat;
 import org.apache.impala.common.AnalysisException;
-import org.apache.impala.thrift.THdfsFileFormat;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.permission.FsAction;
 
 /**
@@ -46,25 +39,12 @@ import org.apache.hadoop.fs.permission.FsAction;
  */
 public class CreateTableDataSrcStmt extends CreateTableStmt {
 
-  public CreateTableDataSrcStmt(TableName tableName, List<ColumnDef> columnDefs,
-      String dataSourceName, String initString, String comment, boolean ifNotExists) {
-    super(tableName, columnDefs, Lists.<ColumnDef>newArrayList(), false, comment,
-        RowFormat.DEFAULT_ROW_FORMAT, THdfsFileFormat.TEXT, null, null, ifNotExists,
-        createInitialTableProperties(dataSourceName, initString),
-        Maps.<String, String>newHashMap(), null);
-  }
-
-  /**
-   * Creates the initial map of table properties containing the name of the data
-   * source and the table init string.
-   */
-  private static Map<String, String> createInitialTableProperties(
-      String dataSourceName, String initString) {
+  public CreateTableDataSrcStmt(CreateTableStmt createTableStmt, String dataSourceName,
+      String initString) {
+    super(createTableStmt);
     Preconditions.checkNotNull(dataSourceName);
-    Map<String, String> tableProperties = Maps.newHashMap();
-    tableProperties.put(TBL_PROP_DATA_SRC_NAME, dataSourceName.toLowerCase());
-    tableProperties.put(TBL_PROP_INIT_STRING, Strings.nullToEmpty(initString));
-    return tableProperties;
+    getTblProperties().put(TBL_PROP_DATA_SRC_NAME, dataSourceName.toLowerCase());
+    getTblProperties().put(TBL_PROP_INIT_STRING, Strings.nullToEmpty(initString));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
index a9a8a90..a653323 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
@@ -21,12 +21,12 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
-
 import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.metadata.ParquetMetadata;
 import parquet.schema.OriginalType;
@@ -36,8 +36,8 @@ import org.apache.impala.authorization.Privilege;
 import org.apache.impala.catalog.ArrayType;
 import org.apache.impala.catalog.HdfsCompression;
 import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.MapType;
-import org.apache.impala.catalog.RowFormat;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.StructField;
 import org.apache.impala.catalog.StructType;
@@ -45,8 +45,6 @@ import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.thrift.THdfsFileFormat;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 
 /**
@@ -60,16 +58,9 @@ public class CreateTableLikeFileStmt extends CreateTableStmt {
   private final static String ERROR_MSG =
       "Failed to convert Parquet type\n%s\nto an Impala %s type:\n%s\n";
 
-  public CreateTableLikeFileStmt(TableName tableName, THdfsFileFormat schemaFileFormat,
-      HdfsUri schemaLocation, List<ColumnDef> partitionColumnDescs,
-      boolean isExternal, String comment, RowFormat rowFormat,
-      THdfsFileFormat fileFormat, HdfsUri location, HdfsCachingOp cachingOp,
-      boolean ifNotExists, Map<String, String> tblProperties,
-      Map<String, String> serdeProperties) {
-    super(tableName, new ArrayList<ColumnDef>(), partitionColumnDescs,
-        isExternal, comment, rowFormat,
-        fileFormat, location, cachingOp, ifNotExists, tblProperties, serdeProperties,
-        null);
+  public CreateTableLikeFileStmt(CreateTableStmt createTableStmt,
+      THdfsFileFormat schemaFileFormat, HdfsUri schemaLocation) {
+    super(createTableStmt);
     schemaLocation_ = schemaLocation;
     schemaFileFormat_ = schemaFileFormat;
   }
@@ -351,8 +342,8 @@ public class CreateTableLikeFileStmt extends CreateTableStmt {
         schemaLocation_.toString());
     String s = ToSqlUtils.getCreateTableSql(getDb(),
         getTbl() + " __LIKE_FILEFORMAT__ ",  getComment(), colsSql, partitionColsSql,
-        getTblProperties(), getSerdeProperties(), isExternal(), getIfNotExists(),
-        getRowFormat(), HdfsFileFormat.fromThrift(getFileFormat()),
+        null, null, getTblProperties(), getSerdeProperties(), isExternal(),
+        getIfNotExists(), getRowFormat(), HdfsFileFormat.fromThrift(getFileFormat()),
         compression, null, getLocation());
     s = s.replace("__LIKE_FILEFORMAT__", "LIKE " + schemaFileFormat_ + " " +
         schemaLocation_.toString());
@@ -361,6 +352,10 @@ public class CreateTableLikeFileStmt extends CreateTableStmt {
 
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
+    if (getFileFormat() == THdfsFileFormat.KUDU) {
+      throw new AnalysisException("CREATE TABLE LIKE FILE statement is not supported " +
+          "for Kudu tables.");
+    }
     schemaLocation_.analyze(analyzer, Privilege.ALL, FsAction.READ_WRITE);
     switch (schemaFileFormat_) {
       case PARQUET:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/041fa6d9/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
index 72843e8..6fde627 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
@@ -17,16 +17,18 @@
 
 package org.apache.impala.analysis;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.permission.FsAction;
 
 import org.apache.impala.authorization.Privilege;
+import org.apache.impala.catalog.KuduTable;
+import org.apache.impala.catalog.Table;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TAccessEvent;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TCreateTableLikeParams;
 import org.apache.impala.thrift.THdfsFileFormat;
 import org.apache.impala.thrift.TTableName;
-import com.google.common.base.Preconditions;
 
 /**
  * Represents a CREATE TABLE LIKE statement which creates a new table based on
@@ -134,10 +136,19 @@ public class CreateTableLikeStmt extends StatementBase {
   public void analyze(Analyzer analyzer) throws AnalysisException {
     Preconditions.checkState(tableName_ != null && !tableName_.isEmpty());
     Preconditions.checkState(srcTableName_ != null && !srcTableName_.isEmpty());
+    // We currently don't support creating a Kudu table using a CREATE TABLE LIKE
+    // statement (see IMPALA-4052).
+    if (fileFormat_ == THdfsFileFormat.KUDU) {
+      throw new AnalysisException("CREATE TABLE LIKE is not supported for Kudu tables");
+    }
+
     // Make sure the source table exists and the user has permission to access it.
-    srcDbName_ = analyzer
-        .getTable(srcTableName_, Privilege.VIEW_METADATA)
-        .getDb().getName();
+    Table srcTable = analyzer.getTable(srcTableName_, Privilege.VIEW_METADATA);
+    if (KuduTable.isKuduTable(srcTable.getMetaStoreTable())) {
+      throw new AnalysisException("Cloning a Kudu table using CREATE TABLE LIKE is " +
+          "not supported.");
+    }
+    srcDbName_ = srcTable.getDb().getName();
     tableName_.analyze();
     dbName_ = analyzer.getTargetDbName(tableName_);
     owner_ = analyzer.getUser().getName();


[09/14] incubator-impala git commit: IMPALA-3739: Enable stress tests on Kudu

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q8.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q8.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q8.test
new file mode 100644
index 0000000..be99b2f
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q8.test
@@ -0,0 +1,71 @@
+====
+---- QUERY: TPCDS-Q8
+select
+  s_store_name,
+  round(sum(ss_net_profit), 2)
+from
+  store_sales
+  join store on (store_sales.ss_store_sk = store.s_store_sk)
+  join
+  (select
+    a.ca_zip
+  from
+    (select
+      substr(ca_zip, 1, 5) ca_zip,
+      count( *) cnt
+    from
+      customer_address
+      join  customer on (customer_address.ca_address_sk = customer.c_current_addr_sk)
+    where
+      c_preferred_cust_flag = 'Y'
+    group by
+      ca_zip
+    having
+      count(*) > 10
+    ) a
+    left semi join
+    (select
+      substr(ca_zip, 1, 5) ca_zip
+    from
+      customer_address
+    where
+      substr(ca_zip, 1, 5) in ('89436', '30868', '65085', '22977', '83927', '77557', '58429', '40697', '80614', '10502', '32779',
+      '91137', '61265', '98294', '17921', '18427', '21203', '59362', '87291', '84093', '21505', '17184', '10866', '67898', '25797',
+      '28055', '18377', '80332', '74535', '21757', '29742', '90885', '29898', '17819', '40811', '25990', '47513', '89531', '91068',
+      '10391', '18846', '99223', '82637', '41368', '83658', '86199', '81625', '26696', '89338', '88425', '32200', '81427', '19053',
+      '77471', '36610', '99823', '43276', '41249', '48584', '83550', '82276', '18842', '78890', '14090', '38123', '40936', '34425',
+      '19850', '43286', '80072', '79188', '54191', '11395', '50497', '84861', '90733', '21068', '57666', '37119', '25004', '57835',
+      '70067', '62878', '95806', '19303', '18840', '19124', '29785', '16737', '16022', '49613', '89977', '68310', '60069', '98360',
+      '48649', '39050', '41793', '25002', '27413', '39736', '47208', '16515', '94808', '57648', '15009', '80015', '42961', '63982',
+      '21744', '71853', '81087', '67468', '34175', '64008', '20261', '11201', '51799', '48043', '45645', '61163', '48375', '36447',
+      '57042', '21218', '41100', '89951', '22745', '35851', '83326', '61125', '78298', '80752', '49858', '52940', '96976', '63792',
+      '11376', '53582', '18717', '90226', '50530', '94203', '99447', '27670', '96577', '57856', '56372', '16165', '23427', '54561',
+      '28806', '44439', '22926', '30123', '61451', '92397', '56979', '92309', '70873', '13355', '21801', '46346', '37562', '56458',
+      '28286', '47306', '99555', '69399', '26234', '47546', '49661', '88601', '35943', '39936', '25632', '24611', '44166', '56648',
+      '30379', '59785', '11110', '14329', '93815', '52226', '71381', '13842', '25612', '63294', '14664', '21077', '82626', '18799',
+      '60915', '81020', '56447', '76619', '11433', '13414', '42548', '92713', '70467', '30884', '47484', '16072', '38936', '13036',
+      '88376', '45539', '35901', '19506', '65690', '73957', '71850', '49231', '14276', '20005', '18384', '76615', '11635', '38177',
+      '55607', '41369', '95447', '58581', '58149', '91946', '33790', '76232', '75692', '95464', '22246', '51061', '56692', '53121',
+      '77209', '15482', '10688', '14868', '45907', '73520', '72666', '25734', '17959', '24677', '66446', '94627', '53535', '15560',
+      '41967', '69297', '11929', '59403', '33283', '52232', '57350', '43933', '40921', '36635', '10827', '71286', '19736', '80619',
+      '25251', '95042', '15526', '36496', '55854', '49124', '81980', '35375', '49157', '63512', '28944', '14946', '36503', '54010',
+      '18767', '23969', '43905', '66979', '33113', '21286', '58471', '59080', '13395', '79144', '70373', '67031', '38360', '26705',
+      '50906', '52406', '26066', '73146', '15884', '31897', '30045', '61068', '45550', '92454', '13376', '14354', '19770', '22928',
+      '97790', '50723', '46081', '30202', '14410', '20223', '88500', '67298', '13261', '14172', '81410', '93578', '83583', '46047',
+      '94167', '82564', '21156', '15799', '86709', '37931', '74703', '83103', '23054', '70470', '72008', '49247', '91911', '69998',
+      '20961', '70070', '63197', '54853', '88191', '91830', '49521', '19454', '81450', '89091', '62378', '25683', '61869', '51744',
+      '36580', '85778', '36871', '48121', '28810', '83712', '45486', '67393', '26935', '42393', '20132', '55349', '86057', '21309',
+      '80218', '10094', '11357', '48819', '39734', '40758', '30432', '21204', '29467', '30214', '61024', '55307', '74621', '11622',
+      '68908', '33032', '52868', '99194', '99900', '84936', '69036', '99149', '45013', '32895', '59004', '32322', '14933', '32936',
+      '33562', '72550', '27385', '58049', '58200', '16808', '21360', '32961', '18586', '79307', '15492')
+    ) b
+  on (a.ca_zip = b.ca_zip)
+  ) v1 on (substr(store.s_zip, 1, 2) = substr(v1.ca_zip, 1, 2))
+where
+  ss_sold_date_sk between 2452276 and 2452366
+group by
+  s_store_name
+order by
+  s_store_name
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q88.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q88.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q88.test
new file mode 100644
index 0000000..bb2beb8
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q88.test
@@ -0,0 +1,93 @@
+====
+---- QUERY: TPCDS-Q88
+select  *
+from
+ (select count(*) h8_30_to_9
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 8
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s1,
+ (select count(*) h9_to_9_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 9
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s2,
+ (select count(*) h9_30_to_10
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 9
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s3,
+ (select count(*) h10_to_10_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 10
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s4,
+ (select count(*) h10_30_to_11
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 10
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s5,
+ (select count(*) h11_to_11_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 11
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s6,
+ (select count(*) h11_30_to_12
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 11
+     and time_dim.t_minute >= 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s7,
+ (select count(*) h12_to_12_30
+ from store_sales, household_demographics , time_dim, store
+ where ss_sold_time_sk = time_dim.t_time_sk
+     and ss_hdemo_sk = household_demographics.hd_demo_sk
+     and ss_store_sk = s_store_sk
+     and time_dim.t_hour = 12
+     and time_dim.t_minute < 30
+     and ((household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or
+          (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or
+          (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
+     and store.s_store_name = 'ese') s8;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q89.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q89.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q89.test
new file mode 100644
index 0000000..6677b07
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q89.test
@@ -0,0 +1,33 @@
+====
+---- QUERY: TPCDS-Q89
+/* Modifications: Added Partition Key filter because Impala does not do dynamic partition
+   pruning.*/
+select * from (select i_category, i_class, i_brand, s_store_name, s_company_name
+	       d_moy, round(sum_sales, 2) sum_sales,
+	       round(avg_monthly_sales, 2) avg_monthly_sales
+from (
+select i_category, i_class, i_brand,
+       s_store_name, s_company_name,
+       d_moy,
+       sum(ss_sales_price) sum_sales,
+       avg(sum(ss_sales_price)) over
+         (partition by i_category, i_brand, s_store_name, s_company_name)
+         avg_monthly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
+      d_year in (2000) and
+        ((i_category in ('Children','Music','Home') and
+          i_class in ('toddlers','pop','lighting')
+         )
+      or (i_category in ('Jewelry','Books','Sports') and
+          i_class in ('costume','travel','football')
+        ))
+      and ss_sold_date_sk between 2451545 and 2451910  -- partition key filter
+group by i_category, i_class, i_brand,
+         s_store_name, s_company_name, d_moy) tmp1
+where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1
+order by sum_sales - avg_monthly_sales, s_store_name
+limit 100) tmp2;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q96.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q96.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q96.test
new file mode 100644
index 0000000..1687bc4
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q96.test
@@ -0,0 +1,17 @@
+====
+---- QUERY: TPCDS-Q96
+SELECT
+  COUNT(*) AS total
+FROM store_sales ss
+JOIN time_dim td
+  ON (ss.ss_sold_time_sk = td.t_time_sk)
+JOIN household_demographics hd
+  ON (ss.ss_hdemo_sk = hd.hd_demo_sk)
+JOIN store s
+  ON (ss.ss_store_sk = s.s_store_sk)
+WHERE
+  td.t_hour = 8
+  AND td.t_minute >= 30
+  AND hd.hd_dep_count = 5
+  AND s.s_store_name = 'ese';
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q98.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q98.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q98.test
new file mode 100644
index 0000000..fcd2081
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q98.test
@@ -0,0 +1,34 @@
+====
+---- QUERY: TPCDS-Q98
+select
+  i_item_desc,
+  i_category,
+  i_class,
+  i_current_price,
+  round(sum(ss_ext_sales_price), 2) as itemrevenue,
+  round(sum(ss_ext_sales_price)*100/ sum(sum(ss_ext_sales_price))
+  over (partition by i_class), 2) as revenueratio
+from
+  store_sales,
+  item,
+  date_dim
+where
+  ss_item_sk = i_item_sk
+  and i_category in ('Jewelry', 'Sports', 'Books')
+  and ss_sold_date_sk = d_date_sk
+  and ss_sold_date_sk between 2451911 and 2451941
+  and d_date between '2001-01-01' and '2001-01-31' -- original uses interval and the
+group by
+  i_item_id,
+  i_item_desc,
+  i_category,
+  i_class,
+  i_current_price
+order by
+  i_category,
+  i_class,
+  i_item_id,
+  i_item_desc,
+  revenueratio
+limit 1000;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/tests/comparison/db_connection.py
----------------------------------------------------------------------
diff --git a/tests/comparison/db_connection.py b/tests/comparison/db_connection.py
index 13125c2..9076da8 100644
--- a/tests/comparison/db_connection.py
+++ b/tests/comparison/db_connection.py
@@ -225,6 +225,11 @@ class DbCursor(object):
     db_name = db_name.lower()
     self.execute('CREATE DATABASE ' + db_name)
 
+  def create_db_if_not_exists(self, db_name):
+    LOG.info("Creating database %s", db_name)
+    db_name = db_name.lower()
+    self.execute('CREATE DATABASE IF NOT EXISTS ' + db_name)
+
   def drop_db_if_exists(self, db_name):
     '''This should not be called from a conn to the database being dropped.'''
     db_name = db_name.lower()

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/tests/stress/concurrent_select.py
----------------------------------------------------------------------
diff --git a/tests/stress/concurrent_select.py b/tests/stress/concurrent_select.py
index b76d713..fb24f65 100755
--- a/tests/stress/concurrent_select.py
+++ b/tests/stress/concurrent_select.py
@@ -897,13 +897,16 @@ class QueryRunner(object):
     return hash_thread.result
 
 
-def load_tpc_queries(workload):
-  """Returns a list of TPC queries. 'workload' should either be 'tpch' or 'tpcds'."""
+def load_tpc_queries(workload, load_in_kudu=False):
+  """Returns a list of TPC queries. 'workload' should either be 'tpch' or 'tpcds'.
+  If 'load_in_kudu' is True, it loads only queries specified for the Kudu storage
+  engine."""
   LOG.info("Loading %s queries", workload)
   queries = list()
   query_dir = os.path.join(os.path.dirname(__file__), "..", "..",
       "testdata", "workloads", workload, "queries")
-  file_name_pattern = re.compile(r"-(q\d+).test$")
+  engine = 'kudu-' if load_in_kudu else ''
+  file_name_pattern = re.compile(r"%s-%s(q\d+).test$" % (workload, engine))
   for query_file in os.listdir(query_dir):
     match = file_name_pattern.search(query_file)
     if not match:
@@ -1339,6 +1342,10 @@ def main():
   parser.add_argument("--tpch-db", help="If provided, TPC-H queries will be used.")
   parser.add_argument("--tpch-nested-db",
       help="If provided, nested TPC-H queries will be used.")
+  parser.add_argument("--tpch-kudu-db",
+      help="If provided, TPC-H queries for Kudu will be used.")
+  parser.add_argument("--tpcds-kudu-db",
+      help="If provided, TPC-DS queries for Kudu will be used.")
   parser.add_argument("--random-db",
       help="If provided, random queries will be used.")
   parser.add_argument("--random-query-count", type=int, default=50,
@@ -1375,9 +1382,10 @@ def main():
   LOG.debug("CLI args: %s" % (args, ))
 
   if not args.tpcds_db and not args.tpch_db and not args.random_db \
-      and not args.tpch_nested_db and not args.query_file_path:
-    raise Exception("At least one of --tpcds-db, --tpch-db,"
-        "--tpch-nested-db, --random-db, --query-file-path is required")
+      and not args.tpch_nested_db and not args.tpch_kudu_db \
+      and not args.tpcds_kudu_db and not args.query_file_path:
+    raise Exception("At least one of --tpcds-db, --tpch-db, --tpch-kudu-db,"
+        "--tpcds-kudu-db, --tpch-nested-db, --random-db, --query-file-path is required")
 
   # The stress test sets these, so callers cannot override them.
   IGNORE_QUERY_OPTIONS = frozenset([
@@ -1451,6 +1459,16 @@ def main():
     for query in tpch_nested_queries:
       query.db_name = args.tpch_nested_db
     queries.extend(tpch_nested_queries)
+  if args.tpch_kudu_db:
+    tpch_kudu_queries = load_tpc_queries("tpch", load_in_kudu=True)
+    for query in tpch_kudu_queries:
+      query.db_name = args.tpch_kudu_db
+    queries.extend(tpch_kudu_queries)
+  if args.tpcds_kudu_db:
+    tpcds_kudu_queries = load_tpc_queries("tpcds", load_in_kudu=True)
+    for query in tpcds_kudu_queries:
+      query.db_name = args.tpcds_kudu_db
+    queries.extend(tpcds_kudu_queries)
   for idx in xrange(len(queries) - 1, -1, -1):
     query = queries[idx]
     if query.sql in queries_with_runtime_info_by_db_and_sql[query.db_name]:


[12/14] incubator-impala git commit: IMPALA-4241: remove spurious child queries event

Posted by ta...@apache.org.
IMPALA-4241: remove spurious child queries event

"IMPALA-4037,IMPALA-4038: fix locking during query
cancellation" accidentally added the "Child queries
finished" event unconditionally. We should only do
this if there are actually child queries.

Change-Id: I3881d032622750444d750f161ad6843bdbd16c30
Reviewed-on: http://gerrit.cloudera.org:8080/4768
Reviewed-by: Tim Armstrong <ta...@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/d1d88aac
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/d1d88aac
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/d1d88aac

Branch: refs/heads/master
Commit: d1d88aaccd03c461c4ce6224f765afe94b7c073b
Parents: 8d7b01f
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Oct 20 08:33:07 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Oct 22 01:27:39 2016 +0000

----------------------------------------------------------------------
 be/src/service/query-exec-state.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d1d88aac/be/src/service/query-exec-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-exec-state.cc b/be/src/service/query-exec-state.cc
index 7ec8c27..69473c5 100644
--- a/be/src/service/query-exec-state.cc
+++ b/be/src/service/query-exec-state.cc
@@ -620,7 +620,7 @@ Status ImpalaServer::QueryExecState::WaitInternal() {
     RETURN_IF_ERROR(query_status_);
     RETURN_IF_ERROR(UpdateQueryStatus(child_queries_status));
   }
-  query_events_->MarkEvent("Child queries finished");
+  if (!child_queries.empty()) query_events_->MarkEvent("Child queries finished");
 
   if (coord_.get() != NULL) {
     RETURN_IF_ERROR(coord_->Wait());


[10/14] incubator-impala git commit: IMPALA-3739: Enable stress tests on Kudu

Posted by ta...@apache.org.
IMPALA-3739: Enable stress tests on Kudu

This commit modifies the stress test framework to run TPC-H and TPC-DS
workloads against Kudu. The follwing changes are included in this
commit:
1. Created template files with DDL and DML statements for loading TPC-H and
   TPC-DS data in Kudu
2. Created a script (load-tpc-kudu.py) to load data in Kudu. The
   script is invoked by the stress test runner to load test data in an
   existing Impala/Kudu cluster (both local and CM-managed clusters are
   supported).
3. Created SQL files with TPC-DS queries to be executed in Kudu. SQL
   files with TPC-H queries for Kudu were added in a previous patch.
4. Modified the stress test runner to take additional parameters
   specific to Kudu (e.g. kudu master addr)

The stress test runner for Kudu was tested on EC2 clusters for both TPC-H
and TPC-DS workloads.

Missing functionality:
* No CRUD operations in the existing TPC-H/TPC-DS workloads for Kudu.
* Not all supported TPC-DS queries are included. Currently, only the
  TPC-DS queries from the testdata/workloads/tpcds/queries directory
  were modified to run against Kudu.

Change-Id: I3c9fc3dae24b761f031ee8e014bd611a49029d34
Reviewed-on: http://gerrit.cloudera.org:8080/4327
Reviewed-by: Dimitris Tsirogiannis <dt...@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/8a49ceaa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/8a49ceaa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/8a49ceaa

Branch: refs/heads/master
Commit: 8a49ceaae532163f17836b1050b639329424ee5c
Parents: 041fa6d
Author: Dimitris Tsirogiannis <dt...@cloudera.com>
Authored: Wed Aug 24 10:20:48 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 11:01:37 2016 +0000

----------------------------------------------------------------------
 testdata/bin/load-tpc-kudu.py                   | 119 +++
 .../functional/functional_schema_template.sql   |  13 +
 testdata/datasets/tpcds/tpcds_kudu_template.sql | 945 +++++++++++++++++++
 testdata/datasets/tpch/tpch_kudu_template.sql   | 198 ++++
 .../workloads/tpcds/queries/tpcds-kudu-q19.test |  39 +
 .../workloads/tpcds/queries/tpcds-kudu-q27.test |  36 +
 .../workloads/tpcds/queries/tpcds-kudu-q3.test  |  32 +
 .../workloads/tpcds/queries/tpcds-kudu-q34.test |  47 +
 .../workloads/tpcds/queries/tpcds-kudu-q42.test |  29 +
 .../workloads/tpcds/queries/tpcds-kudu-q43.test |  37 +
 .../workloads/tpcds/queries/tpcds-kudu-q46.test |  81 ++
 .../workloads/tpcds/queries/tpcds-kudu-q47.test |  53 ++
 .../workloads/tpcds/queries/tpcds-kudu-q52.test |  28 +
 .../workloads/tpcds/queries/tpcds-kudu-q53.test |  38 +
 .../workloads/tpcds/queries/tpcds-kudu-q55.test |  25 +
 .../workloads/tpcds/queries/tpcds-kudu-q59.test |  84 ++
 .../workloads/tpcds/queries/tpcds-kudu-q6.test  |  28 +
 .../workloads/tpcds/queries/tpcds-kudu-q61.test |  43 +
 .../workloads/tpcds/queries/tpcds-kudu-q63.test |  38 +
 .../workloads/tpcds/queries/tpcds-kudu-q65.test |  63 ++
 .../workloads/tpcds/queries/tpcds-kudu-q68.test |  62 ++
 .../workloads/tpcds/queries/tpcds-kudu-q7.test  |  32 +
 .../workloads/tpcds/queries/tpcds-kudu-q73.test |  51 +
 .../workloads/tpcds/queries/tpcds-kudu-q79.test |  61 ++
 .../workloads/tpcds/queries/tpcds-kudu-q8.test  |  71 ++
 .../workloads/tpcds/queries/tpcds-kudu-q88.test |  93 ++
 .../workloads/tpcds/queries/tpcds-kudu-q89.test |  33 +
 .../workloads/tpcds/queries/tpcds-kudu-q96.test |  17 +
 .../workloads/tpcds/queries/tpcds-kudu-q98.test |  34 +
 tests/comparison/db_connection.py               |   5 +
 tests/stress/concurrent_select.py               |  30 +-
 31 files changed, 2459 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/bin/load-tpc-kudu.py
----------------------------------------------------------------------
diff --git a/testdata/bin/load-tpc-kudu.py b/testdata/bin/load-tpc-kudu.py
new file mode 100755
index 0000000..01de79e
--- /dev/null
+++ b/testdata/bin/load-tpc-kudu.py
@@ -0,0 +1,119 @@
+#!/usr/bin/env impala-python
+#
+# 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.
+
+# Script to load TPC-[H|DS] data in a Kudu cluster.
+#
+# Kudu tables are created in the specified 'target-db' using the existing HDFS tables
+# from 'source-db'.
+
+import logging
+import os
+import sqlparse
+import sys
+
+LOG = logging.getLogger(os.path.splitext(os.path.basename(__file__))[0])
+
+source_db = None
+target_db = None
+kudu_master = None
+verbose = False
+buckets = None
+workload = None
+
+tpch_tables = ['customer', 'lineitem', 'nation', 'orders', 'part', 'partsupp', 'region',
+    'revenue', 'supplier']
+
+tpcds_tables = ['call_center', 'catalog_page', 'catalog_returns', 'catalog_sales',
+    'customer', 'customer_address', 'customer_demographics', 'date_dim',
+    'household_demographics', 'income_band', 'inventory', 'item', 'promotion',
+    'reason', 'ship_mode', 'store', 'store_returns', 'store_sales', 'time_dim',
+    'warehouse', 'web_page', 'web_returns', 'web_sales', 'web_site']
+
+def clean_data():
+  """Drop the specified 'target_db' and all its tables"""
+  with cluster.impala.cursor() as impala:
+    tbls_to_clean = tpch_tables if workload.lower() == 'tpch' else tpcds_tables
+    # TODO: Replace with DROP DATABASE CASCADE when it is supported for Kudu tables
+    for table_name in tbls_to_clean:
+      impala.execute("drop table if exists {}.{}".format(target_db, table_name))
+    impala.drop_db_if_exists(target_db)
+
+def load_data():
+  sql_params = {
+      "source_db_name": source_db,
+      "target_db_name": target_db,
+      "kudu_master": kudu_master,
+      "buckets": buckets}
+
+  sql_file_path = get_test_file_path(workload)
+  with open(sql_file_path, "r") as test:
+    queries = sqlparse.split(test.read())
+
+  with cluster.impala.cursor() as impala:
+    impala.create_db_if_not_exists(target_db)
+    impala.execute("USE %s" % target_db)
+    for query in queries:
+      query = sqlparse.format(query.rstrip(';'), strip_comments=True)
+      query_str = query.format(**sql_params)
+      if (len(query_str)) == 0: continue
+      if verbose: print query_str
+      impala.execute(query_str)
+
+def get_test_file_path(workload):
+  if "IMPALA_HOME" not in os.environ:
+    raise Exception("IMPALA_HOME must be set")
+  sql_file_path = os.path.join(os.environ["IMPALA_HOME"], "testdata", "datasets",
+      workload, "%s_kudu_template.sql" % (workload))
+  return sql_file_path
+
+if __name__ == "__main__":
+  from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser
+  import tests.comparison.cli_options as cli_options
+
+  parser = ArgumentParser(formatter_class=ArgumentDefaultsHelpFormatter)
+  cli_options.add_logging_options(parser)
+  cli_options.add_cluster_options(parser)
+  parser.add_argument("-s", "--source-db", required=True,
+      help="Source DB to load data from.")
+  parser.add_argument("-t", "--target-db", required=True,
+      help="Target DB to load data to.")
+  parser.add_argument("-w", "--workload", choices=['tpch', 'tpcds'],
+      required=True)
+  parser.add_argument("--kudu_master", required=True,
+      help="Address or host name of Kudu master")
+  # TODO: Automatically set #buckets as a function of cluster nodes and/or
+  # scale
+  parser.add_argument("-b", "--buckets", default="9",
+      help="Number of buckets to partition Kudu tables (only for hash-based).")
+  parser.add_argument("-v", "--verbose", action='store_true',
+      help="Print the executed statements.")
+  parser.add_argument("--clean", action='store_true',
+      help="Drop all tables in the speficied target database.")
+  args = parser.parse_args()
+
+  cli_options.configure_logging(args.log_level, debug_log_file=args.debug_log_file)
+  cluster = cli_options.create_cluster(args)
+  source_db = args.source_db
+  target_db = args.target_db
+  buckets = args.buckets
+  kudu_master = args.kudu_master
+  workload = args.workload
+  verbose = args.verbose
+  if args.clean: clean_data()
+  load_data()

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/datasets/functional/functional_schema_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index 76e1427..7b929b7 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -77,6 +77,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypes/101001.txt' OVERW
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypes/101101.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=11);
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypes/101201.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=12);
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
   id INT PRIMARY KEY,
   bool_col BOOLEAN,
@@ -154,6 +155,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesSmall/090201.txt'
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesSmall/090301.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=3);
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesSmall/090401.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=4);
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
   id INT PRIMARY KEY,
   bool_col BOOLEAN,
@@ -212,6 +214,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesTiny/090201.txt' O
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesTiny/090301.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=3);
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesTiny/090401.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2009, month=4);
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
   id INT PRIMARY KEY,
   bool_col BOOLEAN,
@@ -625,6 +628,7 @@ LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesAggNoNulls/100108.
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesAggNoNulls/100109.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=1, day=9);
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/target/AllTypesAggNoNulls/100110.txt' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name} PARTITION(year=2010, month=1, day=10);
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
   id INT PRIMARY KEY,
   bool_col BOOLEAN,
@@ -756,6 +760,7 @@ zip int
 ---- ROW_FORMAT
 delimited fields terminated by ','  escaped by '\\'
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   id bigint primary key,
   name string,
@@ -778,6 +783,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/DimTbl/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   id bigint primary key,
   name string,
@@ -801,6 +807,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/JoinTbl/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   test_id bigint,
   test_name string,
@@ -1142,6 +1149,7 @@ f2 int
 ---- COLUMNS
 field string
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 CREATE TABLE {db_name}{db_suffix}.{table_name} (
   field STRING PRIMARY KEY,
   f2 INT
@@ -1248,6 +1256,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/TinyTable/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   a string primary key,
   b string
@@ -1267,6 +1276,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/TinyIntTable/data.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   int_col int primary key
 )
@@ -1292,6 +1302,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} select * from functiona
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/NullTable/data.csv'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   a string primary key, b string, c string, d int, e double, f string, g string
 )
@@ -1317,6 +1328,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} select * from functiona
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/NullTable/data.csv'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   a string primary key, b string, c string, d int, e double, f string, g string
 )
@@ -1391,6 +1403,7 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} SELECT * FROM {db_name}
 ---- LOAD
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/ImpalaDemoDataset/DEC_00_SF3_P077_with_ann_noheader.csv' OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
 create table {db_name}{db_suffix}.{table_name} (
   id string,
   zip string,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/datasets/tpcds/tpcds_kudu_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/tpcds/tpcds_kudu_template.sql b/testdata/datasets/tpcds/tpcds_kudu_template.sql
new file mode 100644
index 0000000..dd65fef
--- /dev/null
+++ b/testdata/datasets/tpcds/tpcds_kudu_template.sql
@@ -0,0 +1,945 @@
+---- Template SQL statements to create and load TPCDS tables in KUDU.
+---- TODO: Change to the new syntax for CREATE TABLE statements (IMPALA-3719)
+---- TODO: Use range partitioning for some tables
+---- TODO: Fix the primary key column order
+---- TODO: Use different number of buckets for fact and dimension tables
+
+---- STORE_SALES
+CREATE TABLE IF NOT EXISTS {target_db_name}.store_sales (
+  ss_ticket_number BIGINT,
+  ss_item_sk BIGINT,
+  ss_sold_date_sk BIGINT,
+  ss_sold_time_sk BIGINT,
+  ss_customer_sk BIGINT,
+  ss_cdemo_sk BIGINT,
+  ss_hdemo_sk BIGINT,
+  ss_addr_sk BIGINT,
+  ss_store_sk BIGINT,
+  ss_promo_sk BIGINT,
+  ss_quantity BIGINT,
+  ss_wholesale_cost DOUBLE,
+  ss_list_price DOUBLE,
+  ss_sales_price DOUBLE,
+  ss_ext_discount_amt DOUBLE,
+  ss_ext_sales_price DOUBLE,
+  ss_ext_wholesale_cost DOUBLE,
+  ss_ext_list_price DOUBLE,
+  ss_ext_tax DOUBLE,
+  ss_coupon_amt DOUBLE,
+  ss_net_paid DOUBLE,
+  ss_net_paid_inc_tax DOUBLE,
+  ss_net_profit DOUBLE
+)
+DISTRIBUTE BY HASH (ss_ticket_number,ss_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES(
+'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+'kudu.table_name' = '{target_db_name}_store_sales',
+'kudu.master_addresses' = '{kudu_master}:7051',
+'kudu.key_columns' = 'ss_ticket_number, ss_item_sk'
+);
+
+INSERT INTO {target_db_name}.store_sales
+SELECT
+  ss_ticket_number,
+  ss_item_sk,
+  ss_sold_date_sk,
+  ss_sold_time_sk,
+  ss_customer_sk,
+  ss_cdemo_sk,
+  ss_hdemo_sk,
+  ss_addr_sk,
+  ss_store_sk,
+  ss_promo_sk,
+  ss_quantity,
+  ss_wholesale_cost,
+  ss_list_price,
+  ss_sales_price,
+  ss_ext_discount_amt,
+  ss_ext_sales_price,
+  ss_ext_wholesale_cost,
+  ss_ext_list_price,
+  ss_ext_tax,
+  ss_coupon_amt,
+  ss_net_paid,
+  ss_net_paid_inc_tax,ss_net_profit
+FROM {source_db_name}.store_sales;
+
+---- WEB_SALES
+CREATE TABLE IF NOT EXISTS {target_db_name}.web_sales (
+  ws_order_number BIGINT,
+  ws_item_sk BIGINT,
+  ws_sold_date_sk BIGINT,
+  ws_sold_time_sk BIGINT,
+  ws_ship_date_sk BIGINT,
+  ws_bill_customer_sk BIGINT,
+  ws_bill_cdemo_sk BIGINT,
+  ws_bill_hdemo_sk BIGINT,
+  ws_bill_addr_sk BIGINT,
+  ws_ship_customer_sk BIGINT,
+  ws_ship_cdemo_sk BIGINT,
+  ws_ship_hdemo_sk BIGINT,
+  ws_ship_addr_sk BIGINT,
+  ws_web_page_sk BIGINT,
+  ws_web_site_sk BIGINT,
+  ws_ship_mode_sk BIGINT,
+  ws_warehouse_sk BIGINT,
+  ws_promo_sk BIGINT,
+  ws_quantity BIGINT,
+  ws_wholesale_cost DOUBLE,
+  ws_list_price DOUBLE,
+  ws_sales_price DOUBLE,
+  ws_ext_discount_amt DOUBLE,
+  ws_ext_sales_price DOUBLE,
+  ws_ext_wholesale_cost DOUBLE,
+  ws_ext_list_price DOUBLE,
+  ws_ext_tax DOUBLE,
+  ws_coupon_amt DOUBLE,
+  ws_ext_ship_cost DOUBLE,
+  ws_net_paid DOUBLE,
+  ws_net_paid_inc_tax DOUBLE,
+  ws_net_paid_inc_ship DOUBLE,
+  ws_net_paid_inc_ship_tax DOUBLE,
+  ws_net_profit DOUBLE
+)
+DISTRIBUTE BY HASH (ws_order_number,ws_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES(
+'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+'kudu.table_name' = '{target_db_name}_web_sales',
+'kudu.master_addresses' = '{kudu_master}:7051',
+'kudu.key_columns' = 'ws_order_number, ws_item_sk'
+);
+
+INSERT INTO {target_db_name}.web_sales
+SELECT
+  ws_order_number,
+  ws_item_sk,
+  ws_sold_date_sk,
+  ws_sold_time_sk,
+  ws_ship_date_sk,
+  ws_bill_customer_sk,
+  ws_bill_cdemo_sk,
+  ws_bill_hdemo_sk,
+  ws_bill_addr_sk,
+  ws_ship_customer_sk,
+  ws_ship_cdemo_sk,
+  ws_ship_hdemo_sk,
+  ws_ship_addr_sk,
+  ws_web_page_sk,
+  ws_web_site_sk,
+  ws_ship_mode_sk,
+  ws_warehouse_sk,
+  ws_promo_sk,
+  ws_quantity,
+  ws_wholesale_cost,
+  ws_list_price,
+  ws_sales_price,
+  ws_ext_discount_amt,
+  ws_ext_sales_price,
+  ws_ext_wholesale_cost,
+  ws_ext_list_price,
+  ws_ext_tax,
+  ws_coupon_amt,
+  ws_ext_ship_cost,
+  ws_net_paid,
+  ws_net_paid_inc_tax,
+  ws_net_paid_inc_ship,
+  ws_net_paid_inc_ship_tax,
+  ws_net_profit
+FROM {source_db_name}.web_sales;
+
+---- CATALOG_SALES
+CREATE TABLE IF NOT EXISTS {target_db_name}.catalog_sales (
+  cs_order_number BIGINT,
+  cs_item_sk BIGINT,
+  cs_sold_date_sk BIGINT,
+  cs_sold_time_sk BIGINT,
+  cs_ship_date_sk BIGINT,
+  cs_bill_customer_sk BIGINT,
+  cs_bill_cdemo_sk BIGINT,
+  cs_bill_hdemo_sk BIGINT,
+  cs_bill_addr_sk BIGINT,
+  cs_ship_customer_sk BIGINT,
+  cs_ship_cdemo_sk BIGINT,
+  cs_ship_hdemo_sk BIGINT,
+  cs_ship_addr_sk BIGINT,
+  cs_call_center_sk BIGINT,
+  cs_catalog_page_sk BIGINT,
+  cs_ship_mode_sk BIGINT,
+  cs_warehouse_sk BIGINT,
+  cs_promo_sk BIGINT,
+  cs_quantity BIGINT,
+  cs_wholesale_cost DOUBLE,
+  cs_list_price DOUBLE,
+  cs_sales_price DOUBLE,
+  cs_ext_discount_amt DOUBLE,
+  cs_ext_sales_price DOUBLE,
+  cs_ext_wholesale_cost DOUBLE,
+  cs_ext_list_price DOUBLE,
+  cs_ext_tax DOUBLE,
+  cs_coupon_amt DOUBLE,
+  cs_ext_ship_cost DOUBLE,
+  cs_net_paid DOUBLE,
+  cs_net_paid_inc_tax DOUBLE,
+  cs_net_paid_inc_ship DOUBLE,
+  cs_net_paid_inc_ship_tax DOUBLE,
+  cs_net_profit DOUBLE
+)
+DISTRIBUTE BY HASH (cs_order_number,cs_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES(
+'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+'kudu.table_name' = '{target_db_name}_catalog_sales',
+'kudu.master_addresses' = '{kudu_master}:7051',
+'kudu.key_columns' = 'cs_order_number, cs_item_sk'
+);
+
+INSERT INTO {target_db_name}.catalog_sales
+SELECT
+  cs_order_number,
+  cs_item_sk,
+  cs_sold_date_sk,
+  cs_sold_time_sk,
+  cs_ship_date_sk,
+  cs_bill_customer_sk,
+  cs_bill_cdemo_sk,
+  cs_bill_hdemo_sk,
+  cs_bill_addr_sk,
+  cs_ship_customer_sk,
+  cs_ship_cdemo_sk,
+  cs_ship_hdemo_sk,
+  cs_ship_addr_sk,
+  cs_call_center_sk,
+  cs_catalog_page_sk,
+  cs_ship_mode_sk,
+  cs_warehouse_sk,
+  cs_promo_sk,
+  cs_quantity,
+  cs_wholesale_cost,
+  cs_list_price,
+  cs_sales_price,
+  cs_ext_discount_amt,
+  cs_ext_sales_price,
+  cs_ext_wholesale_cost,
+  cs_ext_list_price,
+  cs_ext_tax,
+  cs_coupon_amt,
+  cs_ext_ship_cost,
+  cs_net_paid,
+  cs_net_paid_inc_tax,
+  cs_net_paid_inc_ship,
+  cs_net_paid_inc_ship_tax,
+  cs_net_profit
+FROM {source_db_name}.catalog_sales;
+
+---- STORE_RETURNS
+CREATE TABLE IF NOT EXISTS {target_db_name}.store_returns (
+  sr_ticket_number BIGINT,
+  sr_item_sk BIGINT,
+  sr_returned_date_sk BIGINT,
+  sr_return_time_sk BIGINT,
+  sr_customer_sk BIGINT,
+  sr_cdemo_sk BIGINT,
+  sr_hdemo_sk BIGINT,
+  sr_addr_sk BIGINT,
+  sr_store_sk BIGINT,
+  sr_reason_sk BIGINT,
+  sr_return_quantity BIGINT,
+  sr_return_amt DOUBLE,
+  sr_return_tax DOUBLE,
+  sr_return_amt_inc_tax DOUBLE,
+  sr_fee DOUBLE,
+  sr_return_ship_cost DOUBLE,
+  sr_refunded_cash DOUBLE,
+  sr_reversed_charge DOUBLE,
+  sr_store_credit DOUBLE,
+  sr_net_loss DOUBLE
+)
+DISTRIBUTE BY HASH (sr_ticket_number,sr_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='sr_ticket_number, sr_item_sk',
+'kudu.table_name'='{target_db_name}_store_returns',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.store_returns
+SELECT
+  sr_ticket_number,
+  sr_item_sk,
+  sr_returned_date_sk,
+  sr_return_time_sk,
+  sr_customer_sk,
+  sr_cdemo_sk,
+  sr_hdemo_sk,
+  sr_addr_sk,
+  sr_store_sk,
+  sr_reason_sk,
+  sr_return_quantity,
+  sr_return_amt,
+  sr_return_tax,
+  sr_return_amt_inc_tax,
+  sr_fee,
+  sr_return_ship_cost,
+  sr_refunded_cash,
+  sr_reversed_charge,
+  sr_store_credit,
+  sr_net_loss
+FROM {source_db_name}.store_returns;
+
+---- WEB_RETURNS
+CREATE TABLE IF NOT EXISTS {target_db_name}.web_returns (
+  wr_order_number BIGINT,
+  wr_item_sk BIGINT,
+  wr_returned_date_sk BIGINT,
+  wr_returned_time_sk BIGINT,
+  wr_refunded_customer_sk BIGINT,
+  wr_refunded_cdemo_sk BIGINT,
+  wr_refunded_hdemo_sk BIGINT,
+  wr_refunded_addr_sk BIGINT,
+  wr_returning_customer_sk BIGINT,
+  wr_returning_cdemo_sk BIGINT,
+  wr_returning_hdemo_sk BIGINT,
+  wr_returning_addr_sk BIGINT,
+  wr_web_page_sk BIGINT,
+  wr_reason_sk BIGINT,
+  wr_return_quantity BIGINT,
+  wr_return_amt DOUBLE,
+  wr_return_tax DOUBLE,
+  wr_return_amt_inc_tax DOUBLE,
+  wr_fee DOUBLE,
+  wr_return_ship_cost DOUBLE,
+  wr_refunded_cash DOUBLE,
+  wr_reversed_charge DOUBLE,
+  wr_account_credit DOUBLE,
+  wr_net_loss DOUBLE
+)
+DISTRIBUTE BY HASH (wr_order_number,wr_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='wr_order_number, wr_item_sk',
+'kudu.table_name'='{target_db_name}_web_returns',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.web_returns
+SELECT
+  wr_order_number,
+  wr_item_sk,
+  wr_returned_date_sk,
+  wr_returned_time_sk,
+  wr_refunded_customer_sk,
+  wr_refunded_cdemo_sk,
+  wr_refunded_hdemo_sk,
+  wr_refunded_addr_sk,
+  wr_returning_customer_sk,
+  wr_returning_cdemo_sk,
+  wr_returning_hdemo_sk,
+  wr_returning_addr_sk,
+  wr_web_page_sk,
+  wr_reason_sk,
+  wr_return_quantity,
+  wr_return_amt,
+  wr_return_tax,
+  wr_return_amt_inc_tax,
+  wr_fee,
+  wr_return_ship_cost,
+  wr_refunded_cash,
+  wr_reversed_charge,
+  wr_account_credit,
+  wr_net_loss
+FROM {source_db_name}.web_returns;
+
+---- CATALOG_RETURNS
+CREATE TABLE IF NOT EXISTS {target_db_name}.catalog_returns (
+  cr_order_number BIGINT,
+  cr_item_sk BIGINT,
+  cr_returned_date_sk BIGINT,
+  cr_returned_time_sk BIGINT,
+  cr_refunded_customer_sk BIGINT,
+  cr_refunded_cdemo_sk BIGINT,
+  cr_refunded_hdemo_sk BIGINT,
+  cr_refunded_addr_sk BIGINT,
+  cr_returning_customer_sk BIGINT,
+  cr_returning_cdemo_sk BIGINT,
+  cr_returning_hdemo_sk BIGINT,
+  cr_returning_addr_sk BIGINT,
+  cr_call_center_sk BIGINT,
+  cr_catalog_page_sk BIGINT,
+  cr_ship_mode_sk BIGINT,
+  cr_warehouse_sk BIGINT,
+  cr_reason_sk BIGINT,
+  cr_return_quantity BIGINT,
+  cr_return_amount DOUBLE,
+  cr_return_tax DOUBLE,
+  cr_return_amt_inc_tax DOUBLE,
+  cr_fee DOUBLE,
+  cr_return_ship_cost DOUBLE,
+  cr_refunded_cash DOUBLE,
+  cr_reversed_charge DOUBLE,
+  cr_store_credit DOUBLE,
+  cr_net_loss DOUBLE
+)
+DISTRIBUTE BY HASH (cr_order_number,cr_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='cr_order_number, cr_item_sk',
+'kudu.table_name'='{target_db_name}_catalog_returns',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.catalog_returns
+SELECT
+  cr_order_number,
+  cr_item_sk,
+  cr_returned_date_sk,
+  cr_returned_time_sk,
+  cr_refunded_customer_sk,
+  cr_refunded_cdemo_sk,
+  cr_refunded_hdemo_sk,
+  cr_refunded_addr_sk,
+  cr_returning_customer_sk,
+  cr_returning_cdemo_sk,
+  cr_returning_hdemo_sk,
+  cr_returning_addr_sk,
+  cr_call_center_sk,
+  cr_catalog_page_sk,
+  cr_ship_mode_sk,
+  cr_warehouse_sk,
+  cr_reason_sk,
+  cr_return_quantity,
+  cr_return_amount,
+  cr_return_tax,
+  cr_return_amt_inc_tax,
+  cr_fee,
+  cr_return_ship_cost,
+  cr_refunded_cash,
+  cr_reversed_charge,
+  cr_store_credit,
+  cr_net_loss
+FROM {source_db_name}.catalog_returns;
+
+---- INVENTORY
+CREATE TABLE IF NOT EXISTS {target_db_name}.inventory (
+  inv_date_sk BIGINT,
+  inv_item_sk BIGINT,
+  inv_warehouse_sk BIGINT,
+  inv_quantity_on_hand BIGINT
+)
+DISTRIBUTE BY HASH (inv_item_sk,inv_date_sk,inv_warehouse_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='inv_date_sk,inv_item_sk,inv_warehouse_sk',
+'kudu.table_name'='{target_db_name}_inventory',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.inventory SELECT * FROM {source_db_name}.inventory;
+
+---- CUSTOMER
+
+CREATE TABLE {target_db_name}.customer (
+  c_customer_sk BIGINT,
+  c_customer_id STRING,
+  c_current_cdemo_sk BIGINT,
+  c_current_hdemo_sk BIGINT,
+  c_current_addr_sk BIGINT,
+  c_first_shipto_date_sk BIGINT,
+  c_first_sales_date_sk BIGINT,
+  c_salutation STRING,
+  c_first_name STRING,
+  c_last_name STRING,
+  c_preferred_cust_flag STRING,
+  c_birth_day INT,
+  c_birth_month INT,
+  c_birth_year INT,
+  c_birth_country STRING,
+  c_login STRING,
+  c_email_address STRING,
+  c_last_review_date BIGINT
+)
+DISTRIBUTE BY HASH (c_customer_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='c_customer_sk',
+'kudu.table_name'='{target_db_name}_customer',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.customer SELECT * FROM {source_db_name}.customer;
+
+---- CUSTOMER_ADDRESS
+CREATE TABLE IF NOT EXISTS {target_db_name}.customer_address (
+  ca_address_sk BIGINT,
+  ca_address_id STRING,
+  ca_street_number STRING,
+  ca_street_name STRING,
+  ca_street_type STRING,
+  ca_suite_number STRING,
+  ca_city STRING,
+  ca_county STRING,
+  ca_state STRING,
+  ca_zip STRING,
+  ca_country STRING,
+  ca_gmt_offset DOUBLE,
+  ca_location_type STRING
+)
+DISTRIBUTE BY HASH (ca_address_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='ca_address_sk',
+'kudu.table_name'='{target_db_name}_customer_address',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.customer_address
+SELECT * FROM {source_db_name}.customer_address;
+
+---- CUSTOMER_DEMOGRAPHICS
+CREATE TABLE IF NOT EXISTS {target_db_name}.customer_demographics (
+  cd_demo_sk BIGINT,
+  cd_gender STRING,
+  cd_marital_status STRING,
+  cd_education_status STRING,
+  cd_purchase_estimate BIGINT,
+  cd_credit_rating STRING,
+  cd_dep_count BIGINT,
+  cd_dep_employed_count BIGINT,
+  cd_dep_college_count BIGINT
+)
+DISTRIBUTE BY HASH (cd_demo_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='cd_demo_sk',
+'kudu.table_name'='{target_db_name}_customer_demographics',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.customer_demographics
+SELECT * FROM {source_db_name}.customer_demographics;
+
+---- DATE_DIM
+CREATE TABLE IF NOT EXISTS {target_db_name}.date_dim (
+  d_date_sk BIGINT,
+  d_date_id STRING,
+  d_date STRING,
+  d_month_seq BIGINT,
+  d_week_seq BIGINT,
+  d_quarter_seq BIGINT,
+  d_year BIGINT,
+  d_dow BIGINT,
+  d_moy BIGINT,
+  d_dom BIGINT,
+  d_qoy BIGINT,
+  d_fy_year BIGINT,
+  d_fy_quarter_seq BIGINT,
+  d_fy_week_seq BIGINT,
+  d_day_name STRING,
+  d_quarter_name STRING,
+  d_holiday STRING,
+  d_weekend STRING,
+  d_following_holiday STRING,
+  d_first_dom BIGINT,
+  d_last_dom BIGINT,
+  d_same_day_ly BIGINT,
+  d_same_day_lq BIGINT,
+  d_current_day STRING,
+  d_current_week STRING,
+  d_current_month STRING,
+  d_current_quarter STRING,
+  d_current_year STRING
+)
+DISTRIBUTE BY HASH (d_date_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='d_date_sk',
+'kudu.table_name'='{target_db_name}_date_dim',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.date_dim SELECT * FROM {source_db_name}.date_dim;
+
+---- HOUSEHOLD_DEMOGRAPHICS
+CREATE TABLE IF NOT EXISTS {target_db_name}.household_demographics (
+  hd_demo_sk BIGINT,
+  hd_income_band_sk BIGINT,
+  hd_buy_potential STRING,
+  hd_dep_count BIGINT,
+  hd_vehicle_count BIGINT
+)
+DISTRIBUTE BY HASH (hd_demo_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='hd_demo_sk',
+'kudu.table_name'='{target_db_name}_household_demographics',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.household_demographics
+SELECT * FROM {source_db_name}.household_demographics;
+
+---- ITEM
+CREATE TABLE IF NOT EXISTS {target_db_name}.item (
+  i_item_sk BIGINT,
+  i_item_id STRING,
+  i_rec_start_date STRING,
+  i_rec_end_date STRING,
+  i_item_desc STRING,
+  i_current_price DOUBLE,
+  i_wholesale_cost DOUBLE,
+  i_brand_id BIGINT,
+  i_brand STRING,
+  i_class_id BIGINT,
+  i_class STRING,
+  i_category_id BIGINT,
+  i_category STRING,
+  i_manufact_id BIGINT,
+  i_manufact STRING,
+  i_size STRING,
+  i_formulation STRING,
+  i_color STRING,
+  i_units STRING,
+  i_container STRING,
+  i_manager_id BIGINT,
+  i_product_name STRING
+)
+DISTRIBUTE BY HASH (i_item_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='i_item_sk',
+'kudu.table_name'='{target_db_name}_item',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.item SELECT * FROM {source_db_name}.item;
+
+---- PROMOTION
+CREATE TABLE IF NOT EXISTS {target_db_name}.promotion (
+  p_promo_sk BIGINT,
+  p_item_sk BIGINT,
+  p_start_date_sk BIGINT,
+  p_end_date_sk BIGINT,
+  p_promo_id STRING,
+  p_cost DOUBLE,
+  p_response_target BIGINT,
+  p_promo_name STRING,
+  p_channel_dmail STRING,
+  p_channel_email STRING,
+  p_channel_catalog STRING,
+  p_channel_tv STRING,
+  p_channel_radio STRING,
+  p_channel_press STRING,
+  p_channel_event STRING,
+  p_channel_demo STRING,
+  p_channel_details STRING,
+  p_purpose STRING,
+  p_discount_active STRING
+)
+DISTRIBUTE BY HASH (p_promo_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='p_promo_sk',
+'kudu.table_name'='{target_db_name}_promotion',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.promotion
+SELECT
+  p_promo_sk,
+  p_item_sk,
+  p_start_date_sk,
+  p_end_date_sk,
+  p_promo_id,
+  p_cost,
+  p_response_target,
+  p_promo_name,
+  p_channel_dmail,
+  p_channel_email,
+  p_channel_catalog,
+  p_channel_tv,
+  p_channel_radio,
+  p_channel_press,
+  p_channel_event,
+  p_channel_demo,
+  p_channel_details,
+  p_purpose,
+  p_discount_active
+FROM {source_db_name}.promotion;
+
+---- STORE
+CREATE TABLE IF NOT EXISTS {target_db_name}.store (
+  s_store_sk BIGINT,
+  s_store_id STRING,
+  s_rec_start_date STRING,
+  s_rec_end_date STRING,
+  s_closed_date_sk BIGINT,
+  s_store_name STRING,
+  s_number_employees BIGINT,
+  s_floor_space BIGINT,
+  s_hours STRING,
+  s_manager STRING,
+  s_market_id BIGINT,
+  s_geography_class STRING,
+  s_market_desc STRING,
+  s_market_manager STRING,
+  s_division_id BIGINT,
+  s_division_name STRING,
+  s_company_id BIGINT,
+  s_company_name STRING,
+  s_street_number STRING,
+  s_street_name STRING,
+  s_street_type STRING,
+  s_suite_number STRING,
+  s_city STRING,
+  s_county STRING,
+  s_state STRING,
+  s_zip STRING,
+  s_country STRING,
+  s_gmt_offset DOUBLE,
+  s_tax_precentage DOUBLE
+)
+DISTRIBUTE BY HASH (s_store_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='s_store_sk',
+'kudu.table_name'='{target_db_name}_store',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.store SELECT * FROM {source_db_name}.store;
+
+---- TIME_DIM
+CREATE TABLE IF NOT EXISTS {target_db_name}.time_dim (
+  t_time_sk BIGINT,
+  t_time_id STRING,
+  t_time BIGINT,
+  t_hour BIGINT,
+  t_minute BIGINT,
+  t_second BIGINT,
+  t_am_pm STRING,
+  t_shift STRING,
+  t_sub_shift STRING,
+  t_meal_time STRING
+)
+DISTRIBUTE BY HASH (t_time_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='t_time_sk',
+'kudu.table_name'='{target_db_name}_time_dim',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.time_dim SELECT * FROM {source_db_name}.time_dim;
+
+---- CALL_CENTER
+CREATE TABLE IF NOT EXISTS {target_db_name}.call_center (
+  cc_call_center_sk BIGINT,
+  cc_call_center_id STRING,
+  cc_rec_start_date STRING,
+  cc_rec_end_date STRING,
+  cc_closed_date_sk BIGINT,
+  cc_open_date_sk BIGINT,
+  cc_name STRING,
+  cc_class STRING,
+  cc_employees BIGINT,
+  cc_sq_ft BIGINT,
+  cc_hours STRING,
+  cc_manager STRING,
+  cc_mkt_id BIGINT,
+  cc_mkt_class STRING,
+  cc_mkt_desc STRING,
+  cc_market_manager STRING,
+  cc_division BIGINT,
+  cc_division_name STRING,
+  cc_company BIGINT,
+  cc_company_name STRING,
+  cc_street_number STRING,
+  cc_street_name STRING,
+  cc_street_type STRING,
+  cc_suite_number STRING,
+  cc_city STRING,
+  cc_county STRING,
+  cc_state STRING,
+  cc_zip STRING,
+  cc_country STRING,
+  cc_gmt_offset DOUBLE,
+  cc_tax_percentage DOUBLE
+)
+DISTRIBUTE BY HASH (cc_call_center_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='cc_call_center_sk',
+'kudu.table_name'='{target_db_name}_call_center',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.call_center SELECT * FROM {source_db_name}.call_center;
+
+---- CATALOG_PAGE
+CREATE TABLE IF NOT EXISTS {target_db_name}.catalog_page (
+  cp_catalog_page_sk BIGINT,
+  cp_catalog_page_id STRING,
+  cp_start_date_sk BIGINT,
+  cp_end_date_sk BIGINT,
+  cp_department STRING,
+  cp_catalog_number BIGINT,
+  cp_catalog_page_number BIGINT,
+  cp_description STRING,
+  cp_type STRING
+)
+DISTRIBUTE BY HASH (cp_catalog_page_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='cp_catalog_page_sk',
+'kudu.table_name'='{target_db_name}_catalog_page',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.catalog_page SELECT * FROM {source_db_name}.catalog_page;
+
+---- INCOME_BANDS
+CREATE TABLE IF NOT EXISTS {target_db_name}.income_band (
+  ib_income_band_sk BIGINT,
+  ib_lower_bound BIGINT,
+  ib_upper_bound BIGINT
+)
+DISTRIBUTE BY HASH (ib_income_band_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='ib_income_band_sk',
+'kudu.table_name'='{target_db_name}_income_band',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.income_band SELECT * FROM {source_db_name}.income_band;
+
+---- REASON
+CREATE TABLE IF NOT EXISTS {target_db_name}.reason (
+  r_reason_sk BIGINT,
+  r_reason_id STRING,
+  r_reason_desc STRING
+)
+DISTRIBUTE BY HASH (r_reason_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='r_reason_sk',
+'kudu.table_name'='{target_db_name}_reason',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.reason SELECT * FROM {source_db_name}.reason;
+
+---- SHIP_MODE
+CREATE TABLE IF NOT EXISTS {target_db_name}.ship_mode (
+  sm_ship_mode_sk BIGINT,
+  sm_ship_mode_id STRING,
+  sm_type STRING,
+  sm_code STRING,
+  sm_carrier STRING,
+  sm_contract STRING
+)
+DISTRIBUTE BY HASH (sm_ship_mode_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='sm_ship_mode_sk',
+'kudu.table_name'='{target_db_name}_ship_mode',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.ship_mode SELECT * FROM {source_db_name}.ship_mode;
+
+---- WAREHOUSE
+CREATE TABLE IF NOT EXISTS {target_db_name}.warehouse (
+  w_warehouse_sk BIGINT,
+  w_warehouse_id STRING,
+  w_warehouse_name STRING,
+  w_warehouse_sq_ft BIGINT,
+  w_street_number STRING,
+  w_street_name STRING,
+  w_street_type STRING,
+  w_suite_number STRING,
+  w_city STRING,
+  w_county STRING,
+  w_state STRING,
+  w_zip STRING,
+  w_country STRING,
+  w_gmt_offset DOUBLE
+)
+DISTRIBUTE BY HASH (w_warehouse_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='w_warehouse_sk',
+'kudu.table_name'='{target_db_name}_warehouse',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.warehouse SELECT * FROM {source_db_name}.warehouse;
+
+---- WEB_PAGE
+CREATE TABLE IF NOT EXISTS {target_db_name}.web_page (
+  wp_web_page_sk BIGINT,
+  wp_web_page_id STRING,
+  wp_rec_start_date STRING,
+  wp_rec_end_date STRING,
+  wp_creation_date_sk BIGINT,
+  wp_access_date_sk BIGINT,
+  wp_autogen_flag STRING,
+  wp_customer_sk BIGINT,
+  wp_url STRING,
+  wp_type STRING,
+  wp_char_count BIGINT,
+  wp_link_count BIGINT,
+  wp_image_count BIGINT,
+  wp_max_ad_count BIGINT
+)
+DISTRIBUTE BY HASH (wp_web_page_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='wp_web_page_sk',
+'kudu.table_name'='{target_db_name}_web_page',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.web_page SELECT * FROM {source_db_name}.web_page;
+
+---- WEB_SITE
+CREATE TABLE IF NOT EXISTS {target_db_name}.web_site (
+  web_site_sk BIGINT,
+  web_site_id STRING,
+  web_rec_start_date STRING,
+  web_rec_end_date STRING,
+  web_name STRING,
+  web_open_date_sk BIGINT,
+  web_close_date_sk BIGINT,
+  web_class STRING,
+  web_manager STRING,
+  web_mkt_id BIGINT,
+  web_mkt_class STRING,
+  web_mkt_desc STRING,
+  web_market_manager STRING,
+  web_company_id BIGINT,
+  web_company_name STRING,
+  web_street_number STRING,
+  web_street_name STRING,
+  web_street_type STRING,
+  web_suite_number STRING,
+  web_city STRING,
+  web_county STRING,
+  web_state STRING,
+  web_zip STRING,
+  web_country STRING,
+  web_gmt_offset DOUBLE,
+  web_tax_percentage DOUBLE
+)
+DISTRIBUTE BY HASH (web_site_sk) INTO {buckets} BUCKETS
+TBLPROPERTIES (
+'kudu.master_addresses'='{kudu_master}:7051',
+'kudu.key_columns'='web_site_sk',
+'kudu.table_name'='{target_db_name}_web_site',
+'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler');
+
+INSERT INTO {target_db_name}.web_site SELECT * FROM {source_db_name}.web_site;
+
+---- COMPUTE STATS
+compute stats {target_db_name}.call_center;
+compute stats {target_db_name}.catalog_page;
+compute stats {target_db_name}.catalog_returns;
+compute stats {target_db_name}.catalog_sales;
+compute stats {target_db_name}.customer;
+compute stats {target_db_name}.customer_address;
+compute stats {target_db_name}.customer_demographics;
+compute stats {target_db_name}.date_dim;
+compute stats {target_db_name}.household_demographics;
+compute stats {target_db_name}.income_band;
+compute stats {target_db_name}.inventory;
+compute stats {target_db_name}.item;
+compute stats {target_db_name}.reason;
+compute stats {target_db_name}.ship_mode;
+compute stats {target_db_name}.store;
+compute stats {target_db_name}.store_returns;
+compute stats {target_db_name}.store_sales;
+compute stats {target_db_name}.time_dim;
+compute stats {target_db_name}.warehouse;
+compute stats {target_db_name}.web_page;
+compute stats {target_db_name}.web_returns;
+compute stats {target_db_name}.web_sales;
+compute stats {target_db_name}.web_site;
+compute stats {target_db_name}.promotion;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/datasets/tpch/tpch_kudu_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/tpch/tpch_kudu_template.sql b/testdata/datasets/tpch/tpch_kudu_template.sql
new file mode 100644
index 0000000..5abde0b
--- /dev/null
+++ b/testdata/datasets/tpch/tpch_kudu_template.sql
@@ -0,0 +1,198 @@
+---- Template SQL statements to create and load TPCH tables in KUDU.
+---- TODO: Change to the new syntax for CREATE TABLE statements (IMPALA-3719)
+---- TODO: Fix the primary key column order
+---- TODO: Remove the CREATE_KUDU sections from tpch_schema_template.sql and use
+---- this file instead for loading TPC-H data in Kudu.
+
+--- LINEITEM
+CREATE TABLE IF NOT EXISTS {target_db_name}.lineitem (
+  L_ORDERKEY BIGINT,
+  L_LINENUMBER BIGINT,
+  L_PARTKEY BIGINT,
+  L_SUPPKEY BIGINT,
+  L_QUANTITY DOUBLE,
+  L_EXTENDEDPRICE DOUBLE,
+  L_DISCOUNT DOUBLE,
+  L_TAX DOUBLE,
+  L_RETURNFLAG STRING,
+  L_LINESTATUS STRING,
+  L_SHIPDATE STRING,
+  L_COMMITDATE STRING,
+  L_RECEIPTDATE STRING,
+  L_SHIPINSTRUCT STRING,
+  L_SHIPMODE STRING,
+  L_COMMENT STRING
+)
+distribute by hash (l_orderkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_lineitem',
+  'kudu.key_columns' = 'l_orderkey, l_linenumber'
+);
+
+INSERT INTO TABLE {target_db_name}.lineitem
+SELECT
+  L_ORDERKEY,
+  L_LINENUMBER,
+  L_PARTKEY,
+  L_SUPPKEY,
+  L_QUANTITY,
+  L_EXTENDEDPRICE,
+  L_DISCOUNT,
+  L_TAX,
+  L_RETURNFLAG,
+  L_LINESTATUS,
+  L_SHIPDATE,
+  L_COMMITDATE,
+  L_RECEIPTDATE,
+  L_SHIPINSTRUCT,
+  L_SHIPMODE,
+  L_COMMENT
+FROM {source_db_name}.lineitem;
+
+---- PART
+CREATE TABLE IF NOT EXISTS {target_db_name}.part (
+  P_PARTKEY BIGINT,
+  P_NAME STRING,
+  P_MFGR STRING,
+  P_BRAND STRING,
+  P_TYPE STRING,
+  P_SIZE BIGINT,
+  P_CONTAINER STRING,
+  P_RETAILPRICE DOUBLE,
+  P_COMMENT STRING
+)
+distribute by hash (p_partkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_part',
+  'kudu.key_columns' = 'p_partkey'
+);
+
+INSERT INTO TABLE {target_db_name}.part SELECT * FROM {source_db_name}.part;
+
+---- PARTSUPP
+CREATE TABLE IF NOT EXISTS {target_db_name}.partsupp (
+  PS_PARTKEY BIGINT,
+  PS_SUPPKEY BIGINT,
+  PS_AVAILQTY BIGINT,
+  PS_SUPPLYCOST DOUBLE,
+  PS_COMMENT STRING
+)
+distribute by hash (ps_partkey, ps_suppkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_partsupp',
+  'kudu.key_columns' = 'ps_partkey, ps_suppkey'
+);
+
+INSERT INTO TABLE {target_db_name}.partsupp SELECT * FROM {source_db_name}.partsupp;
+
+---- SUPPLIER
+CREATE TABLE IF NOT EXISTS {target_db_name}.supplier (
+  S_SUPPKEY BIGINT,
+  S_NAME STRING,
+  S_ADDRESS STRING,
+  S_NATIONKEY BIGINT,
+  S_PHONE STRING,
+  S_ACCTBAL DOUBLE,
+  S_COMMENT STRING
+)
+distribute by hash (s_suppkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_supplier',
+  'kudu.key_columns' = 's_suppkey'
+);
+
+INSERT INTO TABLE {target_db_name}.supplier SELECT * FROM {source_db_name}.supplier;
+
+---- NATION
+CREATE TABLE IF NOT EXISTS {target_db_name}.nation (
+  N_NATIONKEY BIGINT,
+  N_NAME STRING,
+  N_REGIONKEY BIGINT,
+  N_COMMENT STRING
+)
+distribute by hash (n_nationkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_nation',
+  'kudu.key_columns' = 'n_nationkey'
+);
+
+INSERT INTO TABLE {target_db_name}.nation SELECT * FROM {source_db_name}.nation;
+
+---- REGION
+CREATE TABLE IF NOT EXISTS {target_db_name}.region (
+  R_REGIONKEY BIGINT,
+  R_NAME STRING,
+  R_COMMENT STRING
+)
+distribute by hash (r_regionkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_region',
+  'kudu.key_columns' = 'r_regionkey'
+);
+
+INSERT INTO TABLE {target_db_name}.region SELECT * FROM {source_db_name}.region;
+
+---- ORDERS
+CREATE TABLE IF NOT EXISTS {target_db_name}.orders (
+  O_ORDERKEY BIGINT,
+  O_CUSTKEY BIGINT,
+  O_ORDERSTATUS STRING,
+  O_TOTALPRICE DOUBLE,
+  O_ORDERDATE STRING,
+  O_ORDERPRIORITY STRING,
+  O_CLERK STRING,
+  O_SHIPPRIORITY BIGINT,
+  O_COMMENT STRING
+)
+distribute by hash (o_orderkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_orders',
+  'kudu.key_columns' = 'o_orderkey'
+);
+
+INSERT INTO TABLE {target_db_name}.orders SELECT * FROM {source_db_name}.orders;
+
+---- CUSTOMER
+CREATE TABLE IF NOT EXISTS {target_db_name}.customer (
+  C_CUSTKEY BIGINT,
+  C_NAME STRING,
+  C_ADDRESS STRING,
+  C_NATIONKEY BIGINT,
+  C_PHONE STRING,
+  C_ACCTBAL DOUBLE,
+  C_MKTSEGMENT STRING,
+  C_COMMENT STRING
+)
+distribute by hash (c_custkey) into {buckets} buckets
+tblproperties(
+  'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
+  'kudu.master_addresses' = '{kudu_master}:7051',
+  'kudu.table_name' = '{target_db_name}_customer',
+  'kudu.key_columns' = 'c_custkey'
+);
+
+INSERT INTO TABLE {target_db_name}.customer SELECT * FROM {source_db_name}.customer;
+
+---- COMPUTE STATS
+compute stats {target_db_name}.customer;
+compute stats {target_db_name}.lineitem;
+compute stats {target_db_name}.nation;
+compute stats {target_db_name}.orders;
+compute stats {target_db_name}.part;
+compute stats {target_db_name}.partsupp;
+compute stats {target_db_name}.region;
+compute stats {target_db_name}.supplier;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q19.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q19.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q19.test
new file mode 100644
index 0000000..46f358e
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q19.test
@@ -0,0 +1,39 @@
+====
+---- QUERY: TPCDS-Q19
+select
+  i_brand_id brand_id,
+  i_brand brand,
+  i_manufact_id,
+  i_manufact,
+  round(sum(ss_ext_sales_price), 2) ext_price
+from
+  date_dim,
+  store_sales,
+  item,
+  customer,
+  customer_address,
+  store
+where
+  d_date_sk = ss_sold_date_sk
+  and ss_item_sk = i_item_sk
+  and i_manager_id = 7
+  and d_moy = 11
+  and d_year = 1999
+  and ss_customer_sk = c_customer_sk
+  and c_current_addr_sk = ca_address_sk
+  and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5)
+  and ss_store_sk = s_store_sk
+  and ss_sold_date_sk between 2451484 and 2451513
+group by
+  i_brand,
+  i_brand_id,
+  i_manufact_id,
+  i_manufact
+order by
+  ext_price desc,
+  i_brand,
+  i_brand_id,
+  i_manufact_id,
+  i_manufact
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q27.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q27.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q27.test
new file mode 100644
index 0000000..f2723c8
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q27.test
@@ -0,0 +1,36 @@
+====
+---- QUERY: TPCDS-Q27
+select
+  i_item_id,
+  s_state,
+  -- grouping(s_state) g_state,
+  round(avg(ss_quantity), 2) agg1,
+  round(avg(ss_list_price), 2) agg2,
+  round(avg(ss_coupon_amt), 2) agg3,
+  round(avg(ss_sales_price), 2) agg4
+from
+  store_sales,
+  customer_demographics,
+  date_dim,
+  store,
+  item
+where
+  ss_sold_date_sk = d_date_sk
+  and ss_item_sk = i_item_sk
+  and ss_store_sk = s_store_sk
+  and ss_cdemo_sk = cd_demo_sk
+  and cd_gender = 'F'
+  and cd_marital_status = 'W'
+  and cd_education_status = 'Primary'
+  and d_year = 1998
+  and s_state in ('WI', 'CA', 'TX', 'FL', 'WA', 'TN')
+  and ss_sold_date_sk between 2450815 and 2451179  -- partition key filter
+group by
+  -- rollup (i_item_id, s_state)
+  i_item_id,
+  s_state
+order by
+  i_item_id,
+  s_state
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q3.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q3.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q3.test
new file mode 100644
index 0000000..ba1dd67
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q3.test
@@ -0,0 +1,32 @@
+====
+---- QUERY: TPCDS-Q3
+select
+  dt.d_year,
+  item.i_brand_id brand_id,
+  item.i_brand brand,
+  round(sum(ss_ext_sales_price), 2) sum_agg
+from
+  date_dim dt,
+  store_sales,
+  item
+where
+  dt.d_date_sk = store_sales.ss_sold_date_sk
+  and store_sales.ss_item_sk = item.i_item_sk
+  and item.i_manufact_id = 436
+  and dt.d_moy = 12
+  -- partition key filters
+  and (ss_sold_date_sk between 2451149 and 2451179
+    or ss_sold_date_sk between 2451514 and 2451544
+    or ss_sold_date_sk between 2451880 and 2451910
+    or ss_sold_date_sk between 2452245 and 2452275
+    or ss_sold_date_sk between 2452610 and 2452640)
+group by
+  dt.d_year,
+  item.i_brand,
+  item.i_brand_id
+order by
+  dt.d_year,
+  sum_agg desc,
+  brand_id
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q34.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q34.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q34.test
new file mode 100644
index 0000000..2e6e033
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q34.test
@@ -0,0 +1,47 @@
+====
+---- QUERY: TPCDS-Q34
+select
+  c_last_name,
+  c_first_name,
+  c_salutation,
+  c_preferred_cust_flag,
+  ss_ticket_number,
+  cnt
+from
+  (select
+    ss_ticket_number,
+    ss_customer_sk,
+    count(*) cnt
+  from
+    store_sales,
+    date_dim,
+    store,
+    household_demographics
+  where
+    store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (date_dim.d_dom between 1 and 3
+      or date_dim.d_dom between 25 and 28)
+    and (household_demographics.hd_buy_potential = '>10000'
+      or household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and (case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count / household_demographics.hd_vehicle_count else null end) > 1.2
+    and date_dim.d_year in (1998, 1998 + 1, 1998 + 2)
+    and store.s_county in ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County', 'Fairfield County', 'Raleigh County', 'Ziebach County', 'Williamson County')
+    and ss_sold_date_sk between 2450816 and 2451910 -- partition key filter
+  group by
+    ss_ticket_number,
+    ss_customer_sk
+  ) dn,
+  customer
+where
+  ss_customer_sk = c_customer_sk
+  and cnt between 15 and 20
+order by
+  c_last_name,
+  c_first_name,
+  c_salutation,
+  c_preferred_cust_flag desc
+limit 100000;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q42.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q42.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q42.test
new file mode 100644
index 0000000..6d173e3
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q42.test
@@ -0,0 +1,29 @@
+====
+---- QUERY: TPCDS-Q42
+select
+  dt.d_year,
+  item.i_category_id,
+  item.i_category,
+  round(sum(ss_ext_sales_price), 2)
+from
+  date_dim dt,
+  store_sales,
+  item
+where
+  dt.d_date_sk = store_sales.ss_sold_date_sk
+  and store_sales.ss_item_sk = item.i_item_sk
+  and item.i_manager_id = 1
+  and dt.d_moy = 12
+  and dt.d_year = 1998
+  and ss_sold_date_sk between 2451149 and 2451179  -- partition key filter
+group by
+  dt.d_year,
+  item.i_category_id,
+  item.i_category
+order by
+  round(sum(ss_ext_sales_price), 2) desc,
+  dt.d_year,
+  item.i_category_id,
+  item.i_category
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q43.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q43.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q43.test
new file mode 100644
index 0000000..5c7ccda
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q43.test
@@ -0,0 +1,37 @@
+====
+---- QUERY: TPCDS-Q43
+select
+  s_store_name,
+  s_store_id,
+  round(sum(case when (d_day_name = 'Sunday') then ss_sales_price else null end), 2) sun_sales,
+  round(sum(case when (d_day_name = 'Monday') then ss_sales_price else null end), 2) mon_sales,
+  round(sum(case when (d_day_name = 'Tuesday') then ss_sales_price else null end), 2) tue_sales,
+  round(sum(case when (d_day_name = 'Wednesday') then ss_sales_price else null end), 2) wed_sales,
+  round(sum(case when (d_day_name = 'Thursday') then ss_sales_price else null end), 2) thu_sales,
+  round(sum(case when (d_day_name = 'Friday') then ss_sales_price else null end), 2) fri_sales,
+  round(sum(case when (d_day_name = 'Saturday') then ss_sales_price else null end), 2) sat_sales
+from
+  date_dim,
+  store_sales,
+  store
+where
+  d_date_sk = ss_sold_date_sk
+  and s_store_sk = ss_store_sk
+  and s_gmt_offset = -5
+  and d_year = 1998
+  and ss_sold_date_sk between 2450816 and 2451179  -- partition key filter
+group by
+  s_store_name,
+  s_store_id
+order by
+  s_store_name,
+  s_store_id,
+  sun_sales,
+  mon_sales,
+  tue_sales,
+  wed_sales,
+  thu_sales,
+  fri_sales,
+  sat_sales
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q46.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q46.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q46.test
new file mode 100644
index 0000000..dffecc3
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q46.test
@@ -0,0 +1,81 @@
+====
+---- QUERY: TPCDS-Q46
+select
+  c_last_name,
+  c_first_name,
+  ca_city,
+  bought_city,
+  ss_ticket_number,
+  amt,
+  profit
+from
+  (select
+    ss_ticket_number,
+    ss_customer_sk,
+    ca_city bought_city,
+    round(sum(ss_coupon_amt), 2) amt,
+    round(sum(ss_net_profit), 2) profit
+  from
+    store_sales,
+    date_dim,
+    store,
+    household_demographics,
+    customer_address
+  where
+    store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    and (household_demographics.hd_dep_count = 5
+      or household_demographics.hd_vehicle_count = 3)
+    and date_dim.d_dow in (6, 0)
+    and date_dim.d_year in (1999, 1999 + 1, 1999 + 2)
+    and store.s_city in ('Midway', 'Concord', 'Spring Hill', 'Brownsville', 'Greenville')
+    -- partition key filter
+    and ss_sold_date_sk in (2451181, 2451182, 2451188, 2451189, 2451195, 2451196, 2451202, 2451203, 2451209, 2451210, 2451216, 2451217,
+                            2451223, 2451224, 2451230, 2451231, 2451237, 2451238, 2451244, 2451245, 2451251, 2451252, 2451258, 2451259,
+                            2451265, 2451266, 2451272, 2451273, 2451279, 2451280, 2451286, 2451287, 2451293, 2451294, 2451300, 2451301,
+                            2451307, 2451308, 2451314, 2451315, 2451321, 2451322, 2451328, 2451329, 2451335, 2451336, 2451342, 2451343,
+                            2451349, 2451350, 2451356, 2451357, 2451363, 2451364, 2451370, 2451371, 2451377, 2451378, 2451384, 2451385,
+                            2451391, 2451392, 2451398, 2451399, 2451405, 2451406, 2451412, 2451413, 2451419, 2451420, 2451426, 2451427,
+                            2451433, 2451434, 2451440, 2451441, 2451447, 2451448, 2451454, 2451455, 2451461, 2451462, 2451468, 2451469,
+                            2451475, 2451476, 2451482, 2451483, 2451489, 2451490, 2451496, 2451497, 2451503, 2451504, 2451510, 2451511,
+                            2451517, 2451518, 2451524, 2451525, 2451531, 2451532, 2451538, 2451539, 2451545, 2451546, 2451552, 2451553,
+                            2451559, 2451560, 2451566, 2451567, 2451573, 2451574, 2451580, 2451581, 2451587, 2451588, 2451594, 2451595,
+                            2451601, 2451602, 2451608, 2451609, 2451615, 2451616, 2451622, 2451623, 2451629, 2451630, 2451636, 2451637,
+                            2451643, 2451644, 2451650, 2451651, 2451657, 2451658, 2451664, 2451665, 2451671, 2451672, 2451678, 2451679,
+                            2451685, 2451686, 2451692, 2451693, 2451699, 2451700, 2451706, 2451707, 2451713, 2451714, 2451720, 2451721,
+                            2451727, 2451728, 2451734, 2451735, 2451741, 2451742, 2451748, 2451749, 2451755, 2451756, 2451762, 2451763,
+                            2451769, 2451770, 2451776, 2451777, 2451783, 2451784, 2451790, 2451791, 2451797, 2451798, 2451804, 2451805,
+                            2451811, 2451812, 2451818, 2451819, 2451825, 2451826, 2451832, 2451833, 2451839, 2451840, 2451846, 2451847,
+                            2451853, 2451854, 2451860, 2451861, 2451867, 2451868, 2451874, 2451875, 2451881, 2451882, 2451888, 2451889,
+                            2451895, 2451896, 2451902, 2451903, 2451909, 2451910, 2451916, 2451917, 2451923, 2451924, 2451930, 2451931,
+                            2451937, 2451938, 2451944, 2451945, 2451951, 2451952, 2451958, 2451959, 2451965, 2451966, 2451972, 2451973,
+                            2451979, 2451980, 2451986, 2451987, 2451993, 2451994, 2452000, 2452001, 2452007, 2452008, 2452014, 2452015,
+                            2452021, 2452022, 2452028, 2452029, 2452035, 2452036, 2452042, 2452043, 2452049, 2452050, 2452056, 2452057,
+                            2452063, 2452064, 2452070, 2452071, 2452077, 2452078, 2452084, 2452085, 2452091, 2452092, 2452098, 2452099,
+                            2452105, 2452106, 2452112, 2452113, 2452119, 2452120, 2452126, 2452127, 2452133, 2452134, 2452140, 2452141,
+                            2452147, 2452148, 2452154, 2452155, 2452161, 2452162, 2452168, 2452169, 2452175, 2452176, 2452182, 2452183,
+                            2452189, 2452190, 2452196, 2452197, 2452203, 2452204, 2452210, 2452211, 2452217, 2452218, 2452224, 2452225,
+                            2452231, 2452232, 2452238, 2452239, 2452245, 2452246, 2452252, 2452253, 2452259, 2452260, 2452266, 2452267,
+                            2452273, 2452274)
+  group by
+    ss_ticket_number,
+    ss_customer_sk,
+    ss_addr_sk,
+    ca_city
+  ) dn,
+  customer,
+  customer_address current_addr
+where
+  ss_customer_sk = c_customer_sk
+  and customer.c_current_addr_sk = current_addr.ca_address_sk
+  and current_addr.ca_city <> bought_city
+order by
+  c_last_name,
+  c_first_name,
+  ca_city,
+  bought_city,
+  ss_ticket_number
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q47.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q47.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q47.test
new file mode 100644
index 0000000..51305e9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q47.test
@@ -0,0 +1,53 @@
+====
+---- QUERY: TPCDS-Q47
+with v1 as (
+ select i_category, i_brand,
+        s_store_name, s_company_name,
+        d_year, d_moy,
+        sum(ss_sales_price) sum_sales,
+        avg(sum(ss_sales_price)) over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name
+           order by d_year, d_moy) rn
+ from item, store_sales, date_dim, store
+ where ss_item_sk = i_item_sk and
+       ss_sold_date_sk = d_date_sk and
+       ss_store_sk = s_store_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          s_store_name, s_company_name,
+          d_year, d_moy),
+ v2 as(
+ select v1.i_category, v1.i_brand
+        ,v1.d_year
+        ,round(v1.avg_monthly_sales, 2) avg_monthly_sales
+        ,round(v1.sum_sales, 2) sum_sales, round(v1_lag.sum_sales, 2) psum
+        ,round(v1_lead.sum_sales, 2) nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1.s_store_name = v1_lag.s_store_name and
+       v1.s_store_name = v1_lead.s_store_name and
+       v1.s_company_name = v1_lag.s_company_name and
+       v1.s_company_name = v1_lead.s_company_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+ select * from ( select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, d_year
+ limit 100
+) as v3;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q52.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q52.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q52.test
new file mode 100644
index 0000000..c3a252a
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q52.test
@@ -0,0 +1,28 @@
+====
+---- QUERY: TPCDS-Q52
+select
+  dt.d_year,
+  item.i_brand_id brand_id,
+  item.i_brand brand,
+  round(sum(ss_ext_sales_price), 2) ext_price
+from
+  date_dim dt,
+  store_sales,
+  item
+where
+  dt.d_date_sk = store_sales.ss_sold_date_sk
+  and store_sales.ss_item_sk = item.i_item_sk
+  and item.i_manager_id = 1
+  and dt.d_moy = 12
+  and dt.d_year = 1998
+  and ss_sold_date_sk between 2451149 and 2451179 -- added for partition pruning
+group by
+  dt.d_year,
+  item.i_brand,
+  item.i_brand_id
+order by
+  dt.d_year,
+  ext_price desc,
+  brand_id
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q53.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q53.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q53.test
new file mode 100644
index 0000000..a3ac8e7
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q53.test
@@ -0,0 +1,38 @@
+====
+---- QUERY: TPCDS-Q53
+select
+  *
+from
+  (select
+    i_manufact_id,
+    round(sum(ss_sales_price), 2) sum_sales
+    -- avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales
+  from
+    item,
+    store_sales,
+    date_dim,
+    store
+  where
+    ss_item_sk = i_item_sk
+    and ss_sold_date_sk = d_date_sk
+    and ss_store_sk = s_store_sk
+    and d_month_seq in (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
+    and ((i_category in ('Books', 'Children', 'Electronics')
+      and i_class in ('personal', 'portable', 'reference', 'self-help')
+      and i_brand in ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9'))
+    or (i_category in ('Women', 'Music', 'Men')
+      and i_class in ('accessories', 'classical', 'fragrances', 'pants')
+      and i_brand in ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))
+    and ss_sold_date_sk between 2451911 and 2452275 -- partition key filter
+  group by
+    i_manufact_id,
+    d_qoy
+  ) tmp1
+-- where
+--   case when avg_quarterly_sales > 0 then abs (sum_sales - avg_quarterly_sales) / avg_quarterly_sales else null end > 0.1
+order by
+  -- avg_quarterly_sales,
+  sum_sales,
+  i_manufact_id
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q55.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q55.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q55.test
new file mode 100644
index 0000000..15223c5
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q55.test
@@ -0,0 +1,25 @@
+====
+---- QUERY: TPCDS-Q55
+select
+  i_brand_id brand_id,
+  i_brand brand,
+  round(sum(ss_ext_sales_price), 2) ext_price
+from
+  date_dim,
+  store_sales,
+  item
+where
+  d_date_sk = ss_sold_date_sk
+  and ss_item_sk = i_item_sk
+  and i_manager_id = 36
+  and d_moy = 12
+  and d_year = 2001
+  and ss_sold_date_sk between 2452245 and 2452275 -- partition key filter
+group by
+  i_brand,
+  i_brand_id
+order by
+  ext_price desc,
+  i_brand_id
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q59.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q59.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q59.test
new file mode 100644
index 0000000..e4e3afb
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q59.test
@@ -0,0 +1,84 @@
+====
+---- QUERY: TPCDS-Q59
+with
+  wss as
+  (select
+    d_week_seq,
+    ss_store_sk,
+    round(sum(case when (d_day_name = 'Sunday') then ss_sales_price else null end), 2) sun_sales,
+    round(sum(case when (d_day_name = 'Monday') then ss_sales_price else null end), 2) mon_sales,
+    round(sum(case when (d_day_name = 'Tuesday') then ss_sales_price else null end), 2) tue_sales,
+    round(sum(case when (d_day_name = 'Wednesday') then ss_sales_price else null end), 2) wed_sales,
+    round(sum(case when (d_day_name = 'Thursday') then ss_sales_price else null end), 2) thu_sales,
+    round(sum(case when (d_day_name = 'Friday') then ss_sales_price else null end), 2) fri_sales,
+    round(sum(case when (d_day_name = 'Saturday') then ss_sales_price else null end), 2) sat_sales
+  from
+    store_sales,
+    date_dim
+  where
+    d_date_sk = ss_sold_date_sk
+  group by
+    d_week_seq,
+    ss_store_sk
+  )
+select
+  s_store_name1,
+  s_store_id1,
+  d_week_seq1,
+  sun_sales1 / sun_sales2,
+  mon_sales1 / mon_sales2,
+  tue_sales1 / tue_sales2,
+  wed_sales1 / wed_sales2,
+  thu_sales1 / thu_sales2,
+  fri_sales1 / fri_sales2,
+  sat_sales1 / sat_sales2
+from
+  (select
+    s_store_name s_store_name1,
+    wss.d_week_seq d_week_seq1,
+    s_store_id s_store_id1,
+    sun_sales sun_sales1,
+    mon_sales mon_sales1,
+    tue_sales tue_sales1,
+    wed_sales wed_sales1,
+    thu_sales thu_sales1,
+    fri_sales fri_sales1,
+    sat_sales sat_sales1
+  from
+    wss,
+    store,
+    date_dim d
+  where
+    d.d_week_seq = wss.d_week_seq
+    and ss_store_sk = s_store_sk
+    and d_month_seq between 1185 and 1185 + 11
+  ) y,
+  (select
+    s_store_name s_store_name2,
+    wss.d_week_seq d_week_seq2,
+    s_store_id s_store_id2,
+    sun_sales sun_sales2,
+    mon_sales mon_sales2,
+    tue_sales tue_sales2,
+    wed_sales wed_sales2,
+    thu_sales thu_sales2,
+    fri_sales fri_sales2,
+    sat_sales sat_sales2
+  from
+    wss,
+    store,
+    date_dim d
+  where
+    d.d_week_seq = wss.d_week_seq
+    and ss_store_sk = s_store_sk
+    and d_month_seq between 1185 + 12 and 1185 + 23
+  ) x
+where
+  s_store_id1 = s_store_id2
+  and d_week_seq1 = d_week_seq2 - 52
+order by
+  s_store_name1,
+  s_store_id1,
+  d_week_seq1
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q6.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q6.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q6.test
new file mode 100644
index 0000000..82e3f66
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q6.test
@@ -0,0 +1,28 @@
+====
+---- QUERY: TPCDS-Q6
+select * from (
+ select  a.ca_state state, count(*) cnt
+ from customer_address a
+     ,customer c
+     ,store_sales s
+     ,date_dim d
+     ,item i
+ where
+        a.ca_address_sk = c.c_current_addr_sk
+        and c.c_customer_sk = s.ss_customer_sk
+        and s.ss_sold_date_sk = d.d_date_sk
+        and s.ss_item_sk = i.i_item_sk
+        and d.d_month_seq =
+             (select distinct (d_month_seq)
+              from date_dim
+               where d_year = 1999
+                and d_moy = 1
+               limit 1)
+        and i.i_current_price > 1.2 *
+             (select round(avg(j.i_current_price), 2)
+             from item j
+             where j.i_category = i.i_category)
+ group by a.ca_state
+ having count(*) >= 10
+ order by cnt limit 100) as t;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q61.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q61.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q61.test
new file mode 100644
index 0000000..c31e167
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q61.test
@@ -0,0 +1,43 @@
+====
+---- QUERY: TPCDS-Q61
+select promotions,total,cast(promotions as DOUBLE)/cast(total as DOUBLE)*100
+from
+  (select round(sum(ss_ext_sales_price), 2) promotions
+   from  store_sales
+        ,store
+        ,promotion
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_promo_sk = p_promo_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -5
+   and   i_category = 'Books'
+   and   (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y')
+   and   s_gmt_offset = -5
+   and   d_year = 2000
+   and   d_moy  = 11) promotional_sales,
+  (select round(sum(ss_ext_sales_price), 2) total
+   from  store_sales
+        ,store
+        ,date_dim
+        ,customer
+        ,customer_address
+        ,item
+   where ss_sold_date_sk = d_date_sk
+   and   ss_store_sk = s_store_sk
+   and   ss_customer_sk= c_customer_sk
+   and   ca_address_sk = c_current_addr_sk
+   and   ss_item_sk = i_item_sk
+   and   ca_gmt_offset = -5
+   and   i_category = 'Books'
+   and   s_gmt_offset = -5
+   and   d_year = 2000
+   and   d_moy  = 11) all_sales
+order by promotions, total;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q63.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q63.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q63.test
new file mode 100644
index 0000000..387db25
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q63.test
@@ -0,0 +1,38 @@
+====
+---- QUERY: TPCDS-Q63
+select
+  *
+from
+  (select
+    i_manager_id,
+    round(sum(ss_sales_price), 2) sum_sales,
+    round(avg(sum(ss_sales_price)) over (partition by i_manager_id), 2) avg_monthly_sales
+  from
+    item,
+    store_sales,
+    date_dim,
+    store
+  where
+    ss_item_sk = i_item_sk
+    and ss_sold_date_sk = d_date_sk
+    and ss_store_sk = s_store_sk
+    and d_month_seq in (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
+    and ((i_category in ('Books', 'Children', 'Electronics')
+      and i_class in ('personal', 'portable', 'refernece', 'self-help')
+      and i_brand in ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9'))
+    or (i_category in ('Women', 'Music', 'Men')
+      and i_class in ('accessories', 'classical', 'fragrances', 'pants')
+      and i_brand in ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))
+    and ss_sold_date_sk between 2451911 and 2452275  -- partition key filter
+  group by
+    i_manager_id,
+    d_moy
+  ) tmp1
+where
+case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by
+  i_manager_id,
+  avg_monthly_sales,
+  sum_sales
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q65.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q65.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q65.test
new file mode 100644
index 0000000..e1d3af0
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q65.test
@@ -0,0 +1,63 @@
+====
+---- QUERY: TPCDS-Q65
+select
+  s_store_name,
+  i_item_desc,
+  sc.revenue,
+  i_current_price,
+  i_wholesale_cost,
+  i_brand
+from
+  store,
+  item,
+  (select
+    ss_store_sk,
+    round(avg(revenue), 2) as ave
+  from
+    (select
+      ss_store_sk,
+      ss_item_sk,
+      round(sum(ss_sales_price), 2) as revenue
+    from
+      store_sales,
+      date_dim
+    where
+      ss_sold_date_sk = d_date_sk
+      and d_month_seq between 1212 and 1212 + 11
+      and ss_sold_date_sk between 2451911 and 2452275  -- partition key filter
+    group by
+      ss_store_sk,
+      ss_item_sk
+    ) sa
+  group by
+    ss_store_sk
+  ) sb,
+  (select
+    ss_store_sk,
+    ss_item_sk,
+    round(sum(ss_sales_price), 2) as revenue
+  from
+    store_sales,
+    date_dim
+  where
+    ss_sold_date_sk = d_date_sk
+    and d_month_seq between 1212 and 1212 + 11
+    and ss_sold_date_sk between 2451911 and 2452275  -- partition key filter
+  group by
+    ss_store_sk,
+    ss_item_sk
+  ) sc
+where
+  sb.ss_store_sk = sc.ss_store_sk
+  and sc.revenue <= 0.1 * sb.ave
+  and s_store_sk = sc.ss_store_sk
+  and i_item_sk = sc.ss_item_sk
+order by
+  s_store_name,
+  i_item_desc,
+  sc.revenue, -- for consistent ordering of results
+  i_current_price, -- for consistent ordering of results
+  i_wholesale_cost, -- for consistent ordering of results
+  i_brand -- for consistent ordering of results
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q68.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q68.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q68.test
new file mode 100644
index 0000000..2e0f994
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q68.test
@@ -0,0 +1,62 @@
+====
+---- QUERY: TPCDS-Q68
+select
+  c_last_name,
+  c_first_name,
+  ca_city,
+  bought_city,
+  ss_ticket_number,
+  extended_price,
+  extended_tax,
+  list_price
+from
+  (select
+    ss_ticket_number,
+    ss_customer_sk,
+    ca_city bought_city,
+    round(sum(ss_ext_sales_price), 2) extended_price,
+    round(sum(ss_ext_list_price), 2) list_price,
+    round(sum(ss_ext_tax), 2) extended_tax
+  from
+    store_sales,
+    date_dim,
+    store,
+    household_demographics,
+    customer_address
+  where
+    store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and store_sales.ss_addr_sk = customer_address.ca_address_sk
+    -- and date_dim.d_dom between 1 and 2
+    and (household_demographics.hd_dep_count = 5
+      or household_demographics.hd_vehicle_count = 3)
+    -- and date_dim.d_year in (1999, 1999 + 1, 1999 + 2)
+    and store.s_city in ('Midway', 'Fairview')
+    -- partition key filter
+    -- and ss_sold_date_sk in (2451180, 2451181, 2451211, 2451212, 2451239, 2451240, 2451270, 2451271, 2451300, 2451301, 2451331,
+    --                         2451332, 2451361, 2451362, 2451392, 2451393, 2451423, 2451424, 2451453, 2451454, 2451484, 2451485,
+    --                         2451514, 2451515, 2451545, 2451546, 2451576, 2451577, 2451605, 2451606, 2451636, 2451637, 2451666,
+    --                         2451667, 2451697, 2451698, 2451727, 2451728, 2451758, 2451759, 2451789, 2451790, 2451819, 2451820,
+    --                         2451850, 2451851, 2451880, 2451881, 2451911, 2451912, 2451942, 2451943, 2451970, 2451971, 2452001,
+    --                         2452002, 2452031, 2452032, 2452062, 2452063, 2452092, 2452093, 2452123, 2452124, 2452154, 2452155,
+    --                         2452184, 2452185, 2452215, 2452216, 2452245, 2452246)
+    and ss_sold_date_sk between 2451180 and 2451269 -- partition key filter (3 months)
+    and d_date between '1999-01-01' and '1999-03-31'
+  group by
+    ss_ticket_number,
+    ss_customer_sk,
+    ss_addr_sk,
+    ca_city
+  ) dn,
+  customer,
+  customer_address current_addr
+where
+  ss_customer_sk = c_customer_sk
+  and customer.c_current_addr_sk = current_addr.ca_address_sk
+  and current_addr.ca_city <> bought_city
+order by
+  c_last_name,
+  ss_ticket_number
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q7.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q7.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q7.test
new file mode 100644
index 0000000..0130d4c
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q7.test
@@ -0,0 +1,32 @@
+====
+---- QUERY: TPCDS-Q7
+select
+  i_item_id,
+  round(avg(ss_quantity), 2) agg1,
+  round(avg(ss_list_price), 2) agg2,
+  round(avg(ss_coupon_amt), 2) agg3,
+  round(avg(ss_sales_price), 1) agg4
+from
+  store_sales,
+  customer_demographics,
+  date_dim,
+  item,
+  promotion
+where
+  ss_sold_date_sk = d_date_sk
+  and ss_item_sk = i_item_sk
+  and ss_cdemo_sk = cd_demo_sk
+  and ss_promo_sk = p_promo_sk
+  and cd_gender = 'F'
+  and cd_marital_status = 'W'
+  and cd_education_status = 'Primary'
+  and (p_channel_email = 'N'
+    or p_channel_event = 'N')
+  and d_year = 1998
+  and ss_sold_date_sk between 2450815 and 2451179 -- partition key filter
+group by
+  i_item_id
+order by
+  i_item_id
+limit 100;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q73.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q73.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q73.test
new file mode 100644
index 0000000..66e2a8f
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q73.test
@@ -0,0 +1,51 @@
+====
+---- QUERY: TPCDS-Q73
+select
+  c_last_name,
+  c_first_name,
+  c_salutation,
+  c_preferred_cust_flag,
+  ss_ticket_number,
+  cnt
+from
+  (select
+    ss_ticket_number,
+    ss_customer_sk,
+    count(*) cnt
+  from
+    store_sales,
+    date_dim,
+    store,
+    household_demographics
+  where
+    store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    -- and date_dim.d_dom between 1 and 2
+    and (household_demographics.hd_buy_potential = '>10000'
+      or household_demographics.hd_buy_potential = 'unknown')
+    and household_demographics.hd_vehicle_count > 0
+    and case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count / household_demographics.hd_vehicle_count else null end > 1
+    -- and date_dim.d_year in (1998, 1998 + 1, 1998 + 2)
+    and store.s_county in ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County')
+    -- partition key filter
+    -- and ss_sold_date_sk in (2450816, 2450846, 2450847, 2450874, 2450875, 2450905, 2450906, 2450935, 2450936, 2450966, 2450967,
+    --                         2450996, 2450997, 2451027, 2451028, 2451058, 2451059, 2451088, 2451089, 2451119, 2451120, 2451149,
+    --                         2451150, 2451180, 2451181, 2451211, 2451212, 2451239, 2451240, 2451270, 2451271, 2451300, 2451301,
+    --                         2451331, 2451332, 2451361, 2451362, 2451392, 2451393, 2451423, 2451424, 2451453, 2451454, 2451484,
+    --                         2451485, 2451514, 2451515, 2451545, 2451546, 2451576, 2451577, 2451605, 2451606, 2451636, 2451637,
+    --                         2451666, 2451667, 2451697, 2451698, 2451727, 2451728, 2451758, 2451759, 2451789, 2451790, 2451819,
+    --                         2451820, 2451850, 2451851, 2451880, 2451881)
+    and ss_sold_date_sk between 2451180 and 2451269 -- partition key filter (3 months)
+  group by
+    ss_ticket_number,
+    ss_customer_sk
+  ) dj,
+  customer
+where
+  ss_customer_sk = c_customer_sk
+  and cnt between 1 and 5
+order by
+  cnt desc
+limit 1000;
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8a49ceaa/testdata/workloads/tpcds/queries/tpcds-kudu-q79.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds/queries/tpcds-kudu-q79.test b/testdata/workloads/tpcds/queries/tpcds-kudu-q79.test
new file mode 100644
index 0000000..fcaca50
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-kudu-q79.test
@@ -0,0 +1,61 @@
+====
+---- QUERY: TPCDS-Q79
+select
+  c_last_name,
+  c_first_name,
+  substr(s_city, 1, 30),
+  ss_ticket_number,
+  amt,
+  profit
+from
+  (select
+    ss_ticket_number,
+    ss_customer_sk,
+    store.s_city,
+    round(sum(ss_coupon_amt), 2) amt,
+    round(sum(ss_net_profit), 2) profit
+  from
+    store_sales,
+    date_dim,
+    store,
+    household_demographics
+  where
+    store_sales.ss_sold_date_sk = date_dim.d_date_sk
+    and store_sales.ss_store_sk = store.s_store_sk
+    and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+    and (household_demographics.hd_dep_count = 8
+      or household_demographics.hd_vehicle_count > 0)
+    -- and date_dim.d_dow = 1
+    -- and date_dim.d_year in (1998, 1998 + 1, 1998 + 2)
+    and store.s_number_employees between 200 and 295
+    -- partition key filter
+    -- and ss_sold_date_sk in (2450819, 2450826, 2450833, 2450840, 2450847, 2450854, 2450861, 2450868, 2450875, 2450882, 2450889,
+    -- 2450896, 2450903, 2450910, 2450917, 2450924, 2450931, 2450938, 2450945, 2450952, 2450959, 2450966, 2450973, 2450980, 2450987,
+    -- 2450994, 2451001, 2451008, 2451015, 2451022, 2451029, 2451036, 2451043, 2451050, 2451057, 2451064, 2451071, 2451078, 2451085,
+    -- 2451092, 2451099, 2451106, 2451113, 2451120, 2451127, 2451134, 2451141, 2451148, 2451155, 2451162, 2451169, 2451176, 2451183,
+    -- 2451190, 2451197, 2451204, 2451211, 2451218, 2451225, 2451232, 2451239, 2451246, 2451253, 2451260, 2451267, 2451274, 2451281,
+    -- 2451288, 2451295, 2451302, 2451309, 2451316, 2451323, 2451330, 2451337, 2451344, 2451351, 2451358, 2451365, 2451372, 2451379,
+    -- 2451386, 2451393, 2451400, 2451407, 2451414, 2451421, 2451428, 2451435, 2451442, 2451449, 2451456, 2451463, 2451470, 2451477,
+    -- 2451484, 2451491, 2451498, 2451505, 2451512, 2451519, 2451526, 2451533, 2451540, 2451547, 2451554, 2451561, 2451568, 2451575,
+    -- 2451582, 2451589, 2451596, 2451603, 2451610, 2451617, 2451624, 2451631, 2451638, 2451645, 2451652, 2451659, 2451666, 2451673,
+    -- 2451680, 2451687, 2451694, 2451701, 2451708, 2451715, 2451722, 2451729, 2451736, 2451743, 2451750, 2451757, 2451764, 2451771,
+    -- 2451778, 2451785, 2451792, 2451799, 2451806, 2451813, 2451820, 2451827, 2451834, 2451841, 2451848, 2451855, 2451862, 2451869,
+    -- 2451876, 2451883, 2451890, 2451897, 2451904)
+    and d_date between '1999-01-01' and '1999-03-31'
+    and ss_sold_date_sk between 2451180 and 2451269  -- partition key filter
+  group by
+    ss_ticket_number,
+    ss_customer_sk,
+    ss_addr_sk,
+    store.s_city
+  ) ms,
+  customer
+where
+  ss_customer_sk = c_customer_sk
+order by
+  c_last_name,
+  c_first_name,
+  substr(s_city, 1, 30),
+  profit
+limit 100;
+====



[03/14] incubator-impala git commit: IMPALA-4325: StmtRewrite lost parentheses of CompoundPredicate

Posted by ta...@apache.org.
IMPALA-4325: StmtRewrite lost parentheses of CompoundPredicate

StmtRewrite lost parentheses of CompoundPredicate in pushNegationToOperands()
and leads to incorrect toSql() result. Even though this issue would not leads
to incorrect result of query, it makes user confuse of the logical operator
precedence of predicates shown in EXPLAIN statement.

Change-Id: I79bfc67605206e0e026293bf7032a88227a95623
Reviewed-on: http://gerrit.cloudera.org:8080/4753
Reviewed-by: Alex Behm <al...@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/f8d48b85
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/f8d48b85
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/f8d48b85

Branch: refs/heads/master
Commit: f8d48b8582b9e460c2e0e3dbb4881636f179ae73
Parents: 9ef9512
Author: Yuanhao Luo <lu...@software.ict.ac.cn>
Authored: Wed Oct 19 17:10:39 2016 +0800
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Oct 21 07:45:53 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/impala/analysis/Expr.java   |  6 ++++-
 .../queries/PlannerTest/subquery-rewrite.test   | 23 ++++++++++++++++++++
 2 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f8d48b85/fe/src/main/java/org/apache/impala/analysis/Expr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/Expr.java b/fe/src/main/java/org/apache/impala/analysis/Expr.java
index 6eed7a8..23ae65f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -238,6 +238,7 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
   }
   public boolean hasCost() { return evalCost_ >= 0; }
   public long getNumDistinctValues() { return numDistinctValues_; }
+  public boolean getPrintSqlInParens() { return printSqlInParens_; }
   public void setPrintSqlInParens(boolean b) { printSqlInParens_ = b; }
   public boolean isOnClauseConjunct() { return isOnClauseConjunct_; }
   public void setIsOnClauseConjunct(boolean b) { isOnClauseConjunct_ = b; }
@@ -1160,7 +1161,10 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     if (root instanceof CompoundPredicate) {
       Expr left = pushNegationToOperands(root.getChild(0));
       Expr right = pushNegationToOperands(root.getChild(1));
-      return new CompoundPredicate(((CompoundPredicate)root).getOp(), left, right);
+      CompoundPredicate compoundPredicate =
+        new CompoundPredicate(((CompoundPredicate)root).getOp(), left, right);
+      compoundPredicate.setPrintSqlInParens(root.getPrintSqlInParens());
+      return compoundPredicate;
     }
 
     return root;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f8d48b85/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
index 060d470..1f52597 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
@@ -99,6 +99,29 @@ PLAN-ROOT SINK
    predicates: bool_col = FALSE
    runtime filters: RF000 -> a.id, RF001 -> int_col
 ====
+# IMPALA-4325: Preserve parenthesis of expressions when rewriting subqueries
+select *
+from functional.alltypes t1
+where t1.int_col in
+  (select t2.int_col
+   from functional.alltypes t2
+   where (t2.int_col is not null and (t2.int_col < 0 or t2.int_col > 10)
+   or t2.bigint_col is not null and (t2.bigint_col < 0 or t2.bigint_col > 10)))
+---- PLAN
+PLAN-ROOT SINK
+|
+02:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: t1.int_col = t2.int_col
+|  runtime filters: RF000 <- t2.int_col
+|
+|--01:SCAN HDFS [functional.alltypes t2]
+|     partitions=24/24 files=24 size=478.45KB
+|     predicates: (t2.int_col IS NOT NULL AND (t2.int_col < 0 OR t2.int_col > 10) OR t2.bigint_col IS NOT NULL AND (t2.bigint_col < 0 OR t2.bigint_col > 10))
+|
+00:SCAN HDFS [functional.alltypes t1]
+   partitions=24/24 files=24 size=478.45KB
+   runtime filters: RF000 -> t1.int_col
+====
 # Complex expression in the IN predicate
 select *
 from functional.alltypes t