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/25 20:15:48 UTC

[01/33] incubator-impala git commit: IMPALA-4080, IMPALA-3638: Introduce ExecNode::Codegen()

Repository: incubator-impala
Updated Branches:
  refs/heads/hadoop-next 3cb3f34d6 -> eae4f307d


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/topn-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.h b/be/src/exec/topn-node.h
index 6c2bcae..5bd7ded 100644
--- a/be/src/exec/topn-node.h
+++ b/be/src/exec/topn-node.h
@@ -45,6 +45,7 @@ class TopNNode : public ExecNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -57,9 +58,6 @@ class TopNNode : public ExecNode {
 
   friend class TupleLessThan;
 
-  /// Creates a codegen'd version of InsertBatch() that is used in Open().
-  Status Codegen(RuntimeState* state);
-
   /// Inserts all the rows in 'batch' into the queue.
   void InsertBatch(RowBatch* batch);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/case-expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/case-expr.cc b/be/src/exprs/case-expr.cc
index 5771f09..2262c22 100644
--- a/be/src/exprs/case-expr.cc
+++ b/be/src/exprs/case-expr.cc
@@ -182,7 +182,7 @@ string CaseExpr::DebugString() const {
 //                                   %"class.impala::TupleRow"* %row)
 //   ret i16 %else_val
 // }
-Status CaseExpr::GetCodegendComputeFn(RuntimeState* state, Function** fn) {
+Status CaseExpr::GetCodegendComputeFn(LlvmCodeGen* codegen, Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
@@ -191,11 +191,9 @@ Status CaseExpr::GetCodegendComputeFn(RuntimeState* state, Function** fn) {
   const int num_children = GetNumChildren();
   Function* child_fns[num_children];
   for (int i = 0; i < num_children; ++i) {
-    RETURN_IF_ERROR(children()[i]->GetCodegendComputeFn(state, &child_fns[i]));
+    RETURN_IF_ERROR(children()[i]->GetCodegendComputeFn(codegen, &child_fns[i]));
   }
 
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   LLVMContext& context = codegen->context();
   LlvmCodeGen::LlvmBuilder builder(context);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/case-expr.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/case-expr.h b/be/src/exprs/case-expr.h
index fbf6f85..87bab76 100644
--- a/be/src/exprs/case-expr.h
+++ b/be/src/exprs/case-expr.h
@@ -30,7 +30,7 @@ class TExprNode;
 
 class CaseExpr: public Expr {
  public:
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 
   virtual BooleanVal GetBooleanVal(ExprContext* ctx, const TupleRow* row);
   virtual TinyIntVal GetTinyIntVal(ExprContext* ctx, const TupleRow* row);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/compound-predicates.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/compound-predicates.cc b/be/src/exprs/compound-predicates.cc
index 94eb269..531a70c 100644
--- a/be/src/exprs/compound-predicates.cc
+++ b/be/src/exprs/compound-predicates.cc
@@ -65,8 +65,8 @@ string OrPredicate::DebugString() const {
   return out.str();
 }
 
-// IR codegen for compound and/or predicates.  Compound predicate has non trivial 
-// null handling as well as many branches so this is pretty complicated.  The IR 
+// IR codegen for compound and/or predicates.  Compound predicate has non trivial
+// null handling as well as many branches so this is pretty complicated.  The IR
 // for x && y is:
 //
 // define i16 @CompoundPredicate(%"class.impala::ExprContext"* %context,
@@ -86,30 +86,30 @@ string OrPredicate::DebugString() const {
 //   %val2 = trunc i8 %3 to i1
 //   %tmp_and = and i1 %val, %val2
 //   br i1 %is_null, label %lhs_null, label %lhs_not_null
-// 
+//
 // lhs_null:                                         ; preds = %entry
 //   br i1 %is_null1, label %null_block, label %lhs_null_rhs_not_null
-// 
+//
 // lhs_not_null:                                     ; preds = %entry
 //   br i1 %is_null1, label %lhs_not_null_rhs_null, label %not_null_block
-// 
+//
 // lhs_null_rhs_not_null:                            ; preds = %lhs_null
 //   br i1 %val2, label %null_block, label %not_null_block
-// 
+//
 // lhs_not_null_rhs_null:                            ; preds = %lhs_not_null
 //   br i1 %val, label %null_block, label %not_null_block
-// 
+//
 // null_block:                                       ; preds = %lhs_null_rhs_not_null,
 //                                                     %lhs_not_null_rhs_null, %lhs_null
 //   br label %ret
-// 
+//
 // not_null_block:                                   ; preds = %lhs_null_rhs_not_null,
 //                                                   %lhs_not_null_rhs_null, %lhs_not_null
 //   %4 = phi i1 [ false, %lhs_null_rhs_not_null ],
 //               [ false, %lhs_not_null_rhs_null ],
 //               [ %tmp_and, %lhs_not_null ]
 //   br label %ret
-// 
+//
 // ret:                                              ; preds = %not_null_block, %null_block
 //   %ret3 = phi i1 [ false, %null_block ], [ %4, %not_null_block ]
 //   %5 = zext i1 %ret3 to i16
@@ -118,21 +118,18 @@ string OrPredicate::DebugString() const {
 //   ret i16 %7
 // }
 Status CompoundPredicate::CodegenComputeFn(
-    bool and_fn, RuntimeState* state, Function** fn) {
+    bool and_fn, LlvmCodeGen* codegen, Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
   DCHECK_EQ(GetNumChildren(), 2);
-
   Function* lhs_function;
-  RETURN_IF_ERROR(children()[0]->GetCodegendComputeFn(state, &lhs_function));
+  RETURN_IF_ERROR(children()[0]->GetCodegendComputeFn(codegen, &lhs_function));
   Function* rhs_function;
-  RETURN_IF_ERROR(children()[1]->GetCodegendComputeFn(state, &rhs_function));
-  
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
+  RETURN_IF_ERROR(children()[1]->GetCodegendComputeFn(codegen, &rhs_function));
+
   LLVMContext& context = codegen->context();
   LlvmCodeGen::LlvmBuilder builder(context);
   Value* args[2];
@@ -143,11 +140,11 @@ Status CompoundPredicate::CodegenComputeFn(
 
   // Control blocks for aggregating results
   BasicBlock* lhs_null_block = BasicBlock::Create(context, "lhs_null", function);
-  BasicBlock* lhs_not_null_block = 
+  BasicBlock* lhs_not_null_block =
       BasicBlock::Create(context, "lhs_not_null", function);
-  BasicBlock* lhs_null_rhs_not_null_block = 
+  BasicBlock* lhs_null_rhs_not_null_block =
       BasicBlock::Create(context, "lhs_null_rhs_not_null", function);
-  BasicBlock* lhs_not_null_rhs_null_block = 
+  BasicBlock* lhs_not_null_rhs_null_block =
       BasicBlock::Create(context, "lhs_not_null_rhs_null", function);
   BasicBlock* null_block = BasicBlock::Create(context, "null_block", function);
   BasicBlock* not_null_block = BasicBlock::Create(context, "not_null_block", function);
@@ -159,7 +156,7 @@ Status CompoundPredicate::CodegenComputeFn(
   // Call rhs
   CodegenAnyVal rhs_result = CodegenAnyVal::CreateCallWrapped(
       codegen, &builder, TYPE_BOOLEAN, rhs_function, args, "rhs_call");
-  
+
   Value* lhs_is_null = lhs_result.GetIsNull();
   Value* rhs_is_null = rhs_result.GetIsNull();
   Value* lhs_value = lhs_result.GetVal();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/compound-predicates.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/compound-predicates.h b/be/src/exprs/compound-predicates.h
index 3ba4ad0..0e403ba 100644
--- a/be/src/exprs/compound-predicates.h
+++ b/be/src/exprs/compound-predicates.h
@@ -34,7 +34,7 @@ class CompoundPredicate: public Predicate {
  protected:
   CompoundPredicate(const TExprNode& node) : Predicate(node) { }
 
-  Status CodegenComputeFn(bool and_fn, RuntimeState* state, llvm::Function** fn);
+  Status CodegenComputeFn(bool and_fn, LlvmCodeGen* codegen, llvm::Function** fn);
 };
 
 /// Expr for evaluating and (&&) operators
@@ -42,8 +42,8 @@ class AndPredicate: public CompoundPredicate {
  public:
   virtual impala_udf::BooleanVal GetBooleanVal(ExprContext* context, const TupleRow*);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
-    return CompoundPredicate::CodegenComputeFn(true, state, fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
+    return CompoundPredicate::CodegenComputeFn(true, codegen, fn);
   }
 
  protected:
@@ -61,8 +61,8 @@ class OrPredicate: public CompoundPredicate {
  public:
   virtual impala_udf::BooleanVal GetBooleanVal(ExprContext* context, const TupleRow*);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
-    return CompoundPredicate::CodegenComputeFn(false, state, fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
+    return CompoundPredicate::CodegenComputeFn(false, codegen, fn);
   }
 
  protected:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/conditional-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/conditional-functions.cc b/be/src/exprs/conditional-functions.cc
index eed9b8e..6f8f362 100644
--- a/be/src/exprs/conditional-functions.cc
+++ b/be/src/exprs/conditional-functions.cc
@@ -25,8 +25,8 @@ using namespace impala;
 using namespace impala_udf;
 
 #define CONDITIONAL_CODEGEN_FN(expr_class) \
-  Status expr_class::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) { \
-    return GetCodegendComputeFnWrapper(state, fn); \
+  Status expr_class::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) { \
+    return GetCodegendComputeFnWrapper(codegen, fn); \
   }
 
 CONDITIONAL_CODEGEN_FN(IsNullExpr);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/conditional-functions.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/conditional-functions.h b/be/src/exprs/conditional-functions.h
index 8783c6e..12c9b1e 100644
--- a/be/src/exprs/conditional-functions.h
+++ b/be/src/exprs/conditional-functions.h
@@ -73,7 +73,7 @@ class IsNullExpr : public Expr {
   virtual TimestampVal GetTimestampVal(ExprContext* context, const TupleRow* row);
   virtual DecimalVal GetDecimalVal(ExprContext* context, const TupleRow* row);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual std::string DebugString() const { return Expr::DebugString("IsNullExpr"); }
 
  protected:
@@ -94,7 +94,7 @@ class NullIfExpr : public Expr {
   virtual TimestampVal GetTimestampVal(ExprContext* context, const TupleRow* row);
   virtual DecimalVal GetDecimalVal(ExprContext* context, const TupleRow* row);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual std::string DebugString() const { return Expr::DebugString("NullIfExpr"); }
 
  protected:
@@ -115,7 +115,7 @@ class IfExpr : public Expr {
   virtual TimestampVal GetTimestampVal(ExprContext* context, const TupleRow* row);
   virtual DecimalVal GetDecimalVal(ExprContext* context, const TupleRow* row);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual std::string DebugString() const { return Expr::DebugString("IfExpr"); }
 
  protected:
@@ -141,7 +141,7 @@ class CoalesceExpr : public Expr {
  protected:
   friend class Expr;
   CoalesceExpr(const TExprNode& node) : Expr(node) { }
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr.cc b/be/src/exprs/expr.cc
index 0e3610d..8d21830 100644
--- a/be/src/exprs/expr.cc
+++ b/be/src/exprs/expr.cc
@@ -267,6 +267,16 @@ Status Expr::CreateExpr(ObjectPool* pool, const TExprNode& texpr_node, Expr** ex
   }
 }
 
+bool Expr::NeedCodegen(const TExpr& texpr) {
+  for (const TExprNode& texpr_node : texpr.nodes) {
+    if (texpr_node.node_type == TExprNodeType::FUNCTION_CALL && texpr_node.__isset.fn &&
+        texpr_node.fn.binary_type == TFunctionBinaryType::IR) {
+      return true;
+    }
+  }
+  return false;
+}
+
 struct MemLayoutData {
   int expr_idx;
   int byte_size;
@@ -655,13 +665,11 @@ int Expr::InlineConstants(const FunctionContext::TypeDesc& return_type,
   return replaced;
 }
 
-Status Expr::GetCodegendComputeFnWrapper(RuntimeState* state, Function** fn) {
+Status Expr::GetCodegendComputeFnWrapper(LlvmCodeGen* codegen, Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   Function* static_getval_fn = GetStaticGetValWrapper(type(), codegen);
 
   // Call it passing this as the additional first argument.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/expr.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h
index 0cdafeb..99d463d 100644
--- a/be/src/exprs/expr.h
+++ b/be/src/exprs/expr.h
@@ -173,6 +173,10 @@ class Expr {
   /// Returns the number of slots added to the vector
   virtual int GetSlotIds(std::vector<SlotId>* slot_ids) const;
 
+  /// Returns true iff the expression 'texpr' contains UDF available only as LLVM IR. In
+  /// which case, it's impossible to interpret this expression and codegen must be used.
+  static bool NeedCodegen(const TExpr& texpr);
+
   /// Create expression tree from the list of nodes contained in texpr within 'pool'.
   /// Returns the root of expression tree in 'expr' and the corresponding ExprContext in
   /// 'ctx'.
@@ -234,7 +238,7 @@ class Expr {
   //
   /// The function should evaluate this expr over 'row' and return the result as the
   /// appropriate type of AnyVal.
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) = 0;
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) = 0;
 
   /// If this expr is constant, evaluates the expr with no input row argument and returns
   /// the output. Returns NULL if the argument is not constant. The returned AnyVal* is
@@ -389,7 +393,7 @@ class Expr {
   /// functions that use the IRBuilder. It doesn't provide any performance benefit over
   /// the interpreted path.
   /// TODO: this should be replaced with fancier xcompiling infrastructure
-  Status GetCodegendComputeFnWrapper(RuntimeState* state, llvm::Function** fn);
+  Status GetCodegendComputeFnWrapper(LlvmCodeGen* codegen, llvm::Function** fn);
 
   /// Returns the IR version of the static Get*Val() wrapper function corresponding to
   /// 'type'. This is used for calling interpreted Get*Val() functions from codegen'd

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/hive-udf-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc
index c7a3f32..453f876 100644
--- a/be/src/exprs/hive-udf-call.cc
+++ b/be/src/exprs/hive-udf-call.cc
@@ -277,8 +277,8 @@ void HiveUdfCall::Close(RuntimeState* state, ExprContext* ctx,
   Expr::Close(state, ctx, scope);
 }
 
-Status HiveUdfCall::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
-  return GetCodegendComputeFnWrapper(state, fn);
+Status HiveUdfCall::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
+  return GetCodegendComputeFnWrapper(codegen, fn);
 }
 
 string HiveUdfCall::DebugString() const {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/hive-udf-call.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.h b/be/src/exprs/hive-udf-call.h
index 8a70540..74302eb 100644
--- a/be/src/exprs/hive-udf-call.h
+++ b/be/src/exprs/hive-udf-call.h
@@ -82,7 +82,7 @@ class HiveUdfCall : public Expr {
   virtual TimestampVal GetTimestampVal(ExprContext* ctx, const TupleRow*);
   virtual DecimalVal GetDecimalVal(ExprContext* ctx, const TupleRow*);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 
  protected:
   friend class Expr;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/is-not-empty-predicate.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/is-not-empty-predicate.cc b/be/src/exprs/is-not-empty-predicate.cc
index 88aac42..521ebf4 100644
--- a/be/src/exprs/is-not-empty-predicate.cc
+++ b/be/src/exprs/is-not-empty-predicate.cc
@@ -42,9 +42,9 @@ Status IsNotEmptyPredicate::Prepare(RuntimeState* state,
   return Status::OK();
 }
 
-Status IsNotEmptyPredicate::GetCodegendComputeFn(RuntimeState* state,
+Status IsNotEmptyPredicate::GetCodegendComputeFn(LlvmCodeGen* codegen,
     llvm::Function** fn) {
-  return GetCodegendComputeFnWrapper(state, fn);
+  return GetCodegendComputeFnWrapper(codegen, fn);
 }
 
 string IsNotEmptyPredicate::DebugString() const {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/is-not-empty-predicate.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/is-not-empty-predicate.h b/be/src/exprs/is-not-empty-predicate.h
index d8e15be..2454a6d 100644
--- a/be/src/exprs/is-not-empty-predicate.h
+++ b/be/src/exprs/is-not-empty-predicate.h
@@ -31,7 +31,7 @@ class IsNotEmptyPredicate: public Predicate {
  public:
   virtual Status Prepare(RuntimeState* state, const RowDescriptor& row_desc,
                          ExprContext* ctx);
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual BooleanVal GetBooleanVal(ExprContext* context, const TupleRow* row);
   virtual std::string DebugString() const;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/literal.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/literal.cc b/be/src/exprs/literal.cc
index 1d816c3..caf6c14 100644
--- a/be/src/exprs/literal.cc
+++ b/be/src/exprs/literal.cc
@@ -375,15 +375,13 @@ string Literal::DebugString() const {
 // entry:
 //   ret { i8, i64 } { i8 0, i64 10 }
 // }
-Status Literal::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
+Status Literal::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
   DCHECK_EQ(GetNumChildren(), 0);
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   Value* args[2];
   *fn = CreateIrFunctionPrototype(codegen, "Literal", &args);
   BasicBlock* entry_block = BasicBlock::Create(codegen->context(), "entry", *fn);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/literal.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/literal.h b/be/src/exprs/literal.h
index 24bc913..65219e0 100644
--- a/be/src/exprs/literal.h
+++ b/be/src/exprs/literal.h
@@ -45,7 +45,7 @@ class Literal: public Expr {
   /// Literal.
   static Literal* CreateLiteral(const ColumnType& type, const std::string& str);
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 
   virtual impala_udf::BooleanVal GetBooleanVal(ExprContext*, const TupleRow*);
   virtual impala_udf::TinyIntVal GetTinyIntVal(ExprContext*, const TupleRow*);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/null-literal.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/null-literal.cc b/be/src/exprs/null-literal.cc
index 54d6247..a8f4241 100644
--- a/be/src/exprs/null-literal.cc
+++ b/be/src/exprs/null-literal.cc
@@ -91,15 +91,13 @@ CollectionVal NullLiteral::GetCollectionVal(ExprContext* context, const TupleRow
 // entry:
 //   ret { i8, i64 } { i8 1, i64 0 }
 // }
-Status NullLiteral::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
+Status NullLiteral::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
   }
 
   DCHECK_EQ(GetNumChildren(), 0);
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   Value* args[2];
   *fn = CreateIrFunctionPrototype(codegen, "NullLiteral", &args);
   BasicBlock* entry_block = BasicBlock::Create(codegen->context(), "entry", *fn);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/null-literal.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/null-literal.h b/be/src/exprs/null-literal.h
index 9297cf6..0a0f9a1 100644
--- a/be/src/exprs/null-literal.h
+++ b/be/src/exprs/null-literal.h
@@ -28,7 +28,7 @@ class TExprNode;
 class NullLiteral: public Expr {
  public:
   NullLiteral(PrimitiveType type) : Expr(type) { }
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 
   virtual impala_udf::BooleanVal GetBooleanVal(ExprContext*, const TupleRow*);
   virtual impala_udf::TinyIntVal GetTinyIntVal(ExprContext*, const TupleRow*);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/scalar-fn-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-fn-call.cc b/be/src/exprs/scalar-fn-call.cc
index b2fce53..17cbd64 100644
--- a/be/src/exprs/scalar-fn-call.cc
+++ b/be/src/exprs/scalar-fn-call.cc
@@ -22,6 +22,11 @@
 #include <llvm/IR/Attributes.h>
 #include <llvm/ExecutionEngine/ExecutionEngine.h>
 
+#include <boost/preprocessor/punctuation/comma_if.hpp>
+#include <boost/preprocessor/repetition/repeat.hpp>
+#include <boost/preprocessor/repetition/enum_params.hpp>
+#include <boost/preprocessor/repetition/repeat_from_to.hpp>
+
 #include "codegen/codegen-anyval.h"
 #include "codegen/llvm-codegen.h"
 #include "exprs/anyval-util.h"
@@ -41,6 +46,9 @@ using namespace impala;
 using namespace impala_udf;
 using namespace strings;
 
+// Maximum number of arguments the interpretation path supports.
+#define MAX_INTERP_ARGS 20
+
 ScalarFnCall::ScalarFnCall(const TExprNode& node)
   : Expr(node),
     vararg_start_idx_(node.__isset.vararg_start_idx ?
@@ -89,33 +97,30 @@ Status ScalarFnCall::Prepare(RuntimeState* state, const RowDescriptor& desc,
       varargs_buffer_size);
 
   // Use the interpreted path and call the builtin without codegen if:
-  // 1. there are char arguments (as they aren't supported yet)
-  // OR
-  // if all of the following conditions are satisfied:
-  // 2. the codegen object hasn't been created yet.
-  // 3. the planner doesn't insist on using codegen.
-  // 4. we're calling a builtin or native UDF with <= 8 non-variadic arguments.
-  //    The templates for UDFs used in the interpretation path support up to 8
-  //    arguments only.
-  //
-  // This saves us the overhead of creating the codegen object when it's not necessary
-  // (i.e., in plan fragments with no codegen-enabled operators).
+  // 1. codegen is disabled or
+  // 2. there are char arguments (as they aren't supported yet)
   //
   // TODO: codegen for char arguments
-  // TODO: remove condition 2 above and put a flag in the RuntimeState to indicate
-  // if codegen should be enabled for the entire fragment.
-  bool skip_codegen = false;
-  if (has_char_arg_or_result) {
-    skip_codegen = true;
-  } else if (!state->codegen_created() && !state->ShouldCodegenExpr()) {
-    skip_codegen = fn_.binary_type != TFunctionBinaryType::IR && NumFixedArgs() <= 8;
-  }
-  if (skip_codegen) {
-    // Builtins with char arguments must still have <= 8 arguments.
-    // TODO: delete when we have codegen for char arguments
-    if (has_char_arg_or_result) {
-      DCHECK(NumFixedArgs() <= 8 && fn_.binary_type == TFunctionBinaryType::BUILTIN);
+  bool codegen_enabled = state->codegen_enabled();
+  if (!codegen_enabled || has_char_arg_or_result) {
+    if (fn_.binary_type == TFunctionBinaryType::IR) {
+      // CHAR or VARCHAR are not supported as input arguments or return values for UDFs.
+      DCHECK(!has_char_arg_or_result && !codegen_enabled);
+      return Status(Substitute("Cannot interpret LLVM IR UDF '$0': Codegen is needed. "
+          "Please set DISABLE_CODEGEN to false.", fn_.name.function_name));
     }
+
+    // The templates for builtin or native UDFs used in the interpretation path
+    // support up to 20 arguments only.
+    if (NumFixedArgs() > MAX_INTERP_ARGS) {
+      DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::NATIVE);
+      // CHAR or VARCHAR are not supported as input arguments or return values for UDFs.
+      DCHECK(!has_char_arg_or_result && !codegen_enabled);
+      return Status(Substitute("Cannot interpret native UDF '$0': number of arguments is "
+          "more than $1. Codegen is needed. Please set DISABLE_CODEGEN to false.",
+          fn_.name.function_name, MAX_INTERP_ARGS));
+    }
+
     Status status = LibCache::instance()->GetSoFunctionPtr(
         fn_.hdfs_location, fn_.scalar_fn.symbol, &scalar_fn_, &cache_entry_);
     if (!status.ok()) {
@@ -127,13 +132,12 @@ Status ScalarFnCall::Prepare(RuntimeState* state, const RowDescriptor& desc,
         DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::NATIVE);
         return Status(Substitute("Problem loading UDF '$0':\n$1",
             fn_.name.function_name, status.GetDetail()));
-        return status;
       }
     }
   } else {
     // If we got here, either codegen is enabled or we need codegen to run this function.
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
+    LlvmCodeGen* codegen = state->codegen();
+    DCHECK(codegen != NULL);
 
     if (fn_.binary_type == TFunctionBinaryType::IR) {
       string local_path;
@@ -146,7 +150,7 @@ Status ScalarFnCall::Prepare(RuntimeState* state, const RowDescriptor& desc,
     }
 
     llvm::Function* ir_udf_wrapper;
-    RETURN_IF_ERROR(GetCodegendComputeFn(state, &ir_udf_wrapper));
+    RETURN_IF_ERROR(GetCodegendComputeFn(codegen, &ir_udf_wrapper));
     // TODO: don't do this for child exprs
     codegen->AddFunctionToJit(ir_udf_wrapper, &scalar_fn_wrapper_);
   }
@@ -260,7 +264,7 @@ bool ScalarFnCall::IsConstant() const {
 //        i32 4,
 //        i64* inttoptr (i64 89111072 to i64*))
 //   ret { i8, double } %result
-Status ScalarFnCall::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
+Status ScalarFnCall::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (ir_compute_fn_ != NULL) {
     *fn = ir_compute_fn_;
     return Status::OK();
@@ -275,11 +279,8 @@ Status ScalarFnCall::GetCodegendComputeFn(RuntimeState* state, llvm::Function**
     }
   }
 
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
   llvm::Function* udf;
-  RETURN_IF_ERROR(GetUdf(state, &udf));
+  RETURN_IF_ERROR(GetUdf(codegen, &udf));
 
   // Create wrapper that computes args and calls UDF
   stringstream fn_name;
@@ -326,10 +327,9 @@ Status ScalarFnCall::GetCodegendComputeFn(RuntimeState* state, llvm::Function**
   for (int i = 0; i < GetNumChildren(); ++i) {
     llvm::Function* child_fn = NULL;
     vector<llvm::Value*> child_fn_args;
-    if (state->codegen_enabled()) {
-      // Set 'child_fn' to the codegen'd function, sets child_fn = NULL if codegen fails
-      children_[i]->GetCodegendComputeFn(state, &child_fn);
-    }
+    // Set 'child_fn' to the codegen'd function, sets child_fn = NULL if codegen fails
+    children_[i]->GetCodegendComputeFn(codegen, &child_fn);
+
     if (child_fn == NULL) {
       // Set 'child_fn' to the interpreted function
       child_fn = GetStaticGetValWrapper(children_[i]->type(), codegen);
@@ -401,21 +401,16 @@ Status ScalarFnCall::GetCodegendComputeFn(RuntimeState* state, llvm::Function**
   return Status::OK();
 }
 
-Status ScalarFnCall::GetUdf(RuntimeState* state, llvm::Function** udf) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
-  // from_utc_timestamp and to_utc_timestamp have inline ASM that cannot be JIT'd.
+Status ScalarFnCall::GetUdf(LlvmCodeGen* codegen, llvm::Function** udf) {
+  // from_utc_timestamp() and to_utc_timestamp() have inline ASM that cannot be JIT'd.
   // TimestampFunctions::AddSub() contains a try/catch which doesn't work in JIT'd
-  // code.  Always use the statically compiled versions of these functions so the
-  // xcompiled versions are not included in the final module to be JIT'd.
-  // TODO: fix this
+  // code. Always use the interpreted version of these functions.
+  // TODO: fix these built-in functions so we don't need 'broken_builtin' below.
   bool broken_builtin = fn_.name.function_name == "from_utc_timestamp" ||
                         fn_.name.function_name == "to_utc_timestamp" ||
                         fn_.scalar_fn.symbol.find("AddSub") != string::npos;
   if (fn_.binary_type == TFunctionBinaryType::NATIVE ||
-      (fn_.binary_type == TFunctionBinaryType::BUILTIN &&
-       (!state->codegen_enabled() || broken_builtin))) {
+      (fn_.binary_type == TFunctionBinaryType::BUILTIN && broken_builtin)) {
     // In this path, we are code that has been statically compiled to assembly.
     // This can either be a UDF implemented in a .so or a builtin using the UDF
     // interface with the code in impalad.
@@ -478,7 +473,6 @@ Status ScalarFnCall::GetUdf(RuntimeState* state, llvm::Function** udf) {
   } else if (fn_.binary_type == TFunctionBinaryType::BUILTIN) {
     // In this path, we're running a builtin with the UDF interface. The IR is
     // in the llvm module.
-    DCHECK(state->codegen_enabled());
     *udf = codegen->GetFunction(fn_.scalar_fn.symbol);
     if (*udf == NULL) {
       // Builtins symbols should exist unless there is a version mismatch.
@@ -522,8 +516,8 @@ Status ScalarFnCall::GetFunction(RuntimeState* state, const string& symbol, void
                                                   &cache_entry_);
   } else {
     DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::IR);
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
+    LlvmCodeGen* codegen = state->codegen();
+    DCHECK(codegen != NULL);
     llvm::Function* ir_fn = codegen->GetFunction(symbol);
     if (ir_fn == NULL) {
       stringstream ss;
@@ -563,122 +557,56 @@ RETURN_TYPE ScalarFnCall::InterpretEval(ExprContext* context, const TupleRow* ro
 
   if (vararg_start_idx_ == -1) {
     switch (children_.size()) {
-      case 0:
-        typedef RETURN_TYPE (*ScalarFn0)(FunctionContext*);
-        return reinterpret_cast<ScalarFn0>(scalar_fn_)(fn_ctx);
-      case 1:
-        typedef RETURN_TYPE (*ScalarFn1)(FunctionContext*, const AnyVal& a1);
-        return reinterpret_cast<ScalarFn1>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0]);
-      case 2:
-        typedef RETURN_TYPE (*ScalarFn2)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2);
-        return reinterpret_cast<ScalarFn2>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1]);
-      case 3:
-        typedef RETURN_TYPE (*ScalarFn3)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3);
-        return reinterpret_cast<ScalarFn3>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2]);
-      case 4:
-        typedef RETURN_TYPE (*ScalarFn4)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4);
-        return reinterpret_cast<ScalarFn4>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3]);
-      case 5:
-        typedef RETURN_TYPE (*ScalarFn5)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5);
-        return reinterpret_cast<ScalarFn5>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4]);
-      case 6:
-        typedef RETURN_TYPE (*ScalarFn6)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6);
-        return reinterpret_cast<ScalarFn6>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5]);
-      case 7:
-        typedef RETURN_TYPE (*ScalarFn7)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6, const AnyVal& a7);
-        return reinterpret_cast<ScalarFn7>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5], *(*input_vals)[6]);
-      case 8:
-        typedef RETURN_TYPE (*ScalarFn8)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6, const AnyVal& a7, const AnyVal& a8);
-        return reinterpret_cast<ScalarFn8>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5], *(*input_vals)[6], *(*input_vals)[7]);
+
+#define ARG_DECL_ONE(z, n, data) BOOST_PP_COMMA_IF(n) const AnyVal&
+#define ARG_DECL_LIST(n) \
+    FunctionContext* BOOST_PP_COMMA_IF(n) BOOST_PP_REPEAT(n, ARG_DECL_ONE, unused)
+#define ARG_ONE(z, n, data) BOOST_PP_COMMA_IF(n) *(*input_vals)[n]
+#define ARG_LIST(n) fn_ctx BOOST_PP_COMMA_IF(n) BOOST_PP_REPEAT(n, ARG_ONE, unused)
+
+   // Expands to code snippet like the following for X from 0 to 20:
+   // case X:
+   //     typedef RETURN_TYPE (*ScalarFnX)(FunctionContext*, const AnyVal& a1, ...,
+   //         const AnyVal& aX);
+   //     return reinterpret_cast<ScalarFnn>(scalar_fn_)(fn_ctx, *(*input_vals)[0], ...,
+   //         *(*input_vals)[X-1]);
+#define SCALAR_FN_TYPE(n) BOOST_PP_CAT(ScalarFn, n)
+#define INTERP_SCALAR_FN(z, n, unused)                                       \
+      case n:                                                                \
+        typedef RETURN_TYPE (*SCALAR_FN_TYPE(n))(ARG_DECL_LIST(n));          \
+        return reinterpret_cast<SCALAR_FN_TYPE(n)>(scalar_fn_)(ARG_LIST(n));
+
+      // Support up to MAX_INTERP_ARGS arguments in the interpretation path
+      BOOST_PP_REPEAT_FROM_TO(0, BOOST_PP_ADD(MAX_INTERP_ARGS, 1),
+          INTERP_SCALAR_FN, unused)
+
       default:
-        DCHECK(false) << "Interpreted path not implemented. We should have "
-                      << "codegen'd the wrapper";
+        DCHECK(false) << "Interpreted path not implemented.";
     }
-   } else {
+  } else {
     int num_varargs = children_.size() - NumFixedArgs();
     const AnyVal* varargs = reinterpret_cast<AnyVal*>(fn_ctx->impl()->varargs_buffer());
     switch (NumFixedArgs()) {
-      case 0:
-        typedef RETURN_TYPE (*VarargFn0)(FunctionContext*, int num_varargs,
-            const AnyVal* varargs);
-        return reinterpret_cast<VarargFn0>(scalar_fn_)(fn_ctx, num_varargs, varargs);
-      case 1:
-        typedef RETURN_TYPE (*VarargFn1)(FunctionContext*, const AnyVal& a1,
-            int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn1>(scalar_fn_)(fn_ctx, *(*input_vals)[0],
-            num_varargs, varargs);
-      case 2:
-        typedef RETURN_TYPE (*VarargFn2)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn2>(scalar_fn_)(fn_ctx, *(*input_vals)[0],
-            *(*input_vals)[1], num_varargs, varargs);
-      case 3:
-        typedef RETURN_TYPE (*VarargFn3)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn3>(scalar_fn_)(fn_ctx, *(*input_vals)[0],
-            *(*input_vals)[1], *(*input_vals)[2], num_varargs, varargs);
-      case 4:
-        typedef RETURN_TYPE (*VarargFn4)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, int num_varargs,
-            const AnyVal* varargs);
-        return reinterpret_cast<VarargFn4>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            num_varargs, varargs);
-      case 5:
-        typedef RETURN_TYPE (*VarargFn5)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn5>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], num_varargs, varargs);
-      case 6:
-        typedef RETURN_TYPE (*VarargFn6)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6, int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn6>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5], num_varargs, varargs);
-      case 7:
-        typedef RETURN_TYPE (*VarargFn7)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6, const AnyVal& a7, int num_varargs, const AnyVal* varargs);
-        return reinterpret_cast<VarargFn7>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5], *(*input_vals)[6], num_varargs, varargs);
-      case 8:
-        typedef RETURN_TYPE (*VarargFn8)(FunctionContext*, const AnyVal& a1,
-            const AnyVal& a2, const AnyVal& a3, const AnyVal& a4, const AnyVal& a5,
-            const AnyVal& a6, const AnyVal& a7, const AnyVal& a8, int num_varargs,
-            const AnyVal* varargs);
-        return reinterpret_cast<VarargFn8>(scalar_fn_)(fn_ctx,
-            *(*input_vals)[0], *(*input_vals)[1], *(*input_vals)[2], *(*input_vals)[3],
-            *(*input_vals)[4], *(*input_vals)[5], *(*input_vals)[6], *(*input_vals)[7],
+
+   // Expands to code snippet like the following for X from 0 to 20:
+   // case X:
+   //     typedef RETURN_TYPE (*VarargFnX)(FunctionContext*, const AnyVal& a1, ...,
+   //         const AnyVal& aX, int num_varargs, const AnyVal* varargs);
+   //     return reinterpret_cast<VarargFnX>(scalar_fn_)(fn_ctx, *(*input_vals)[0], ...,
+   //         *(*input_vals)[X-1], num_varargs, varargs);
+#define SCALAR_VARARG_FN_TYPE(n) BOOST_PP_CAT(VarargFn, n)
+#define INTERP_SCALAR_VARARG_FN(z, n, text)                                        \
+      case n:                                                                      \
+        typedef RETURN_TYPE (*SCALAR_VARARG_FN_TYPE(n))(ARG_DECL_LIST(n), int,     \
+            const AnyVal*);                                                        \
+        return reinterpret_cast<SCALAR_VARARG_FN_TYPE(n)>(scalar_fn_)(ARG_LIST(n), \
             num_varargs, varargs);
+
+      BOOST_PP_REPEAT_FROM_TO(0, BOOST_PP_ADD(MAX_INTERP_ARGS, 1),
+         INTERP_SCALAR_VARARG_FN, unused)
+
       default:
-        DCHECK(false) << "Interpreted path not implemented. We should have "
-                      << "codegen'd the wrapper";
+        DCHECK(false) << "Interpreted path not implemented.";
     }
   }
   return RETURN_TYPE::null();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/scalar-fn-call.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-fn-call.h b/be/src/exprs/scalar-fn-call.h
index 8cc8a19..e48b9df 100644
--- a/be/src/exprs/scalar-fn-call.h
+++ b/be/src/exprs/scalar-fn-call.h
@@ -60,7 +60,7 @@ class ScalarFnCall: public Expr {
                          ExprContext* context);
   virtual Status Open(RuntimeState* state, ExprContext* context,
       FunctionContext::FunctionStateScope scope = FunctionContext::FRAGMENT_LOCAL);
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual void Close(RuntimeState* state, ExprContext* context,
       FunctionContext::FunctionStateScope scope = FunctionContext::FRAGMENT_LOCAL);
 
@@ -107,7 +107,7 @@ class ScalarFnCall: public Expr {
   }
 
   /// Loads the native or IR function from HDFS and puts the result in *udf.
-  Status GetUdf(RuntimeState* state, llvm::Function** udf);
+  Status GetUdf(LlvmCodeGen* codegen, llvm::Function** udf);
 
   /// Loads the native or IR function 'symbol' from HDFS and puts the result in *fn.
   /// If the function is loaded from an IR module, it cannot be called until the module

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/slot-ref.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/slot-ref.cc b/be/src/exprs/slot-ref.cc
index 69f555b..0a766fa 100644
--- a/be/src/exprs/slot-ref.cc
+++ b/be/src/exprs/slot-ref.cc
@@ -155,7 +155,7 @@ string SlotRef::DebugString() const {
 //
 // TODO: We could generate a typed struct (and not a char*) for Tuple for llvm.  We know
 // the types from the TupleDesc.  It will likey make this code simpler to reason about.
-Status SlotRef::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
+Status SlotRef::GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn) {
   if (type_.type == TYPE_CHAR) {
     *fn = NULL;
     return Status("Codegen for Char not supported.");
@@ -166,9 +166,6 @@ Status SlotRef::GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn) {
   }
 
   DCHECK_EQ(GetNumChildren(), 0);
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
   // SlotRefs are based on the slot_id and tuple_idx.  Combine them to make a
   // query-wide unique id. We also need to combine whether the tuple is nullable. For
   // example, in an outer join the scan node could have the same tuple id and slot id

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/slot-ref.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/slot-ref.h b/be/src/exprs/slot-ref.h
index d3d8368..6ca3c89 100644
--- a/be/src/exprs/slot-ref.h
+++ b/be/src/exprs/slot-ref.h
@@ -43,7 +43,7 @@ class SlotRef : public Expr {
   virtual int GetSlotIds(std::vector<SlotId>* slot_ids) const;
   const SlotId& slot_id() const { return slot_id_; }
 
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
 
   virtual impala_udf::BooleanVal GetBooleanVal(ExprContext* context, const TupleRow*);
   virtual impala_udf::TinyIntVal GetTinyIntVal(ExprContext* context, const TupleRow*);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/tuple-is-null-predicate.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/tuple-is-null-predicate.cc b/be/src/exprs/tuple-is-null-predicate.cc
index dc54273..dd20e8a 100644
--- a/be/src/exprs/tuple-is-null-predicate.cc
+++ b/be/src/exprs/tuple-is-null-predicate.cc
@@ -59,9 +59,9 @@ Status TupleIsNullPredicate::Prepare(RuntimeState* state, const RowDescriptor& r
   return Status::OK();
 }
 
-Status TupleIsNullPredicate::GetCodegendComputeFn(RuntimeState* state,
-                                                  llvm::Function** fn) {
-  return GetCodegendComputeFnWrapper(state, fn);
+Status TupleIsNullPredicate::GetCodegendComputeFn(LlvmCodeGen* codegen,
+    llvm::Function** fn) {
+  return GetCodegendComputeFnWrapper(codegen, fn);
 }
 
 string TupleIsNullPredicate::DebugString() const {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exprs/tuple-is-null-predicate.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/tuple-is-null-predicate.h b/be/src/exprs/tuple-is-null-predicate.h
index 8c48c3a..e20a195 100644
--- a/be/src/exprs/tuple-is-null-predicate.h
+++ b/be/src/exprs/tuple-is-null-predicate.h
@@ -38,7 +38,7 @@ class TupleIsNullPredicate: public Predicate {
 
   virtual Status Prepare(RuntimeState* state, const RowDescriptor& row_desc,
                          ExprContext* ctx);
-  virtual Status GetCodegendComputeFn(RuntimeState* state, llvm::Function** fn);
+  virtual Status GetCodegendComputeFn(LlvmCodeGen* codegen, llvm::Function** fn);
   virtual std::string DebugString() const;
 
   virtual bool IsConstant() const { return false; }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/plan-fragment-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/plan-fragment-executor.cc b/be/src/runtime/plan-fragment-executor.cc
index 5269fe5..a791e61 100644
--- a/be/src/runtime/plan-fragment-executor.cc
+++ b/be/src/runtime/plan-fragment-executor.cc
@@ -201,10 +201,14 @@ Status PlanFragmentExecutor::PrepareInternal(const TExecPlanFragmentParams& requ
     scan_node->SetScanRanges(scan_ranges);
   }
 
+  RuntimeState* state = runtime_state_.get();
   RuntimeProfile::Counter* prepare_timer = ADD_TIMER(profile(), "ExecTreePrepareTime");
   {
     SCOPED_TIMER(prepare_timer);
-    RETURN_IF_ERROR(exec_tree_->Prepare(runtime_state_.get()));
+    // Until IMPALA-4233 is fixed, we still need to create the codegen object before
+    // Prepare() as ScalarFnCall::Prepare() may codegen.
+    if (state->codegen_enabled()) RETURN_IF_ERROR(state->CreateCodegen());
+    RETURN_IF_ERROR(exec_tree_->Prepare(state));
   }
 
   PrintVolumeIds(fragment_instance_ctx.per_node_scan_ranges);
@@ -232,6 +236,8 @@ Status PlanFragmentExecutor::PrepareInternal(const TExecPlanFragmentParams& requ
     ReleaseThreadToken();
   }
 
+  if (state->codegen_enabled()) exec_tree_->Codegen(state);
+
   // set up profile counters
   profile()->AddChild(exec_tree_->runtime_profile());
   rows_produced_counter_ =
@@ -246,12 +252,10 @@ Status PlanFragmentExecutor::PrepareInternal(const TExecPlanFragmentParams& requ
 }
 
 void PlanFragmentExecutor::OptimizeLlvmModule() {
-  if (!runtime_state_->codegen_created()) return;
-  LlvmCodeGen* codegen;
-  Status status = runtime_state_->GetCodegen(&codegen, /* initalize */ false);
-  DCHECK(status.ok());
+  if (!runtime_state_->codegen_enabled()) return;
+  LlvmCodeGen* codegen = runtime_state_->codegen();
   DCHECK(codegen != NULL);
-  status = codegen->FinalizeModule();
+  Status status = codegen->FinalizeModule();
   if (!status.ok()) {
     stringstream ss;
     ss << "Error with codegen for this query: " << status.GetDetail();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index a05b3ef..00ccea1 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -73,7 +73,6 @@ RuntimeState::RuntimeState(
     fragment_params_(fragment_params),
     now_(new TimestampValue(
         query_ctx().now_string.c_str(), query_ctx().now_string.size())),
-    codegen_expr_(false),
     profile_(obj_pool_.get(), "Fragment " + PrintId(fragment_ctx().fragment_instance_id)),
     is_cancelled_(false),
     root_node_id_(-1) {
@@ -86,7 +85,6 @@ RuntimeState::RuntimeState(const TQueryCtx& query_ctx)
     now_(new TimestampValue(query_ctx.now_string.c_str(),
         query_ctx.now_string.size())),
     exec_env_(ExecEnv::GetInstance()),
-    codegen_expr_(false),
     profile_(obj_pool_.get(), "<unnamed>"),
     is_cancelled_(false),
     root_node_id_(-1) {
@@ -286,12 +284,6 @@ Status RuntimeState::CheckQueryState() {
   return GetQueryStatus();
 }
 
-Status RuntimeState::GetCodegen(LlvmCodeGen** codegen, bool initialize) {
-  if (codegen_.get() == NULL && initialize) RETURN_IF_ERROR(CreateCodegen());
-  *codegen = codegen_.get();
-  return Status::OK();
-}
-
 void RuntimeState::AcquireReaderContext(DiskIoRequestContext* reader_context) {
   boost::lock_guard<SpinLock> l(reader_contexts_lock_);
   reader_contexts_.push_back(reader_context);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/runtime-state.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.h b/be/src/runtime/runtime-state.h
index 3496d9c..0d612db 100644
--- a/be/src/runtime/runtime-state.h
+++ b/be/src/runtime/runtime-state.h
@@ -158,9 +158,8 @@ class RuntimeState {
   /// Returns true if codegen is enabled for this query.
   bool codegen_enabled() const { return !query_options().disable_codegen; }
 
-  /// Returns true if the codegen object has been created. Note that this may return false
-  /// even when codegen is enabled if nothing has been codegen'd.
-  bool codegen_created() const { return codegen_.get() != NULL; }
+  /// Returns the LlvmCodeGen object for this fragment instance.
+  LlvmCodeGen* codegen() { return codegen_.get(); }
 
   /// Takes ownership of a scan node's reader context and plan fragment executor will call
   /// UnregisterReaderContexts() to unregister it when the fragment is closed. The IO
@@ -170,18 +169,6 @@ class RuntimeState {
   /// Unregisters all reader contexts acquired through AcquireReaderContext().
   void UnregisterReaderContexts();
 
-  /// Returns codegen_ in 'codegen'. If 'initialize' is true, codegen_ will be created if
-  /// it has not been initialized by a previous call already. If 'initialize' is false,
-  /// 'codegen' will be set to NULL if codegen_ has not been initialized.
-  Status GetCodegen(LlvmCodeGen** codegen, bool initialize = true);
-
-  /// Returns true if codegen should be used for expr evaluation in this plan fragment.
-  bool ShouldCodegenExpr() { return codegen_expr_; }
-
-  /// Records that this fragment should use codegen for expr evaluation whenever
-  /// applicable if codegen is not disabled.
-  void SetCodegenExpr() { codegen_expr_ = codegen_enabled(); }
-
   BufferedBlockMgr* block_mgr() {
     DCHECK(block_mgr_.get() != NULL);
     return block_mgr_.get();
@@ -267,6 +254,9 @@ class RuntimeState {
   /// execution doesn't continue if the query terminates abnormally.
   Status CheckQueryState();
 
+  /// Create a codegen object accessible via codegen() if it doesn't exist already.
+  Status CreateCodegen();
+
  private:
   /// Allow TestEnv to set block_mgr manually for testing.
   friend class TestEnv;
@@ -274,10 +264,6 @@ class RuntimeState {
   /// Set per-fragment state.
   Status Init(ExecEnv* exec_env);
 
-  /// Create a codegen object in codegen_. No-op if it has already been called. This is
-  /// created on first use.
-  Status CreateCodegen();
-
   /// Use a custom block manager for the query for testing purposes.
   void set_block_mgr(const std::shared_ptr<BufferedBlockMgr>& block_mgr) {
     block_mgr_ = block_mgr;
@@ -306,9 +292,6 @@ class RuntimeState {
   ExecEnv* exec_env_;
   boost::scoped_ptr<LlvmCodeGen> codegen_;
 
-  /// True if this fragment should force codegen for expr evaluation.
-  bool codegen_expr_;
-
   /// Thread resource management object for this fragment's execution.  The runtime
   /// state is responsible for returning this pool to the thread mgr.
   ThreadResourceMgr::ResourcePool* resource_pool_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/sorted-run-merger.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/sorted-run-merger.h b/be/src/runtime/sorted-run-merger.h
index dade968..e81ca88 100644
--- a/be/src/runtime/sorted-run-merger.h
+++ b/be/src/runtime/sorted-run-merger.h
@@ -92,7 +92,7 @@ class SortedRunMerger {
   std::vector<SortedRunWrapper*> min_heap_;
 
   /// Row comparator. Returns true if lhs < rhs.
-  TupleRowComparator comparator_;
+  const TupleRowComparator& comparator_;
 
   /// Descriptor for the rows provided by the input runs. Owned by the exec-node through
   /// which this merger was created.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/sorter.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/sorter.cc b/be/src/runtime/sorter.cc
index 6757be0..513aba2 100644
--- a/be/src/runtime/sorter.cc
+++ b/be/src/runtime/sorter.cc
@@ -420,7 +420,7 @@ class Sorter::TupleSorter {
   const int tuple_size_;
 
   /// Tuple comparator with method Less() that returns true if lhs < rhs.
-  const TupleRowComparator comparator_;
+  const TupleRowComparator& comparator_;
 
   /// Number of times comparator_.Less() can be invoked again before
   /// comparator_.FreeLocalAllocations() needs to be called.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/sorter.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/sorter.h b/be/src/runtime/sorter.h
index ed23b75..54e2d22 100644
--- a/be/src/runtime/sorter.h
+++ b/be/src/runtime/sorter.h
@@ -160,7 +160,7 @@ class Sorter {
   RuntimeState* const state_;
 
   /// In memory sorter and less-than comparator.
-  TupleRowComparator compare_less_than_;
+  const TupleRowComparator& compare_less_than_;
   boost::scoped_ptr<TupleSorter> in_mem_tuple_sorter_;
 
   /// Block manager object used to allocate, pin and release runs. Not owned by Sorter.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/tuple.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tuple.cc b/be/src/runtime/tuple.cc
index 656621d..7fa8b2b 100644
--- a/be/src/runtime/tuple.cc
+++ b/be/src/runtime/tuple.cc
@@ -306,19 +306,17 @@ void Tuple::MaterializeExprs(
 //   ; ----- end CodegenAnyVal::WriteToSlot() -------------------------------------------
 //   ret void
 // }
-Status Tuple::CodegenMaterializeExprs(RuntimeState* state, bool collect_string_vals,
+Status Tuple::CodegenMaterializeExprs(LlvmCodeGen* codegen, bool collect_string_vals,
     const TupleDescriptor& desc, const vector<ExprContext*>& materialize_expr_ctxs,
     MemPool* pool, Function** fn) {
   DCHECK(!collect_string_vals) << "CodegenMaterializeExprs: collect_string_vals NYI";
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   SCOPED_TIMER(codegen->codegen_timer());
   LLVMContext& context = codegen->context();
 
   // Codegen each compute function from materialize_expr_ctxs
   Function* materialize_expr_fns[materialize_expr_ctxs.size()];
   for (int i = 0; i < materialize_expr_ctxs.size(); ++i) {
-    Status status = materialize_expr_ctxs[i]->root()->GetCodegendComputeFn(state,
+    Status status = materialize_expr_ctxs[i]->root()->GetCodegendComputeFn(codegen,
         &materialize_expr_fns[i]);
     if (!status.ok()) {
       stringstream ss;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/runtime/tuple.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/tuple.h b/be/src/runtime/tuple.h
index b95492c..82efc24 100644
--- a/be/src/runtime/tuple.h
+++ b/be/src/runtime/tuple.h
@@ -167,7 +167,7 @@ class Tuple {
   /// separate functions for the non-NULL and NULL cases, i.e., the 'pool' argument of the
   /// generated function is ignored. There are two different MaterializeExprs symbols to
   /// differentiate these cases when we replace the function calls during codegen.
-  static Status CodegenMaterializeExprs(RuntimeState* state, bool collect_string_vals,
+  static Status CodegenMaterializeExprs(LlvmCodeGen* codegen, bool collect_string_vals,
       const TupleDescriptor& desc, const vector<ExprContext*>& materialize_expr_ctxs,
       MemPool* pool, llvm::Function** fn);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index 996770c..31aa488 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -84,31 +84,45 @@ Java_org_apache_impala_service_FeSupport_NativeEvalConstExprs(
 
   DeserializeThriftMsg(env, thrift_expr_batch, &expr_batch);
   DeserializeThriftMsg(env, thrift_query_ctx_bytes, &query_ctx);
-  query_ctx.request.query_options.disable_codegen = true;
+  vector<TExpr>& texprs = expr_batch.exprs;
+
+  // Codegen is almost always disabled in this path. The only exception is when the
+  // expression contains IR UDF which cannot be interpreted. Enable codegen in this
+  // case if codegen is not disabled in the query option. Otherwise, we will let it
+  // fail in ScalarFnCall::Prepare().
+  bool need_codegen = false;
+  for (const TExpr& texpr : texprs) {
+    if (Expr::NeedCodegen(texpr)) {
+      need_codegen = true;
+      break;
+    }
+  }
+  query_ctx.request.query_options.disable_codegen |= !need_codegen;
   RuntimeState state(query_ctx);
+  if (!query_ctx.request.query_options.disable_codegen) {
+    THROW_IF_ERROR_RET(
+        state.CreateCodegen(), env, JniUtil::internal_exc_class(), result_bytes);
+  }
 
   THROW_IF_ERROR_RET(jni_frame.push(env), env, JniUtil::internal_exc_class(),
-                     result_bytes);
+      result_bytes);
   // Exprs can allocate memory so we need to set up the mem trackers before
   // preparing/running the exprs.
   state.InitMemTrackers(TUniqueId(), NULL, -1);
 
-  vector<TExpr>& texprs = expr_batch.exprs;
   // Prepare the exprs
   vector<ExprContext*> expr_ctxs;
-  for (vector<TExpr>::iterator it = texprs.begin(); it != texprs.end(); it++) {
+  for (const TExpr& texpr : texprs) {
     ExprContext* ctx;
-    THROW_IF_ERROR_RET(Expr::CreateExprTree(&obj_pool, *it, &ctx), env,
-                       JniUtil::internal_exc_class(), result_bytes);
+    THROW_IF_ERROR_RET(Expr::CreateExprTree(&obj_pool, texpr, &ctx), env,
+        JniUtil::internal_exc_class(), result_bytes);
     THROW_IF_ERROR_RET(ctx->Prepare(&state, RowDescriptor(), state.query_mem_tracker()),
-                       env, JniUtil::internal_exc_class(), result_bytes);
+        env, JniUtil::internal_exc_class(), result_bytes);
     expr_ctxs.push_back(ctx);
   }
 
-  if (state.codegen_created()) {
-    // Finalize the module so any UDF functions are jit'd
-    LlvmCodeGen* codegen = NULL;
-    state.GetCodegen(&codegen, /* initialize */ false);
+  if (!query_ctx.request.query_options.disable_codegen) {
+    LlvmCodeGen* codegen = state.codegen();
     DCHECK(codegen != NULL);
     codegen->EnableOptimizations(false);
     codegen->FinalizeModule();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/service/query-options.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 0823981..eb1d2f9 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -304,6 +304,7 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_schedule_random_replica(
             iequals(value, "true") || iequals(value, "1"));
         break;
+      // TODO: remove this query option (IMPALA-4319).
       case TImpalaQueryOptions::SCAN_NODE_CODEGEN_THRESHOLD:
         query_options->__set_scan_node_codegen_threshold(atol(value.c_str()));
         break;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/testutil/test-udfs.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/test-udfs.cc b/be/src/testutil/test-udfs.cc
index fa86a4a..efdfa0a 100644
--- a/be/src/testutil/test-udfs.cc
+++ b/be/src/testutil/test-udfs.cc
@@ -333,3 +333,32 @@ IntVal EightArgs(FunctionContext* context, const IntVal& v1, const IntVal& v2,
     const IntVal& v7, const IntVal& v8) {
   return IntVal(v1.val + v2.val + v3.val + v4.val + v5.val + v6.val + v7.val + v8.val);
 }
+
+IntVal NineArgs(FunctionContext* context, const IntVal& v1, const IntVal& v2,
+    const IntVal& v3, const IntVal& v4, const IntVal& v5, const IntVal& v6,
+    const IntVal& v7, const IntVal& v8, const IntVal& v9) {
+  return IntVal(v1.val + v2.val + v3.val + v4.val + v5.val + v6.val + v7.val + v8.val +
+      v9.val);
+}
+
+IntVal TwentyArgs(FunctionContext* context, const IntVal& v1, const IntVal& v2,
+    const IntVal& v3, const IntVal& v4, const IntVal& v5, const IntVal& v6,
+    const IntVal& v7, const IntVal& v8, const IntVal& v9, const IntVal& v10,
+    const IntVal& v11, const IntVal& v12, const IntVal& v13, const IntVal& v14,
+    const IntVal& v15, const IntVal& v16, const IntVal& v17, const IntVal& v18,
+    const IntVal& v19, const IntVal& v20) {
+  return IntVal(v1.val + v2.val + v3.val + v4.val + v5.val + v6.val + v7.val + v8.val +
+      v9.val + v10.val + v11.val + v12.val + v13.val + v14.val + v15.val + v16.val +
+      v17.val + v18.val + v19.val + v20.val);
+}
+
+IntVal TwentyOneArgs(FunctionContext* context, const IntVal& v1, const IntVal& v2,
+    const IntVal& v3, const IntVal& v4, const IntVal& v5, const IntVal& v6,
+    const IntVal& v7, const IntVal& v8, const IntVal& v9, const IntVal& v10,
+    const IntVal& v11, const IntVal& v12, const IntVal& v13, const IntVal& v14,
+    const IntVal& v15, const IntVal& v16, const IntVal& v17, const IntVal& v18,
+    const IntVal& v19, const IntVal& v20, const IntVal& v21) {
+  return IntVal(v1.val + v2.val + v3.val + v4.val + v5.val + v6.val + v7.val + v8.val +
+      v9.val + v10.val + v11.val + v12.val + v13.val + v14.val + v15.val + v16.val +
+      v17.val + v18.val + v19.val + v20.val + v21.val);
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/util/tuple-row-compare.cc
----------------------------------------------------------------------
diff --git a/be/src/util/tuple-row-compare.cc b/be/src/util/tuple-row-compare.cc
index 6f68b9e..69892ae 100644
--- a/be/src/util/tuple-row-compare.cc
+++ b/be/src/util/tuple-row-compare.cc
@@ -51,10 +51,9 @@ int TupleRowComparator::CompareInterpreted(
 
 Status TupleRowComparator::Codegen(RuntimeState* state) {
   Function* fn;
-  RETURN_IF_ERROR(CodegenCompare(state, &fn));
-  LlvmCodeGen* codegen;
-  bool got_codegen = state->GetCodegen(&codegen).ok();
-  DCHECK(got_codegen);
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
+  RETURN_IF_ERROR(CodegenCompare(codegen, &fn));
   codegend_compare_fn_ = state->obj_pool()->Add(new CompareFn);
   codegen->AddFunctionToJit(fn, reinterpret_cast<void**>(codegend_compare_fn_));
   return Status::OK();
@@ -175,9 +174,7 @@ Status TupleRowComparator::Codegen(RuntimeState* state) {
 // next_key2:                                        ; preds = %rhs_non_null12, %next_key
 //   ret i32 0
 // }
-Status TupleRowComparator::CodegenCompare(RuntimeState* state, Function** fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
+Status TupleRowComparator::CodegenCompare(LlvmCodeGen* codegen, Function** fn) {
   SCOPED_TIMER(codegen->codegen_timer());
   LLVMContext& context = codegen->context();
 
@@ -188,7 +185,7 @@ Status TupleRowComparator::CodegenCompare(RuntimeState* state, Function** fn) {
   Function* key_fns[key_expr_ctxs_lhs_.size()];
   for (int i = 0; i < key_expr_ctxs_lhs_.size(); ++i) {
     Status status =
-        key_expr_ctxs_lhs_[i]->root()->GetCodegendComputeFn(state, &key_fns[i]);
+        key_expr_ctxs_lhs_[i]->root()->GetCodegendComputeFn(codegen, &key_fns[i]);
     if (!status.ok()) {
       return Status(Substitute("Could not codegen TupleRowComparator::Compare(): $0",
           status.GetDetail()));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/util/tuple-row-compare.h
----------------------------------------------------------------------
diff --git a/be/src/util/tuple-row-compare.h b/be/src/util/tuple-row-compare.h
index a1d8d72..e2beed1 100644
--- a/be/src/util/tuple-row-compare.h
+++ b/be/src/util/tuple-row-compare.h
@@ -127,7 +127,7 @@ class TupleRowComparator {
   /// Codegen Compare(). Returns a non-OK status if codegen is unsuccessful.
   /// TODO: inline this at codegen'd callsites instead of indirectly calling via function
   /// pointer.
-  Status CodegenCompare(RuntimeState* state, llvm::Function** fn);
+  Status CodegenCompare(LlvmCodeGen* codegen, llvm::Function** fn);
 
   /// References to ExprContexts managed by SortExecExprs. The lhs ExprContexts must
   /// be created and prepared before the TupleRowCompator is constructed, but the rhs

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/common/thrift/PlanNodes.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift
index 49fcfbb..6f863b2 100644
--- a/common/thrift/PlanNodes.thrift
+++ b/common/thrift/PlanNodes.thrift
@@ -196,17 +196,14 @@ struct THdfsScanNode {
   // Option to control tie breaks during scan scheduling.
   4: optional bool random_replica
 
-  // Option to control whether codegen should be used for conjuncts evaluation.
-  5: optional bool codegen_conjuncts
-
   // Number of header lines to skip at the beginning of each file of this table. Only set
   // for hdfs text files.
-  6: optional i32 skip_header_line_count
+  5: optional i32 skip_header_line_count
 
   // Indicates whether the MT scan node implementation should be used.
   // If this is true then the MT_DOP query option must be > 0.
   // TODO: Remove this option when the MT scan node supports all file formats.
-  7: optional bool use_mt_scan_node
+  6: optional bool use_mt_scan_node
 }
 
 struct TDataSourceScanNode {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 3d52aa4..95d86e3 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -105,9 +105,6 @@ public class HdfsScanNode extends ScanNode {
   // Total number of bytes from partitions_
   private long totalBytes_ = 0;
 
-  // True if this scan node should use codegen for evaluting conjuncts.
-  private boolean codegenConjuncts_;
-
   // True if this scan node should use the MT implementation in the backend.
   private boolean useMtScanNode_;
 
@@ -188,9 +185,6 @@ public class HdfsScanNode extends ScanNode {
 
     // TODO: do we need this?
     assignedConjuncts_ = analyzer.getAssignedConjuncts();
-
-    // Decide whether codegen should be used for evaluating conjuncts.
-    checkForCodegen(analyzer);
   }
 
   /**
@@ -505,39 +499,6 @@ public class HdfsScanNode extends ScanNode {
         " clusterNodes=" + cluster.numNodes());
   }
 
-  /**
-   * Approximate the cost of evaluating all conjuncts bound by this node by
-   * aggregating total number of nodes in expression trees of all conjuncts.
-   */
-  private int computeConjunctsCost() {
-    int cost = 0;
-    for (Expr expr: getConjuncts()) {
-      cost += expr.numNodes();
-    }
-    for (List<Expr> exprs: collectionConjuncts_.values()) {
-      for (Expr expr: exprs) {
-        cost += expr.numNodes();
-      }
-    }
-    return cost;
-  }
-
-  /**
-   * Scan node is not a codegen-enabled operator. Decide whether to use codegen for
-   * conjuncts evaluation by estimating the cost of interpretation.
-   */
-  private void checkForCodegen(Analyzer analyzer) {
-    long conjunctsCost = computeConjunctsCost();
-    long inputCardinality = getInputCardinality();
-    long threshold =
-        analyzer.getQueryCtx().getRequest().query_options.scan_node_codegen_threshold;
-    if (inputCardinality == -1) {
-      codegenConjuncts_ = conjunctsCost > 0;
-    } else {
-      codegenConjuncts_ = inputCardinality * conjunctsCost > threshold;
-    }
-  }
-
   @Override
   protected void toThrift(TPlanNode msg) {
     msg.hdfs_scan_node = new THdfsScanNode(desc_.getId().asInt());
@@ -546,7 +507,6 @@ public class HdfsScanNode extends ScanNode {
     }
     msg.hdfs_scan_node.setRandom_replica(randomReplica_);
     msg.node_type = TPlanNodeType.HDFS_SCAN_NODE;
-    msg.hdfs_scan_node.setCodegen_conjuncts(codegenConjuncts_);
     if (!collectionConjuncts_.isEmpty()) {
       Map<Integer, List<TExpr>> tcollectionConjuncts = Maps.newLinkedHashMap();
       for (Map.Entry<TupleDescriptor, List<Expr>> entry:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
index f3ff3f9..948d584 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
@@ -24,12 +24,60 @@ symbol='FnDoesNotExist';
 Could not load binary: $FILESYSTEM_PREFIX/test-warehouse/not-a-real-file.so
 ====
 ---- QUERY
+# This test is run with codegen disabled. Interpretation only handles up to 20 arguments.
+create function if not exists udf_test_errors.twenty_args(int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int, int, int, int) returns int
+location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
+symbol='TwentyArgs';
+---- RESULTS
+====
+---- QUERY
+# Verifies that interpretation can support up to 20 arguments
+select udf_test_errors.twenty_args(1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20);
+---- TYPES
+INT
+---- RESULTS
+210
+====
+---- QUERY
+# This test is run with codegen disabled. Interpretation only handles up to 20 arguments.
+create function if not exists udf_test_errors.twenty_one_args(int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int, int, int, int, int) returns int
+location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
+symbol='TwentyOneArgs';
+---- RESULTS
+====
+---- QUERY
+# Verifies that interpretation fails with more than 20 arguments.
+select udf_test_errors.twenty_one_args(1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21);
+---- CATCH
+Cannot interpret native UDF 'twenty_one_args': number of arguments is more than 20. Codegen is needed. Please set DISABLE_CODEGEN to false.
+====
+---- QUERY
+# This test is run with codegen disabled. IR UDF will fail.
+create function if not exists udf_test_errors.nine_args_ir(int, int, int, int, int, int,
+    int, int, int) returns int
+location '$FILESYSTEM_PREFIX/test-warehouse/test-udfs.ll'
+symbol='NineArgs';
+---- RESULTS
+====
+---- QUERY
+select udf_test_errors.nine_args_ir(1,2,3,4,5,6,7,8,9);
+---- CATCH
+Cannot interpret LLVM IR UDF 'nine_args_ir': Codegen is needed. Please set DISABLE_CODEGEN to false.
+====
+---- QUERY
 drop database udf_test_errors;
 ---- CATCH
 Cannot drop non-empty database: udf_test_errors
 ====
 ---- QUERY
 drop function udf_test_errors.hive_pi();
+drop function udf_test_errors.twenty_args(int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int, int);
+drop function udf_test_errors.twenty_one_args(int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int, int, int);
+drop function udf_test_errors.nine_args_ir(int, int, int, int, int, int, int, int, int);
 drop database udf_test_errors;
 ---- RESULTS
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/testdata/workloads/functional-query/queries/QueryTest/udf.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf.test b/testdata/workloads/functional-query/queries/QueryTest/udf.test
index 645c321..5cbbecb 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf.test
@@ -529,3 +529,17 @@ INT
 ---- RESULTS
 36
 ====
+---- QUERY
+select twenty_args(1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20);
+---- TYPES
+INT
+---- RESULTS
+210
+====
+---- QUERY
+select twenty_one_args(1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21);
+---- TYPES
+INT
+---- RESULTS
+231
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 02bdf4f..2658351 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -73,6 +73,11 @@ class TestUdfs(ImpalaTestSuite):
       self.run_test_case('QueryTest/udf-init-close', vector, use_db=database)
 
   def test_udf_errors(self, vector):
+    # Disable codegen to force interpretation path to be taken.
+    # Aim to exercise two failure cases:
+    # 1. too many arguments
+    # 2. IR UDF
+    vector.get_value('exec_option')['disable_codegen'] = 1
     self.run_test_case('QueryTest/udf-errors', vector)
 
   def test_udf_invalid_symbol(self, vector):
@@ -371,6 +376,11 @@ drop function if exists {database}.five_args(int, int, int, int, int);
 drop function if exists {database}.six_args(int, int, int, int, int, int);
 drop function if exists {database}.seven_args(int, int, int, int, int, int, int);
 drop function if exists {database}.eight_args(int, int, int, int, int, int, int, int);
+drop function if exists {database}.nine_args(int, int, int, int, int, int, int, int, int);
+drop function if exists {database}.twenty_args(int, int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int);
+drop function if exists {database}.twenty_one_args(int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int, int, int);
 
 create database if not exists {database};
 
@@ -493,4 +503,12 @@ location '{location}' symbol='SevenArgs';
 
 create function {database}.eight_args(int, int, int, int, int, int, int, int) returns int
 location '{location}' symbol='EightArgs';
+
+create function {database}.twenty_args(int, int, int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int) returns int
+location '{location}' symbol='TwentyArgs';
+
+create function {database}.twenty_one_args(int, int, int, int, int, int, int, int, int, int,
+    int, int, int, int, int, int, int, int, int, int, int) returns int
+location '{location}' symbol='TwentyOneArgs';
 """


[02/33] incubator-impala git commit: IMPALA-4080, IMPALA-3638: Introduce ExecNode::Codegen()

Posted by ta...@apache.org.
IMPALA-4080, IMPALA-3638: Introduce ExecNode::Codegen()

This patch is mostly mechanical move of codegen related logic
from each exec node's Prepare() to its Codegen() function.
After this change, code generation will no longer happen in
Prepare(). Instead, it will happen after Prepare() completes in
PlanFragmentExecutor. This is an intermediate step towards the
final goal of sharing compiled code among fragment instances in
multi-threading.

As part of the clean up, this change also removes the logic for
lazy codegen object creation. In other words, if codegen is enabled,
the codegen object will always be created. This simplifies some
of the logic in ScalarFnCall::Prepare() and various Codegen()
functions by reducing error checking needed. This change also
removes the logic added for tackling IMPALA-1755 as it's not
needed anymore after the clean up.

The clean up also rectifies a not so well documented situation.
Previously, even if a user explicitly sets DISABLE_CODEGEN to true,
we may still codegen a UDF if it was written in LLVM IR or if it
has more than 8 arguments. This patch enforces the query option
by failing the query in both cases. To run the query, the user
must enable codegen. This change also extends the number of
arguments supported in the interpretation path of ScalarFn to 20.

Change-Id: I207566bc9f4c6a159271ecdbc4bbdba3d78c6651
Reviewed-on: http://gerrit.cloudera.org:8080/4651
Reviewed-by: Michael Ho <kw...@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/b15d992a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/b15d992a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/b15d992a

Branch: refs/heads/hadoop-next
Commit: b15d992abe09bc841f6e2112d47099eb15f8454f
Parents: ee2a06d
Author: Michael Ho <kw...@cloudera.com>
Authored: Wed Oct 5 20:05:24 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Oct 19 08:18:37 2016 +0000

----------------------------------------------------------------------
 be/src/exec/aggregation-node.cc                 |  61 +++--
 be/src/exec/aggregation-node.h                  |  11 +-
 be/src/exec/exchange-node.cc                    |   5 +-
 be/src/exec/exchange-node.h                     |   4 +
 be/src/exec/exec-node.cc                        |  21 +-
 be/src/exec/exec-node.h                         |  13 +-
 be/src/exec/hash-join-node.cc                   |  51 ++--
 be/src/exec/hash-join-node.h                    |   6 +-
 be/src/exec/hash-table.cc                       |  22 +-
 be/src/exec/hash-table.h                        |   8 +-
 be/src/exec/hdfs-avro-scanner.cc                |  16 +-
 be/src/exec/hdfs-avro-scanner.h                 |   2 +-
 be/src/exec/hdfs-parquet-scanner.cc             |  10 +-
 be/src/exec/hdfs-scan-node-base.cc              |  28 +--
 be/src/exec/hdfs-scan-node-base.h               |   1 +
 be/src/exec/hdfs-scanner.cc                     |   2 +-
 be/src/exec/hdfs-sequence-scanner.cc            |   8 +-
 be/src/exec/hdfs-text-scanner.cc                |   8 +-
 be/src/exec/old-hash-table.cc                   |  30 +--
 be/src/exec/old-hash-table.h                    |   6 +-
 be/src/exec/partitioned-aggregation-node.cc     |  76 +++---
 be/src/exec/partitioned-aggregation-node.h      |  13 +-
 be/src/exec/partitioned-hash-join-builder.cc    |  83 +++----
 be/src/exec/partitioned-hash-join-builder.h     |  19 +-
 be/src/exec/partitioned-hash-join-node.cc       |  56 +++--
 be/src/exec/partitioned-hash-join-node.h        |   4 +-
 be/src/exec/sort-node.cc                        |  23 +-
 be/src/exec/sort-node.h                         |   4 +
 be/src/exec/topn-node.cc                        |  94 +++----
 be/src/exec/topn-node.h                         |   4 +-
 be/src/exprs/case-expr.cc                       |   6 +-
 be/src/exprs/case-expr.h                        |   2 +-
 be/src/exprs/compound-predicates.cc             |  37 ++-
 be/src/exprs/compound-predicates.h              |  10 +-
 be/src/exprs/conditional-functions.cc           |   4 +-
 be/src/exprs/conditional-functions.h            |   8 +-
 be/src/exprs/expr.cc                            |  14 +-
 be/src/exprs/expr.h                             |   8 +-
 be/src/exprs/hive-udf-call.cc                   |   4 +-
 be/src/exprs/hive-udf-call.h                    |   2 +-
 be/src/exprs/is-not-empty-predicate.cc          |   4 +-
 be/src/exprs/is-not-empty-predicate.h           |   2 +-
 be/src/exprs/literal.cc                         |   4 +-
 be/src/exprs/literal.h                          |   2 +-
 be/src/exprs/null-literal.cc                    |   4 +-
 be/src/exprs/null-literal.h                     |   2 +-
 be/src/exprs/scalar-fn-call.cc                  | 246 +++++++------------
 be/src/exprs/scalar-fn-call.h                   |   4 +-
 be/src/exprs/slot-ref.cc                        |   5 +-
 be/src/exprs/slot-ref.h                         |   2 +-
 be/src/exprs/tuple-is-null-predicate.cc         |   6 +-
 be/src/exprs/tuple-is-null-predicate.h          |   2 +-
 be/src/runtime/plan-fragment-executor.cc        |  16 +-
 be/src/runtime/runtime-state.cc                 |   8 -
 be/src/runtime/runtime-state.h                  |  27 +-
 be/src/runtime/sorted-run-merger.h              |   2 +-
 be/src/runtime/sorter.cc                        |   2 +-
 be/src/runtime/sorter.h                         |   2 +-
 be/src/runtime/tuple.cc                         |   6 +-
 be/src/runtime/tuple.h                          |   2 +-
 be/src/service/fe-support.cc                    |  36 ++-
 be/src/service/query-options.cc                 |   1 +
 be/src/testutil/test-udfs.cc                    |  29 +++
 be/src/util/tuple-row-compare.cc                |  13 +-
 be/src/util/tuple-row-compare.h                 |   2 +-
 common/thrift/PlanNodes.thrift                  |   7 +-
 .../org/apache/impala/planner/HdfsScanNode.java |  40 ---
 .../queries/QueryTest/udf-errors.test           |  48 ++++
 .../functional-query/queries/QueryTest/udf.test |  14 ++
 tests/query_test/test_udfs.py                   |  18 ++
 70 files changed, 647 insertions(+), 693 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node.cc b/be/src/exec/aggregation-node.cc
index 909d42b..67928ca 100644
--- a/be/src/exec/aggregation-node.cc
+++ b/be/src/exec/aggregation-node.cc
@@ -165,25 +165,29 @@ Status AggregationNode::Prepare(RuntimeState* state) {
     hash_tbl_->Insert(singleton_intermediate_tuple_);
     output_iterator_ = hash_tbl_->Begin();
   }
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
+  return Status::OK();
+}
 
+void AggregationNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
   bool codegen_enabled = false;
-  if (state->codegen_enabled()) {
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
-    Function* update_tuple_fn = CodegenUpdateTuple(state);
-    if (update_tuple_fn != NULL) {
-      codegen_process_row_batch_fn_ =
-          CodegenProcessRowBatch(state, update_tuple_fn);
-      if (codegen_process_row_batch_fn_ != NULL) {
-        // Update to using codegen'd process row batch.
-        codegen->AddFunctionToJit(codegen_process_row_batch_fn_,
-            reinterpret_cast<void**>(&process_row_batch_fn_));
-        codegen_enabled = true;
-      }
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
+  Function* update_tuple_fn = CodegenUpdateTuple(codegen);
+  if (update_tuple_fn != NULL) {
+    codegen_process_row_batch_fn_ = CodegenProcessRowBatch(codegen, update_tuple_fn);
+    if (codegen_process_row_batch_fn_ != NULL) {
+      // Update to using codegen'd process row batch.
+      codegen->AddFunctionToJit(codegen_process_row_batch_fn_,
+          reinterpret_cast<void**>(&process_row_batch_fn_));
+      codegen_enabled = true;
     }
   }
   runtime_profile()->AddCodegenMsg(codegen_enabled);
-  return Status::OK();
+  ExecNode::Codegen(state);
 }
 
 Status AggregationNode::Open(RuntimeState* state) {
@@ -525,11 +529,8 @@ IRFunction::Type GetHllUpdateFunction2(const ColumnType& type) {
 // ret:                                              ; preds = %src_not_null, %entry
 //   ret void
 // }
-llvm::Function* AggregationNode::CodegenUpdateSlot(
-    RuntimeState* state, AggFnEvaluator* evaluator, SlotDescriptor* slot_desc) {
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
-
+llvm::Function* AggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
+    AggFnEvaluator* evaluator, SlotDescriptor* slot_desc) {
   // TODO: Fix this DCHECK and Init() once CodegenUpdateSlot() can handle AggFnEvaluator
   // with multiple input expressions (e.g. group_concat).
   DCHECK_EQ(evaluator->input_expr_ctxs().size(), 1);
@@ -538,7 +539,7 @@ llvm::Function* AggregationNode::CodegenUpdateSlot(
   // TODO: implement timestamp
   if (input_expr->type().type == TYPE_TIMESTAMP) return NULL;
   Function* agg_expr_fn;
-  Status status = input_expr->GetCodegendComputeFn(state, &agg_expr_fn);
+  Status status = input_expr->GetCodegendComputeFn(codegen, &agg_expr_fn);
   if (!status.ok()) {
     VLOG_QUERY << "Could not codegen UpdateSlot(): " << status.GetDetail();
     return NULL;
@@ -715,9 +716,7 @@ llvm::Function* AggregationNode::CodegenUpdateSlot(
 //                           %"class.impala::TupleRow"* %tuple_row)
 //   ret void
 // }
-Function* AggregationNode::CodegenUpdateTuple(RuntimeState* state) {
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
+Function* AggregationNode::CodegenUpdateTuple(LlvmCodeGen* codegen) {
   SCOPED_TIMER(codegen->codegen_timer());
 
   int j = probe_expr_ctxs_.size();
@@ -805,7 +804,7 @@ Function* AggregationNode::CodegenUpdateTuple(RuntimeState* state) {
       Value* count_inc = builder.CreateAdd(slot_loaded, const_one, "count_star_inc");
       builder.CreateStore(count_inc, slot_ptr);
     } else {
-      Function* update_slot_fn = CodegenUpdateSlot(state, evaluator, slot_desc);
+      Function* update_slot_fn = CodegenUpdateSlot(codegen, evaluator, slot_desc);
       if (update_slot_fn == NULL) return NULL;
       // Call GetAggFnCtx() to get the function context.
       Value* get_fn_ctx_args[] = { this_arg, codegen->GetIntConstant(TYPE_INT, i) };
@@ -824,10 +823,8 @@ Function* AggregationNode::CodegenUpdateTuple(RuntimeState* state) {
   return codegen->FinalizeFunction(fn);
 }
 
-Function* AggregationNode::CodegenProcessRowBatch(
-    RuntimeState* state, Function* update_tuple_fn) {
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
+Function* AggregationNode::CodegenProcessRowBatch(LlvmCodeGen* codegen,
+    Function* update_tuple_fn) {
   SCOPED_TIMER(codegen->codegen_timer());
   DCHECK(update_tuple_fn != NULL);
 
@@ -847,19 +844,19 @@ Function* AggregationNode::CodegenProcessRowBatch(
     // Aggregation w/o grouping does not use a hash table.
 
     // Codegen for hash
-    Function* hash_fn = hash_tbl_->CodegenHashCurrentRow(state);
+    Function* hash_fn = hash_tbl_->CodegenHashCurrentRow(codegen);
     if (hash_fn == NULL) return NULL;
 
     // Codegen HashTable::Equals
-    Function* equals_fn = hash_tbl_->CodegenEquals(state);
+    Function* equals_fn = hash_tbl_->CodegenEquals(codegen);
     if (equals_fn == NULL) return NULL;
 
     // Codegen for evaluating build rows
-    Function* eval_build_row_fn = hash_tbl_->CodegenEvalTupleRow(state, true);
+    Function* eval_build_row_fn = hash_tbl_->CodegenEvalTupleRow(codegen, true);
     if (eval_build_row_fn == NULL) return NULL;
 
     // Codegen for evaluating probe rows
-    Function* eval_probe_row_fn = hash_tbl_->CodegenEvalTupleRow(state, false);
+    Function* eval_probe_row_fn = hash_tbl_->CodegenEvalTupleRow(codegen, false);
     if (eval_probe_row_fn == NULL) return NULL;
 
     // Replace call sites

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/aggregation-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node.h b/be/src/exec/aggregation-node.h
index 5d87d82..6f4867b 100644
--- a/be/src/exec/aggregation-node.h
+++ b/be/src/exec/aggregation-node.h
@@ -55,6 +55,7 @@ class AggregationNode : public ExecNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -159,16 +160,16 @@ class AggregationNode : public ExecNode {
   /// IR and loaded into the codegen object.  UpdateAggTuple has also been
   /// codegen'd to IR.  This function will modify the loop subsituting the
   /// UpdateAggTuple function call with the (inlined) codegen'd 'update_tuple_fn'.
-  llvm::Function* CodegenProcessRowBatch(
-      RuntimeState* state, llvm::Function* update_tuple_fn);
+  llvm::Function* CodegenProcessRowBatch(LlvmCodeGen* codegen,
+      llvm::Function* update_tuple_fn);
 
   /// Codegen for updating aggregate_exprs at slot_idx. Returns NULL if unsuccessful.
   /// slot_idx is the idx into aggregate_exprs_ (does not include grouping exprs).
-  llvm::Function* CodegenUpdateSlot(
-      RuntimeState* state, AggFnEvaluator* evaluator, SlotDescriptor* slot_desc);
+  llvm::Function* CodegenUpdateSlot(LlvmCodeGen* codegen,
+      AggFnEvaluator* evaluator, SlotDescriptor* slot_desc);
 
   /// Codegen UpdateTuple(). Returns NULL if codegen is unsuccessful.
-  llvm::Function* CodegenUpdateTuple(RuntimeState* state);
+  llvm::Function* CodegenUpdateTuple(LlvmCodeGen* codegen);
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/exchange-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.cc b/be/src/exec/exchange-node.cc
index 833949b..fac2ff5 100644
--- a/be/src/exec/exchange-node.cc
+++ b/be/src/exec/exchange-node.cc
@@ -83,6 +83,8 @@ Status ExchangeNode::Prepare(RuntimeState* state) {
     RETURN_IF_ERROR(sort_exec_exprs_.Prepare(
         state, row_descriptor_, row_descriptor_, expr_mem_tracker()));
     AddExprCtxsToFree(sort_exec_exprs_);
+    less_than_.reset(
+        new TupleRowComparator(sort_exec_exprs_, is_asc_order_, nulls_first_));
   }
   return Status::OK();
 }
@@ -92,10 +94,9 @@ Status ExchangeNode::Open(RuntimeState* state) {
   RETURN_IF_ERROR(ExecNode::Open(state));
   if (is_merging_) {
     RETURN_IF_ERROR(sort_exec_exprs_.Open(state));
-    TupleRowComparator less_than(sort_exec_exprs_, is_asc_order_, nulls_first_);
     // CreateMerger() will populate its merging heap with batches from the stream_recvr_,
     // so it is not necessary to call FillInputRowBatch().
-    RETURN_IF_ERROR(stream_recvr_->CreateMerger(less_than));
+    RETURN_IF_ERROR(stream_recvr_->CreateMerger(*less_than_.get()));
   } else {
     RETURN_IF_ERROR(FillInputRowBatch(state));
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/exchange-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.h b/be/src/exec/exchange-node.h
index 16e9137..f6face4 100644
--- a/be/src/exec/exchange-node.h
+++ b/be/src/exec/exchange-node.h
@@ -27,6 +27,7 @@ namespace impala {
 
 class RowBatch;
 class DataStreamRecvr;
+class TupleRowComparator;
 
 /// Receiver node for data streams. The data stream receiver is created in Prepare()
 /// and closed in Close().
@@ -94,6 +95,9 @@ class ExchangeNode : public ExecNode {
   /// underlying stream_recvr_, and input_batch_ is not used/valid.
   bool is_merging_;
 
+  /// The TupleRowComparator based on 'sort_exec_exprs_' for merging exchange.
+  boost::scoped_ptr<TupleRowComparator> less_than_;
+
   /// Sort expressions and parameters passed to the merging receiver..
   SortExecExprs sort_exec_exprs_;
   std::vector<bool> is_asc_order_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index df491dd..510a7d9 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -158,13 +158,20 @@ Status ExecNode::Prepare(RuntimeState* state) {
 
   RETURN_IF_ERROR(Expr::Prepare(conjunct_ctxs_, state, row_desc(), expr_mem_tracker()));
   AddExprCtxsToFree(conjunct_ctxs_);
-
   for (int i = 0; i < children_.size(); ++i) {
     RETURN_IF_ERROR(children_[i]->Prepare(state));
   }
   return Status::OK();
 }
 
+void ExecNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  DCHECK(state->codegen() != NULL);
+  for (int i = 0; i < children_.size(); ++i) {
+    children_[i]->Codegen(state);
+  }
+}
+
 Status ExecNode::Open(RuntimeState* state) {
   RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::OPEN, state));
   return Expr::Open(conjunct_ctxs_, state);
@@ -272,12 +279,6 @@ Status ExecNode::CreateNode(ObjectPool* pool, const TPlanNode& tnode,
             || state->query_options().num_scanner_threads == 1);
         *node = pool->Add(new HdfsScanNode(pool, tnode, descs));
       }
-      // If true, this node requests codegen over interpretation for conjuncts
-      // evaluation whenever possible. Turn codegen on for expr evaluation for
-      // the entire fragment.
-      if (tnode.hdfs_scan_node.codegen_conjuncts) state->SetCodegenExpr();
-      (*node)->runtime_profile()->AddCodegenMsg(
-          state->ShouldCodegenExpr(), "", "Expr Evaluation");
       break;
     case TPlanNodeType::HBASE_SCAN_NODE:
       *node = pool->Add(new HBaseScanNode(pool, tnode, descs));
@@ -492,15 +493,13 @@ void ExecNode::AddExprCtxsToFree(const SortExecExprs& sort_exec_exprs) {
 // false:                                            ; preds = %continue, %entry
 //   ret i1 false
 // }
-Status ExecNode::CodegenEvalConjuncts(RuntimeState* state,
+Status ExecNode::CodegenEvalConjuncts(LlvmCodeGen* codegen,
     const vector<ExprContext*>& conjunct_ctxs, Function** fn, const char* name) {
   Function* conjunct_fns[conjunct_ctxs.size()];
   for (int i = 0; i < conjunct_ctxs.size(); ++i) {
     RETURN_IF_ERROR(
-        conjunct_ctxs[i]->root()->GetCodegendComputeFn(state, &conjunct_fns[i]));
+        conjunct_ctxs[i]->root()->GetCodegendComputeFn(codegen, &conjunct_fns[i]));
   }
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
 
   // Construct function signature to match
   // bool EvalConjuncts(Expr** exprs, int num_exprs, TupleRow* row)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/exec-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index 9283a8b..4f3f3fc 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -63,12 +63,17 @@ class ExecNode {
   /// Sets up internal structures, etc., without doing any actual work.
   /// Must be called prior to Open(). Will only be called once in this
   /// node's lifetime.
-  /// All code generation (adding functions to the LlvmCodeGen object) must happen
-  /// in Prepare().  Retrieving the jit compiled function pointer must happen in
-  /// Open().
   /// If overridden in subclass, must first call superclass's Prepare().
   virtual Status Prepare(RuntimeState* state);
 
+  /// Recursively calls Codegen() on all children.
+  /// Expected to be overriden in subclass to generate LLVM IR functions and register
+  /// them with the LlvmCodeGen object. The function pointers of the compiled IR functions
+  /// will be set up in PlanFragmentExecutor::Open(). If overridden in subclass, must also
+  /// call superclass's Codegen() before or after the code generation for this exec node.
+  /// Will only be called once in the node's lifetime.
+  virtual void Codegen(RuntimeState* state);
+
   /// Performs any preparatory work prior to calling GetNext().
   /// Caller must not be holding any io buffers. This will cause deadlock.
   /// If overridden in subclass, must first call superclass's Open().
@@ -146,7 +151,7 @@ class ExecNode {
   /// Codegen EvalConjuncts(). Returns a non-OK status if the function couldn't be
   /// codegen'd. The codegen'd version uses inlined, codegen'd GetBooleanVal() functions.
   static Status CodegenEvalConjuncts(
-      RuntimeState* state, const std::vector<ExprContext*>& conjunct_ctxs,
+      LlvmCodeGen* codegen, const std::vector<ExprContext*>& conjunct_ctxs,
       llvm::Function** fn, const char* name = "EvalConjuncts");
 
   /// Returns a string representation in DFS order of the plan rooted at this.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hash-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-join-node.cc b/be/src/exec/hash-join-node.cc
index 9106159..f2ac928 100644
--- a/be/src/exec/hash-join-node.cc
+++ b/be/src/exec/hash-join-node.cc
@@ -143,19 +143,24 @@ Status HashJoinNode::Prepare(RuntimeState* state) {
           child(1)->row_desc().tuple_descriptors().size(), stores_nulls,
           is_not_distinct_from_, state->fragment_hash_seed(), mem_tracker(), filters_));
   build_pool_.reset(new MemPool(mem_tracker()));
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
+  return Status::OK();
+}
 
+void HashJoinNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
   bool build_codegen_enabled = false;
   bool probe_codegen_enabled = false;
-  if (state->codegen_enabled()) {
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
-    // Codegen for hashing rows
-    Function* hash_fn = hash_tbl_->CodegenHashCurrentRow(state);
-    if (hash_fn == NULL) return Status::OK();
 
+  // Codegen for hashing rows
+  Function* hash_fn = hash_tbl_->CodegenHashCurrentRow(codegen);
+  if (hash_fn != NULL) {
     // Codegen for build path
-    codegen_process_build_batch_fn_ = CodegenProcessBuildBatch(state, hash_fn);
+    codegen_process_build_batch_fn_ = CodegenProcessBuildBatch(codegen, hash_fn);
     if (codegen_process_build_batch_fn_ != NULL) {
       codegen->AddFunctionToJit(codegen_process_build_batch_fn_,
           reinterpret_cast<void**>(&process_build_batch_fn_));
@@ -164,7 +169,8 @@ Status HashJoinNode::Prepare(RuntimeState* state) {
 
     // Codegen for probe path (only for left joins)
     if (!match_all_build_) {
-      Function* codegen_process_probe_batch_fn = CodegenProcessProbeBatch(state, hash_fn);
+      Function* codegen_process_probe_batch_fn =
+          CodegenProcessProbeBatch(codegen, hash_fn);
       if (codegen_process_probe_batch_fn != NULL) {
         codegen->AddFunctionToJit(codegen_process_probe_batch_fn,
             reinterpret_cast<void**>(&process_probe_batch_fn_));
@@ -174,7 +180,7 @@ Status HashJoinNode::Prepare(RuntimeState* state) {
   }
   runtime_profile()->AddCodegenMsg(build_codegen_enabled, "", "Build Side");
   runtime_profile()->AddCodegenMsg(probe_codegen_enabled, "", "Probe Side");
-  return Status::OK();
+  ExecNode::Codegen(state);
 }
 
 Status HashJoinNode::Reset(RuntimeState* state) {
@@ -591,18 +597,15 @@ Function* HashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen) {
   return codegen->FinalizeFunction(fn);
 }
 
-Function* HashJoinNode::CodegenProcessBuildBatch(RuntimeState* state,
+Function* HashJoinNode::CodegenProcessBuildBatch(LlvmCodeGen* codegen,
     Function* hash_fn) {
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
-
   // Get cross compiled function
   Function* process_build_batch_fn =
       codegen->GetFunction(IRFunction::HASH_JOIN_PROCESS_BUILD_BATCH, true);
   DCHECK(process_build_batch_fn != NULL);
 
   // Codegen for evaluating build rows
-  Function* eval_row_fn = hash_tbl_->CodegenEvalTupleRow(state, true);
+  Function* eval_row_fn = hash_tbl_->CodegenEvalTupleRow(codegen, true);
   if (eval_row_fn == NULL) return NULL;
 
   int replaced = codegen->ReplaceCallSites(process_build_batch_fn, eval_row_fn,
@@ -615,36 +618,34 @@ Function* HashJoinNode::CodegenProcessBuildBatch(RuntimeState* state,
   return codegen->FinalizeFunction(process_build_batch_fn);
 }
 
-Function* HashJoinNode::CodegenProcessProbeBatch(RuntimeState* state, Function* hash_fn) {
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
-
+Function* HashJoinNode::CodegenProcessProbeBatch(LlvmCodeGen* codegen,
+    Function* hash_fn) {
   // Get cross compiled function
   Function* process_probe_batch_fn =
       codegen->GetFunction(IRFunction::HASH_JOIN_PROCESS_PROBE_BATCH, true);
   DCHECK(process_probe_batch_fn != NULL);
 
-  // Codegen HashTable::Equals
-  Function* equals_fn = hash_tbl_->CodegenEquals(state);
+  // Codegen HashTable::Equals()
+  Function* equals_fn = hash_tbl_->CodegenEquals(codegen);
   if (equals_fn == NULL) return NULL;
 
   // Codegen for evaluating build rows
-  Function* eval_row_fn = hash_tbl_->CodegenEvalTupleRow(state, false);
+  Function* eval_row_fn = hash_tbl_->CodegenEvalTupleRow(codegen, false);
   if (eval_row_fn == NULL) return NULL;
 
-  // Codegen CreateOutputRow
+  // Codegen CreateOutputRow()
   Function* create_output_row_fn = CodegenCreateOutputRow(codegen);
   if (create_output_row_fn == NULL) return NULL;
 
   // Codegen evaluating other join conjuncts
   Function* eval_other_conjuncts_fn;
-  Status status = ExecNode::CodegenEvalConjuncts(state, other_join_conjunct_ctxs_,
+  Status status = ExecNode::CodegenEvalConjuncts(codegen, other_join_conjunct_ctxs_,
       &eval_other_conjuncts_fn, "EvalOtherConjuncts");
   if (!status.ok()) return NULL;
 
   // Codegen evaluating conjuncts
   Function* eval_conjuncts_fn;
-  status = ExecNode::CodegenEvalConjuncts(state, conjunct_ctxs_, &eval_conjuncts_fn);
+  status = ExecNode::CodegenEvalConjuncts(codegen, conjunct_ctxs_, &eval_conjuncts_fn);
   if (!status.ok()) return NULL;
 
   // Replace all call sites with codegen version

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hash-join-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-join-node.h b/be/src/exec/hash-join-node.h
index 2e5ca6e..e65dc16 100644
--- a/be/src/exec/hash-join-node.h
+++ b/be/src/exec/hash-join-node.h
@@ -54,6 +54,7 @@ class HashJoinNode : public BlockingJoinNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -63,7 +64,6 @@ class HashJoinNode : public BlockingJoinNode {
 
  protected:
   virtual void AddToDebugString(int indentation_level, std::stringstream* out) const;
-
   virtual Status ProcessBuildInput(RuntimeState* state);
 
  private:
@@ -144,13 +144,13 @@ class HashJoinNode : public BlockingJoinNode {
   /// hash_fn is the codegen'd function for computing hashes over tuple rows in the
   /// hash table.
   /// Returns NULL if codegen was not possible.
-  llvm::Function* CodegenProcessBuildBatch(RuntimeState* state, llvm::Function* hash_fn);
+  llvm::Function* CodegenProcessBuildBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn);
 
   /// Codegen processing probe batches.  Identical signature to ProcessProbeBatch.
   /// hash_fn is the codegen'd function for computing hashes over tuple rows in the
   /// hash table.
   /// Returns NULL if codegen was not possible.
-  llvm::Function* CodegenProcessProbeBatch(RuntimeState* state, llvm::Function* hash_fn);
+  llvm::Function* CodegenProcessProbeBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn);
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index 6626b33..c39e9e9 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -684,7 +684,7 @@ static void CodegenAssignNullValue(LlvmCodeGen* codegen,
 // Both the null and not null branch into the continue block.  The continue block
 // becomes the start of the next block for codegen (either the next expr or just the
 // end of the function).
-Status HashTableCtx::CodegenEvalRow(RuntimeState* state, bool build, Function** fn) {
+Status HashTableCtx::CodegenEvalRow(LlvmCodeGen* codegen, bool build, Function** fn) {
   // TODO: CodegenAssignNullValue() can't handle TYPE_TIMESTAMP or TYPE_DECIMAL yet
   const vector<ExprContext*>& ctxs = build ? build_expr_ctxs_ : probe_expr_ctxs_;
   for (int i = 0; i < ctxs.size(); ++i) {
@@ -695,9 +695,6 @@ Status HashTableCtx::CodegenEvalRow(RuntimeState* state, bool build, Function**
     }
   }
 
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
   // Get types to generate function prototype
   Type* this_type = codegen->GetType(HashTableCtx::LLVM_CLASS_NAME);
   DCHECK(this_type != NULL);
@@ -746,7 +743,7 @@ Status HashTableCtx::CodegenEvalRow(RuntimeState* state, bool build, Function**
 
     // Call expr
     Function* expr_fn;
-    Status status = ctxs[i]->root()->GetCodegendComputeFn(state, &expr_fn);
+    Status status = ctxs[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
     if (!status.ok()) {
       (*fn)->eraseFromParent(); // deletes function
       *fn = NULL;
@@ -837,7 +834,7 @@ Status HashTableCtx::CodegenEvalRow(RuntimeState* state, bool build, Function**
 //   %hash_phi = phi i32 [ %string_hash, %not_null ], [ %str_null, %null ]
 //   ret i32 %hash_phi
 // }
-Status HashTableCtx::CodegenHashRow(RuntimeState* state, bool use_murmur, Function** fn) {
+Status HashTableCtx::CodegenHashRow(LlvmCodeGen* codegen, bool use_murmur, Function** fn) {
   for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
     // Disable codegen for CHAR
     if (build_expr_ctxs_[i]->root()->type().type == TYPE_CHAR) {
@@ -845,9 +842,6 @@ Status HashTableCtx::CodegenHashRow(RuntimeState* state, bool use_murmur, Functi
     }
   }
 
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-
   // Get types to generate function prototype
   Type* this_type = codegen->GetType(HashTableCtx::LLVM_CLASS_NAME);
   DCHECK(this_type != NULL);
@@ -1044,7 +1038,7 @@ Status HashTableCtx::CodegenHashRow(RuntimeState* state, bool use_murmur, Functi
 //        %"struct.impala_udf::StringVal"* %8, %"struct.impala::StringValue"* %row_val8)
 //   br i1 %cmp_raw10, label %continue3, label %false_block
 // }
-Status HashTableCtx::CodegenEquals(RuntimeState* state, bool force_null_equality,
+Status HashTableCtx::CodegenEquals(LlvmCodeGen* codegen, bool force_null_equality,
     Function** fn) {
   for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
     // Disable codegen for CHAR
@@ -1053,8 +1047,6 @@ Status HashTableCtx::CodegenEquals(RuntimeState* state, bool force_null_equality
     }
   }
 
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   // Get types to generate function prototype
   Type* this_type = codegen->GetType(HashTableCtx::LLVM_CLASS_NAME);
   DCHECK(this_type != NULL);
@@ -1091,7 +1083,7 @@ Status HashTableCtx::CodegenEquals(RuntimeState* state, bool force_null_equality
 
     // call GetValue on build_exprs[i]
     Function* expr_fn;
-    Status status = build_expr_ctxs_[i]->root()->GetCodegendComputeFn(state, &expr_fn);
+    Status status = build_expr_ctxs_[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
     if (!status.ok()) {
       (*fn)->eraseFromParent(); // deletes function
       *fn = NULL;
@@ -1164,11 +1156,9 @@ Status HashTableCtx::CodegenEquals(RuntimeState* state, bool force_null_equality
   return Status::OK();
 }
 
-Status HashTableCtx::ReplaceHashTableConstants(RuntimeState* state,
+Status HashTableCtx::ReplaceHashTableConstants(LlvmCodeGen* codegen,
     bool stores_duplicates, int num_build_tuples, Function* fn,
     HashTableReplacedConstants* replacement_counts) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
 
   replacement_counts->stores_nulls = codegen->ReplaceCallSitesWithBoolConst(
       fn, stores_nulls(), "stores_nulls");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index 4edd130..404b294 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -177,20 +177,20 @@ class HashTableCtx {
   /// Codegen for evaluating a tuple row. Codegen'd function matches the signature
   /// for EvalBuildRow and EvalTupleRow.
   /// If build_row is true, the codegen uses the build_exprs, otherwise the probe_exprs.
-  Status CodegenEvalRow(RuntimeState* state, bool build_row, llvm::Function** fn);
+  Status CodegenEvalRow(LlvmCodeGen* codegen, bool build_row, llvm::Function** fn);
 
   /// Codegen for evaluating a TupleRow and comparing equality. Function signature
   /// matches HashTable::Equals(). 'force_null_equality' is true if the generated
   /// equality function should treat all NULLs as equal. See the template parameter
   /// to HashTable::Equals().
-  Status CodegenEquals(RuntimeState* state, bool force_null_equality,
+  Status CodegenEquals(LlvmCodeGen* codegen, bool force_null_equality,
       llvm::Function** fn);
 
   /// Codegen for hashing expr values. Function prototype matches HashRow identically.
   /// Unlike HashRow(), the returned function only uses a single hash function, rather
   /// than switching based on level_. If 'use_murmur' is true, murmur hash is used,
   /// otherwise CRC is used if the hardware supports it (see hash-util.h).
-  Status CodegenHashRow(RuntimeState* state, bool use_murmur, llvm::Function** fn);
+  Status CodegenHashRow(LlvmCodeGen* codegen, bool use_murmur, llvm::Function** fn);
 
   /// Struct that returns the number of constants replaced by ReplaceConstants().
   struct HashTableReplacedConstants {
@@ -204,7 +204,7 @@ class HashTableCtx {
   /// Replace hash table parameters with constants in 'fn'. Updates 'replacement_counts'
   /// with the number of replacements made. 'num_build_tuples' and 'stores_duplicates'
   /// correspond to HashTable parameters with the same name.
-  Status ReplaceHashTableConstants(RuntimeState* state, bool stores_duplicates,
+  Status ReplaceHashTableConstants(LlvmCodeGen* codegen, bool stores_duplicates,
       int num_build_tuples, llvm::Function* fn,
       HashTableReplacedConstants* replacement_counts);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-avro-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.cc b/be/src/exec/hdfs-avro-scanner.cc
index f511b12..88d6d3a 100644
--- a/be/src/exec/hdfs-avro-scanner.cc
+++ b/be/src/exec/hdfs-avro-scanner.cc
@@ -80,15 +80,13 @@ Status HdfsAvroScanner::Open(ScannerContext* context) {
 Status HdfsAvroScanner::Codegen(HdfsScanNodeBase* node,
     const vector<ExprContext*>& conjunct_ctxs, Function** decode_avro_data_fn) {
   *decode_avro_data_fn = NULL;
-  if (!node->runtime_state()->codegen_enabled()) {
-    return Status("Disabled by query option.");
-  }
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(node->runtime_state()->GetCodegen(&codegen));
+  DCHECK(node->runtime_state()->codegen_enabled());
+  LlvmCodeGen* codegen = node->runtime_state()->codegen();
+  DCHECK(codegen != NULL);
   Function* materialize_tuple_fn;
   RETURN_IF_ERROR(CodegenMaterializeTuple(node, codegen, &materialize_tuple_fn));
   DCHECK(materialize_tuple_fn != NULL);
-  RETURN_IF_ERROR(CodegenDecodeAvroData(node->runtime_state(), materialize_tuple_fn,
+  RETURN_IF_ERROR(CodegenDecodeAvroData(codegen, materialize_tuple_fn,
       conjunct_ctxs, decode_avro_data_fn));
   DCHECK(*decode_avro_data_fn != NULL);
   return Status::OK();
@@ -1016,11 +1014,9 @@ Status HdfsAvroScanner::CodegenReadScalar(const AvroSchemaElement& element,
   return Status::OK();
 }
 
-Status HdfsAvroScanner::CodegenDecodeAvroData(RuntimeState* state,
+Status HdfsAvroScanner::CodegenDecodeAvroData(LlvmCodeGen* codegen,
     Function* materialize_tuple_fn, const vector<ExprContext*>& conjunct_ctxs,
     Function** decode_avro_data_fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
   SCOPED_TIMER(codegen->codegen_timer());
   DCHECK(materialize_tuple_fn != NULL);
 
@@ -1030,7 +1026,7 @@ Status HdfsAvroScanner::CodegenDecodeAvroData(RuntimeState* state,
   DCHECK_EQ(replaced, 1);
 
   Function* eval_conjuncts_fn;
-  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(state, conjunct_ctxs,
+  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, conjunct_ctxs,
       &eval_conjuncts_fn));
 
   replaced = codegen->ReplaceCallSites(fn, eval_conjuncts_fn, "EvalConjuncts");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-avro-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.h b/be/src/exec/hdfs-avro-scanner.h
index 274cf4d..595a733 100644
--- a/be/src/exec/hdfs-avro-scanner.h
+++ b/be/src/exec/hdfs-avro-scanner.h
@@ -197,7 +197,7 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   /// Produces a version of DecodeAvroData that uses codegen'd instead of interpreted
   /// functions. Stores the resulting function in 'decode_avro_data_fn' if codegen was
   /// successful or returns an error.
-  static Status CodegenDecodeAvroData(RuntimeState* state,
+  static Status CodegenDecodeAvroData(LlvmCodeGen* codegen,
       llvm::Function* materialize_tuple_fn,
       const std::vector<ExprContext*>& conjunct_ctxs,
       llvm::Function** decode_avro_data_fn);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 86dbbf8..e91a7ec 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -620,13 +620,9 @@ int HdfsParquetScanner::TransferScratchTuples(RowBatch* dst_batch) {
 
 Status HdfsParquetScanner::Codegen(HdfsScanNodeBase* node,
     const vector<ExprContext*>& conjunct_ctxs, Function** process_scratch_batch_fn) {
+  DCHECK(node->runtime_state()->codegen_enabled());
   *process_scratch_batch_fn = NULL;
-  if (!node->runtime_state()->codegen_enabled()) {
-    return Status("Disabled by query option.");
-  }
-
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(node->runtime_state()->GetCodegen(&codegen));
+  LlvmCodeGen* codegen = node->runtime_state()->codegen();
   DCHECK(codegen != NULL);
   SCOPED_TIMER(codegen->codegen_timer());
 
@@ -634,7 +630,7 @@ Status HdfsParquetScanner::Codegen(HdfsScanNodeBase* node,
   DCHECK(fn != NULL);
 
   Function* eval_conjuncts_fn;
-  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(node->runtime_state(), conjunct_ctxs,
+  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, conjunct_ctxs,
       &eval_conjuncts_fn));
   DCHECK(eval_conjuncts_fn != NULL);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 4acf3f5..cf6708c 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -146,7 +146,6 @@ Status HdfsScanNodeBase::Init(const TPlanNode& tnode, RuntimeState* state) {
   // Add row batch conjuncts
   DCHECK(conjuncts_map_[tuple_id_].empty());
   conjuncts_map_[tuple_id_] = conjunct_ctxs_;
-
   return Status::OK();
 }
 
@@ -293,10 +292,15 @@ Status HdfsScanNodeBase::Prepare(RuntimeState* state) {
   UpdateHdfsSplitStats(*scan_range_params_, &per_volume_stats);
   PrintHdfsSplitStats(per_volume_stats, &str);
   runtime_profile()->AddInfoString(HDFS_SPLIT_STATS_DESC, str.str());
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
+  return Status::OK();
+}
 
+void HdfsScanNodeBase::Codegen(RuntimeState* state) {
   // Create codegen'd functions
-  for (int format = THdfsFileFormat::TEXT;
-       format <= THdfsFileFormat::PARQUET; ++format) {
+  for (int format = THdfsFileFormat::TEXT; format <= THdfsFileFormat::PARQUET; ++format) {
     vector<HdfsFileDesc*>& file_descs =
         per_type_files_[static_cast<THdfsFileFormat::type>(format)];
 
@@ -332,20 +336,16 @@ Status HdfsScanNodeBase::Prepare(RuntimeState* state) {
         status = Status("Not implemented for this format.");
     }
     DCHECK(fn != NULL || !status.ok());
-
     const char* format_name = _THdfsFileFormat_VALUES_TO_NAMES.find(format)->second;
-    if (!status.ok()) {
-      runtime_profile()->AddCodegenMsg(false, status, format_name);
-    } else {
-      runtime_profile()->AddCodegenMsg(true, status, format_name);
-      LlvmCodeGen* codegen;
-      RETURN_IF_ERROR(runtime_state_->GetCodegen(&codegen));
-      codegen->AddFunctionToJit(
-          fn, &codegend_fn_map_[static_cast<THdfsFileFormat::type>(format)]);
+    if (status.ok()) {
+      LlvmCodeGen* codegen = state->codegen();
+      DCHECK(codegen != NULL);
+      codegen->AddFunctionToJit(fn,
+          &codegend_fn_map_[static_cast<THdfsFileFormat::type>(format)]);
     }
+    runtime_profile()->AddCodegenMsg(status.ok(), status, format_name);
   }
-
-  return Status::OK();
+  ExecNode::Codegen(state);
 }
 
 Status HdfsScanNodeBase::Open(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-scan-node-base.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.h b/be/src/exec/hdfs-scan-node-base.h
index 7ea4b9d..3531c9e 100644
--- a/be/src/exec/hdfs-scan-node-base.h
+++ b/be/src/exec/hdfs-scan-node-base.h
@@ -119,6 +119,7 @@ class HdfsScanNodeBase : public ScanNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status Reset(RuntimeState* state);
   virtual void Close(RuntimeState* state);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index 81542ec..0b6e8c5 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -530,7 +530,7 @@ Status HdfsScanner::CodegenWriteCompleteTuple(HdfsScanNodeBase* node,
       parse_block = BasicBlock::Create(context, "parse", fn, eval_fail_block);
       Function* conjunct_fn;
       Status status =
-          conjunct_ctxs[conjunct_idx]->root()->GetCodegendComputeFn(state, &conjunct_fn);
+          conjunct_ctxs[conjunct_idx]->root()->GetCodegendComputeFn(codegen, &conjunct_fn);
       if (!status.ok()) {
         stringstream ss;
         ss << "Failed to codegen conjunct: " << status.GetDetail();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-sequence-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-sequence-scanner.cc b/be/src/exec/hdfs-sequence-scanner.cc
index 03a07ce..fd552be 100644
--- a/be/src/exec/hdfs-sequence-scanner.cc
+++ b/be/src/exec/hdfs-sequence-scanner.cc
@@ -55,11 +55,9 @@ HdfsSequenceScanner::~HdfsSequenceScanner() {
 Status HdfsSequenceScanner::Codegen(HdfsScanNodeBase* node,
     const vector<ExprContext*>& conjunct_ctxs, Function** write_aligned_tuples_fn) {
   *write_aligned_tuples_fn = NULL;
-  if (!node->runtime_state()->codegen_enabled()) {
-    return Status("Disabled by query option.");
-  }
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(node->runtime_state()->GetCodegen(&codegen));
+  DCHECK(node->runtime_state()->codegen_enabled());
+  LlvmCodeGen* codegen = node->runtime_state()->codegen();
+  DCHECK(codegen != NULL);
   Function* write_complete_tuple_fn;
   RETURN_IF_ERROR(CodegenWriteCompleteTuple(node, codegen, conjunct_ctxs,
       &write_complete_tuple_fn));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/hdfs-text-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-text-scanner.cc b/be/src/exec/hdfs-text-scanner.cc
index f3f0a7c..cc63408 100644
--- a/be/src/exec/hdfs-text-scanner.cc
+++ b/be/src/exec/hdfs-text-scanner.cc
@@ -696,11 +696,9 @@ Status HdfsTextScanner::CheckForSplitDelimiter(bool* split_delimiter) {
 Status HdfsTextScanner::Codegen(HdfsScanNodeBase* node,
     const vector<ExprContext*>& conjunct_ctxs, Function** write_aligned_tuples_fn) {
   *write_aligned_tuples_fn = NULL;
-  if (!node->runtime_state()->codegen_enabled()) {
-    return Status("Disabled by query option.");
-  }
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(node->runtime_state()->GetCodegen(&codegen));
+  DCHECK(node->runtime_state()->codegen_enabled());
+  LlvmCodeGen* codegen = node->runtime_state()->codegen();
+  DCHECK(codegen != NULL);
   Function* write_complete_tuple_fn;
   RETURN_IF_ERROR(CodegenWriteCompleteTuple(node, codegen, conjunct_ctxs,
       &write_complete_tuple_fn));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/old-hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table.cc b/be/src/exec/old-hash-table.cc
index 0e38924..db89782 100644
--- a/be/src/exec/old-hash-table.cc
+++ b/be/src/exec/old-hash-table.cc
@@ -255,7 +255,7 @@ static void CodegenAssignNullValue(LlvmCodeGen* codegen,
 // Both the null and not null branch into the continue block.  The continue block
 // becomes the start of the next block for codegen (either the next expr or just the
 // end of the function).
-Function* OldHashTable::CodegenEvalTupleRow(RuntimeState* state, bool build) {
+Function* OldHashTable::CodegenEvalTupleRow(LlvmCodeGen* codegen, bool build) {
   // TODO: CodegenAssignNullValue() can't handle TYPE_TIMESTAMP or TYPE_DECIMAL yet
   const vector<ExprContext*>& ctxs = build ? build_expr_ctxs_ : probe_expr_ctxs_;
   for (int i = 0; i < ctxs.size(); ++i) {
@@ -263,9 +263,6 @@ Function* OldHashTable::CodegenEvalTupleRow(RuntimeState* state, bool build) {
     if (type == TYPE_TIMESTAMP || type == TYPE_DECIMAL || type == TYPE_CHAR) return NULL;
   }
 
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
-
   // Get types to generate function prototype
   Type* tuple_row_type = codegen->GetType(TupleRow::LLVM_CLASS_NAME);
   DCHECK(tuple_row_type != NULL);
@@ -307,12 +304,10 @@ Function* OldHashTable::CodegenEvalTupleRow(RuntimeState* state, bool build) {
 
       // Call expr
       Function* expr_fn;
-      Status status = ctxs[i]->root()->GetCodegendComputeFn(state, &expr_fn);
+      Status status = ctxs[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
       if (!status.ok()) {
-        stringstream ss;
-        ss << "Problem with codegen: " << status.GetDetail();
-        state->LogError(ErrorMsg(TErrorCode::GENERAL, ss.str()));
         fn->eraseFromParent(); // deletes function
+        VLOG_QUERY << "Failed to codegen EvalTupleRow(): " << status.GetDetail();
         return NULL;
       }
 
@@ -352,11 +347,9 @@ Function* OldHashTable::CodegenEvalTupleRow(RuntimeState* state, bool build) {
     }
   }
   builder.CreateRet(has_null);
-
   return codegen->FinalizeFunction(fn);
 }
 
-
 uint32_t OldHashTable::HashVariableLenRow() {
   uint32_t hash = initial_seed_;
   // Hash the non-var length portions (if there are any)
@@ -410,15 +403,12 @@ uint32_t OldHashTable::HashVariableLenRow() {
 //   ret i32 %7
 // }
 // TODO: can this be cross-compiled?
-Function* OldHashTable::CodegenHashCurrentRow(RuntimeState* state) {
+Function* OldHashTable::CodegenHashCurrentRow(LlvmCodeGen* codegen) {
   for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
     // Disable codegen for CHAR
     if (build_expr_ctxs_[i]->root()->type().type == TYPE_CHAR) return NULL;
   }
 
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
-
   // Get types to generate function prototype
   Type* this_type = codegen->GetType(OldHashTable::LLVM_CLASS_NAME);
   DCHECK(this_type != NULL);
@@ -592,14 +582,12 @@ bool OldHashTable::Equals(TupleRow* build_row) {
 // continue3:                                        ; preds = %not_null2, %null1
 //   ret i1 true
 // }
-Function* OldHashTable::CodegenEquals(RuntimeState* state) {
+Function* OldHashTable::CodegenEquals(LlvmCodeGen* codegen) {
   for (int i = 0; i < build_expr_ctxs_.size(); ++i) {
     // Disable codegen for CHAR
     if (build_expr_ctxs_[i]->root()->type().type == TYPE_CHAR) return NULL;
   }
 
-  LlvmCodeGen* codegen;
-  if (!state->GetCodegen(&codegen).ok()) return NULL;
   // Get types to generate function prototype
   Type* tuple_row_type = codegen->GetType(TupleRow::LLVM_CLASS_NAME);
   DCHECK(tuple_row_type != NULL);
@@ -629,12 +617,11 @@ Function* OldHashTable::CodegenEquals(RuntimeState* state) {
 
       // call GetValue on build_exprs[i]
       Function* expr_fn;
-      Status status = build_expr_ctxs_[i]->root()->GetCodegendComputeFn(state, &expr_fn);
+      Status status =
+          build_expr_ctxs_[i]->root()->GetCodegendComputeFn(codegen, &expr_fn);
       if (!status.ok()) {
-        stringstream ss;
-        ss << "Problem with codegen: " << status.GetDetail();
-        state->LogError(ErrorMsg(TErrorCode::GENERAL, ss.str()));
         fn->eraseFromParent(); // deletes function
+        VLOG_QUERY << "Failed to codegen Equals(): " << status.GetDetail();
         return NULL;
       }
 
@@ -690,7 +677,6 @@ Function* OldHashTable::CodegenEquals(RuntimeState* state) {
   } else {
     builder.CreateRet(codegen->true_value());
   }
-
   return codegen->FinalizeFunction(fn);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/old-hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table.h b/be/src/exec/old-hash-table.h
index 5ef24ab..3a9b5b2 100644
--- a/be/src/exec/old-hash-table.h
+++ b/be/src/exec/old-hash-table.h
@@ -233,15 +233,15 @@ class OldHashTable {
   /// Codegen for evaluating a tuple row.  Codegen'd function matches the signature
   /// for EvalBuildRow and EvalTupleRow.
   /// if build_row is true, the codegen uses the build_exprs, otherwise the probe_exprs
-  llvm::Function* CodegenEvalTupleRow(RuntimeState* state, bool build_row);
+  llvm::Function* CodegenEvalTupleRow(LlvmCodeGen* codegen, bool build_row);
 
   /// Codegen for hashing the expr values in 'expr_values_buffer_'.  Function
   /// prototype matches HashCurrentRow identically.
-  llvm::Function* CodegenHashCurrentRow(RuntimeState* state);
+  llvm::Function* CodegenHashCurrentRow(LlvmCodeGen* codegen);
 
   /// Codegen for evaluating a TupleRow and comparing equality against
   /// 'expr_values_buffer_'.  Function signature matches OldHashTable::Equals()
-  llvm::Function* CodegenEquals(RuntimeState* state);
+  llvm::Function* CodegenEquals(LlvmCodeGen* codegen);
 
   static const char* LLVM_CLASS_NAME;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-aggregation-node.cc b/be/src/exec/partitioned-aggregation-node.cc
index 629e407..6862bb3 100644
--- a/be/src/exec/partitioned-aggregation-node.cc
+++ b/be/src/exec/partitioned-aggregation-node.cc
@@ -172,14 +172,6 @@ Status PartitionedAggregationNode::Init(const TPlanNode& tnode, RuntimeState* st
 Status PartitionedAggregationNode::Prepare(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
 
-  // Create the codegen object before preparing conjunct_ctxs_ and children_, so that any
-  // ScalarFnCalls will use codegen.
-  // TODO: this is brittle and hard to reason about, revisit
-  if (state->codegen_enabled()) {
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
-  }
-
   RETURN_IF_ERROR(ExecNode::Prepare(state));
   state_ = state;
 
@@ -292,18 +284,24 @@ Status PartitionedAggregationNode::Prepare(RuntimeState* state) {
     }
     DCHECK(serialize_stream_->has_write_block());
   }
-
-  bool codegen_enabled = false;
-  Status codegen_status;
-  if (state->codegen_enabled()) {
-    codegen_status =
-        is_streaming_preagg_ ? CodegenProcessBatchStreaming() : CodegenProcessBatch();
-    codegen_enabled = codegen_status.ok();
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
   }
-  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
   return Status::OK();
 }
 
+void PartitionedAggregationNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
+  TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
+  Status codegen_status =
+     is_streaming_preagg_ ? CodegenProcessBatchStreaming(codegen, prefetch_mode) :
+          CodegenProcessBatch(codegen, prefetch_mode);
+  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+  ExecNode::Codegen(state);
+}
+
 Status PartitionedAggregationNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Open(state));
@@ -1531,11 +1529,8 @@ Status PartitionedAggregationNode::QueryMaintenance(RuntimeState* state) {
 // ret:                                              ; preds = %src_not_null, %entry
 //   ret void
 // }
-Status PartitionedAggregationNode::CodegenUpdateSlot(
+Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen,
     AggFnEvaluator* evaluator, SlotDescriptor* slot_desc, Function** fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state_->GetCodegen(&codegen));
-
   // TODO: Fix this DCHECK and Init() once CodegenUpdateSlot() can handle AggFnEvaluator
   // with multiple input expressions (e.g. group_concat).
   DCHECK_EQ(evaluator->input_expr_ctxs().size(), 1);
@@ -1550,7 +1545,7 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(
   }
 
   Function* agg_expr_fn;
-  RETURN_IF_ERROR(agg_expr->GetCodegendComputeFn(state_, &agg_expr_fn));
+  RETURN_IF_ERROR(agg_expr->GetCodegendComputeFn(codegen, &agg_expr_fn));
 
   PointerType* fn_ctx_type =
       codegen->GetPtrType(FunctionContextImpl::LLVM_FUNCTIONCONTEXT_NAME);
@@ -1746,9 +1741,8 @@ Status PartitionedAggregationNode::CodegenUpdateSlot(
 //                          %"class.impala::TupleRow"* %row)
 //   ret void
 // }
-Status PartitionedAggregationNode::CodegenUpdateTuple(Function** fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state_->GetCodegen(&codegen));
+Status PartitionedAggregationNode::CodegenUpdateTuple(LlvmCodeGen* codegen,
+    Function** fn) {
   SCOPED_TIMER(codegen->codegen_timer());
 
   int j = grouping_expr_ctxs_.size();
@@ -1838,7 +1832,7 @@ Status PartitionedAggregationNode::CodegenUpdateTuple(Function** fn) {
       builder.CreateStore(count_inc, slot_ptr);
     } else {
       Function* update_slot_fn;
-      RETURN_IF_ERROR(CodegenUpdateSlot(evaluator, slot_desc, &update_slot_fn));
+      RETURN_IF_ERROR(CodegenUpdateSlot(codegen, evaluator, slot_desc, &update_slot_fn));
       Value* agg_fn_ctx_ptr = builder.CreateConstGEP1_32(agg_fn_ctxs_arg, i);
       Value* agg_fn_ctx = builder.CreateLoad(agg_fn_ctx_ptr, "agg_fn_ctx");
       // Call GetExprCtx() to get the expression context.
@@ -1860,13 +1854,12 @@ Status PartitionedAggregationNode::CodegenUpdateTuple(Function** fn) {
   return Status::OK();
 }
 
-Status PartitionedAggregationNode::CodegenProcessBatch() {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state_->GetCodegen(&codegen));
+Status PartitionedAggregationNode::CodegenProcessBatch(LlvmCodeGen* codegen,
+    TPrefetchMode::type prefetch_mode) {
   SCOPED_TIMER(codegen->codegen_timer());
 
   Function* update_tuple_fn;
-  RETURN_IF_ERROR(CodegenUpdateTuple(&update_tuple_fn));
+  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
 
   // Get the cross compiled update row batch function
   IRFunction::Type ir_fn = (!grouping_expr_ctxs_.empty() ?
@@ -1880,7 +1873,6 @@ Status PartitionedAggregationNode::CodegenProcessBatch() {
     // Codegen for grouping using hash table
 
     // Replace prefetch_mode with constant so branches can be optimised out.
-    TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
     Value* prefetch_mode_arg = codegen->GetArgument(process_batch_fn, 3);
     prefetch_mode_arg->replaceAllUsesWith(
         ConstantInt::get(Type::getInt32Ty(codegen->context()), prefetch_mode));
@@ -1888,15 +1880,15 @@ Status PartitionedAggregationNode::CodegenProcessBatch() {
     // The codegen'd ProcessBatch function is only used in Open() with level_ = 0,
     // so don't use murmur hash
     Function* hash_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(state_, /* use murmur */ false, &hash_fn));
+    RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, /* use murmur */ false, &hash_fn));
 
     // Codegen HashTable::Equals<true>
     Function* build_equals_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenEquals(state_, true, &build_equals_fn));
+    RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &build_equals_fn));
 
     // Codegen for evaluating input rows
     Function* eval_grouping_expr_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(state_, false, &eval_grouping_expr_fn));
+    RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
 
     // Replace call sites
     replaced = codegen->ReplaceCallSites(process_batch_fn, eval_grouping_expr_fn,
@@ -1911,7 +1903,7 @@ Status PartitionedAggregationNode::CodegenProcessBatch() {
 
     HashTableCtx::HashTableReplacedConstants replaced_constants;
     const bool stores_duplicates = false;
-    RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(state_, stores_duplicates, 1,
+    RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates, 1,
         process_batch_fn, &replaced_constants));
     DCHECK_GE(replaced_constants.stores_nulls, 1);
     DCHECK_GE(replaced_constants.finds_some_nulls, 1);
@@ -1935,10 +1927,9 @@ Status PartitionedAggregationNode::CodegenProcessBatch() {
   return Status::OK();
 }
 
-Status PartitionedAggregationNode::CodegenProcessBatchStreaming() {
+Status PartitionedAggregationNode::CodegenProcessBatchStreaming(
+    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
   DCHECK(is_streaming_preagg_);
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state_->GetCodegen(&codegen));
   SCOPED_TIMER(codegen->codegen_timer());
 
   IRFunction::Type ir_fn = IRFunction::PART_AGG_NODE_PROCESS_BATCH_STREAMING;
@@ -1951,25 +1942,24 @@ Status PartitionedAggregationNode::CodegenProcessBatchStreaming() {
       ConstantInt::get(Type::getInt1Ty(codegen->context()), needs_serialize_));
 
   // Replace prefetch_mode with constant so branches can be optimised out.
-  TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
   Value* prefetch_mode_arg = codegen->GetArgument(process_batch_streaming_fn, 3);
   prefetch_mode_arg->replaceAllUsesWith(
       ConstantInt::get(Type::getInt32Ty(codegen->context()), prefetch_mode));
 
   Function* update_tuple_fn;
-  RETURN_IF_ERROR(CodegenUpdateTuple(&update_tuple_fn));
+  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
 
   // We only use the top-level hash function for streaming aggregations.
   Function* hash_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(state_, false, &hash_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, false, &hash_fn));
 
   // Codegen HashTable::Equals
   Function* equals_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(state_, true, &equals_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &equals_fn));
 
   // Codegen for evaluating input rows
   Function* eval_grouping_expr_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(state_, false, &eval_grouping_expr_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
 
   // Replace call sites
   int replaced = codegen->ReplaceCallSites(process_batch_streaming_fn, update_tuple_fn,
@@ -1988,7 +1978,7 @@ Status PartitionedAggregationNode::CodegenProcessBatchStreaming() {
 
   HashTableCtx::HashTableReplacedConstants replaced_constants;
   const bool stores_duplicates = false;
-  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(state_, stores_duplicates, 1,
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates, 1,
       process_batch_streaming_fn, &replaced_constants));
   DCHECK_GE(replaced_constants.stores_nulls, 1);
   DCHECK_GE(replaced_constants.finds_some_nulls, 1);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-aggregation-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-aggregation-node.h b/be/src/exec/partitioned-aggregation-node.h
index 0c0f3e8..9e14e66 100644
--- a/be/src/exec/partitioned-aggregation-node.h
+++ b/be/src/exec/partitioned-aggregation-node.h
@@ -126,6 +126,7 @@ class PartitionedAggregationNode : public ExecNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -136,7 +137,6 @@ class PartitionedAggregationNode : public ExecNode {
  protected:
   /// Frees local allocations from aggregate_evaluators_ and agg_fn_ctxs
   virtual Status QueryMaintenance(RuntimeState* state);
-
   virtual void DebugString(int indentation_level, std::stringstream* out) const;
 
  private:
@@ -637,11 +637,11 @@ class PartitionedAggregationNode : public ExecNode {
 
   /// Codegen UpdateSlot(). Returns non-OK status if codegen is unsuccessful.
   /// Assumes is_merge = false;
-  Status CodegenUpdateSlot(AggFnEvaluator* evaluator, SlotDescriptor* slot_desc,
-      llvm::Function** fn);
+  Status CodegenUpdateSlot(LlvmCodeGen* codegen, AggFnEvaluator* evaluator,
+      SlotDescriptor* slot_desc, llvm::Function** fn);
 
   /// Codegen UpdateTuple(). Returns non-OK status if codegen is unsuccessful.
-  Status CodegenUpdateTuple(llvm::Function** fn);
+  Status CodegenUpdateTuple(LlvmCodeGen* codegen, llvm::Function** fn);
 
   /// Codegen the non-streaming process row batch loop. The loop has already been
   /// compiled to IR and loaded into the codegen object. UpdateAggTuple has also been
@@ -650,11 +650,12 @@ class PartitionedAggregationNode : public ExecNode {
   /// 'process_batch_no_grouping_fn_' will be updated with the codegened function
   /// depending on whether this is a grouping or non-grouping aggregation.
   /// Assumes AGGREGATED_ROWS = false.
-  Status CodegenProcessBatch();
+  Status CodegenProcessBatch(LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode);
 
   /// Codegen the materialization loop for streaming preaggregations.
   /// 'process_batch_streaming_fn_' will be updated with the codegened function.
-  Status CodegenProcessBatchStreaming();
+  Status CodegenProcessBatchStreaming(
+      LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode);
 
   /// We need two buffers per partition, one for the aggregated stream and one
   /// for the unaggregated stream. We need an additional buffer to read the stream

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-hash-join-builder.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index bf5b42a..0f15876 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -138,8 +138,9 @@ Status PhjBuilder::Prepare(RuntimeState* state, MemTracker* mem_tracker) {
   partition_build_rows_timer_ = ADD_TIMER(profile(), "BuildRowsPartitionTime");
   build_hash_table_timer_ = ADD_TIMER(profile(), "HashTablesBuildTime");
   repartition_timer_ = ADD_TIMER(profile(), "RepartitionTime");
-
-  Codegen(state);
+  if (!state->codegen_enabled()) {
+    profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
   return Status::OK();
 }
 
@@ -704,39 +705,34 @@ not_built:
   return Status::OK();
 }
 
-void PhjBuilder::Codegen(RuntimeState* state) {
-  bool build_codegen_enabled = false;
-  bool insert_codegen_enabled = false;
-  Status build_codegen_status, insert_codegen_status;
-  if (state->codegen_enabled()) {
-    Status codegen_status;
-    // Codegen for hashing rows with the builder's hash table context.
-    Function* hash_fn;
-    codegen_status = ht_ctx_->CodegenHashRow(runtime_state_, false, &hash_fn);
-    Function* murmur_hash_fn;
-    codegen_status.MergeStatus(
-        ht_ctx_->CodegenHashRow(runtime_state_, true, &murmur_hash_fn));
-
-    // Codegen for evaluating build rows
-    Function* eval_build_row_fn;
-    codegen_status.MergeStatus(
-        ht_ctx_->CodegenEvalRow(runtime_state_, true, &eval_build_row_fn));
-
-    if (codegen_status.ok()) {
-      build_codegen_status =
-          CodegenProcessBuildBatch(hash_fn, murmur_hash_fn, eval_build_row_fn);
-      insert_codegen_status =
-          CodegenInsertBatch(hash_fn, murmur_hash_fn, eval_build_row_fn);
-    } else {
-      build_codegen_status = codegen_status;
-      insert_codegen_status = codegen_status;
-    }
-    build_codegen_enabled = build_codegen_status.ok();
-    insert_codegen_enabled = insert_codegen_status.ok();
+void PhjBuilder::Codegen(LlvmCodeGen* codegen) {
+  Status build_codegen_status;
+  Status insert_codegen_status;
+  Status codegen_status;
+
+  // Codegen for hashing rows with the builder's hash table context.
+  Function* hash_fn;
+  codegen_status = ht_ctx_->CodegenHashRow(codegen, false, &hash_fn);
+  Function* murmur_hash_fn;
+  codegen_status.MergeStatus(ht_ctx_->CodegenHashRow(codegen, true, &murmur_hash_fn));
+
+  // Codegen for evaluating build rows
+  Function* eval_build_row_fn;
+  codegen_status.MergeStatus(ht_ctx_->CodegenEvalRow(codegen, true, &eval_build_row_fn));
+
+  if (codegen_status.ok()) {
+    TPrefetchMode::type prefetch_mode = runtime_state_->query_options().prefetch_mode;
+    build_codegen_status =
+        CodegenProcessBuildBatch(codegen, hash_fn, murmur_hash_fn, eval_build_row_fn);
+    insert_codegen_status = CodegenInsertBatch(codegen, hash_fn, murmur_hash_fn,
+        eval_build_row_fn, prefetch_mode);
+  } else {
+    build_codegen_status = codegen_status;
+    insert_codegen_status = codegen_status;
   }
-  profile()->AddCodegenMsg(build_codegen_enabled, build_codegen_status, "Build Side");
-  profile()->AddCodegenMsg(
-      insert_codegen_enabled, insert_codegen_status, "Hash Table Construction");
+  profile()->AddCodegenMsg(build_codegen_status.ok(), build_codegen_status, "Build Side");
+  profile()->AddCodegenMsg(insert_codegen_status.ok(), insert_codegen_status,
+      "Hash Table Construction");
 }
 
 string PhjBuilder::DebugString() const {
@@ -763,11 +759,8 @@ string PhjBuilder::DebugString() const {
   return ss.str();
 }
 
-Status PhjBuilder::CodegenProcessBuildBatch(
+Status PhjBuilder::CodegenProcessBuildBatch(LlvmCodeGen* codegen,
     Function* hash_fn, Function* murmur_hash_fn, Function* eval_row_fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(runtime_state_->GetCodegen(&codegen));
-
   Function* process_build_batch_fn =
       codegen->GetFunction(IRFunction::PHJ_PROCESS_BUILD_BATCH, true);
   DCHECK(process_build_batch_fn != NULL);
@@ -781,7 +774,7 @@ Status PhjBuilder::CodegenProcessBuildBatch(
   HashTableCtx::HashTableReplacedConstants replaced_constants;
   const bool stores_duplicates = true;
   const int num_build_tuples = row_desc_.tuple_descriptors().size();
-  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(runtime_state_, stores_duplicates,
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates,
       num_build_tuples, process_build_batch_fn, &replaced_constants));
   DCHECK_GE(replaced_constants.stores_nulls, 1);
   DCHECK_EQ(replaced_constants.finds_some_nulls, 0);
@@ -838,18 +831,14 @@ Status PhjBuilder::CodegenProcessBuildBatch(
   return Status::OK();
 }
 
-Status PhjBuilder::CodegenInsertBatch(
-    Function* hash_fn, Function* murmur_hash_fn, Function* eval_row_fn) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(runtime_state_->GetCodegen(&codegen));
-
+Status PhjBuilder::CodegenInsertBatch(LlvmCodeGen* codegen, Function* hash_fn,
+    Function* murmur_hash_fn, Function* eval_row_fn, TPrefetchMode::type prefetch_mode) {
   Function* insert_batch_fn = codegen->GetFunction(IRFunction::PHJ_INSERT_BATCH, true);
   Function* build_equals_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(runtime_state_, true, &build_equals_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &build_equals_fn));
 
   // Replace the parameter 'prefetch_mode' with constant.
   Value* prefetch_mode_arg = codegen->GetArgument(insert_batch_fn, 1);
-  TPrefetchMode::type prefetch_mode = runtime_state_->query_options().prefetch_mode;
   DCHECK_GE(prefetch_mode, TPrefetchMode::NONE);
   DCHECK_LE(prefetch_mode, TPrefetchMode::HT_BUCKET);
   prefetch_mode_arg->replaceAllUsesWith(
@@ -867,7 +856,7 @@ Status PhjBuilder::CodegenInsertBatch(
   HashTableCtx::HashTableReplacedConstants replaced_constants;
   const bool stores_duplicates = true;
   const int num_build_tuples = row_desc_.tuple_descriptors().size();
-  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(runtime_state_, stores_duplicates,
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates,
       num_build_tuples, insert_batch_fn, &replaced_constants));
   DCHECK_GE(replaced_constants.stores_nulls, 1);
   DCHECK_EQ(replaced_constants.finds_some_nulls, 0);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-hash-join-builder.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-builder.h b/be/src/exec/partitioned-hash-join-builder.h
index 7f81e5a..650452c 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -84,6 +84,11 @@ class PhjBuilder : public DataSink {
   virtual Status FlushFinal(RuntimeState* state) override;
   virtual void Close(RuntimeState* state) override;
 
+  /// Does all codegen for the builder (if codegen is enabled).
+  /// Updates the the builder's runtime profile with info about whether any errors
+  /// occured during codegen.
+  void Codegen(LlvmCodeGen* codegen);
+
   /////////////////////////////////////////
   // The following functions are used only by PartitionedHashJoinNode.
   /////////////////////////////////////////
@@ -312,20 +317,16 @@ class PhjBuilder : public DataSink {
   /// unacceptably high false-positive rate.
   void PublishRuntimeFilters(int64_t num_build_rows);
 
-  /// Does all codegen for the builder (if codegen is enabled).
-  /// Updates the the builder's runtime profile with info about whether the codegen was
-  /// enabled and whether any errors occured during codegen.
-  void Codegen(RuntimeState* state);
-
   /// Codegen processing build batches. Identical signature to ProcessBuildBatch().
   /// Returns non-OK status if codegen was not possible.
-  Status CodegenProcessBuildBatch(llvm::Function* hash_fn, llvm::Function* murmur_hash_fn,
-      llvm::Function* eval_row_fn);
+  Status CodegenProcessBuildBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn,
+      llvm::Function* murmur_hash_fn, llvm::Function* eval_row_fn);
 
   /// Codegen inserting batches into a partition's hash table. Identical signature to
   /// Partition::InsertBatch(). Returns non-OK if codegen was not possible.
-  Status CodegenInsertBatch(llvm::Function* hash_fn, llvm::Function* murmur_hash_fn,
-      llvm::Function* eval_row_fn);
+  Status CodegenInsertBatch(LlvmCodeGen* codegen, llvm::Function* hash_fn,
+      llvm::Function* murmur_hash_fn, llvm::Function* eval_row_fn,
+      TPrefetchMode::type prefetch_mode);
 
   RuntimeState* const runtime_state_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-hash-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-node.cc b/be/src/exec/partitioned-hash-join-node.cc
index 46b91ba..024fdc3 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -98,14 +98,6 @@ Status PartitionedHashJoinNode::Init(const TPlanNode& tnode, RuntimeState* state
 Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
 
-  // Create the codegen object before preparing conjunct_ctxs_ and children_, so that any
-  // ScalarFnCalls will use codegen.
-  // TODO: this is brittle and hard to reason about, revisit
-  if (state->codegen_enabled()) {
-    LlvmCodeGen* codegen;
-    RETURN_IF_ERROR(state->GetCodegen(&codegen));
-  }
-
   RETURN_IF_ERROR(BlockingJoinNode::Prepare(state));
   runtime_state_ = state;
 
@@ -143,18 +135,30 @@ Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
 
   num_probe_rows_partitioned_ =
       ADD_COUNTER(runtime_profile(), "ProbeRowsPartitioned", TUnit::UNIT);
-
-  bool probe_codegen_enabled = false;
-  Status probe_codegen_status;
-  if (state->codegen_enabled()) {
-    probe_codegen_status = CodegenProcessProbeBatch(state);
-    probe_codegen_enabled = probe_codegen_status.ok();
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
   }
-  runtime_profile()->AddCodegenMsg(
-      probe_codegen_enabled, probe_codegen_status, "Probe Side");
   return Status::OK();
 }
 
+void PartitionedHashJoinNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
+
+  // Codegen the build side.
+  builder_->Codegen(codegen);
+
+  // Codegen the probe side.
+  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
+  Status probe_codegen_status = CodegenProcessProbeBatch(codegen, prefetch_mode);
+  runtime_profile()->AddCodegenMsg(probe_codegen_status.ok(), probe_codegen_status,
+      "Probe Side");
+
+  // Codegen the children node;
+  ExecNode::Codegen(state);
+}
+
 Status PartitionedHashJoinNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(BlockingJoinNode::Open(state));
@@ -1190,14 +1194,13 @@ Status PartitionedHashJoinNode::CodegenCreateOutputRow(LlvmCodeGen* codegen,
   return Status::OK();
 }
 
-Status PartitionedHashJoinNode::CodegenProcessProbeBatch(RuntimeState* state) {
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
+Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
+    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
   // Codegen for hashing rows
   Function* hash_fn;
   Function* murmur_hash_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(state, false, &hash_fn));
-  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(state, true, &murmur_hash_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, false, &hash_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, true, &murmur_hash_fn));
 
   // Get cross compiled function
   IRFunction::Type ir_fn = IRFunction::FN_END;
@@ -1243,7 +1246,6 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(RuntimeState* state) {
 
   // Replace the parameter 'prefetch_mode' with constant.
   Value* prefetch_mode_arg = codegen->GetArgument(process_probe_batch_fn, 1);
-  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
   DCHECK_GE(prefetch_mode, TPrefetchMode::NONE);
   DCHECK_LE(prefetch_mode, TPrefetchMode::HT_BUCKET);
   prefetch_mode_arg->replaceAllUsesWith(
@@ -1251,11 +1253,11 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(RuntimeState* state) {
 
   // Codegen HashTable::Equals
   Function* probe_equals_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(state, false, &probe_equals_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, false, &probe_equals_fn));
 
   // Codegen for evaluating probe rows
   Function* eval_row_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(state, false, &eval_row_fn));
+  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_row_fn));
 
   // Codegen CreateOutputRow
   Function* create_output_row_fn;
@@ -1263,12 +1265,12 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(RuntimeState* state) {
 
   // Codegen evaluating other join conjuncts
   Function* eval_other_conjuncts_fn;
-  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(state, other_join_conjunct_ctxs_,
+  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, other_join_conjunct_ctxs_,
       &eval_other_conjuncts_fn, "EvalOtherConjuncts"));
 
   // Codegen evaluating conjuncts
   Function* eval_conjuncts_fn;
-  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(state, conjunct_ctxs_,
+  RETURN_IF_ERROR(ExecNode::CodegenEvalConjuncts(codegen, conjunct_ctxs_,
       &eval_conjuncts_fn));
 
   // Replace all call sites with codegen version
@@ -1317,7 +1319,7 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(RuntimeState* state) {
   HashTableCtx::HashTableReplacedConstants replaced_constants;
   const bool stores_duplicates = true;
   const int num_build_tuples = child(1)->row_desc().tuple_descriptors().size();
-  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(state, stores_duplicates,
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates,
       num_build_tuples, process_probe_batch_fn, &replaced_constants));
   DCHECK_GE(replaced_constants.stores_nulls, 1);
   DCHECK_GE(replaced_constants.finds_some_nulls, 1);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/partitioned-hash-join-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-node.h b/be/src/exec/partitioned-hash-join-node.h
index 5b9264c..504dc7b 100644
--- a/be/src/exec/partitioned-hash-join-node.h
+++ b/be/src/exec/partitioned-hash-join-node.h
@@ -110,6 +110,7 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -355,7 +356,8 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
 
   /// Codegen processing probe batches.  Identical signature to ProcessProbeBatch.
   /// Returns non-OK if codegen was not possible.
-  Status CodegenProcessProbeBatch(RuntimeState* state);
+  Status CodegenProcessProbeBatch(
+      LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode);
 
   /// Returns the current state of the partition as a string.
   std::string PrintState() const;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/sort-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/sort-node.cc b/be/src/exec/sort-node.cc
index 9271721..140e662 100644
--- a/be/src/exec/sort-node.cc
+++ b/be/src/exec/sort-node.cc
@@ -50,22 +50,23 @@ Status SortNode::Prepare(RuntimeState* state) {
   RETURN_IF_ERROR(sort_exec_exprs_.Prepare(
       state, child(0)->row_desc(), row_descriptor_, expr_mem_tracker()));
   AddExprCtxsToFree(sort_exec_exprs_);
-  TupleRowComparator less_than(sort_exec_exprs_, is_asc_order_, nulls_first_);
-
-  bool codegen_enabled = false;
-  Status codegen_status;
-  if (state->codegen_enabled()) {
-    codegen_status = less_than.Codegen(state);
-    codegen_enabled = codegen_status.ok();
-  }
-  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
-
-  sorter_.reset(new Sorter(less_than, sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
+  less_than_.reset(new TupleRowComparator(sort_exec_exprs_, is_asc_order_, nulls_first_));
+  sorter_.reset(new Sorter(*less_than_.get(), sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
       &row_descriptor_, mem_tracker(), runtime_profile(), state));
   RETURN_IF_ERROR(sorter_->Init());
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
   return Status::OK();
 }
 
+void SortNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  Status codegen_status = less_than_->Codegen(state);
+  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+  ExecNode::Codegen(state);
+}
+
 Status SortNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Open(state));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/sort-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/sort-node.h b/be/src/exec/sort-node.h
index a90c99e..75513ba 100644
--- a/be/src/exec/sort-node.h
+++ b/be/src/exec/sort-node.h
@@ -41,6 +41,7 @@ class SortNode : public ExecNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
   virtual Status Reset(RuntimeState* state);
@@ -56,6 +57,9 @@ class SortNode : public ExecNode {
   /// Number of rows to skip.
   int64_t offset_;
 
+  /// The tuple row comparator derived based on 'sort_exec_exprs_'.
+  boost::scoped_ptr<TupleRowComparator> less_than_;
+
   /// Expressions and parameters used for tuple materialization and tuple comparison.
   SortExecExprs sort_exec_exprs_;
   std::vector<bool> is_asc_order_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b15d992a/be/src/exec/topn-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.cc b/be/src/exec/topn-node.cc
index 6dd1e34..e72249f 100644
--- a/be/src/exec/topn-node.cc
+++ b/be/src/exec/topn-node.cc
@@ -63,39 +63,6 @@ Status TopNNode::Init(const TPlanNode& tnode, RuntimeState* state) {
   return Status::OK();
 }
 
-Status TopNNode::Codegen(RuntimeState* state) {
-  DCHECK(materialized_tuple_desc_ != NULL);
-  LlvmCodeGen* codegen;
-  RETURN_IF_ERROR(state->GetCodegen(&codegen));
-  Function* insert_batch_fn =
-      codegen->GetFunction(IRFunction::TOPN_NODE_INSERT_BATCH, true);
-
-  // Generate two MaterializeExprs() functions, one using tuple_pool_ and one with no
-  // pool.
-  Function* materialize_exprs_tuple_pool_fn;
-  RETURN_IF_ERROR(Tuple::CodegenMaterializeExprs(state, false, *materialized_tuple_desc_,
-      sort_exec_exprs_.sort_tuple_slot_expr_ctxs(), tuple_pool_.get(),
-      &materialize_exprs_tuple_pool_fn));
-
-  Function* materialize_exprs_no_pool_fn;
-  RETURN_IF_ERROR(Tuple::CodegenMaterializeExprs(state, false, *materialized_tuple_desc_,
-      sort_exec_exprs_.sort_tuple_slot_expr_ctxs(), NULL, &materialize_exprs_no_pool_fn));
-
-  int replaced = codegen->ReplaceCallSites(insert_batch_fn,
-      materialize_exprs_tuple_pool_fn, Tuple::MATERIALIZE_EXPRS_SYMBOL);
-  DCHECK_EQ(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
-
-  replaced = codegen->ReplaceCallSites(insert_batch_fn, materialize_exprs_no_pool_fn,
-      Tuple::MATERIALIZE_EXPRS_NULL_POOL_SYMBOL);
-  DCHECK_EQ(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
-
-  insert_batch_fn = codegen->FinalizeFunction(insert_batch_fn);
-  DCHECK(insert_batch_fn != NULL);
-  codegen->AddFunctionToJit(insert_batch_fn,
-      reinterpret_cast<void**>(&codegend_insert_batch_fn_));
-  return Status::OK();
-}
-
 Status TopNNode::Prepare(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Prepare(state));
@@ -106,23 +73,66 @@ Status TopNNode::Prepare(RuntimeState* state) {
   AddExprCtxsToFree(sort_exec_exprs_);
   tuple_row_less_than_.reset(
       new TupleRowComparator(sort_exec_exprs_, is_asc_order_, nulls_first_));
-  bool codegen_enabled = false;
-  Status codegen_status;
-  if (state->codegen_enabled()) {
-    // TODO: inline tuple_row_less_than_->Compare()
-    codegen_status = tuple_row_less_than_->Codegen(state);
-    codegen_status.MergeStatus(Codegen(state));
-    codegen_enabled = codegen_status.ok();
-  }
-  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
   priority_queue_.reset(
       new priority_queue<Tuple*, vector<Tuple*>, ComparatorWrapper<TupleRowComparator>>(
           *tuple_row_less_than_));
   materialized_tuple_desc_ = row_descriptor_.tuple_descriptors()[0];
   insert_batch_timer_ = ADD_TIMER(runtime_profile(), "InsertBatchTime");
+  if (!state->codegen_enabled()) {
+    runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+  }
+
+runtime_profile()->AddCodegenMsg(false);
   return Status::OK();
 }
 
+void TopNNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != NULL);
+
+  // TODO: inline tuple_row_less_than_->Compare()
+  Status codegen_status = tuple_row_less_than_->Codegen(state);
+  if (codegen_status.ok()) {
+    Function* insert_batch_fn =
+        codegen->GetFunction(IRFunction::TOPN_NODE_INSERT_BATCH, true);
+    DCHECK(insert_batch_fn != NULL);
+
+    // Generate two MaterializeExprs() functions, one using tuple_pool_ and
+    // one with no pool.
+    DCHECK(materialized_tuple_desc_ != NULL);
+    Function* materialize_exprs_tuple_pool_fn;
+    Function* materialize_exprs_no_pool_fn;
+
+    codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
+        *materialized_tuple_desc_, sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
+        tuple_pool_.get(), &materialize_exprs_tuple_pool_fn);
+
+    if (codegen_status.ok()) {
+      codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
+          *materialized_tuple_desc_, sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
+          NULL, &materialize_exprs_no_pool_fn);
+
+      if (codegen_status.ok()) {
+        int replaced = codegen->ReplaceCallSites(insert_batch_fn,
+            materialize_exprs_tuple_pool_fn, Tuple::MATERIALIZE_EXPRS_SYMBOL);
+        DCHECK_EQ(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
+
+        replaced = codegen->ReplaceCallSites(insert_batch_fn,
+            materialize_exprs_no_pool_fn, Tuple::MATERIALIZE_EXPRS_NULL_POOL_SYMBOL);
+        DCHECK_EQ(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
+
+        insert_batch_fn = codegen->FinalizeFunction(insert_batch_fn);
+        DCHECK(insert_batch_fn != NULL);
+        codegen->AddFunctionToJit(insert_batch_fn,
+            reinterpret_cast<void**>(&codegend_insert_batch_fn_));
+      }
+    }
+  }
+  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+  ExecNode::Codegen(state);
+}
+
 Status TopNNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Open(state));



[15/33] 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_; }
 }


[21/33] 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/hadoop-next
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. """
 


[20/33] 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/hadoop-next
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());


[26/33] incubator-impala git commit: IMPALA-4300: Speed up BloomFilter::Or with SIMD

Posted by ta...@apache.org.
IMPALA-4300: Speed up BloomFilter::Or with SIMD

The previous code was not written in a way that GCC could
auto-vectorize it. Manually vectorizing speeds up BloomFilter::Or by
up to 184x.

Change-Id: I840799d9cfb81285c796e2abfe2029bb869b0f67
Reviewed-on: http://gerrit.cloudera.org:8080/4813
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/61fcb489
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/61fcb489
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/61fcb489

Branch: refs/heads/hadoop-next
Commit: 61fcb489745f3f0b3f1abbf9fbf666a29a6363de
Parents: 0fbb5b7
Author: Jim Apple <jb...@cloudera.com>
Authored: Fri Oct 21 07:46:42 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Oct 24 18:07:25 2016 +0000

----------------------------------------------------------------------
 be/src/benchmarks/bloom-filter-benchmark.cc | 198 +++++++++++++++--------
 be/src/testutil/mem-util.h                  |   2 +
 be/src/util/bloom-filter.cc                 |  49 +++++-
 be/src/util/cpu-info.cc                     |   1 +
 be/src/util/cpu-info.h                      |   3 +-
 5 files changed, 178 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/61fcb489/be/src/benchmarks/bloom-filter-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/bloom-filter-benchmark.cc b/be/src/benchmarks/bloom-filter-benchmark.cc
index 1aa0619..d9019c8 100644
--- a/be/src/benchmarks/bloom-filter-benchmark.cc
+++ b/be/src/benchmarks/bloom-filter-benchmark.cc
@@ -30,13 +30,14 @@
 using namespace std;
 using namespace impala;
 
-// Tests Bloom filter performance on four tasks:
+// Tests Bloom filter performance on:
 //
 // 1. Construct/destruct pairs
 // 2. Inserts
 // 3. Lookups when the item is present
 // 4. Lookups when the item is absent (this is theoretically faster than when the item is
 //    present in some Bloom filter variants)
+// 5. Unions
 //
 // As in bloom-filter.h, ndv refers to the number of unique items inserted into a filter
 // and fpp is the probability of false positives.
@@ -46,89 +47,116 @@ using namespace impala;
 // initialize:                Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           7.05e+03 7.27e+03 7.34e+03         1X         1X         1X
-//            ndv      10k fpp    1.0%           3.79e+03 3.93e+03 3.96e+03     0.538X     0.541X      0.54X
-//            ndv      10k fpp    0.1%           1.39e+03 1.42e+03 1.44e+03     0.198X     0.196X     0.196X
-//            ndv    1000k fpp   10.0%               4.62     4.78     4.81  0.000655X  0.000658X  0.000655X
-//            ndv    1000k fpp    1.0%               2.49     2.55      2.6  0.000354X  0.000351X  0.000354X
-//            ndv    1000k fpp    0.1%               2.45     2.55      2.6  0.000347X  0.000351X  0.000354X
-//            ndv  100000k fpp   10.0%              0.035   0.0358    0.037  4.96e-06X  4.93e-06X  5.04e-06X
-//            ndv  100000k fpp    1.0%             0.0347   0.0361   0.0372  4.93e-06X  4.96e-06X  5.06e-06X
-//            ndv  100000k fpp    0.1%             0.0176   0.0181   0.0186   2.5e-06X  2.49e-06X  2.53e-06X
+//            ndv      10k fpp   10.0%           5.89e+03 5.98e+03 6.03e+03         1X         1X         1X
+//            ndv      10k fpp    1.0%           3.22e+03 3.25e+03 3.27e+03     0.546X     0.543X     0.542X
+//            ndv      10k fpp    0.1%           1.13e+03 1.17e+03 1.18e+03     0.191X     0.195X     0.195X
+//            ndv    1000k fpp   10.0%               3.85     3.93     3.93  0.000654X  0.000657X  0.000652X
+//            ndv    1000k fpp    1.0%               2.04     2.12     2.12  0.000346X  0.000354X  0.000351X
+//            ndv    1000k fpp    0.1%               2.04     2.12     2.12  0.000346X  0.000354X  0.000351X
+//            ndv  100000k fpp   10.0%             0.0281    0.029   0.0294  4.77e-06X  4.85e-06X  4.87e-06X
+//            ndv  100000k fpp    1.0%             0.0284    0.029   0.0298  4.82e-06X  4.85e-06X  4.93e-06X
+//            ndv  100000k fpp    0.1%             0.0144   0.0147   0.0149  2.44e-06X  2.47e-06X  2.47e-06X
 //
 // With AVX2:
 //
 // insert:                    Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           2.03e+05 2.05e+05 2.08e+05         1X         1X         1X
-//            ndv      10k fpp    1.0%           2.03e+05 2.06e+05 2.08e+05     0.997X         1X         1X
-//            ndv      10k fpp    0.1%           2.03e+05 2.05e+05 2.07e+05     0.997X     0.998X     0.997X
-//            ndv    1000k fpp   10.0%           1.82e+05 1.87e+05 1.89e+05     0.896X      0.91X     0.907X
-//            ndv    1000k fpp    1.0%           1.49e+05 1.53e+05 1.56e+05     0.731X     0.747X      0.75X
-//            ndv    1000k fpp    0.1%           1.79e+05 1.82e+05 1.83e+05     0.881X     0.886X     0.882X
-//            ndv  100000k fpp   10.0%           4.08e+04 4.49e+04 5.44e+04     0.201X     0.219X     0.262X
-//            ndv  100000k fpp    1.0%           3.94e+04  4.4e+04 5.04e+04     0.194X     0.214X     0.242X
-//            ndv  100000k fpp    0.1%           4.08e+04 4.48e+04 5.68e+04     0.201X     0.218X     0.273X
+//            ndv      10k fpp   10.0%           1.17e+05 1.23e+05 1.25e+05         1X         1X         1X
+//            ndv      10k fpp    1.0%           1.17e+05 1.24e+05 1.25e+05         1X         1X         1X
+//            ndv      10k fpp    0.1%            1.2e+05 1.23e+05 1.24e+05      1.02X     0.996X     0.991X
+//            ndv    1000k fpp   10.0%            1.1e+05 1.18e+05  1.2e+05     0.944X     0.959X      0.96X
+//            ndv    1000k fpp    1.0%           1.11e+05 1.16e+05 1.17e+05     0.954X     0.938X     0.934X
+//            ndv    1000k fpp    0.1%           9.73e+04 1.16e+05 1.17e+05     0.834X     0.937X     0.936X
+//            ndv  100000k fpp   10.0%           2.96e+04 4.19e+04 5.44e+04     0.254X      0.34X     0.436X
+//            ndv  100000k fpp    1.0%           2.92e+04 3.81e+04 4.89e+04      0.25X     0.308X     0.391X
+//            ndv  100000k fpp    0.1%           2.44e+04 3.28e+04 4.31e+04     0.209X     0.266X     0.345X
 //
 // find:                      Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//    present ndv      10k fpp   10.0%           2.48e+05 2.51e+05 2.53e+05         1X         1X         1X
-//    absent  ndv      10k fpp   10.0%           2.47e+05 2.52e+05 2.55e+05     0.995X         1X      1.01X
-//    present ndv      10k fpp    1.0%           2.49e+05 2.52e+05 2.55e+05         1X      1.01X      1.01X
-//    absent  ndv      10k fpp    1.0%           2.47e+05 2.53e+05 2.56e+05     0.997X      1.01X      1.01X
-//    present ndv      10k fpp    0.1%           2.49e+05 2.53e+05 2.54e+05         1X      1.01X      1.01X
-//    absent  ndv      10k fpp    0.1%           2.47e+05 2.53e+05 2.56e+05     0.997X      1.01X      1.01X
-//    present ndv    1000k fpp   10.0%           1.98e+05 2.04e+05 2.06e+05       0.8X     0.814X     0.812X
-//    absent  ndv    1000k fpp   10.0%           2.01e+05 2.07e+05  2.1e+05     0.808X     0.826X     0.829X
-//    present ndv    1000k fpp    1.0%           1.83e+05 1.95e+05 2.02e+05     0.737X      0.78X     0.798X
-//    absent  ndv    1000k fpp    1.0%           2.01e+05 2.04e+05 2.08e+05     0.808X     0.815X      0.82X
-//    present ndv    1000k fpp    0.1%           1.96e+05 2.01e+05 2.03e+05     0.788X       0.8X     0.801X
-//    absent  ndv    1000k fpp    0.1%              2e+05 2.05e+05 2.07e+05     0.808X     0.817X     0.818X
-//    present ndv  100000k fpp   10.0%            4.6e+04 5.09e+04 6.08e+04     0.185X     0.203X      0.24X
-//    absent  ndv  100000k fpp   10.0%           4.11e+04 4.36e+04 4.53e+04     0.166X     0.174X     0.179X
-//    present ndv  100000k fpp    1.0%           4.55e+04 4.96e+04 6.19e+04     0.184X     0.198X     0.245X
-//    absent  ndv  100000k fpp    1.0%           3.83e+04 4.15e+04 4.69e+04     0.154X     0.166X     0.186X
-//    present ndv  100000k fpp    0.1%           4.73e+04 5.43e+04 6.58e+04     0.191X     0.217X      0.26X
-//    absent  ndv  100000k fpp    0.1%           3.77e+04 4.07e+04 4.37e+04     0.152X     0.163X     0.173X
+//    present ndv      10k fpp   10.0%           1.16e+05 1.17e+05 1.18e+05         1X         1X         1X
+//    absent  ndv      10k fpp   10.0%           1.16e+05 1.17e+05 1.18e+05         1X         1X     0.998X
+//    present ndv      10k fpp    1.0%           1.15e+05 1.17e+05 1.18e+05     0.988X         1X     0.999X
+//    absent  ndv      10k fpp    1.0%           1.14e+05 1.17e+05 1.19e+05     0.978X         1X         1X
+//    present ndv      10k fpp    0.1%           1.09e+05 1.17e+05 1.18e+05     0.939X         1X         1X
+//    absent  ndv      10k fpp    0.1%           1.13e+05 1.17e+05 1.18e+05      0.97X         1X         1X
+//    present ndv    1000k fpp   10.0%           1.09e+05 1.13e+05 1.15e+05     0.942X     0.968X      0.97X
+//    absent  ndv    1000k fpp   10.0%           1.09e+05 1.15e+05 1.16e+05     0.937X     0.982X     0.982X
+//    present ndv    1000k fpp    1.0%           9.44e+04 1.12e+05 1.13e+05     0.814X     0.952X     0.951X
+//    absent  ndv    1000k fpp    1.0%           1.02e+05 1.14e+05 1.15e+05     0.877X     0.973X     0.972X
+//    present ndv    1000k fpp    0.1%           1.01e+05 1.11e+05 1.12e+05     0.868X     0.951X     0.949X
+//    absent  ndv    1000k fpp    0.1%           1.08e+05 1.14e+05 1.15e+05     0.927X     0.975X     0.975X
+//    present ndv  100000k fpp   10.0%           3.18e+04 3.94e+04 5.18e+04     0.274X     0.336X     0.437X
+//    absent  ndv  100000k fpp   10.0%           2.74e+04 3.07e+04 3.49e+04     0.236X     0.262X     0.294X
+//    present ndv  100000k fpp    1.0%           3.07e+04 4.29e+04 5.51e+04     0.265X     0.366X     0.465X
+//    absent  ndv  100000k fpp    1.0%           2.67e+04  2.9e+04 3.25e+04      0.23X     0.248X     0.274X
+//    present ndv  100000k fpp    0.1%           2.78e+04 3.88e+04  4.9e+04      0.24X     0.331X     0.413X
+//    absent  ndv  100000k fpp    0.1%           2.44e+04 2.84e+04 3.02e+04     0.211X     0.242X     0.255X
 //
-// Without AVX2:
+// union:                     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+//                                                                          (relative) (relative) (relative)
+// ---------------------------------------------------------------------------------------------------------
+//            ndv      10k fpp   10.0%           1.81e+04 1.84e+04 1.86e+04         1X         1X         1X
+//            ndv      10k fpp    1.0%           8.25e+03 8.39e+03 8.47e+03     0.455X     0.455X     0.455X
+//            ndv      10k fpp    0.1%           4.02e+03 4.31e+03 4.35e+03     0.222X     0.234X     0.234X
+//            ndv    1000k fpp   10.0%                105      111      112   0.00577X   0.00603X   0.00602X
+//            ndv    1000k fpp    1.0%               45.9     46.4     46.9   0.00253X   0.00252X   0.00252X
+//            ndv    1000k fpp    0.1%               46.2     46.6     46.9   0.00255X   0.00253X   0.00252X
+//            ndv  100000k fpp   10.0%                0.2      0.2      0.2   1.1e-05X  1.08e-05X  1.07e-05X
+//            ndv  100000k fpp    1.0%                0.2      0.2      0.2   1.1e-05X  1.08e-05X  1.07e-05X
+//            ndv  100000k fpp    0.1%              0.133    0.143    0.145  7.35e-06X  7.75e-06X  7.79e-06X
+//
+//
+// Without AVX or AVX2:
 //
 // insert:                    Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           1.25e+05 1.27e+05 1.28e+05         1X         1X         1X
-//            ndv      10k fpp    1.0%           1.27e+05 1.29e+05  1.3e+05      1.01X      1.02X      1.02X
-//            ndv      10k fpp    0.1%           1.26e+05 1.28e+05  1.3e+05         1X      1.01X      1.01X
-//            ndv    1000k fpp   10.0%           1.23e+05 1.25e+05 1.26e+05     0.977X     0.981X     0.985X
-//            ndv    1000k fpp    1.0%           1.16e+05 1.22e+05 1.23e+05     0.925X     0.958X     0.958X
-//            ndv    1000k fpp    0.1%           1.16e+05 1.22e+05 1.23e+05     0.928X     0.958X     0.957X
-//            ndv  100000k fpp   10.0%           3.77e+04 4.06e+04 5.62e+04     0.301X     0.319X     0.438X
-//            ndv  100000k fpp    1.0%           3.71e+04 4.06e+04 5.45e+04     0.296X      0.32X     0.425X
-//            ndv  100000k fpp    0.1%           3.37e+04 3.68e+04 5.15e+04     0.269X      0.29X     0.401X
+//            ndv      10k fpp   10.0%           9.27e+04 9.33e+04  9.4e+04         1X         1X         1X
+//            ndv      10k fpp    1.0%           9.43e+04 9.49e+04 9.61e+04      1.02X      1.02X      1.02X
+//            ndv      10k fpp    0.1%           9.36e+04  9.5e+04 9.58e+04      1.01X      1.02X      1.02X
+//            ndv    1000k fpp   10.0%            8.4e+04 9.49e+04 9.61e+04     0.906X      1.02X      1.02X
+//            ndv    1000k fpp    1.0%           7.64e+04 9.34e+04 9.45e+04     0.824X         1X      1.01X
+//            ndv    1000k fpp    0.1%           8.24e+04 9.34e+04 9.44e+04     0.888X         1X         1X
+//            ndv  100000k fpp   10.0%           3.22e+04    4e+04 5.03e+04     0.347X     0.429X     0.535X
+//            ndv  100000k fpp    1.0%           2.77e+04  3.6e+04  4.8e+04     0.298X     0.386X      0.51X
+//            ndv  100000k fpp    0.1%           2.54e+04 2.93e+04 4.32e+04     0.274X     0.314X      0.46X
 //
 // find:                      Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//    present ndv      10k fpp   10.0%            1.6e+05 1.64e+05 1.66e+05         1X         1X         1X
-//    absent  ndv      10k fpp   10.0%           1.11e+05 1.14e+05 1.15e+05     0.696X     0.697X     0.695X
-//    present ndv      10k fpp    1.0%           1.57e+05 1.63e+05 1.64e+05     0.982X     0.994X     0.989X
-//    absent  ndv      10k fpp    1.0%            1.3e+05 1.33e+05 1.35e+05     0.814X     0.813X     0.812X
-//    present ndv      10k fpp    0.1%           1.55e+05 1.58e+05 1.61e+05     0.967X     0.968X     0.969X
-//    absent  ndv      10k fpp    0.1%           2.26e+05 2.29e+05 2.31e+05      1.41X       1.4X       1.4X
-//    present ndv    1000k fpp   10.0%           1.21e+05 1.23e+05 1.25e+05     0.758X     0.753X     0.756X
-//    absent  ndv    1000k fpp   10.0%            7.6e+04 7.72e+04 7.81e+04     0.475X     0.472X     0.471X
-//    present ndv    1000k fpp    1.0%           1.23e+05 1.27e+05 1.28e+05     0.771X     0.773X      0.77X
-//    absent  ndv    1000k fpp    1.0%           1.19e+05 1.21e+05 1.22e+05     0.744X     0.739X     0.738X
-//    present ndv    1000k fpp    0.1%           1.17e+05 1.18e+05  1.2e+05     0.731X     0.724X     0.723X
-//    absent  ndv    1000k fpp    0.1%           1.13e+05 1.16e+05 1.17e+05     0.707X     0.706X     0.705X
-//    present ndv  100000k fpp   10.0%           3.42e+04 3.63e+04  3.9e+04     0.214X     0.222X     0.235X
-//    absent  ndv  100000k fpp   10.0%            3.6e+04 3.77e+04 3.82e+04     0.225X      0.23X      0.23X
-//    present ndv  100000k fpp    1.0%           3.18e+04 3.42e+04 3.57e+04     0.199X     0.209X     0.216X
-//    absent  ndv  100000k fpp    1.0%           3.63e+04 3.73e+04 3.79e+04     0.227X     0.228X     0.229X
-//    present ndv  100000k fpp    0.1%           2.89e+04  3.2e+04 3.33e+04      0.18X     0.196X     0.201X
-//    absent  ndv  100000k fpp    0.1%           4.56e+04 4.78e+04 4.86e+04     0.285X     0.292X     0.293X
+//    present ndv      10k fpp   10.0%            1.3e+05 1.31e+05 1.33e+05         1X         1X         1X
+//    absent  ndv      10k fpp   10.0%           8.74e+04 8.83e+04 8.92e+04     0.674X     0.673X     0.671X
+//    present ndv      10k fpp    1.0%           1.25e+05  1.3e+05 1.31e+05      0.96X     0.991X     0.988X
+//    absent  ndv      10k fpp    1.0%           1.04e+05 1.06e+05 1.07e+05     0.805X     0.809X     0.807X
+//    present ndv      10k fpp    0.1%           1.28e+05  1.3e+05 1.31e+05     0.986X     0.988X     0.984X
+//    absent  ndv      10k fpp    0.1%           1.69e+05 1.72e+05 1.74e+05       1.3X      1.31X      1.31X
+//    present ndv    1000k fpp   10.0%           9.33e+04  9.6e+04 9.69e+04     0.719X     0.732X     0.729X
+//    absent  ndv    1000k fpp   10.0%           5.99e+04 6.07e+04 6.12e+04     0.462X     0.462X     0.461X
+//    present ndv    1000k fpp    1.0%           9.48e+04 1.01e+05 1.02e+05     0.731X     0.768X     0.768X
+//    absent  ndv    1000k fpp    1.0%           9.49e+04 9.67e+04 9.74e+04     0.731X     0.737X     0.734X
+//    present ndv    1000k fpp    0.1%           8.46e+04  9.3e+04 9.41e+04     0.652X     0.709X     0.709X
+//    absent  ndv    1000k fpp    0.1%           9.05e+04 9.18e+04 9.28e+04     0.697X       0.7X     0.699X
+//    present ndv  100000k fpp   10.0%            2.6e+04 2.88e+04 3.11e+04     0.201X      0.22X     0.235X
+//    absent  ndv  100000k fpp   10.0%           2.88e+04 2.99e+04 3.08e+04     0.222X     0.228X     0.232X
+//    present ndv  100000k fpp    1.0%           2.34e+04 2.76e+04 2.91e+04      0.18X      0.21X     0.219X
+//    absent  ndv  100000k fpp    1.0%           2.86e+04 2.97e+04 3.03e+04      0.22X     0.227X     0.228X
+//    present ndv  100000k fpp    0.1%           2.34e+04 2.65e+04 2.81e+04      0.18X     0.202X     0.211X
+//    absent  ndv  100000k fpp    0.1%           3.73e+04 3.85e+04 3.91e+04     0.287X     0.293X     0.295X
+//
+// union:                     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+//                                                                          (relative) (relative) (relative)
+// ---------------------------------------------------------------------------------------------------------
+//            ndv      10k fpp   10.0%           3.06e+03  3.1e+03 3.12e+03         1X         1X         1X
+//            ndv      10k fpp    1.0%           1.51e+03 1.55e+03 1.57e+03     0.493X     0.502X     0.503X
+//            ndv      10k fpp    0.1%                748      775      782     0.244X      0.25X     0.251X
+//            ndv    1000k fpp   10.0%               19.6       20     20.2    0.0064X   0.00646X   0.00647X
+//            ndv    1000k fpp    1.0%               9.41       10     10.1   0.00307X   0.00324X   0.00323X
+//            ndv    1000k fpp    0.1%                9.9       10     10.1   0.00323X   0.00324X   0.00323X
+//            ndv  100000k fpp   10.0%             0.0671   0.0714   0.0725  2.19e-05X   2.3e-05X  2.32e-05X
+//            ndv  100000k fpp    1.0%             0.0676   0.0709   0.0719  2.21e-05X  2.29e-05X  2.31e-05X
+//            ndv  100000k fpp    0.1%             0.0338    0.035   0.0356   1.1e-05X  1.13e-05X  1.14e-05X
 
 // Make a random uint32_t, avoiding the absent high bit and the low-entropy low bits
 // produced by rand().
@@ -221,6 +249,27 @@ void Absent(int batch_size, void* data) {
 
 }  // namespace find
 
+// Benchmark or
+namespace either {
+
+struct TestData {
+  explicit TestData(int log_heap_size) {
+    BloomFilter bf(log_heap_size);
+    BloomFilter::ToThrift(&bf, &tbf);
+  }
+
+  TBloomFilter tbf;
+};
+
+void Benchmark(int batch_size, void* data) {
+  TestData* d = reinterpret_cast<TestData*>(data);
+  for (int i = 0; i < batch_size; ++i) {
+    BloomFilter::Or(d->tbf, &d->tbf);
+  }
+}
+
+} // namespace either
+
 void RunBenchmarks() {
 
   char name[120];
@@ -254,6 +303,20 @@ void RunBenchmarks() {
     }
     cout << suite.Measure() << endl;
   }
+
+  {
+    Benchmark suite("union");
+    vector<unique_ptr<either::TestData> > testdata;
+    for (int ndv = 10000; ndv <= 100 * 1000 * 1000; ndv *= 100) {
+      for (double fpp = 0.1; fpp >= 0.001; fpp /= 10) {
+        testdata.emplace_back(
+            new either::TestData(BloomFilter::MinLogSpace(ndv, fpp)));
+        snprintf(name, sizeof(name), "ndv %7dk fpp %6.1f%%", ndv/1000, fpp*100);
+        suite.AddBenchmark(name, either::Benchmark, testdata.back().get());
+      }
+    }
+    cout << suite.Measure() << endl;
+  }
 }
 
 int main(int argc, char **argv) {
@@ -277,7 +340,8 @@ int main(int argc, char **argv) {
 
   cout << "With AVX2:" << endl << endl;
   RunBenchmarks();
-  cout << endl << "Without AVX2:" << endl << endl;
-  CpuInfo::TempDisable t(CpuInfo::AVX2);
+  cout << endl << "Without AVX or AVX2:" << endl << endl;
+  CpuInfo::TempDisable t1(CpuInfo::AVX);
+  CpuInfo::TempDisable t2(CpuInfo::AVX2);
   RunBenchmarks();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/61fcb489/be/src/testutil/mem-util.h
----------------------------------------------------------------------
diff --git a/be/src/testutil/mem-util.h b/be/src/testutil/mem-util.h
index 78b7b48..b4ce9ea 100644
--- a/be/src/testutil/mem-util.h
+++ b/be/src/testutil/mem-util.h
@@ -21,6 +21,8 @@
 #include <cstdint>
 #include <cstdlib>
 
+#include <glog/logging.h>
+
 #include "gutil/macros.h"
 
 namespace impala {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/61fcb489/be/src/util/bloom-filter.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.cc b/be/src/util/bloom-filter.cc
index 6fd53f5..2aadc05 100644
--- a/be/src/util/bloom-filter.cc
+++ b/be/src/util/bloom-filter.cc
@@ -154,6 +154,21 @@ bool BloomFilter::BucketFind(
   return true;
 }
 
+namespace {
+// Computes out[i] |= in[i] for the arrays 'in' and 'out' of length 'n' using AVX
+// instructions. 'n' must be a multiple of 32.
+void __attribute__((target("avx"))) OrEqualArrayAvx(size_t n, const char* in, char* out) {
+  constexpr size_t REGISTER_SIZE = sizeof(__m256d);
+  DCHECK_EQ(n % REGISTER_SIZE, 0) << "Invalid Bloom Filter directory size";
+  const double* simd_in = reinterpret_cast<const double*>(in);
+  double* simd_out = reinterpret_cast<double*>(out);
+  const size_t simd_size = n / REGISTER_SIZE;
+  for (size_t i = 0; i < simd_size; i += REGISTER_SIZE / sizeof(simd_in[0])) {
+    _mm256_storeu_pd(simd_out + i,
+        _mm256_or_pd(_mm256_loadu_pd(simd_out + i), _mm256_loadu_pd(simd_in + i)));
+  }
+}
+} //namespace
 
 void BloomFilter::Or(const TBloomFilter& in, TBloomFilter* out) {
   DCHECK(out != NULL);
@@ -163,8 +178,29 @@ void BloomFilter::Or(const TBloomFilter& in, TBloomFilter* out) {
     out->directory.resize(0);
     return;
   }
-
-  for (int i = 0; i < in.directory.size(); ++i) out->directory[i] |= in.directory[i];
+  // The trivial loop out[i] |= in[i] should auto-vectorize with gcc at -O3, but it is not
+  // written in a way that is very friendly to auto-vectorization. Instead, we manually
+  // vectorize, increasing the speed by up to 184x.
+  //
+  // TODO: Tune gcc flags to auto-vectorize the trivial loop instead of hand-vectorizing
+  // it. This might not be possible.
+  if (CpuInfo::IsSupported(CpuInfo::AVX)) {
+    OrEqualArrayAvx(in.directory.size(), &in.directory[0], &out->directory[0]);
+  } else {
+    const __m128i* simd_in = reinterpret_cast<const __m128i*>(&in.directory[0]);
+    __m128i* simd_out = reinterpret_cast<__m128i*>(&out->directory[0]);
+    const size_t simd_size =
+        (in.directory.size() * sizeof(in.directory[0])) / sizeof(simd_in[0]);
+    // in.directory has a size (in bytes) that is a multiple of 32. Since sizeof(__m128i)
+    // == 16, we can do two _mm_or_si128's in each iteration without checking array
+    // bounds.
+    for (size_t i = 0; i < simd_size; i += 2) {
+      _mm_storeu_si128(simd_out + i,
+          _mm_or_si128(_mm_loadu_si128(simd_out + i), _mm_loadu_si128(simd_in + i)));
+      _mm_storeu_si128(simd_out + i + 1, _mm_or_si128(_mm_loadu_si128(simd_out + i + 1),
+                                             _mm_loadu_si128(simd_in + i + 1)));
+    }
+  }
 }
 
 // The following three methods are derived from
@@ -187,14 +223,13 @@ int BloomFilter::MinLogSpace(const size_t ndv, const double fpp) {
   const double m = -k * ndv / log(1 - pow(fpp, 1.0 / k));
 
   // Handle case where ndv == 1 => ceil(log2(m/8)) < 0.
-  return max(0, static_cast<int>(ceil(log2(m/8))));
+  return max(0, static_cast<int>(ceil(log2(m / 8))));
 }
 
 double BloomFilter::FalsePositiveProb(const size_t ndv, const int log_heap_space) {
-  return pow(
-      1 - exp((-1.0 * static_cast<double>(BUCKET_WORDS) * static_cast<double>(ndv)) /
-              static_cast<double>(1ull << (log_heap_space + 3))),
+  return pow(1 - exp((-1.0 * static_cast<double>(BUCKET_WORDS) * static_cast<double>(ndv))
+                     / static_cast<double>(1ull << (log_heap_space + 3))),
       BUCKET_WORDS);
 }
 
-}  // namespace impala
+} // namespace impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/61fcb489/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 532c9dd..6329ca8 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -77,6 +77,7 @@ static struct {
   { "sse4_1", CpuInfo::SSE4_1 },
   { "sse4_2", CpuInfo::SSE4_2 },
   { "popcnt", CpuInfo::POPCNT },
+  { "avx",    CpuInfo::AVX },
   { "avx2",   CpuInfo::AVX2 },
 };
 static const long num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/61fcb489/be/src/util/cpu-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index cb577c2..868d2dd 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -36,7 +36,8 @@ class CpuInfo {
   static const int64_t SSE4_1  = (1 << 2);
   static const int64_t SSE4_2  = (1 << 3);
   static const int64_t POPCNT  = (1 << 4);
-  static const int64_t AVX2    = (1 << 5);
+  static const int64_t AVX     = (1 << 5);
+  static const int64_t AVX2    = (1 << 6);
 
   /// Cache enums for L1 (data), L2 and L3
   enum CacheLevel {


[06/33] incubator-impala git commit: IMPALA-4269: Codegen merging exchange node

Posted by ta...@apache.org.
IMPALA-4269: Codegen merging exchange node

This change enables codegen for the tuple row comparator
used in merging-exchange node.

With this change, merging-exchange operator improves by
40% and 50% respectively for primitive_orderby_bigint and
primitive_orderby_all on TPCH-300, speeding up the query by
6% and 11% respectively.

Change-Id: I944b8d52ea63ede58e4dc6fbe6e6953756394d41
Reviewed-on: http://gerrit.cloudera.org:8080/4759
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/502220c6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/502220c6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/502220c6

Branch: refs/heads/hadoop-next
Commit: 502220c69d483f785632eaf0030babf40be78ff4
Parents: f1f54fe
Author: Michael Ho <kw...@cloudera.com>
Authored: Tue Oct 18 22:37:33 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Oct 20 04:42:09 2016 +0000

----------------------------------------------------------------------
 be/src/exec/exchange-node.cc | 12 ++++++++++++
 be/src/exec/exchange-node.h  |  1 +
 2 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/502220c6/be/src/exec/exchange-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.cc b/be/src/exec/exchange-node.cc
index fac2ff5..13522a7 100644
--- a/be/src/exec/exchange-node.cc
+++ b/be/src/exec/exchange-node.cc
@@ -85,10 +85,22 @@ Status ExchangeNode::Prepare(RuntimeState* state) {
     AddExprCtxsToFree(sort_exec_exprs_);
     less_than_.reset(
         new TupleRowComparator(sort_exec_exprs_, is_asc_order_, nulls_first_));
+    if (!state->codegen_enabled()) {
+      runtime_profile()->AddCodegenMsg(false, "disabled by query option DISABLE_CODEGEN");
+    }
   }
   return Status::OK();
 }
 
+void ExchangeNode::Codegen(RuntimeState* state) {
+  DCHECK(state->codegen_enabled());
+  if (is_merging_) {
+    Status codegen_status = less_than_->Codegen(state);
+    runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+  }
+  ExecNode::Codegen(state);
+}
+
 Status ExchangeNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Open(state));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/502220c6/be/src/exec/exchange-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.h b/be/src/exec/exchange-node.h
index f6face4..6feaff3 100644
--- a/be/src/exec/exchange-node.h
+++ b/be/src/exec/exchange-node.h
@@ -46,6 +46,7 @@ class ExchangeNode : public ExecNode {
 
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
   virtual Status Prepare(RuntimeState* state);
+  virtual void Codegen(RuntimeState* state);
   /// Blocks until the first batch is available for consumption via GetNext().
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);


[33/33] incubator-impala git commit: Merge remote-tracking branch 'gerrit/master' into HEAD

Posted by ta...@apache.org.
Merge remote-tracking branch 'gerrit/master' into HEAD

Change-Id: I1b56e4dbb67889bbe4ff8462ccc2b49968821a1f


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

Branch: refs/heads/hadoop-next
Commit: eae4f307d880d98293ef85633cfe13a2ebb3b06b
Parents: 3cb3f34 13455b5
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Oct 25 13:12:51 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Oct 25 13:13:53 2016 -0700

----------------------------------------------------------------------
 .gitignore                                      |   2 +-
 LICENSE.txt                                     | 130 ++-
 README.md                                       |  10 +-
 be/CMakeLists.txt                               |   2 +-
 be/src/benchmarks/bitmap-benchmark.cc           |   6 +-
 be/src/benchmarks/bloom-filter-benchmark.cc     | 198 ++--
 be/src/codegen/CMakeLists.txt                   |   7 +
 be/src/codegen/codegen-anyval.cc                |   8 +-
 be/src/codegen/llvm-codegen.cc                  |  10 +-
 be/src/codegen/llvm-codegen.h                   |   9 +-
 be/src/exec/aggregation-node.cc                 |  61 +-
 be/src/exec/aggregation-node.h                  |  11 +-
 be/src/exec/analytic-eval-node.cc               |  88 +-
 be/src/exec/analytic-eval-node.h                |  28 +-
 be/src/exec/data-sink.cc                        |   5 +
 be/src/exec/exchange-node.cc                    |  17 +-
 be/src/exec/exchange-node.h                     |   5 +
 be/src/exec/exec-node.cc                        |  21 +-
 be/src/exec/exec-node.h                         |  13 +-
 be/src/exec/hash-join-node.cc                   |  51 +-
 be/src/exec/hash-join-node.h                    |   6 +-
 be/src/exec/hash-table.cc                       |  58 +-
 be/src/exec/hash-table.h                        |  12 +-
 be/src/exec/hbase-table-sink.cc                 |   4 +-
 be/src/exec/hdfs-avro-scanner.cc                |  18 +-
 be/src/exec/hdfs-avro-scanner.h                 |   2 +-
 be/src/exec/hdfs-parquet-scanner.cc             |  12 +-
 be/src/exec/hdfs-rcfile-scanner.cc              |   2 +-
 be/src/exec/hdfs-scan-node-base.cc              |  30 +-
 be/src/exec/hdfs-scan-node-base.h               |   1 +
 be/src/exec/hdfs-scanner.cc                     |  66 +-
 be/src/exec/hdfs-scanner.h                      |  14 +-
 be/src/exec/hdfs-sequence-scanner.cc            |  12 +-
 be/src/exec/hdfs-table-sink.cc                  |   4 +-
 be/src/exec/hdfs-text-scanner.cc                |  10 +-
 be/src/exec/kudu-scan-node.cc                   |   3 -
 be/src/exec/kudu-scan-node.h                    |   3 -
 be/src/exec/kudu-scanner.cc                     |   4 +-
 be/src/exec/kudu-table-sink.cc                  | 141 ++-
 be/src/exec/kudu-table-sink.h                   |  53 +-
 be/src/exec/nested-loop-join-node.cc            |  12 +-
 be/src/exec/old-hash-table.cc                   |  62 +-
 be/src/exec/old-hash-table.h                    |   6 +-
 be/src/exec/partitioned-aggregation-node.cc     |  76 +-
 be/src/exec/partitioned-aggregation-node.h      |  13 +-
 be/src/exec/partitioned-hash-join-builder.cc    |  83 +-
 be/src/exec/partitioned-hash-join-builder.h     |  19 +-
 be/src/exec/partitioned-hash-join-node.cc       |  56 +-
 be/src/exec/partitioned-hash-join-node.h        |   4 +-
 be/src/exec/plan-root-sink.cc                   |   8 +-
 be/src/exec/sort-node.cc                        |  23 +-
 be/src/exec/sort-node.h                         |   4 +
 be/src/exec/topn-node.cc                        |  94 +-
 be/src/exec/topn-node.h                         |   4 +-
 be/src/exprs/agg-fn-evaluator.h                 |  27 +-
 be/src/exprs/aggregate-functions-ir.cc          |  11 +-
 be/src/exprs/case-expr.cc                       |   6 +-
 be/src/exprs/case-expr.h                        |   2 +-
 be/src/exprs/compound-predicates.cc             |  37 +-
 be/src/exprs/compound-predicates.h              |  10 +-
 be/src/exprs/conditional-functions.cc           |   4 +-
 be/src/exprs/conditional-functions.h            |   8 +-
 be/src/exprs/expr.cc                            |  14 +-
 be/src/exprs/expr.h                             |   8 +-
 be/src/exprs/hive-udf-call.cc                   |   4 +-
 be/src/exprs/hive-udf-call.h                    |   2 +-
 be/src/exprs/is-not-empty-predicate.cc          |   4 +-
 be/src/exprs/is-not-empty-predicate.h           |   2 +-
 be/src/exprs/literal.cc                         |   4 +-
 be/src/exprs/literal.h                          |   2 +-
 be/src/exprs/null-literal.cc                    |   4 +-
 be/src/exprs/null-literal.h                     |   2 +-
 be/src/exprs/scalar-fn-call.cc                  | 246 ++---
 be/src/exprs/scalar-fn-call.h                   |   4 +-
 be/src/exprs/slot-ref.cc                        |   5 +-
 be/src/exprs/slot-ref.h                         |   2 +-
 be/src/exprs/tuple-is-null-predicate.cc         |   6 +-
 be/src/exprs/tuple-is-null-predicate.h          |   2 +-
 be/src/runtime/coordinator.cc                   |  14 +-
 be/src/runtime/coordinator.h                    |   1 -
 be/src/runtime/plan-fragment-executor.cc        |  16 +-
 be/src/runtime/runtime-state.cc                 |   8 -
 be/src/runtime/runtime-state.h                  |  27 +-
 be/src/runtime/sorted-run-merger.h              |   2 +-
 be/src/runtime/sorter.cc                        |   2 +-
 be/src/runtime/sorter.h                         |   2 +-
 be/src/runtime/tuple.cc                         |   6 +-
 be/src/runtime/tuple.h                          |   2 +-
 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 +-
 be/src/service/fe-support.cc                    |  36 +-
 be/src/service/frontend.cc                      |   9 +-
 be/src/service/impala-beeswax-server.cc         |   4 +-
 be/src/service/query-exec-state.cc              |   4 +-
 be/src/service/query-options.cc                 |   1 +
 be/src/testutil/mem-util.h                      |   2 +
 be/src/testutil/test-udfs.cc                    |  29 +
 be/src/udf/udf.cc                               |   3 +-
 be/src/util/bit-util.h                          |   4 +
 be/src/util/bitmap-test.cc                      |  82 +-
 be/src/util/bitmap.cc                           |   8 +-
 be/src/util/bitmap.h                            |  42 +-
 be/src/util/bloom-filter.cc                     |  49 +-
 be/src/util/cpu-info.cc                         |   1 +
 be/src/util/cpu-info.h                          |   3 +-
 be/src/util/tuple-row-compare.cc                |  13 +-
 be/src/util/tuple-row-compare.h                 |   2 +-
 bin/bootstrap_toolchain.py                      |   9 +-
 bin/distcc/.gitignore                           |   1 +
 bin/distcc/README.md                            | 106 +++
 bin/distcc/distcc.sh                            |  62 ++
 bin/distcc/distcc_env.sh                        | 160 ++++
 bin/impala-config.sh                            |  12 +-
 bin/rat_exclude_files.txt                       |   1 -
 bin/start-catalogd.sh                           |  30 +-
 bin/start-impala-cluster.py                     |  11 +-
 buildall.sh                                     |   3 +
 common/thrift/CatalogObjects.thrift             |  39 +-
 common/thrift/ImpalaInternalService.thrift      |   4 +-
 common/thrift/ImpalaService.thrift              |   8 +-
 common/thrift/JniCatalog.thrift                 |  44 +-
 common/thrift/PlanNodes.thrift                  |   7 +-
 common/thrift/generate_error_codes.py           |   9 +
 fe/src/main/cup/sql-parser.cup                  | 407 ++++----
 .../apache/impala/analysis/AnalysisUtils.java   |  43 +
 .../apache/impala/analysis/AnalyticExpr.java    |  10 +-
 .../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 +++--
 .../java/org/apache/impala/analysis/Expr.java   |   6 +-
 .../apache/impala/analysis/FunctionParams.java  |   3 +-
 .../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 +-
 .../org/apache/impala/catalog/BuiltinsDb.java   |   4 +-
 .../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 +-
 .../org/apache/impala/catalog/HdfsTable.java    |  29 +-
 .../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 -
 .../apache/impala/common/FileSystemUtil.java    |   2 +-
 .../impala/planner/HdfsPartitionFilter.java     |   2 +-
 .../org/apache/impala/planner/HdfsScanNode.java |  40 -
 .../org/apache/impala/planner/KuduScanNode.java |   2 +-
 .../impala/service/CatalogOpExecutor.java       | 416 ++++----
 .../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 ++++----
 .../impala/analysis/AnalyzeExprsTest.java       |  11 +-
 .../org/apache/impala/analysis/ParserTest.java  |  32 +-
 .../org/apache/impala/service/JdbcTest.java     |   6 +-
 .../impala/testutil/ImpaladTestCatalog.java     |   6 +-
 fe/src/test/resources/hbase-site.xml.template   |   2 -
 infra/python/deps/download_requirements         |   2 +-
 infra/python/deps/requirements.txt              |   6 +-
 llvm-ir/test-loop.bc                            | Bin 1828 -> 0 bytes
 shell/impala_client.py                          |   2 +-
 testdata/bin/create-load-data.sh                |   4 +-
 testdata/bin/generate-schema-statements.py      |  20 +-
 testdata/bin/load-tpc-kudu.py                   | 119 +++
 testdata/bin/run-step.sh                        |   3 +-
 .../common/etc/hadoop/conf/hadoop-policy.xml    |   2 -
 .../etc/hadoop/conf/log4j.properties.tmpl       |   2 -
 .../functional/functional_schema_template.sql   | 156 +--
 testdata/datasets/tpcds/tpcds_kudu_template.sql | 945 +++++++++++++++++++
 testdata/datasets/tpch/tpch_kudu_template.sql   | 198 ++++
 testdata/datasets/tpch/tpch_schema_template.sql | 120 +--
 .../queries/PlannerTest/kudu.test               |  28 +-
 .../queries/PlannerTest/subquery-rewrite.test   |  23 +
 .../queries/QueryTest/alter-table.test          |  38 +-
 .../queries/QueryTest/analytic-fns.test         |  14 +
 .../queries/QueryTest/create_kudu.test          |  90 --
 .../queries/QueryTest/joins.test                |  10 +
 .../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            | 161 ++--
 .../queries/QueryTest/kudu_partition_ddl.test   | 103 +-
 .../queries/QueryTest/kudu_stats.test           |  12 +-
 .../queries/QueryTest/mt-dop.test               |   9 +
 .../queries/QueryTest/udf-errors.test           |  48 +
 .../functional-query/queries/QueryTest/udf.test |  14 +
 .../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/beeswax/impala_beeswax.py                 |   4 +-
 tests/common/__init__.py                        |   4 +-
 tests/common/kudu_test_suite.py                 | 148 +++
 tests/comparison/db_connection.py               |   5 +
 tests/conftest.py                               |  10 +-
 tests/custom_cluster/test_client_ssl.py         |   1 +
 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_cancellation.py           |  49 +-
 tests/query_test/test_kudu.py                   | 560 ++++++++---
 tests/query_test/test_mt_dop.py                 |  47 +
 tests/query_test/test_udfs.py                   |  18 +
 tests/stress/concurrent_select.py               |  30 +-
 241 files changed, 7624 insertions(+), 3492 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/bin/impala-config.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/eae4f307/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------


[27/33] incubator-impala git commit: Minor compute stats script fixes

Posted by ta...@apache.org.
Minor compute stats script fixes

* Change run-step to output full log path
* Change text to say "Computing table stats" rather than "Computing
  HBase stats" when running compute-table-stats.sh

Change-Id: I326f4c370fda8d5e388af8e2395623185c06bc07
Reviewed-on: http://gerrit.cloudera.org:8080/4825
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/e0a32721
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/e0a32721
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/e0a32721

Branch: refs/heads/hadoop-next
Commit: e0a32721297db9d85da4a1d749672289537aca50
Parents: 61fcb48
Author: Henry Robinson <he...@cloudera.com>
Authored: Mon Oct 24 11:45:46 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Oct 25 00:13:54 2016 +0000

----------------------------------------------------------------------
 testdata/bin/create-load-data.sh | 4 ++--
 testdata/bin/run-step.sh         | 3 +--
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e0a32721/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index 38e38df..7631c1c 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -425,9 +425,9 @@ if [ "${TARGET_FILESYSTEM}" = "hdfs" ]; then
       create-internal-hbase-table
 fi
 
-# TODO: Investigate why all stats are not preserved. Theorectically, we only need to
+# TODO: Investigate why all stats are not preserved. Theoretically, we only need to
 # recompute stats for HBase.
-run-step "Computing HBase stats" compute-hbase-stats.log \
+run-step "Computing table stats" compute-table-stats.log \
     ${IMPALA_HOME}/testdata/bin/compute-table-stats.sh
 
 run-step "Copying auth policy file" copy-auth-policy.log copy-auth-policy

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/e0a32721/testdata/bin/run-step.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-step.sh b/testdata/bin/run-step.sh
old mode 100644
new mode 100755
index b7d1671..faef360
--- a/testdata/bin/run-step.sh
+++ b/testdata/bin/run-step.sh
@@ -36,7 +36,7 @@ function run-step {
   fi
   local LOG=${LOG_DIR}/${LOG_FILE_NAME}
 
-  echo -n "${MSG} (logging to ${LOG_FILE_NAME})... "
+  echo -n "${MSG} (logging to ${LOG})... "
   echo "Log for command '$@'" > ${LOG}
   if ! "$@" >> ${LOG} 2>&1 ; then
     echo "FAILED"
@@ -46,4 +46,3 @@ function run-step {
   fi
   echo OK
 }
-


[10/33] 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/hadoop-next
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);
   }
 
   /**


[25/33] incubator-impala git commit: Remove unused Bitmap code.

Posted by ta...@apache.org.
Remove unused Bitmap code.

These methods and code paths have been made obsolete by the switch to
Bloom filters.

Change-Id: I95fcaaa40243999800c2ec2ead5b3479d66a63e7
Reviewed-on: http://gerrit.cloudera.org:8080/4801
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/0fbb5b7e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/0fbb5b7e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/0fbb5b7e

Branch: refs/heads/hadoop-next
Commit: 0fbb5b7e71e55346c8b97ec143854dba0088f124
Parents: ff6b450
Author: Jim Apple <jb...@cloudera.com>
Authored: Sat Oct 22 10:42:51 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Oct 24 17:53:33 2016 +0000

----------------------------------------------------------------------
 be/src/benchmarks/bitmap-benchmark.cc |  6 +--
 be/src/exec/hash-table.h              |  4 +-
 be/src/exec/nested-loop-join-node.cc  | 12 ++---
 be/src/util/bitmap-test.cc            | 82 +++++-------------------------
 be/src/util/bitmap.cc                 |  8 +--
 be/src/util/bitmap.h                  | 42 ++-------------
 6 files changed, 32 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/benchmarks/bitmap-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/bitmap-benchmark.cc b/be/src/benchmarks/bitmap-benchmark.cc
index 26c51d8..5c1c9e6 100644
--- a/be/src/benchmarks/bitmap-benchmark.cc
+++ b/be/src/benchmarks/bitmap-benchmark.cc
@@ -110,7 +110,7 @@ struct TestData {
 void Benchmark(int batch_size, void* data) {
   TestData* d = reinterpret_cast<TestData*>(data);
   for (int i = 0; i < batch_size; ++i) {
-    d->bm.Set<true>(d->data[i & (d->data.size() - 1)], true);
+    d->bm.Set(d->data[i & (d->data.size() - 1)] % d->bm.num_bits(), true);
   }
 }
 
@@ -122,7 +122,7 @@ struct TestData {
   TestData(int64_t size)
     : bm(size), data (1ull << 20) {
     for (size_t i = 0; i < size/2; ++i) {
-      bm.Set<true>(MakeNonNegativeRand(), true);
+      bm.Set(MakeNonNegativeRand() % size, true);
     }
     for (size_t i = 0; i < data.size(); ++i) {
       data[i] = MakeNonNegativeRand();
@@ -138,7 +138,7 @@ struct TestData {
 void Benchmark(int batch_size, void* data) {
   TestData* d = reinterpret_cast<TestData*>(data);
   for (int i = 0; i < batch_size; ++i) {
-    d->result += d->bm.Get<true>(d->data[i & (d->data.size() - 1)]);
+    d->result += d->bm.Get(d->data[i & (d->data.size() - 1)] % d->bm.num_bits());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/exec/hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index 404b294..2ebc22f 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -290,11 +290,11 @@ class HashTableCtx {
 
     /// Returns true if the current row is null but nulls are not considered in the current
     /// phase (build or probe).
-    bool ALWAYS_INLINE IsRowNull() const { return null_bitmap_.Get<false>(CurIdx()); }
+    bool ALWAYS_INLINE IsRowNull() const { return null_bitmap_.Get(CurIdx()); }
 
     /// Record in a bitmap that the current row is null but nulls are not considered in
     /// the current phase (build or probe).
-    void ALWAYS_INLINE SetRowNull() { null_bitmap_.Set<false>(CurIdx(), true); }
+    void ALWAYS_INLINE SetRowNull() { null_bitmap_.Set(CurIdx(), true); }
 
     /// Returns the hash values of the current row.
     uint32_t ALWAYS_INLINE CurExprValuesHash() const { return *cur_expr_values_hash_; }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/exec/nested-loop-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/nested-loop-join-node.cc b/be/src/exec/nested-loop-join-node.cc
index 0a115c6..6c75797 100644
--- a/be/src/exec/nested-loop-join-node.cc
+++ b/be/src/exec/nested-loop-join-node.cc
@@ -410,7 +410,7 @@ Status NestedLoopJoinNode::GetNextRightSemiJoin(RuntimeState* state,
       }
 
       // Check if we already have a match for the build row.
-      if (matching_build_rows_->Get<false>(current_build_row_idx_)) {
+      if (matching_build_rows_->Get(current_build_row_idx_)) {
         build_row_iterator_.Next();
         ++current_build_row_idx_;
         continue;
@@ -424,7 +424,7 @@ Status NestedLoopJoinNode::GetNextRightSemiJoin(RuntimeState* state,
         continue;
       }
       TupleRow* output_row = output_batch->GetRow(output_batch->AddRow());
-      matching_build_rows_->Set<false>(current_build_row_idx_, true);
+      matching_build_rows_->Set(current_build_row_idx_, true);
       output_batch->CopyRow(build_row_iterator_.GetRow(), output_row);
       build_row_iterator_.Next();
       ++current_build_row_idx_;
@@ -461,7 +461,7 @@ Status NestedLoopJoinNode::GetNextRightAntiJoin(RuntimeState* state,
         RETURN_IF_ERROR(QueryMaintenance(state));
       }
 
-      if (matching_build_rows_->Get<false>(current_build_row_idx_)) {
+      if (matching_build_rows_->Get(current_build_row_idx_)) {
         build_row_iterator_.Next();
         ++current_build_row_idx_;
         continue;
@@ -469,7 +469,7 @@ Status NestedLoopJoinNode::GetNextRightAntiJoin(RuntimeState* state,
       CreateOutputRow(semi_join_staging_row_, current_probe_row_,
           build_row_iterator_.GetRow());
       if (EvalConjuncts(join_conjunct_ctxs, num_join_ctxs, semi_join_staging_row_)) {
-        matching_build_rows_->Set<false>(current_build_row_idx_, true);
+        matching_build_rows_->Set(current_build_row_idx_, true);
       }
       build_row_iterator_.Next();
       ++current_build_row_idx_;
@@ -548,7 +548,7 @@ Status NestedLoopJoinNode::ProcessUnmatchedBuildRows(
       RETURN_IF_ERROR(QueryMaintenance(state));
     }
 
-    if (matching_build_rows_->Get<false>(current_build_row_idx_)) {
+    if (matching_build_rows_->Get(current_build_row_idx_)) {
       build_row_iterator_.Next();
       ++current_build_row_idx_;
       continue;
@@ -612,7 +612,7 @@ Status NestedLoopJoinNode::FindBuildMatches(
     if (!EvalConjuncts(join_conjunct_ctxs, num_join_ctxs, output_row)) continue;
     matched_probe_ = true;
     if (matching_build_rows_ != NULL) {
-      matching_build_rows_->Set<false>(current_build_row_idx_ - 1, true);
+      matching_build_rows_->Set(current_build_row_idx_ - 1, true);
     }
     if (!EvalConjuncts(conjunct_ctxs, num_ctxs, output_row)) continue;
     VLOG_ROW << "match row: " << PrintRow(output_row, row_desc());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/util/bitmap-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bitmap-test.cc b/be/src/util/bitmap-test.cc
index 238b5fc..c0a3899 100644
--- a/be/src/util/bitmap-test.cc
+++ b/be/src/util/bitmap-test.cc
@@ -30,14 +30,14 @@ namespace impala {
 
 void CreateRandomBitmap(Bitmap* bitmap) {
   for (int64_t i = 0; i < bitmap->num_bits(); ++i) {
-    bitmap->Set<false>(i, rand() % 2 == 0);
+    bitmap->Set(i, rand() % 2 == 0);
   }
 }
 
 // Returns true if all the bits in the bitmap are equal to 'value'.
 bool CheckAll(const Bitmap& bitmap, const bool value) {
   for (int64_t i = 0; i < bitmap.num_bits(); ++i) {
-    if (bitmap.Get<false>(i) != value) return false;
+    if (bitmap.Get(i) != value) return false;
   }
   return true;
 }
@@ -70,24 +70,6 @@ TEST(Bitmap, SetAllTest) {
   EXPECT_TRUE(CheckAll(bm, false));
 }
 
-TEST(Bitmap, AndTest) {
-  Bitmap bm(1024);
-  CreateRandomBitmap(&bm);
-  Bitmap bm_zeros(1024);
-  bm_zeros.SetAllBits(false);
-  bm.And(&bm_zeros);
-  EXPECT_TRUE(CheckAll(bm, false));
-}
-
-TEST(Bitmap, OrTest) {
-  Bitmap bm(1024);
-  CreateRandomBitmap(&bm);
-  Bitmap bm_ones(1024);
-  bm_ones.SetAllBits(true);
-  bm.Or(&bm_ones);
-  EXPECT_TRUE(CheckAll(bm, true));
-}
-
 // Regression test for IMPALA-2155.
 TEST(Bitmap, SetGetTest) {
   Bitmap bm(1024);
@@ -96,36 +78,16 @@ TEST(Bitmap, SetGetTest) {
   // to 0 and 1.
   for (int64_t bit_idx = 0; bit_idx < 63; ++bit_idx) {
     for (int64_t i = 0; i < 4; ++i) {
-      bm.Set<false>((1 << (6 + i)) + bit_idx, (i + bit_idx) % 2 == 0);
+      bm.Set((1 << (6 + i)) + bit_idx, (i + bit_idx) % 2 == 0);
     }
   }
   for (int64_t bit_idx = 0; bit_idx < 63; ++bit_idx) {
     for (int64_t i = 0; i < 4; ++i) {
-      EXPECT_EQ(bm.Get<false>((1 << (6 + i)) + bit_idx), (i + bit_idx) % 2 == 0);
+      EXPECT_EQ(bm.Get((1 << (6 + i)) + bit_idx), (i + bit_idx) % 2 == 0);
     }
   }
 }
 
-TEST(Bitmap, SetGetModTest) {
-  Bitmap bm(256);
-  bm.SetAllBits(false);
-  for (int64_t bit_idx = 0; bit_idx < 1024; ++bit_idx) {
-    bm.Set<true>(bit_idx, true);
-    EXPECT_TRUE(bm.Get<true>(bit_idx));
-    bm.Set<true>(bit_idx, false);
-    EXPECT_FALSE(bm.Get<true>(bit_idx));
-  }
-
-  bm.SetAllBits(false);
-  EXPECT_TRUE(CheckAll(bm, false));
-  for (int64_t bit_idx = 0; bit_idx < 1024; ++bit_idx) {
-    bm.Set<true>(bit_idx, bit_idx % 2 == 0);
-  }
-  for (int64_t bit_idx = 0; bit_idx < 1024; ++bit_idx) {
-    EXPECT_EQ(bm.Get<true>(bit_idx), bit_idx % 2 == 0);
-  }
-}
-
 /// Regression test for IMPALA-2307.
 TEST(Bitmap, OverflowTest) {
   Bitmap bm(64);
@@ -133,36 +95,20 @@ TEST(Bitmap, OverflowTest) {
   int64_t bit_idx = 45;
   int64_t ovr_idx = 13;
 
-  bm.Set<false>(bit_idx, true);
-  EXPECT_FALSE(bm.Get<false>(ovr_idx));
-  EXPECT_TRUE(bm.Get<false>(bit_idx));
-
-  bm.SetAllBits(false);
-  bm.Set<false>(ovr_idx, true);
-  EXPECT_FALSE(bm.Get<false>(bit_idx));
-  EXPECT_TRUE(bm.Get<false>(ovr_idx));
-
-  bm.SetAllBits(false);
-  bm.Set<false>(ovr_idx, true);
-  bm.Set<false>(bit_idx, false);
-  EXPECT_TRUE(bm.Get<false>(ovr_idx));
-  EXPECT_FALSE(bm.Get<false>(bit_idx));
-
-  bm.SetAllBits(false);
-  bm.Set<true>(bit_idx, true);
-  EXPECT_FALSE(bm.Get<true>(ovr_idx));
-  EXPECT_TRUE(bm.Get<true>(bit_idx));
+  bm.Set(bit_idx, true);
+  EXPECT_FALSE(bm.Get(ovr_idx));
+  EXPECT_TRUE(bm.Get(bit_idx));
 
   bm.SetAllBits(false);
-  bm.Set<true>(ovr_idx, true);
-  EXPECT_FALSE(bm.Get<true>(bit_idx));
-  EXPECT_TRUE(bm.Get<true>(ovr_idx));
+  bm.Set(ovr_idx, true);
+  EXPECT_FALSE(bm.Get(bit_idx));
+  EXPECT_TRUE(bm.Get(ovr_idx));
 
   bm.SetAllBits(false);
-  bm.Set<true>(ovr_idx, true);
-  bm.Set<true>(bit_idx, false);
-  EXPECT_TRUE(bm.Get<true>(ovr_idx));
-  EXPECT_FALSE(bm.Get<true>(bit_idx));
+  bm.Set(ovr_idx, true);
+  bm.Set(bit_idx, false);
+  EXPECT_TRUE(bm.Get(ovr_idx));
+  EXPECT_FALSE(bm.Get(bit_idx));
 }
 
 /// Test that bitmap memory usage calculation is correct.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/util/bitmap.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bitmap.cc b/be/src/util/bitmap.cc
index b1f54bc..504c919 100644
--- a/be/src/util/bitmap.cc
+++ b/be/src/util/bitmap.cc
@@ -23,21 +23,21 @@
 
 using namespace impala;
 
-string Bitmap::DebugString(bool print_bits) {
+string Bitmap::DebugString(bool print_bits) const {
   int64_t words = BitUtil::RoundUp(num_bits_, 64) / 64;
   stringstream ss;
   ss << "Size (" << num_bits_ << ") words (" << words << ") ";
   if (print_bits) {
     for (int i = 0; i < num_bits(); ++i) {
-      if (Get<false>(i)) {
+      if (Get(i)) {
         ss << "1";
       } else {
         ss << "0";
       }
     }
   } else {
-    for (vector<uint64_t>::iterator it = buffer_.begin(); it != buffer_.end(); ++it) {
-      ss << *it << ".";
+    for (auto v : buffer_) {
+      ss << v << ".";
     }
   }
   ss << endl;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0fbb5b7e/be/src/util/bitmap.h
----------------------------------------------------------------------
diff --git a/be/src/util/bitmap.h b/be/src/util/bitmap.h
index 1ff8050..5f02f60 100644
--- a/be/src/util/bitmap.h
+++ b/be/src/util/bitmap.h
@@ -37,10 +37,6 @@ class Bitmap {
     num_bits_ = num_bits;
   }
 
-  Bitmap(const uint64_t* from_buf, int64_t num_bits) {
-    SetFromBuffer(from_buf, num_bits);
-  }
-
   /// Resize bitmap and set all bits to zero.
   void Reset(int64_t num_bits) {
     DCHECK_GE(num_bits, 0);
@@ -49,30 +45,17 @@ class Bitmap {
     SetAllBits(false);
   }
 
-  void SetFromBuffer(const uint64_t* from_buf, int64_t num_bits) {
-    buffer_.resize(BitUtil::RoundUpNumi64(num_bits));
-    for (int i = 0; i < buffer_.size(); ++i) {
-      buffer_[i] = from_buf[i];
-    }
-    num_bits_ = num_bits;
-  }
-
   /// Compute memory usage of a bitmap, not including the Bitmap object itself.
   static int64_t MemUsage(int64_t num_bits) {
     DCHECK_GE(num_bits, 0);
     return BitUtil::RoundUpNumi64(num_bits) * sizeof(int64_t);
   }
 
-  static int64_t DefaultHashSeed() { return 1234; }
-
   /// Compute memory usage of this bitmap, not including the Bitmap object itself.
   int64_t MemUsage() const { return MemUsage(num_bits_); }
 
-  /// Sets the bit at 'bit_index' to v. If mod is true, this
-  /// function will first mod the bit_index by the bitmap size.
-  template<bool mod>
+  /// Sets the bit at 'bit_index' to v.
   void Set(int64_t bit_index, bool v) {
-    if (mod) bit_index &= (num_bits() - 1);
     int64_t word_index = bit_index >> NUM_OFFSET_BITS;
     bit_index &= BIT_INDEX_MASK;
     DCHECK_LT(word_index, buffer_.size());
@@ -83,33 +66,14 @@ class Bitmap {
     }
   }
 
-  /// Returns true if the bit at 'bit_index' is set. If mod is true, this
-  /// function will first mod the bit_index by the bitmap size.
-  template<bool mod>
+  /// Returns true if the bit at 'bit_index' is set.
   bool Get(int64_t bit_index) const {
-    if (mod) bit_index &= (num_bits() - 1);
     int64_t word_index = bit_index >> NUM_OFFSET_BITS;
     bit_index &= BIT_INDEX_MASK;
     DCHECK_LT(word_index, buffer_.size());
     return (buffer_[word_index] & (1LL << bit_index)) != 0;
   }
 
-  /// Bitwise ANDs the src bitmap into this one.
-  void And(const Bitmap* src) {
-    DCHECK_EQ(num_bits(), src->num_bits());
-    for (int i = 0; i < buffer_.size(); ++i) {
-      buffer_[i] &= src->buffer_[i];
-    }
-  }
-
-  /// Bitwise ORs the src bitmap into this one.
-  void Or(const Bitmap* src) {
-    DCHECK_EQ(num_bits(), src->num_bits());
-    for (int i = 0; i < buffer_.size(); ++i) {
-      buffer_[i] |= src->buffer_[i];
-    }
-  }
-
   void SetAllBits(bool b) {
     memset(&buffer_[0], 255 * b, buffer_.size() * sizeof(uint64_t));
   }
@@ -117,7 +81,7 @@ class Bitmap {
   int64_t num_bits() const { return num_bits_; }
 
   /// If 'print_bits' prints 0/1 per bit, otherwise it prints the int64_t value.
-  std::string DebugString(bool print_bits);
+  std::string DebugString(bool print_bits) const;
 
  private:
   std::vector<uint64_t> buffer_;


[16/33] 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/hadoop-next
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();


[14/33] 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.");



[12/33] 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)



[08/33] incubator-impala git commit: IMPALA-4155: Update default partition when table is altered

Posted by ta...@apache.org.
IMPALA-4155: Update default partition when table is altered

If the table format is changed by the Alter Table statement, the
default partition in partitioned tables used to not get updated. This
caused a problem because Insert picks up the file format for new
partitions from the default partition. This patch fixes the problem by
calling addDefaultPartition().

Also removed "drop table if not exists" in tests in alter-table.test
because we already have the unique_database fixture.

Change-Id: I59bf21caa5c5e7867d07d87cda0c0a5b4b994859
Reviewed-on: http://gerrit.cloudera.org:8080/4750
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/bf1d9677
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/bf1d9677
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/bf1d9677

Branch: refs/heads/hadoop-next
Commit: bf1d9677fc09dabc88ab6c71f82ec99ec59ec164
Parents: c6fc899
Author: Taras Bobrovytsky <tb...@cloudera.com>
Authored: Mon Oct 10 12:14:52 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Oct 20 23:47:52 2016 +0000

----------------------------------------------------------------------
 .../org/apache/impala/catalog/HdfsTable.java    | 29 ++++++++-------
 .../impala/service/CatalogOpExecutor.java       |  3 ++
 .../queries/QueryTest/alter-table.test          | 38 ++++++++++----------
 3 files changed, 39 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/bf1d9677/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index e664f6f..e6206fc 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -965,7 +965,10 @@ public class HdfsTable extends Table {
     return partition;
   }
 
-  private void addDefaultPartition(StorageDescriptor storageDescriptor)
+  /**
+   * Adds or replaces the default partition.
+   */
+  public void addDefaultPartition(StorageDescriptor storageDescriptor)
       throws CatalogException {
     // Default partition has no files and is not referred to by scan nodes. Data sinks
     // refer to this to understand how to create new partitions.
@@ -1025,15 +1028,7 @@ public class HdfsTable extends Table {
         return;
       }
       // Load partition and file metadata
-      if (!reuseMetadata) {
-        // Load all partitions from Hive Metastore, including file metadata.
-        LOG.debug("load table from Hive Metastore: " + db_.getName() + "." + name_);
-        List<org.apache.hadoop.hive.metastore.api.Partition> msPartitions =
-            Lists.newArrayList();
-        msPartitions.addAll(MetaStoreUtil.fetchAllPartitions(
-            client, db_.getName(), name_, NUM_PARTITION_FETCH_RETRIES));
-        loadAllPartitions(msPartitions, msTbl);
-      } else {
+      if (reuseMetadata) {
         // Incrementally update this table's partitions and file metadata
         LOG.debug("incremental update for table: " + db_.getName() + "." + name_);
         Preconditions.checkState(partitionsToUpdate == null || loadFileMetadata);
@@ -1043,6 +1038,14 @@ public class HdfsTable extends Table {
         } else {
           updatePartitionsFromHms(client, partitionsToUpdate, loadFileMetadata);
         }
+      } else {
+        // Load all partitions from Hive Metastore, including file metadata.
+        LOG.debug("load table from Hive Metastore: " + db_.getName() + "." + name_);
+        List<org.apache.hadoop.hive.metastore.api.Partition> msPartitions =
+            Lists.newArrayList();
+        msPartitions.addAll(MetaStoreUtil.fetchAllPartitions(
+            client, db_.getName(), name_, NUM_PARTITION_FETCH_RETRIES));
+        loadAllPartitions(msPartitions, msTbl);
       }
       if (loadTableSchema) setAvroSchema(client, msTbl);
       updateStatsFromHmsTable(msTbl);
@@ -1420,9 +1423,9 @@ public class HdfsTable extends Table {
     HdfsStorageDescriptor fileFormatDescriptor =
         HdfsStorageDescriptor.fromStorageDescriptor(this.name_, msTbl.getSd());
     Map<FsKey, FileBlocksInfo> perFsFileBlocks = Maps.newHashMap();
-    for (HdfsPartition part: partitions) {
+    for (HdfsPartition partition: partitions) {
       org.apache.hadoop.hive.metastore.api.Partition msPart =
-          part.toHmsPartition();
+          partition.toHmsPartition();
       StorageDescriptor sd = null;
       if (msPart == null) {
         // If this partition is not stored in the Hive Metastore (e.g. default partition
@@ -1432,7 +1435,7 @@ public class HdfsTable extends Table {
       } else {
         sd = msPart.getSd();
       }
-      loadPartitionFileMetadata(sd, part, fileFormatDescriptor.getFileFormat(),
+      loadPartitionFileMetadata(sd, partition, fileFormatDescriptor.getFileFormat(),
           perFsFileBlocks);
     }
     loadDiskIds(perFsFileBlocks);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/bf1d9677/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 780bf13..5743a59 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -1869,6 +1869,9 @@ public class CatalogOpExecutor {
       org.apache.hadoop.hive.metastore.api.Table msTbl =
           tbl.getMetaStoreTable().deepCopy();
       setStorageDescriptorFileFormat(msTbl.getSd(), fileFormat);
+      // The default partition must be updated if the file format is changed so that new
+      // partitions are created with the new file format.
+      if (tbl instanceof HdfsTable) ((HdfsTable) tbl).addDefaultPartition(msTbl.getSd());
       applyAlterTable(msTbl);
       reloadFileMetadata = true;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/bf1d9677/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
index 1f63c2b..3b1d86b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
@@ -612,9 +612,6 @@ STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ====
 ---- QUERY
 # IMPALA-1016: Testing scanning newly added columns
-DROP TABLE IF EXISTS imp1016
-====
----- QUERY
 CREATE TABLE imp1016 (string1 string)
 ---- RESULTS
 ====
@@ -672,9 +669,6 @@ bigint
 ====
 ---- QUERY
 # Create a larger table to test scanning newly added columns
-DROP TABLE IF EXISTS imp1016Large
-====
----- QUERY
 CREATE TABLE imp1016Large (string1 string)
 ---- RESULTS
 ====
@@ -929,30 +923,38 @@ STRING, STRING, STRING, STRING, STRING, STRING, STRING, STRING, STRING, STRING,
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
-drop table if exists i1740_alter;
-create table i1740_alter (i1 integer);
-alter table i1740_alter set tblproperties ('skip.header.line.count'='2');
+create table i1740_alter_1 (i1 integer);
+alter table i1740_alter_1 set tblproperties ('skip.header.line.count'='2');
 ---- RESULTS
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
-drop table if exists i1740_alter;
-create table i1740_alter (i1 integer);
-alter table i1740_alter set tblproperties ('skip.header.line.count'='-1');
+create table i1740_alter_2 (i1 integer);
+alter table i1740_alter_2 set tblproperties ('skip.header.line.count'='-1');
 ---- CATCH
 Invalid value for table property skip.header.line.count:
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
-drop table if exists i1740_alter;
-create table i1740_alter (i1 integer);
-alter table i1740_alter set tblproperties ('skip.header.line.count'='foo');
+create table i1740_alter_3 (i1 integer);
+alter table i1740_alter_3 set tblproperties ('skip.header.line.count'='foo');
 ---- CATCH
 Invalid value for table property skip.header.line.count:
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
-drop table if exists i1740_alter;
-create table i1740_alter (i1 integer) stored as parquet;
-alter table i1740_alter set tblproperties ('skip.header.line.count'='2');
+create table i1740_alter_4 (i1 integer) stored as parquet;
+alter table i1740_alter_4 set tblproperties ('skip.header.line.count'='2');
+====
+---- QUERY
+# IMPALA-4155: Verify that the default partition is updated by the alter table statement
+create table i4155_alter (c1 int)
+partitioned by (p1 string) row format delimited fields terminated by ',';
+alter table i4155_alter set fileformat PARQUET;
+insert into i4155_alter partition(p1="abc") values (1);
+select * from i4155_alter;
+---- RESULTS
+1,'abc'
+---- TYPES
+INT, STRING
 ====


[18/33] 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/hadoop-next
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;
+====



[07/33] incubator-impala git commit: IMPALA-4301: Fix IGNORE NULLS with subquery rewriting.

Posted by ta...@apache.org.
IMPALA-4301: Fix IGNORE NULLS with subquery rewriting.

AnayticExpr.analyze() replaces the original FIRST/LAST_VALUE
function with a FIRST/LAST_VALUE_IGNORE_NULLS function if
the IGNORE NULLS clause is specified.

The bug was that several places in AnalyticExpr.analyze() assumed
and asserted that only the original FIRST/LAST_VALUE function
could be encountered during analysis. However, with subquery
rewriting the IGNORE NULLS version of the function may also be
seen because the whole statement is re-analyzed after rewriting.

The fix is to unset the IGNORE NULLS flag of the function params
after changing the analytic function name.

Change-Id: I708de7925fe6aeef582fd7510da93d24c71229d9
Reviewed-on: http://gerrit.cloudera.org:8080/4732
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/c6fc8991
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/c6fc8991
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/c6fc8991

Branch: refs/heads/hadoop-next
Commit: c6fc89913453befbf03d206d20c35323288702f4
Parents: 502220c
Author: Alex Behm <al...@cloudera.com>
Authored: Sat Oct 15 21:35:56 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Oct 20 09:14:19 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/impala/analysis/AnalyticExpr.java    | 10 ++++++----
 .../java/org/apache/impala/analysis/FunctionParams.java  |  3 ++-
 .../org/apache/impala/analysis/AnalyzeExprsTest.java     | 11 ++++++++---
 3 files changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c6fc8991/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java b/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
index 7eead02..40fee99 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
@@ -22,9 +22,6 @@ import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.impala.analysis.AnalyticWindow.Boundary;
 import org.apache.impala.analysis.AnalyticWindow.BoundaryType;
 import org.apache.impala.catalog.AggregateFunction;
@@ -38,6 +35,9 @@ import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TColumnValue;
 import org.apache.impala.thrift.TExprNode;
 import org.apache.impala.util.TColumnValueUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
@@ -750,7 +750,8 @@ public class AnalyticExpr extends Expr {
           window_.getRightBoundary());
     }
 
-    // 8. Append IGNORE NULLS to fn name if set.
+    // 8. Change fn name to the IGNORE NULLS version. Also unset the IGNORE NULLS flag
+    // to allow statement rewriting for subqueries.
     if (getFnCall().getParams().isIgnoreNulls()) {
       if (analyticFnName.getFunction().equals(LAST_VALUE)) {
         fnCall_ = new FunctionCallExpr(new FunctionName(LAST_VALUE_IGNORE_NULLS),
@@ -760,6 +761,7 @@ public class AnalyticExpr extends Expr {
         fnCall_ = new FunctionCallExpr(new FunctionName(FIRST_VALUE_IGNORE_NULLS),
             getFnCall().getParams());
       }
+      getFnCall().getParams().setIsIgnoreNulls(false);
 
       fnCall_.setIsAnalyticFnCall(true);
       fnCall_.setIsInternalFnCall(true);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c6fc8991/fe/src/main/java/org/apache/impala/analysis/FunctionParams.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/FunctionParams.java b/fe/src/main/java/org/apache/impala/analysis/FunctionParams.java
index 4a1effb..c0a78c7 100644
--- a/fe/src/main/java/org/apache/impala/analysis/FunctionParams.java
+++ b/fe/src/main/java/org/apache/impala/analysis/FunctionParams.java
@@ -53,9 +53,10 @@ class FunctionParams implements Cloneable {
 
   public boolean isStar() { return isStar_; }
   public boolean isDistinct() { return isDistinct_; }
+  public void setIsDistinct(boolean v) { isDistinct_ = v; }
   public boolean isIgnoreNulls() { return isIgnoreNulls_; }
+  public void setIsIgnoreNulls(boolean b) { isIgnoreNulls_ = b; }
   public List<Expr> exprs() { return exprs_; }
-  public void setIsDistinct(boolean v) { isDistinct_ = v; }
   public int size() { return exprs_ == null ? 0 : exprs_.size(); }
 
   // c'tor for <agg>(*)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c6fc8991/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
index 72b5163..2b77f39 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -26,9 +26,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import org.apache.impala.analysis.TimestampArithmeticExpr.TimeUnit;
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.catalog.Catalog;
@@ -47,6 +44,9 @@ import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TExpr;
 import org.apache.impala.thrift.TFunctionBinaryType;
 import org.apache.impala.thrift.TQueryOptions;
+import org.junit.Assert;
+import org.junit.Test;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -729,6 +729,11 @@ public class AnalyzeExprsTest extends AnalyzerTest {
         + "functional.alltypesagg");
     AnalyzesOk("select last_value(tinyint_col ignore nulls) over (order by id) from "
         + "functional.alltypesagg");
+    // IMPALA-4301: Test IGNORE NULLS with subqueries.
+    AnalyzesOk("select first_value(tinyint_col ignore nulls) over (order by id)," +
+               "last_value(tinyint_col ignore nulls) over (order by id)" +
+               "from functional.alltypesagg a " +
+               "where exists (select 1 from functional.alltypes b where a.id = b.id)");
 
     // legal combinations of analytic and agg functions
     AnalyzesOk("select sum(count(id)) over (partition by min(int_col) "


[17/33] 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]:


[30/33] incubator-impala git commit: IMPALA-4339: ensure coredumps end up in IMPALA_HOME

Posted by ta...@apache.org.
IMPALA-4339: ensure coredumps end up in IMPALA_HOME

Change-Id: Ibc34d152139653374f940dc3edbca08e749bf55e
Reviewed-on: http://gerrit.cloudera.org:8080/4785
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/a6257013
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/a6257013
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/a6257013

Branch: refs/heads/hadoop-next
Commit: a6257013fad4d9bd0fcdef805b189a2cc666057a
Parents: 99ed6dc
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Oct 21 15:06:12 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Oct 25 04:17:58 2016 +0000

----------------------------------------------------------------------
 buildall.sh | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a6257013/buildall.sh
----------------------------------------------------------------------
diff --git a/buildall.sh b/buildall.sh
index d7159e7..d830ae3 100755
--- a/buildall.sh
+++ b/buildall.sh
@@ -39,6 +39,9 @@ if ! . "$ROOT"/bin/impala-config.sh; then
   exit 1
 fi
 
+# Change to IMPALA_HOME so that coredumps, etc end up in IMPALA_HOME.
+cd "${IMPALA_HOME}"
+
 # Defaults that are only changable via the commandline.
 CLEAN_ACTION=1
 TESTDATA_ACTION=0


[05/33] incubator-impala git commit: IMPALA-3420: use gold by default

Posted by ta...@apache.org.
IMPALA-3420: use gold by default

Also pass the flag that enables ld.gold directly to the
compiler. This is understood by both gcc and clang
(if prefixed with -Wl, clang just forwards the flag to ld,
where it is ignored).

Testing:
Did ASAN and debug private builds to validate it works.
Tested shared library, release, ninja and distcc builds locally
as part of my normal workflow.

Change-Id: Ib05c944ced9cdfe54941f4b690574e45a25110a2
Reviewed-on: http://gerrit.cloudera.org:8080/4751
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/f1f54fe6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/f1f54fe6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/f1f54fe6

Branch: refs/heads/hadoop-next
Commit: f1f54fe65db7d5af2e53b217fb7cb50abe8c50fc
Parents: 5a2c50a
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Oct 13 11:00:00 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Oct 20 03:09:59 2016 +0000

----------------------------------------------------------------------
 be/CMakeLists.txt    | 2 +-
 bin/impala-config.sh | 5 ++---
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f1f54fe6/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index a5cc4d8..a44834a 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -47,7 +47,7 @@ SET(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBOOST_DATE_TIME_POSIX_TIME_STD_CONFI
 SET(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -DBOOST_SYSTEM_NO_DEPRECATED")
 SET(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -B $ENV{IMPALA_TOOLCHAIN}/binutils-$ENV{IMPALA_BINUTILS_VERSION}/bin/")
 IF($ENV{USE_GOLD_LINKER} STREQUAL "true")
-  SET(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wl,-fuse-ld=gold")
+  SET(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -fuse-ld=gold")
 ENDIF()
 
 # On Apple we build with clang and need libstdc++ instead of libc++

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f1f54fe6/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 38bdc69..85bcbfc 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -56,9 +56,8 @@ fi
 # If it's 0, Impala will be built with the compiler in the toolchain directory.
 : ${USE_SYSTEM_GCC=0}
 
-# Gold is available on newer systems and a full build with static linking is ~1.5 mins
-# faster using gold. A shared object build using gold is a little faster than using ld.
-: ${USE_GOLD_LINKER=false}
+# Use ld.gold instead of ld by default to speed up builds.
+: ${USE_GOLD_LINKER=true}
 
 # Override the default compiler by setting a path to the new compiler. The default
 # compiler depends on USE_SYSTEM_GCC and IMPALA_GCC_VERSION. The intended use case


[29/33] incubator-impala git commit: IMPALA-4134, IMPALA-3704: Kudu INSERT improvements

Posted by ta...@apache.org.
IMPALA-4134,IMPALA-3704: Kudu INSERT improvements

1.) IMPALA-4134: Use Kudu AUTO FLUSH
Improves performance of writes to Kudu up to 4.2x in
bulk data loading tests (load 200 million rows from
lineitem).

2.) IMPALA-3704: Improve errors on PK conflicts
The Kudu client reports an error for every PK conflict,
and all errors were being returned in the error status.
As a result, inserts/updates/deletes could return errors
with thousands errors reported. This changes the error
handling to log all reported errors as warnings and
return only the first error in the query error status.

3.) Improve the DataSink reporting of the insert stats.
The per-partition stats returned by the data sink weren't
useful for Kudu sinks. Firstly, the number of appended rows
was not being displayed in the profile. Secondly, the
'stats' field isn't populated for Kudu tables and thus was
confusing in the profile, so it is no longer printed if it
is not set in the thrift struct.

Testing: Ran local tests, including new tests to verify
the query profile insert stats. Manual cluster testing was
conducted of the AUTO FLUSH functionality, and that testing
informed the default mutation buffer value of 100MB which
was found to provide good results.

Change-Id: I5542b9a061b01c543a139e8722560b1365f06595
Reviewed-on: http://gerrit.cloudera.org:8080/4728
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/99ed6dc6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/99ed6dc6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/99ed6dc6

Branch: refs/heads/hadoop-next
Commit: 99ed6dc67ae889eb2a45b10c97cb23f52bc83e5d
Parents: 0eaff80
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Wed Oct 19 15:30:58 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Oct 25 02:06:10 2016 +0000

----------------------------------------------------------------------
 be/src/exec/data-sink.cc                        |   5 +
 be/src/exec/hbase-table-sink.cc                 |   4 +-
 be/src/exec/hdfs-table-sink.cc                  |   4 +-
 be/src/exec/kudu-table-sink.cc                  | 141 ++++++++++++-------
 be/src/exec/kudu-table-sink.h                   |  53 ++++---
 be/src/runtime/coordinator.cc                   |  14 +-
 be/src/service/impala-beeswax-server.cc         |   4 +-
 be/src/service/query-exec-state.cc              |   2 +-
 common/thrift/ImpalaInternalService.thrift      |   4 +-
 common/thrift/ImpalaService.thrift              |   8 +-
 common/thrift/generate_error_codes.py           |   6 +
 shell/impala_client.py                          |   2 +-
 .../queries/QueryTest/kudu_crud.test            |  94 ++++++++++++-
 tests/beeswax/impala_beeswax.py                 |   4 +-
 14 files changed, 248 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/exec/data-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index c95b854..6a34543 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -142,6 +142,11 @@ string DataSink::OutputInsertStats(const PartitionStatusMap& stats,
     } else {
       ss << partition_key << endl;
     }
+    if (val.second.__isset.num_modified_rows) {
+      ss << "NumModifiedRows: " << val.second.num_modified_rows << endl;
+    }
+
+    if (!val.second.__isset.stats) continue;
     const TInsertStats& stats = val.second.stats;
     ss << indent << "BytesWritten: "
        << PrettyPrinter::Print(stats.bytes_written, TUnit::BYTES);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/exec/hbase-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hbase-table-sink.cc b/be/src/exec/hbase-table-sink.cc
index e6052cc..3d84fed 100644
--- a/be/src/exec/hbase-table-sink.cc
+++ b/be/src/exec/hbase-table-sink.cc
@@ -72,7 +72,7 @@ Status HBaseTableSink::Prepare(RuntimeState* state, MemTracker* mem_tracker) {
 
   // Add a 'root partition' status in which to collect insert statistics
   TInsertPartitionStatus root_status;
-  root_status.__set_num_appended_rows(0L);
+  root_status.__set_num_modified_rows(0L);
   root_status.__set_stats(TInsertStats());
   root_status.__set_id(-1L);
   state->per_partition_status()->insert(make_pair(ROOT_PARTITION_KEY, root_status));
@@ -90,7 +90,7 @@ Status HBaseTableSink::Send(RuntimeState* state, RowBatch* batch) {
   RETURN_IF_ERROR(state->CheckQueryState());
   // Since everything is set up just forward everything to the writer.
   RETURN_IF_ERROR(hbase_table_writer_->AppendRowBatch(batch));
-  (*state->per_partition_status())[ROOT_PARTITION_KEY].num_appended_rows +=
+  (*state->per_partition_status())[ROOT_PARTITION_KEY].num_modified_rows +=
       batch->num_rows();
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/exec/hdfs-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-table-sink.cc b/be/src/exec/hdfs-table-sink.cc
index 77316d4..63bd648 100644
--- a/be/src/exec/hdfs-table-sink.cc
+++ b/be/src/exec/hdfs-table-sink.cc
@@ -499,7 +499,7 @@ inline Status HdfsTableSink::GetOutputPartition(RuntimeState* state,
     DCHECK(state->per_partition_status()->find(partition->partition_name) ==
         state->per_partition_status()->end());
     TInsertPartitionStatus partition_status;
-    partition_status.__set_num_appended_rows(0L);
+    partition_status.__set_num_modified_rows(0L);
     partition_status.__set_id(partition_descriptor->id());
     partition_status.__set_stats(TInsertStats());
     partition_status.__set_partition_base_dir(table_desc_->hdfs_base_dir());
@@ -594,7 +594,7 @@ Status HdfsTableSink::FinalizePartitionFile(RuntimeState* state,
     // Should have been created in GetOutputPartition() when the partition was
     // initialised.
     DCHECK(it != state->per_partition_status()->end());
-    it->second.num_appended_rows += partition->num_rows;
+    it->second.num_modified_rows += partition->num_rows;
     DataSink::MergeInsertStats(partition->writer->stats(), &it->second.stats);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/exec/kudu-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.cc b/be/src/exec/kudu-table-sink.cc
index 661489f..70a74a9 100644
--- a/be/src/exec/kudu-table-sink.cc
+++ b/be/src/exec/kudu-table-sink.cc
@@ -33,6 +33,8 @@
 
 DEFINE_int32(kudu_session_timeout_seconds, 60, "Timeout set on the Kudu session. "
     "How long to wait before considering a write failed.");
+DEFINE_int32(kudu_mutation_buffer_size, 100 * 1024 * 1024, "The size (bytes) of the "
+    "Kudu client buffer for mutations.");
 
 using kudu::client::KuduColumnSchema;
 using kudu::client::KuduSchema;
@@ -48,6 +50,9 @@ namespace impala {
 const static string& ROOT_PARTITION_KEY =
     g_ImpalaInternalService_constants.ROOT_PARTITION_KEY;
 
+// Send 7MB buffers to Kudu, matching a hard-coded size in Kudu (KUDU-1693).
+const static int INDIVIDUAL_BUFFER_SIZE = 7 * 1024 * 1024;
+
 KuduTableSink::KuduTableSink(const RowDescriptor& row_desc,
     const vector<TExpr>& select_list_texprs,
     const TDataSink& tsink)
@@ -56,8 +61,6 @@ KuduTableSink::KuduTableSink(const RowDescriptor& row_desc,
       select_list_texprs_(select_list_texprs),
       sink_action_(tsink.table_sink.action),
       kudu_table_sink_(tsink.table_sink.kudu_table_sink),
-      kudu_flush_counter_(NULL),
-      kudu_flush_timer_(NULL),
       kudu_error_counter_(NULL),
       rows_written_(NULL),
       rows_written_rate_(NULL) {
@@ -91,16 +94,14 @@ Status KuduTableSink::Prepare(RuntimeState* state, MemTracker* mem_tracker) {
 
   // Add a 'root partition' status in which to collect write statistics
   TInsertPartitionStatus root_status;
-  root_status.__set_num_appended_rows(0L);
-  root_status.__set_stats(TInsertStats());
+  root_status.__set_num_modified_rows(0L);
   root_status.__set_id(-1L);
   state->per_partition_status()->insert(make_pair(ROOT_PARTITION_KEY, root_status));
 
   // Add counters
-  kudu_flush_counter_ = ADD_COUNTER(profile(), "TotalKuduFlushOperations", TUnit::UNIT);
   kudu_error_counter_ = ADD_COUNTER(profile(), "TotalKuduFlushErrors", TUnit::UNIT);
-  kudu_flush_timer_ = ADD_TIMER(profile(), "KuduFlushTimer");
   rows_written_ = ADD_COUNTER(profile(), "RowsWritten", TUnit::UNIT);
+  kudu_apply_timer_ = ADD_TIMER(profile(), "KuduApplyTimer");
   rows_written_rate_ = profile()->AddDerivedCounter(
       "RowsWrittenRate", TUnit::UNIT_PER_SECOND,
       bind<int64_t>(&RuntimeProfile::UnitsPerSecond, rows_written_,
@@ -124,8 +125,40 @@ Status KuduTableSink::Open(RuntimeState* state) {
 
   session_ = client_->NewSession();
   session_->SetTimeoutMillis(FLAGS_kudu_session_timeout_seconds * 1000);
+
+  // KuduSession Set* methods here and below return a status for API compatibility.
+  // As long as the Kudu client is statically linked, these shouldn't fail and thus these
+  // calls could also DCHECK status is OK for debug builds (while still returning errors
+  // for release).
   KUDU_RETURN_IF_ERROR(session_->SetFlushMode(
-      kudu::client::KuduSession::MANUAL_FLUSH), "Unable to set flush mode");
+      kudu::client::KuduSession::AUTO_FLUSH_BACKGROUND), "Unable to set flush mode");
+
+  const int32_t buf_size = FLAGS_kudu_mutation_buffer_size;
+  if (buf_size < 1024 * 1024) {
+    return Status(strings::Substitute(
+        "Invalid kudu_mutation_buffer_size: '$0'. Must be greater than 1MB.", buf_size));
+  }
+  KUDU_RETURN_IF_ERROR(session_->SetMutationBufferSpace(buf_size),
+      "Couldn't set mutation buffer size");
+
+  // Configure client memory used for buffering.
+  // Internally, the Kudu client keeps one or more buffers for writing operations. When a
+  // single buffer is flushed, it is locked (that space cannot be reused) until all
+  // operations within it complete, so it is important to have a number of buffers. In
+  // our testing, we found that allowing a total of 100MB of buffer space to provide good
+  // results; this is the default.  Then, because of some existing 8MB limits in Kudu, we
+  // want to have that total space broken up into 7MB buffers (INDIVIDUAL_BUFFER_SIZE).
+  // The mutation flush watermark is set to flush every INDIVIDUAL_BUFFER_SIZE.
+  int num_buffers = FLAGS_kudu_mutation_buffer_size / INDIVIDUAL_BUFFER_SIZE;
+  if (num_buffers == 0) num_buffers = 1;
+  KUDU_RETURN_IF_ERROR(session_->SetMutationBufferFlushWatermark(1.0 / num_buffers),
+      "Couldn't set mutation buffer watermark");
+
+  // No limit on the buffer count since the settings above imply a max number of buffers.
+  // Note that the Kudu client API has a few too many knobs for configuring the size and
+  // number of these buffers; there are a few ways to accomplish similar behaviors.
+  KUDU_RETURN_IF_ERROR(session_->SetMutationBufferMaxNum(0),
+      "Couldn't set mutation buffer count");
   return Status::OK();
 }
 
@@ -135,7 +168,8 @@ kudu::client::KuduWriteOperation* KuduTableSink::NewWriteOp() {
   } else if (sink_action_ == TSinkAction::UPDATE) {
     return table_->NewUpdate();
   } else {
-    DCHECK(sink_action_ == TSinkAction::DELETE) << "Sink type not supported. " << sink_action_;
+    DCHECK(sink_action_ == TSinkAction::DELETE) << "Sink type not supported: "
+        << sink_action_;
     return table_->NewDelete();
   }
 }
@@ -145,11 +179,15 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
   ExprContext::FreeLocalAllocations(output_expr_ctxs_);
   RETURN_IF_ERROR(state->CheckQueryState());
 
+  // Collect all write operations and apply them together so the time in Apply() can be
+  // easily timed.
+  vector<unique_ptr<kudu::client::KuduWriteOperation>> write_ops;
+
   int rows_added = 0;
   // Since everything is set up just forward everything to the writer.
   for (int i = 0; i < batch->num_rows(); ++i) {
     TupleRow* current_row = batch->GetRow(i);
-    gscoped_ptr<kudu::client::KuduWriteOperation> write(NewWriteOp());
+    unique_ptr<kudu::client::KuduWriteOperation> write(NewWriteOp());
 
     for (int j = 0; j < output_expr_ctxs_.size(); ++j) {
       int col = kudu_table_sink_.referenced_columns.empty() ?
@@ -173,7 +211,7 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
         case TYPE_STRING: {
           StringValue* sv = reinterpret_cast<StringValue*>(value);
           kudu::Slice slice(reinterpret_cast<uint8_t*>(sv->ptr), sv->len);
-          KUDU_RETURN_IF_ERROR(write->mutable_row()->SetStringNoCopy(col, slice),
+          KUDU_RETURN_IF_ERROR(write->mutable_row()->SetString(col, slice),
               "Could not add Kudu WriteOp.");
           break;
         }
@@ -216,46 +254,38 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
           return Status(TErrorCode::IMPALA_KUDU_TYPE_MISSING, TypeToString(type));
       }
     }
+    write_ops.push_back(move(write));
+  }
 
-    KUDU_RETURN_IF_ERROR(session_->Apply(write.release()),
-        "Error while applying Kudu session.");
-    ++rows_added;
+  {
+    SCOPED_TIMER(kudu_apply_timer_);
+    for (auto&& write: write_ops) {
+      KUDU_RETURN_IF_ERROR(session_->Apply(write.release()), "Error applying Kudu Op.");
+      ++rows_added;
+    }
   }
+
   COUNTER_ADD(rows_written_, rows_added);
-  int64_t error_count = 0;
-  RETURN_IF_ERROR(Flush(&error_count));
-  (*state->per_partition_status())[ROOT_PARTITION_KEY].num_appended_rows +=
-      rows_added - error_count;
+  RETURN_IF_ERROR(CheckForErrors(state));
   return Status::OK();
 }
 
-Status KuduTableSink::Flush(int64_t* error_count) {
-  // TODO right now we always flush an entire row batch, if these are small we'll
-  // be inefficient. Consider decoupling impala's batch size from kudu's
-  kudu::Status s;
-  {
-    SCOPED_TIMER(kudu_flush_timer_);
-    COUNTER_ADD(kudu_flush_counter_, 1);
-    s = session_->Flush();
-  }
-  if (LIKELY(s.ok())) return Status::OK();
+Status KuduTableSink::CheckForErrors(RuntimeState* state) {
+  if (session_->CountPendingErrors() == 0) return Status::OK();
 
-  stringstream error_msg_buffer;
   vector<KuduError*> errors;
-
-  // Check if there are pending errors in the Kudu session. If errors overflowed the error
-  // buffer we can't be sure all errors can be ignored and fail immediately, setting 'failed'
-  // to true.
-  bool failed = false;
-  session_->GetPendingErrors(&errors, &failed);
-  if (UNLIKELY(failed)) {
-    error_msg_buffer << "Error overflow in Kudu session, "
-                     << "previous write operation might be inconsistent.\n";
+  Status status = Status::OK();
+
+  // Get the pending errors from the Kudu session. If errors overflowed the error buffer
+  // we can't be sure all errors can be ignored, so an error status will be reported.
+  bool error_overflow = false;
+  session_->GetPendingErrors(&errors, &error_overflow);
+  if (UNLIKELY(error_overflow)) {
+    status = Status("Error overflow in Kudu session.");
   }
 
   // The memory for the errors is manually managed. Iterate over all errors and delete
   // them accordingly.
-  bool first_error = true;
   for (int i = 0; i < errors.size(); ++i) {
     kudu::Status e = errors[i]->status();
     // If the sink has the option "ignore_not_found_or_duplicate" set, duplicate key or
@@ -265,24 +295,39 @@ Status KuduTableSink::Flush(int64_t* error_count) {
         ((sink_action_ == TSinkAction::DELETE && !e.IsNotFound()) ||
             (sink_action_ == TSinkAction::UPDATE && !e.IsNotFound()) ||
             (sink_action_ == TSinkAction::INSERT && !e.IsAlreadyPresent()))) {
-      if (first_error) {
-        error_msg_buffer << "Error while flushing Kudu session: \n";
-        first_error = false;
+      if (status.ok()) {
+        status = Status(strings::Substitute(
+            "Kudu error(s) reported, first error: $0", e.ToString()));
       }
-      error_msg_buffer << e.ToString() << "\n";
-      failed = true;
+    }
+    if (e.IsNotFound()) {
+      state->LogError(ErrorMsg::Init(TErrorCode::KUDU_KEY_NOT_FOUND,
+          table_desc_->table_name()));
+    } else if (e.IsAlreadyPresent()) {
+      state->LogError(ErrorMsg::Init(TErrorCode::KUDU_KEY_ALREADY_PRESENT,
+          table_desc_->table_name()));
+    } else {
+      state->LogError(ErrorMsg::Init(TErrorCode::KUDU_SESSION_ERROR,
+          table_desc_->table_name(), e.ToString()));
     }
     delete errors[i];
   }
   COUNTER_ADD(kudu_error_counter_, errors.size());
-  if (error_count != NULL) *error_count = errors.size();
-  if (failed) return Status(error_msg_buffer.str());
-  return Status::OK();
+  return status;
 }
 
 Status KuduTableSink::FlushFinal(RuntimeState* state) {
-  // No buffered state to flush.
-  return Status::OK();
+  kudu::Status flush_status = session_->Flush();
+
+  // Flush() may return an error status but any errors will also be reported by
+  // CheckForErrors(), so it's safe to ignore and always call CheckForErrors.
+  if (!flush_status.ok()) {
+    VLOG_RPC << "Ignoring Flush() error status: " << flush_status.ToString();
+  }
+  Status status = CheckForErrors(state);
+  (*state->per_partition_status())[ROOT_PARTITION_KEY].__set_num_modified_rows(
+      rows_written_->value() - kudu_error_counter_->value());
+  return status;
 }
 
 void KuduTableSink::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/exec/kudu-table-sink.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.h b/be/src/exec/kudu-table-sink.h
index 2eeb721..fe278c5 100644
--- a/be/src/exec/kudu-table-sink.h
+++ b/be/src/exec/kudu-table-sink.h
@@ -30,17 +30,27 @@
 
 namespace impala {
 
-/// Sink that takes RowBatches and writes them into Kudu.
-/// Currently the data is sent to Kudu on Send(), i.e. the data is batched on the
-/// KuduSession until all the rows in a RowBatch are applied and then the session
-/// is flushed.
+/// Sink that takes RowBatches and writes them into a Kudu table.
 ///
-/// Kudu doesn't have transactions (yet!) so some rows may fail to write while
-/// others are successful. This sink will return an error if any of the rows fails
-/// to be written.
+/// The data is added to Kudu in Send(). The Kudu client is configured to automatically
+/// flush records when enough data has been written (AUTO_FLUSH_BACKGROUND). This
+/// requires specifying a mutation buffer size and a buffer flush watermark percentage in
+/// the Kudu client. The mutation buffer needs to be large enough to buffer rows sent to
+/// all destination nodes because the buffer accounting is not specified per-tablet
+/// server (KUDU-1693). Tests showed that 100MB was a good default, and this is
+/// configurable via the gflag --kudu_mutation_buffer_size. The buffer flush watermark
+/// percentage is set to a value that results in Kudu flushing after 7MB is in a
+/// buffer for a particular destination (of the 100MB of the total mutation buffer space)
+/// because Kudu currently has some 8MB buffer limits.
 ///
-/// TODO Once Kudu actually implements AUTOFLUSH_BACKGROUND flush mode we should
-/// change the flushing behavior as it will likely make writes more efficient.
+/// Kudu doesn't have transactions yet, so some rows may fail to write while others are
+/// successful. The Kudu client reports errors, some of which may be considered to be
+/// expected: rows that fail to be written/updated/deleted due to a key conflict while
+/// the IGNORE option is specified, and these will not result in the sink returning an
+/// error. These errors when IGNORE is not specified, or any other kind of error
+/// reported by Kudu result in the sink returning an error status. The first non-ignored
+/// error is returned in the sink's Status. All reported errors (ignored or not) will be
+/// logged via the RuntimeState.
 class KuduTableSink : public DataSink {
  public:
   KuduTableSink(const RowDescriptor& row_desc,
@@ -59,7 +69,7 @@ class KuduTableSink : public DataSink {
   /// The KuduSession is flushed on each row batch.
   virtual Status Send(RuntimeState* state, RowBatch* batch);
 
-  /// Does nothing. We currently flush on each Send() call.
+  /// Forces any remaining buffered operations to be flushed to Kudu.
   virtual Status FlushFinal(RuntimeState* state);
 
   /// Closes the KuduSession and the expressions.
@@ -72,12 +82,11 @@ class KuduTableSink : public DataSink {
   /// Create a new write operation according to the sink type.
   kudu::client::KuduWriteOperation* NewWriteOp();
 
-  /// Flushes the Kudu session, making sure all previous operations were committed, and handles
-  /// errors returned from Kudu. Passes the number of errors during the flush operations as an
-  /// out parameter.
-  /// Returns a non-OK status if there was an unrecoverable error. This might return an OK
-  /// status even if 'error_count' is > 0, as some errors might be ignored.
-  Status Flush(int64_t* error_count);
+  /// Checks for any errors buffered in the Kudu session, and increments
+  /// appropriate counters for ignored errors.
+  //
+  /// Returns a bad Status if there are non-ignorable errors.
+  Status CheckForErrors(RuntimeState* state);
 
   /// Used to get the KuduTableDescriptor from the RuntimeState
   TableId table_id_;
@@ -102,15 +111,15 @@ class KuduTableSink : public DataSink {
   /// Captures parameters passed down from the frontend
   TKuduTableSink kudu_table_sink_;
 
-  /// Counts the number of calls to KuduSession::flush().
-  RuntimeProfile::Counter* kudu_flush_counter_;
-
-  /// Aggregates the times spent in KuduSession:flush().
-  RuntimeProfile::Counter* kudu_flush_timer_;
-
   /// Total number of errors returned from Kudu.
   RuntimeProfile::Counter* kudu_error_counter_;
 
+  /// Time spent applying Kudu operations. In normal circumstances, Apply() should be
+  /// negligible because it is asynchronous with AUTO_FLUSH_BACKGROUND enabled.
+  /// Significant time spent in Apply() may indicate that Kudu cannot buffer and send
+  /// rows as fast as the sink can write them.
+  RuntimeProfile::Counter* kudu_apply_timer_;
+
   /// Total number of rows written including errors.
   RuntimeProfile::Counter* rows_written_;
   RuntimeProfile::Counter* rows_written_rate_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 4214d4d..0f41deb 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -1668,11 +1668,15 @@ Status Coordinator::UpdateFragmentExecStatus(const TReportExecStatusParams& para
     for (const PartitionStatusMap::value_type& partition:
          params.insert_exec_status.per_partition_status) {
       TInsertPartitionStatus* status = &(per_partition_status_[partition.first]);
-      status->num_appended_rows += partition.second.num_appended_rows;
-      status->id = partition.second.id;
-      status->partition_base_dir = partition.second.partition_base_dir;
-      if (!status->__isset.stats) status->__set_stats(TInsertStats());
-      DataSink::MergeInsertStats(partition.second.stats, &status->stats);
+      status->__set_num_modified_rows(
+          status->num_modified_rows + partition.second.num_modified_rows);
+      status->__set_id(partition.second.id);
+      status->__set_partition_base_dir(partition.second.partition_base_dir);
+
+      if (partition.second.__isset.stats) {
+        if (!status->__isset.stats) status->__set_stats(TInsertStats());
+        DataSink::MergeInsertStats(partition.second.stats, &status->stats);
+      }
     }
     files_to_move_.insert(
         params.insert_exec_status.files_to_move.begin(),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/be/src/service/impala-beeswax-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index b50499e..63fa9cd 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -523,8 +523,8 @@ Status ImpalaServer::CloseInsertInternal(const TUniqueId& query_id,
         for (const PartitionStatusMap::value_type& v:
              exec_state->coord()->per_partition_status()) {
           const pair<string, TInsertPartitionStatus> partition_status = v;
-          insert_result->rows_appended[partition_status.first] =
-              partition_status.second.num_appended_rows;
+          insert_result->rows_modified[partition_status.first] =
+              partition_status.second.num_modified_rows;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/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 69473c5..3dc9d6c 100644
--- a/be/src/service/query-exec-state.cc
+++ b/be/src/service/query-exec-state.cc
@@ -956,7 +956,7 @@ void ImpalaServer::QueryExecState::SetCreateTableAsSelectResultSet() {
   if (catalog_op_executor_->ddl_exec_response()->new_table_created) {
     DCHECK(coord_.get());
     for (const PartitionStatusMap::value_type& p: coord_->per_partition_status()) {
-      total_num_rows_inserted += p.second.num_appended_rows;
+      total_num_rows_inserted += p.second.num_modified_rows;
     }
   }
   const string& summary_msg = Substitute("Inserted $0 row(s)", total_num_rows_inserted);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/common/thrift/ImpalaInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index 736de34..e9c3119 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -422,8 +422,8 @@ struct TInsertPartitionStatus {
   // query). See THdfsTable.partitions.
   1: optional i64 id
 
-  // The number of rows appended to this partition
-  2: optional i64 num_appended_rows
+  // The number of rows modified in this partition
+  2: optional i64 num_modified_rows
 
   // Detailed statistics gathered by table writers for this partition
   3: optional TInsertStats stats

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/common/thrift/ImpalaService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index 794c140..573709b 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -247,10 +247,10 @@ enum TImpalaQueryOptions {
 
 // The summary of an insert.
 struct TInsertResult {
-  // Number of appended rows per modified partition. Only applies to HDFS tables.
-  // The keys represent partitions to create, coded as k1=v1/k2=v2/k3=v3..., with the
-  // root in an unpartitioned table being the empty string.
-  1: required map<string, i64> rows_appended
+  // Number of modified rows per partition. Only applies to HDFS and Kudu tables.
+  // The keys represent partitions to create, coded as k1=v1/k2=v2/k3=v3..., with
+  // the root in an unpartitioned table being the empty string.
+  1: required map<string, i64> rows_modified
 }
 
 // Response from a call to PingImpalaService

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/common/thrift/generate_error_codes.py
----------------------------------------------------------------------
diff --git a/common/thrift/generate_error_codes.py b/common/thrift/generate_error_codes.py
index ae338d5..f03b073 100755
--- a/common/thrift/generate_error_codes.py
+++ b/common/thrift/generate_error_codes.py
@@ -292,6 +292,12 @@ error_codes = (
 
   ("NO_REGISTERED_BACKENDS", 94, "Cannot schedule query: no registered backends "
    "available."),
+
+  ("KUDU_KEY_ALREADY_PRESENT", 95, "Key already present in Kudu table '$0'."),
+
+  ("KUDU_KEY_NOT_FOUND", 96, "Key not found in Kudu table '$0'."),
+
+  ("KUDU_SESSION_ERROR", 97, "Error in Kudu table '$0': $1")
 )
 
 import sys

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/shell/impala_client.py
----------------------------------------------------------------------
diff --git a/shell/impala_client.py b/shell/impala_client.py
index 0d1c835..f57a015 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -354,7 +354,7 @@ class ImpalaClient(object):
     if status != RpcStatus.OK:
       raise RPCException()
 
-    num_rows = sum([int(k) for k in insert_result.rows_appended.values()])
+    num_rows = sum([int(k) for k in insert_result.rows_modified.values()])
     return num_rows
 
   def close_query(self, last_query_handle, query_handle_closed=False):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/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 a06d203..e4f3205 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
@@ -29,10 +29,15 @@ insert into tdata values
 (3, "todd", cast(1.0 as float), 993393939, cast('c' as VARCHAR(20)), true)
 ---- RESULTS
 : 3
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 3.*
 ====
 ---- QUERY
 update tdata set vali=43 where id = 1
 ---- RESULTS
+# TODO: Verify row count after fixing IMPALA-3713 (Here and UPDATE/DELETE below)
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select * from tdata
@@ -48,6 +53,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 # Try updating a varchar col. with a value that is bigger than it's size (truncated).
 update tdata set valv=cast('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' as VARCHAR(20)) where id = 1
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select * from tdata
@@ -58,10 +65,11 @@ select * from tdata
 ---- TYPES
 INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 ====
-====
 ---- QUERY
 update tdata set valb=false where id = 1
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select * from tdata
@@ -75,6 +83,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 ---- QUERY
 update tdata set vali=43 where id > 1
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 2.*
 ====
 ---- QUERY
 select * from tdata
@@ -88,6 +98,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 ---- QUERY
 update tdata set name='unknown' where name = 'martin'
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select * from tdata
@@ -104,6 +116,8 @@ insert into tdata values
 (120, "she", cast(0.0 as float), 99, cast('f' as VARCHAR(20)), true)
 ---- RESULTS
 : 2
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 2.*
 ====
 ---- QUERY
 select * from tdata
@@ -119,6 +133,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 ---- QUERY
 update tdata set name=null where id = 40
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select * from tdata
@@ -133,6 +149,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 ====
 ---- QUERY
 update tdata set name='he' where id = 40
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ---- RESULTS
 ====
 ---- QUERY
@@ -152,6 +170,8 @@ INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN
 insert into tdata values (320, '', 2.0, 932, cast('' as VARCHAR(20)), false)
 ---- RESULTS
 : 1
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select id, name, valv, valb from tdata where id = 320;
@@ -169,6 +189,10 @@ create table ignore_column_case (Id int, NAME string, vAlf float, vali bigint,
 ====
 ---- QUERY
 insert into ignore_column_case values (1, 'Martin', 1.0, 10);
+---- RESULTS
+: 1
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 select ID, nAmE, VALF, VALI from ignore_column_case where NaMe = 'Martin';
@@ -182,36 +206,44 @@ insert into tdata values
 (666, "The Devil", cast(1.2 as float), 43, cast('z' as VARCHAR(20)), true)
 ---- RESULTS
 : 1
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 insert into tdata values
 (666, "The Devil", cast(1.2 as float), 43, cast('z' as VARCHAR(20)), true)
 ---- CATCH
-Error while flushing Kudu session:
+Kudu error(s) reported, first error: Already present
 ====
 ---- QUERY
 insert ignore into tdata values
 (666, "The Devil", cast(1.2 as float), 43, cast('z' as VARCHAR(20)), true)
 ---- RESULTS
 : 0
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 0.*
 ====
 ---- QUERY
--- Updating the same record twice
+-- Updating the same record many times: cross join produces 7 identical updates
 update a set a.name='Satan' from tdata a, tdata b where a.id = 666
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 7.*
 ====
 ---- QUERY
--- Does not exercise any error path in the sink because updating the same record twice
--- is valid. Makes sure IGNORE works.
+-- Does not exercise any error path in the sink because updating the same record multiple
+-- times is valid. Makes sure IGNORE works.
 update ignore a set a.name='Satan' from tdata a, tdata b where a.id = 666
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 7.*
 ====
 ---- QUERY
 -- Using a cross join to generate the same delete twice. After the first delete succeeded,
 -- trying to execute the second delete will fail because the record does not exist.
 delete a from tdata a, tdata b where a.id = 666
 ---- CATCH
-Error while flushing Kudu session:
+Kudu error(s) reported, first error: Not found: key not found
 ====
 ---- QUERY
 -- Re-insert the data
@@ -223,6 +255,8 @@ insert into tdata values
 ---- QUERY
 delete ignore a from tdata a, tdata b where a.id = 666
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 1.*
 ====
 ---- QUERY
 # IMPALA-3454: A delete that requires a rewrite may not get the Kudu column order correct
@@ -242,6 +276,8 @@ insert into impala_3454 values
 ---- QUERY
 delete from impala_3454 where key_1 < (select max(key_2) from impala_3454)
 ---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 2.*
 ====
 ---- QUERY
 select * from impala_3454
@@ -250,3 +286,49 @@ select * from impala_3454
 ---- TYPES
 TINYINT,BIGINT
 ====
+---- QUERY
+CREATE TABLE kudu_test_tbl PRIMARY KEY(id)
+DISTRIBUTE BY RANGE(id) SPLIT ROWS ((100000000))
+STORED AS KUDU AS
+SELECT * FROM functional_kudu.alltypes WHERE id < 100;
+---- RESULTS
+'Inserted 100 row(s)'
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 100.*
+====
+---- QUERY
+INSERT IGNORE INTO kudu_test_tbl
+SELECT * FROM functional_kudu.alltypes WHERE id < 100;
+---- RESULTS
+: 0
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 0.*
+====
+---- QUERY
+INSERT INTO kudu_test_tbl
+SELECT * FROM functional_kudu.alltypes WHERE id < 100;
+---- CATCH
+Kudu error(s) reported, first error: Already present: key already present
+====
+---- QUERY
+INSERT IGNORE INTO kudu_test_tbl
+SELECT * FROM functional_kudu.alltypes;
+---- RESULTS
+: 7200
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 7200.*
+====
+---- QUERY
+# Test a larger UPDATE
+UPDATE kudu_test_tbl SET int_col = -1;
+---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 7300.*
+====
+---- QUERY
+# Test a larger DELETE
+DELETE FROM kudu_test_tbl WHERE id > -1;
+---- RESULTS
+---- RUNTIME_PROFILE
+row_regex: .*NumModifiedRows: 7300.*
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/99ed6dc6/tests/beeswax/impala_beeswax.py
----------------------------------------------------------------------
diff --git a/tests/beeswax/impala_beeswax.py b/tests/beeswax/impala_beeswax.py
index e0f5d55..7ed411e 100644
--- a/tests/beeswax/impala_beeswax.py
+++ b/tests/beeswax/impala_beeswax.py
@@ -416,8 +416,8 @@ class ImpalaBeeswaxClient(object):
     """Executes an insert query"""
     result = self.__do_rpc(lambda: self.imp_service.CloseInsert(handle))
     # The insert was successful
-    num_rows = sum(map(int, result.rows_appended.values()))
-    data = ["%s: %s" % row for row in result.rows_appended.iteritems()]
+    num_rows = sum(map(int, result.rows_modified.values()))
+    data = ["%s: %s" % row for row in result.rows_modified.iteritems()]
     exec_result = ImpalaBeeswaxResult(success=True, data=data)
     exec_result.summary = "Inserted %d rows" % (num_rows,)
     return exec_result


[28/33] incubator-impala git commit: Add distcc infrastructure.

Posted by ta...@apache.org.
Add distcc infrastructure.

This has been working for several months, and it it was written mainly
by Casey Ching while he was at Cloudera working on Impala.

Change-Id: Ia4bc78ad46dda13e4533183195af632f46377cae
Reviewed-on: http://gerrit.cloudera.org:8080/4820
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/0eaff805
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/0eaff805
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/0eaff805

Branch: refs/heads/hadoop-next
Commit: 0eaff805e28dd4afac134f58b294732e414235ce
Parents: e0a3272
Author: Jim Apple <jb...@cloudera.com>
Authored: Sun Oct 23 14:54:08 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Oct 25 01:15:50 2016 +0000

----------------------------------------------------------------------
 .gitignore               |   2 +-
 bin/distcc/.gitignore    |   1 +
 bin/distcc/README.md     | 106 ++++++++++++++++++++++++++++
 bin/distcc/distcc.sh     |  62 ++++++++++++++++
 bin/distcc/distcc_env.sh | 160 ++++++++++++++++++++++++++++++++++++++++++
 5 files changed, 330 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0eaff805/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 849ee61..e63f863 100644
--- a/.gitignore
+++ b/.gitignore
@@ -13,7 +13,7 @@ org.eclipse.jdt.ui.prefs
 load-*-generated.sql
 bin/version.info
 
-# Cloudera distcc options
+# distcc options
 .impala_compiler_opts
 
 pprof.out

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0eaff805/bin/distcc/.gitignore
----------------------------------------------------------------------
diff --git a/bin/distcc/.gitignore b/bin/distcc/.gitignore
new file mode 100644
index 0000000..ce71f70
--- /dev/null
+++ b/bin/distcc/.gitignore
@@ -0,0 +1 @@
+ld

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0eaff805/bin/distcc/README.md
----------------------------------------------------------------------
diff --git a/bin/distcc/README.md b/bin/distcc/README.md
new file mode 100644
index 0000000..2de7d4a
--- /dev/null
+++ b/bin/distcc/README.md
@@ -0,0 +1,106 @@
+# Distcc
+Distcc will speed up compilation by distributing compilation tasks to remote build
+machines. The scripts in this folder make using distcc easier.
+
+# Requirements
+
+The only requirement you should need to be aware of is, the scripts in this folder were
+only tested on Linux. If you are using OS X, things probably won't work out of the box.
+
+Assuming you are using Linux, if you use the scripts in this folder, there shouldn't be
+any other requirements. The distcc program should be installed and configured
+automatically. Still, understanding what is involved could be useful.
+
+**You shouldn't need to do any of this, this scripts do this for you.**
+
+1. Install distcc and ccache. Most Linux distros have these packages. The scripts will
+   install it if you have a yum or apt-get based system. Otherwise you should install
+   distcc and ccache yourself through whatever package manager your system uses.
+1. Configure the remote distcc hosts. Set your environment variable BUILD_FARM to
+   "host1/limit1,lzo host2/limit2,lzo" and so on.
+1. Your local compiler needs to be at the same path as it is on the remote build slaves.
+   That path is /opt/Impala-Toolchain/<gcc-version-folder>/bin/gcc. In other words, make
+   sure the Impala toolchain is available at /opt/Impala-Toolchain. That can be done
+   through a symlink, and that's what the scripts will attempt to setup.
+
+# Usage
+
+### First time
+1. Source bin/impala-config.sh in the Impala repo. Step #2 depends on this.
+
+        source "$IMPALA_HOME"/bin/impala-config.sh
+
+1. Source "distcc_env.sh" in this directory. The script will attempt to install distcc
+   if needed.
+
+        source "$IMPALA_AUX_TEST_HOME"/distcc/distcc_env.sh
+
+1. Run buildall.sh. The main purpose is to regenerate cmakefiles.
+
+        cd "$IMPALA_HOME"
+        ./buildall.sh -skiptests -so   # Do not use -noclean
+
+   You should notice that the build runs quite a bit faster.
+
+### Incremental builds
+At this point you no longer need to run the heavyweight buildall.sh. After editing files
+you can either
+```
+make -j$(distcc -j)
+```
+or
+```
+bin/make_impala.sh
+```
+
+### Switiching back to local compilation
+If you want to compile a very small change, a local build might be faster.
+```
+switch_compiler local
+```
+to switch back
+```
+switch_compiler distcc
+```
+
+### Switch to clang++
+Clang is faster and gives better error messages. This setup is still somewhat
+experimental.
+```
+switch_compiler clang
+```
+to switch back
+```
+switch_compiler gcc
+```
+
+### Second time
+If you open a new terminal and attempt to build with "make" or "bin/make_impala.sh",
+that will fail. To fix:
+```
+source "$IMPALA_HOME"/bin/impala-config.sh   # Skip if already done
+source "$IMPALA_HOME"/bin/distcc/distcc_env.sh
+```
+
+# Setting up a new distcc server
+
+1. Install "distccd" and "ccache".
+1. Configure distccd (edit /etc/sysconfig/distccd on a RHEL server) with the options
+   OPTIONS="--jobs 96 --allow YOUR.IP.ADDRESS.HERE --log-level=warn --nice=-15"
+   Where num jobs = 2x the number of cores on the machine. (2x is recommended by distcc.)
+1. Start distcc.
+1. Edit distcc_env.sh to include the new host.
+1. Install all gcc and binutils versions from the toolchain into /opt/Impala-Toolchain.
+1. ccache stores its cache in $HOME/.ccache. Assuming distcc is running as a non-root user
+   that has no $HOME, you must sudo mkdir /.ccache, then sudo chmod 777 /.ccache.
+1. If distcc runs as "nobody", sudo -u nobody ccache -M 25G. This sets the size of the
+   cache to 25GB. Adjust to your taste.
+
+# Misc notes
+
+1. "pump" doesn't work. Many compilation attempts time out say something like "Include
+   server did not process the request in 3.8 seconds". distcc tries to copy 3rd party
+   headers to the remote hosts and that may be the problem. If we could get the include
+   server to use the remote 3rd party headers that should help.
+1. Having a different local Linux OS on your development machine than on the distcc hosts
+   should be fine.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0eaff805/bin/distcc/distcc.sh
----------------------------------------------------------------------
diff --git a/bin/distcc/distcc.sh b/bin/distcc/distcc.sh
new file mode 100755
index 0000000..a1136e8
--- /dev/null
+++ b/bin/distcc/distcc.sh
@@ -0,0 +1,62 @@
+#!/bin/bash
+
+# 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.
+
+if [[ -z "$DISTCC_HOSTS" || -z "$IMPALA_REAL_CXX_COMPILER" ]]; then
+  # This could be sourced here and the build would work but the parallelization (-j)
+  # should be wrong at this point and it's too late to fix.
+  DIR=$(dirname "$0")
+  echo "You must source '$DIR/distcc_env.sh' before attempting to build." 1>&2
+  exit 1
+fi
+
+TOOLCHAIN_DIR=/opt/Impala-Toolchain
+if [[ ! -d "$TOOLCHAIN_DIR" ]]; then
+  if [[ -n "$IMPALA_TOOLCHAIN" && -d "$IMPALA_TOOLCHAIN" ]]; then
+    if ! sudo -n -- ln -s "$IMPALA_TOOLCHAIN" "$TOOLCHAIN_DIR" &>/dev/null; then
+      echo The toolchain must be available at $TOOLCHAIN_DIR for distcc. \
+          Try running '"sudo ln -s $IMPALA_TOOLCHAIN $TOOLCHAIN_DIR"'. 1>&2
+      exit 1
+    fi
+  fi
+  echo "The toolchain wasn't found at '$TOOLCHAIN_DIR' and IMPALA_TOOLCHAIN is not set." \
+      Make sure the toolchain is available at $TOOLCHAIN_DIR and try again. 1>&2
+  exit 1
+fi
+
+CMD=
+CMD_POST_ARGS=
+if $IMPALA_USE_DISTCC; then
+  CMD="distcc ccache"
+fi
+
+GCC_ROOT="$TOOLCHAIN_DIR/gcc-$IMPALA_GCC_VERSION"
+case "$IMPALA_REAL_CXX_COMPILER" in
+  gcc) CMD+=" $GCC_ROOT/bin/g++";;
+  clang) # Assume the compilation options were setup for gcc, which would happen using
+         # default build options. Now some additional options need to be added for clang.
+         CMD+=" $TOOLCHAIN_DIR/llvm-$IMPALA_LLVM_ASAN_VERSION/bin/clang++"
+         CMD+=" --gcc-toolchain=$GCC_ROOT"
+         # -Wno-unused-local-typedef needs to go after -Wall
+         # -Wno-error is needed, clang generates more warnings than gcc.
+         CMD_POST_ARGS+=" -Wno-unused-local-typedef -Wno-error";;
+  *) echo "Unexpected IMPALA_REAL_CXX_COMPILER: '$IMPALA_REAL_CXX_COMPILER'" 1>&2
+     exit 1;;
+esac
+
+exec $CMD "$@" $CMD_POST_ARGS

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0eaff805/bin/distcc/distcc_env.sh
----------------------------------------------------------------------
diff --git a/bin/distcc/distcc_env.sh b/bin/distcc/distcc_env.sh
new file mode 100644
index 0000000..173cc18
--- /dev/null
+++ b/bin/distcc/distcc_env.sh
@@ -0,0 +1,160 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# This file is intended to be sourced by a shell (zsh and bash have been tested).
+
+if [[ -z $BUILD_FARM ]]
+then
+  echo "BUILD_FARM must be set to configure distcc" >&2
+  return 1
+fi
+
+if [[ ! -z $ZSH_NAME ]]; then
+  DISTCC_ENV_DIR=$(cd $(dirname ${(%):-%x}) && pwd)
+else
+  DISTCC_ENV_DIR=$(cd $(dirname "${BASH_SOURCE[0]}") && pwd)
+fi
+
+function cmd_exists {
+  which $1 &>/dev/null
+}
+
+INSTALLER=
+if cmd_exists apt-get; then
+  INSTALLER=apt-get
+elif cmd_exists yum; then
+  INSTALLER=yum
+fi
+
+if ! cmd_exists distcc; then
+  echo distcc command not found, attempting installation
+  if [[ -z $INSTALLER ]] || ! sudo $INSTALLER -y install distcc; then
+    echo Unable to automatically install distcc. You need to install it manually. 1>&2
+    return 1
+  fi
+fi
+
+# Install CCache if necessary.
+if ! cmd_exists ccache; then
+  echo "ccache command not found, attempting installation"
+  if [[ -z $INSTALLER ]] || ! sudo $INSTALLER -y install ccache; then
+    echo "Unable to automatically install ccache"
+    return 1
+  fi
+fi
+
+# Don't include localhost in the list. It is already the slowest part of the build because
+# it needs to do preprocessing and linking. There shouldn't be a need to add an extra
+# compilation worker.
+export DISTCC_HOSTS=
+DISTCC_HOSTS+=" --localslots=$(nproc)"
+DISTCC_HOSTS+=" --localslots_cpp=$(nproc)"
+DISTCC_HOSTS+=" --randomize"
+DISTCC_HOSTS+=" ${BUILD_FARM}"
+
+# The compiler that distcc.sh should use: gcc or clang.
+: ${IMPALA_REAL_CXX_COMPILER=}
+export IMPALA_REAL_CXX_COMPILER
+
+# Set to false to use local compilation instead of distcc.
+: ${IMPALA_USE_DISTCC=}
+export IMPALA_USE_DISTCC
+
+# Even after generating make files, some state about compiler options would only exist in
+# environment vars. Any such vars should be saved to this file so they can be restored.
+if [[ -z "$IMPALA_HOME" ]]; then
+  echo '$IMPALA_HOME must be set before sourcing this file.' 1>&2
+  return 1
+fi
+IMPALA_COMPILER_CONFIG_FILE="$IMPALA_HOME/.impala_compiler_opts"
+
+# Completely disable anything that could have been setup using this script and clean
+# the make files.
+function disable_distcc {
+  export IMPALA_CXX_COMPILER=default
+  export IMPALA_BUILD_THREADS=$(nproc)
+  save_compiler_opts
+  if ! clean_cmake_files; then
+    echo Failed to clean cmake files. 1>&2
+    return 1
+  fi
+  echo "distcc is not fully disabled, run 'buildall.sh' to complete the change." \
+    "Run 'enable_distcc' to enable."
+}
+
+function enable_distcc {
+  export IMPALA_CXX_COMPILER="$DISTCC_ENV_DIR"/distcc.sh
+  switch_compiler distcc gcc
+  export IMPALA_BUILD_THREADS=$(distcc -j)
+  if ! clean_cmake_files; then
+    echo Failed to clean cmake files. 1>&2
+    return 1
+  fi
+  echo "distcc is not fully enabled, run 'buildall.sh' to complete the change." \
+    "Run 'disable_distcc' or 'switch_compiler local' to disable."
+}
+
+# Cleans old CMake files, this is required when switching between distcc.sh and direct
+# compilation.
+function clean_cmake_files {
+  if [[ -z "$IMPALA_HOME" || ! -d "$IMPALA_HOME" ]]; then
+    echo IMPALA_HOME=$IMPALA_HOME is not valid. 1>&2
+    return 1
+  fi
+  # Copied from $IMPALA_HOME/bin/clean.sh.
+  find "$IMPALA_HOME" -iname '*cmake*' -not -name CMakeLists.txt \
+      -not -path '*cmake_modules*' \
+      -not -path '*thirdparty*'  | xargs rm -rf
+}
+
+function switch_compiler {
+  for ARG in "$@"; do
+    case "$ARG" in
+      "local")
+        IMPALA_USE_DISTCC=false
+        IMPALA_BUILD_THREADS=$(nproc);;
+      distcc)
+        IMPALA_USE_DISTCC=true
+        IMPALA_BUILD_THREADS=$(distcc -j);;
+      gcc) IMPALA_REAL_CXX_COMPILER=gcc;;
+      clang) IMPALA_REAL_CXX_COMPILER=clang;;
+      *) echo "Valid compiler options are:
+    'local'  - Don't use distcc and set -j value to $(nproc). (gcc/clang) remains unchanged.
+    'distcc' - Use distcc and set -j value to $(distcc -j). (gcc/clang) remains unchanged.
+    'gcc'    - Use gcc. (local/distcc remains unchanged).
+    'clang'  - Use clang. (local/distcc remains unchanged)." 2>&1
+        return 1;;
+    esac
+  done
+  save_compiler_opts
+}
+
+function save_compiler_opts {
+  rm -f "$IMPALA_COMPILER_CONFIG_FILE"
+  cat <<EOF > "$IMPALA_COMPILER_CONFIG_FILE"
+IMPALA_CXX_COMPILER=$IMPALA_CXX_COMPILER
+IMPALA_BUILD_THREADS=$IMPALA_BUILD_THREADS
+IMPALA_USE_DISTCC=$IMPALA_USE_DISTCC
+IMPALA_REAL_CXX_COMPILER=$IMPALA_REAL_CXX_COMPILER
+EOF
+}
+
+if [[ -e "$IMPALA_COMPILER_CONFIG_FILE" ]]; then
+  source "$IMPALA_COMPILER_CONFIG_FILE"
+else
+  enable_distcc
+fi


[31/33] incubator-impala git commit: IMPALA-3211: provide toolchain build id for bootstrapping

Posted by ta...@apache.org.
IMPALA-3211: provide toolchain build id for bootstrapping

Testing:
Ran a private build, which succeeded.

Change-Id: Ibcc25ae82511713d0ff05ded37ef162925f2f0fb
Reviewed-on: http://gerrit.cloudera.org:8080/4771
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/aa7741a5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/aa7741a5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/aa7741a5

Branch: refs/heads/hadoop-next
Commit: aa7741a57bbd3c11377e29c59e1d734bcdebf602
Parents: a625701
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Oct 19 15:44:59 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Oct 25 05:10:28 2016 +0000

----------------------------------------------------------------------
 bin/bootstrap_toolchain.py | 9 +++++++--
 bin/impala-config.sh       | 7 +++++++
 2 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/aa7741a5/bin/bootstrap_toolchain.py
----------------------------------------------------------------------
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index 3f161c8..dbe53a2 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -86,9 +86,14 @@ def wget_and_unpack_package(download_path, file_name, destination, wget_no_clobb
 def download_package(destination, product, version, compiler, platform_release=None):
   remove_existing_package(destination, product, version)
 
+  toolchain_build_id = os.environ["IMPALA_TOOLCHAIN_BUILD_ID"]
   label = get_platform_release_label(release=platform_release)
-  file_name = "{0}-{1}-{2}-{3}.tar.gz".format(product, version, compiler, label)
-  url_path="/{0}/{1}-{2}/{0}-{1}-{2}-{3}.tar.gz".format(product, version, compiler, label)
+  format_params = {'product': product, 'version': version, 'compiler': compiler,
+      'label': label, 'toolchain_build_id': toolchain_build_id}
+  file_name = "{product}-{version}-{compiler}-{label}.tar.gz".format(**format_params)
+  format_params['file_name'] = file_name
+  url_path = "/{toolchain_build_id}/{product}/{version}-{compiler}/{file_name}".format(
+      **format_params)
   download_path = HOST + url_path
 
   wget_and_unpack_package(download_path, file_name, destination, True)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/aa7741a5/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 85bcbfc..4e848c1 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -52,6 +52,12 @@ fi
 # If true, will not call $IMPALA_HOME/bin/bootstrap_toolchain.py.
 : ${SKIP_TOOLCHAIN_BOOTSTRAP=false}
 
+# The unique build id of the toolchain to use if bootstrapping. This is generated by the
+# native-toolchain build when publishing its build artifacts. This should be changed when
+# moving to a different build of the toolchain, e.g. when a version is bumped or a
+# compile option is changed.
+: ${IMPALA_TOOLCHAIN_BUILD_ID=249-2267164200}
+
 # This flag is used in $IMPALA_HOME/cmake_modules/toolchain.cmake.
 # If it's 0, Impala will be built with the compiler in the toolchain directory.
 : ${USE_SYSTEM_GCC=0}
@@ -79,6 +85,7 @@ fi
 
 export IMPALA_TOOLCHAIN
 export SKIP_TOOLCHAIN_BOOTSTRAP
+export IMPALA_TOOLCHAIN_BUILD_ID
 export USE_SYSTEM_GCC
 export USE_GOLD_LINKER
 export IMPALA_CXX_COMPILER


[19/33] 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/hadoop-next
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\


[03/33] incubator-impala git commit: IMPALA-3920: TotalStorageWaitTime counter not populated for fragments with Kudu scan node

Posted by ta...@apache.org.
IMPALA-3920: TotalStorageWaitTime counter not populated for fragments
with Kudu scan node

Currently we do not start the TotalStorageWaitTime timer in the
kudu-scanner. This patch replaces the kudu_read_timer with the
TotalStorageWaitTime which measures the intended time.

Change-Id: If0c793930799fdcaff53e705f94b52cadac2f53a
Reviewed-on: http://gerrit.cloudera.org:8080/4639
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/ef5d0c36
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/ef5d0c36
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/ef5d0c36

Branch: refs/heads/hadoop-next
Commit: ef5d0c36aa7d681d91f04be0f0a487c9fe402c51
Parents: b15d992
Author: aphadke <ap...@cloudera.com>
Authored: Wed Oct 5 16:51:07 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Oct 19 23:07:47 2016 +0000

----------------------------------------------------------------------
 be/src/exec/kudu-scan-node.cc | 3 ---
 be/src/exec/kudu-scan-node.h  | 3 ---
 be/src/exec/kudu-scanner.cc   | 4 ++--
 3 files changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ef5d0c36/be/src/exec/kudu-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node.cc b/be/src/exec/kudu-scan-node.cc
index 6e97378..4a162a8 100644
--- a/be/src/exec/kudu-scan-node.cc
+++ b/be/src/exec/kudu-scan-node.cc
@@ -60,7 +60,6 @@ using kudu::Slice;
 
 namespace impala {
 
-const string KuduScanNode::KUDU_READ_TIMER = "TotalKuduReadTime";
 const string KuduScanNode::KUDU_ROUND_TRIPS = "TotalKuduScanRoundTrips";
 const string KuduScanNode::KUDU_REMOTE_TOKENS = "KuduRemoteScanTokens";
 
@@ -95,8 +94,6 @@ Status KuduScanNode::Prepare(RuntimeState* state) {
 
   scan_ranges_complete_counter_ =
       ADD_COUNTER(runtime_profile(), SCAN_RANGES_COMPLETE_COUNTER, TUnit::UNIT);
-  kudu_read_timer_ = ADD_CHILD_TIMER(runtime_profile(), KUDU_READ_TIMER,
-      SCANNER_THREAD_TOTAL_WALLCLOCK_TIME);
   kudu_round_trips_ = ADD_COUNTER(runtime_profile(), KUDU_ROUND_TRIPS, TUnit::UNIT);
   kudu_remote_tokens_ = ADD_COUNTER(runtime_profile(), KUDU_REMOTE_TOKENS, TUnit::UNIT);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ef5d0c36/be/src/exec/kudu-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node.h b/be/src/exec/kudu-scan-node.h
index d4f742a..79ed480 100644
--- a/be/src/exec/kudu-scan-node.h
+++ b/be/src/exec/kudu-scan-node.h
@@ -99,10 +99,8 @@ class KuduScanNode : public ScanNode {
   /// Thread group for all scanner worker threads
   ThreadGroup scanner_threads_;
 
-  RuntimeProfile::Counter* kudu_read_timer_;
   RuntimeProfile::Counter* kudu_round_trips_;
   RuntimeProfile::Counter* kudu_remote_tokens_;
-  static const std::string KUDU_READ_TIMER;
   static const std::string KUDU_ROUND_TRIPS;
   static const std::string KUDU_REMOTE_TOKENS;
 
@@ -136,7 +134,6 @@ class KuduScanNode : public ScanNode {
   // have been open previously.
   Status GetConjunctCtxs(vector<ExprContext*>* ctxs);
 
-  RuntimeProfile::Counter* kudu_read_timer() const { return kudu_read_timer_; }
   RuntimeProfile::Counter* kudu_round_trips() const { return kudu_round_trips_; }
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ef5d0c36/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index a6affbf..ca4ee9a 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -147,7 +147,7 @@ Status KuduScanner::OpenNextScanToken(const string& scan_token)  {
       "Could not set scanner timeout");
 
   {
-    SCOPED_TIMER(scan_node_->kudu_read_timer());
+    SCOPED_TIMER(state_->total_storage_wait_timer());
     KUDU_RETURN_IF_ERROR(scanner_->Open(), "Unable to open scanner");
   }
   return Status::OK();
@@ -335,7 +335,7 @@ Status KuduScanner::KuduRowToImpalaTuple(const KuduScanBatch::RowPtr& row,
 
 
 Status KuduScanner::GetNextScannerBatch() {
-  SCOPED_TIMER(scan_node_->kudu_read_timer());
+  SCOPED_TIMER(state_->total_storage_wait_timer());
   int64_t now = MonotonicMicros();
   KUDU_RETURN_IF_ERROR(scanner_->NextBatch(&cur_kudu_batch_), "Unable to advance iterator");
   COUNTER_ADD(scan_node_->kudu_round_trips(), 1);


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

Posted by ta...@apache.org.
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/hadoop-next
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


[32/33] incubator-impala git commit: IMPALA-3884: Support TYPE_TIMESTAMP for HashTableCtx::CodegenAssignNullValue()

Posted by ta...@apache.org.
IMPALA-3884: Support TYPE_TIMESTAMP for HashTableCtx::CodegenAssignNullValue()

This change implements support for TYPE_TIMESTAMP for
HashTableCtx::CodegenAssignNullValue(). TimestampValue itself
is 16 bytes in size. To match RawValue::Write() in the
interpreted path, CodegenAssignNullValue() emits code to assign
HashUtil::FNV_SEED to both the upper and lower 64-bit of the
destination value. This change also fixes the handling of 128-bit
Decimal16Value in CodegenAssignNullValue() so the emitted code
matches the behavior of the interpreted path.

Change-Id: I0211d38cbef46331e0006fa5ed0680e6e0867bc8
Reviewed-on: http://gerrit.cloudera.org:8080/4794
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Michael Ho <kw...@cloudera.com>


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

Branch: refs/heads/hadoop-next
Commit: 13455b5a24a9d4d009d1dd0d72944c6cacd54829
Parents: aa7741a
Author: Michael Ho <kw...@cloudera.com>
Authored: Fri Oct 21 15:00:56 2016 -0700
Committer: Michael Ho <kw...@cloudera.com>
Committed: Tue Oct 25 05:52:33 2016 +0000

----------------------------------------------------------------------
 be/src/codegen/codegen-anyval.cc                |  8 ++---
 be/src/codegen/llvm-codegen.cc                  | 10 ++++--
 be/src/codegen/llvm-codegen.h                   |  9 +++--
 be/src/exec/hash-table.cc                       | 36 ++++++++++++--------
 be/src/exec/old-hash-table.cc                   | 32 +++++++++++------
 be/src/util/bit-util.h                          |  4 +++
 .../queries/QueryTest/joins.test                | 10 ++++++
 7 files changed, 72 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/codegen/codegen-anyval.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.cc b/be/src/codegen/codegen-anyval.cc
index a2df356..57c08b5 100644
--- a/be/src/codegen/codegen-anyval.cc
+++ b/be/src/codegen/codegen-anyval.cc
@@ -380,12 +380,8 @@ void CodegenAnyVal::SetVal(int64_t val) {
 
 void CodegenAnyVal::SetVal(int128_t val) {
   DCHECK_EQ(type_.type, TYPE_DECIMAL);
-  // TODO: is there a better way to do this?
-  // Set high bits
-  Value* ir_val = ConstantInt::get(codegen_->i128_type(), HighBits(val));
-  ir_val = builder_->CreateShl(ir_val, 64, "tmp");
-  // Set low bits
-  ir_val = builder_->CreateOr(ir_val, LowBits(val), "tmp");
+  vector<uint64_t> vals({LowBits(val), HighBits(val)});
+  Value* ir_val = ConstantInt::get(codegen_->context(), APInt(128, vals));
   SetVal(ir_val);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index d43ad6e..30cc7d0 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -564,7 +564,7 @@ Value* LlvmCodeGen::CastPtrToLlvmPtr(Type* type, const void* ptr) {
   return ConstantExpr::getIntToPtr(const_int, type);
 }
 
-Value* LlvmCodeGen::GetIntConstant(PrimitiveType type, int64_t val) {
+Value* LlvmCodeGen::GetIntConstant(PrimitiveType type, uint64_t val) {
   switch (type) {
     case TYPE_TINYINT:
       return ConstantInt::get(context(), APInt(8, val));
@@ -580,8 +580,12 @@ Value* LlvmCodeGen::GetIntConstant(PrimitiveType type, int64_t val) {
   }
 }
 
-Value* LlvmCodeGen::GetIntConstant(int num_bytes, int64_t val) {
-  return ConstantInt::get(context(), APInt(8 * num_bytes, val));
+Value* LlvmCodeGen::GetIntConstant(int num_bytes, uint64_t low_bits, uint64_t high_bits) {
+  DCHECK_GE(num_bytes, 1);
+  DCHECK_LE(num_bytes, 16);
+  DCHECK(BitUtil::IsPowerOf2(num_bytes));
+  vector<uint64_t> vals({low_bits, high_bits});
+  return ConstantInt::get(context(), APInt(8 * num_bytes, vals));
 }
 
 AllocaInst* LlvmCodeGen::CreateEntryBlockAlloca(Function* f, const NamedVariable& var) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 0b81701..602dc5d 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -380,10 +380,13 @@ class LlvmCodeGen {
   llvm::Value* CastPtrToLlvmPtr(llvm::Type* type, const void* ptr);
 
   /// Returns the constant 'val' of 'type'.
-  llvm::Value* GetIntConstant(PrimitiveType type, int64_t val);
+  llvm::Value* GetIntConstant(PrimitiveType type, uint64_t val);
 
-  /// Returns the constant 'val' of the int type of size 'byte_size'.
-  llvm::Value* GetIntConstant(int byte_size, int64_t val);
+  /// Returns a constant int of 'byte_size' bytes based on 'low_bits' and 'high_bits'
+  /// which stand for the lower and upper 64-bits of the constant respectively. For
+  /// values less than or equal to 64-bits, 'high_bits' is not used. This function
+  /// can generate constant up to 128-bit wide. 'byte_size' must be power of 2.
+  llvm::Value* GetIntConstant(int byte_size, uint64_t low_bits, uint64_t high_bits);
 
   /// Returns true/false constants (bool type)
   llvm::Value* true_value() { return true_value_; }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index c39e9e9..74fc534 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -569,41 +569,51 @@ string HashTable::PrintStats() const {
 // we'll pick a more random value.
 static void CodegenAssignNullValue(LlvmCodeGen* codegen,
     LlvmCodeGen::LlvmBuilder* builder, Value* dst, const ColumnType& type) {
-  int64_t fvn_seed = HashUtil::FNV_SEED;
+  uint64_t fnv_seed = HashUtil::FNV_SEED;
 
   if (type.type == TYPE_STRING || type.type == TYPE_VARCHAR) {
     Value* dst_ptr = builder->CreateStructGEP(NULL, dst, 0, "string_ptr");
     Value* dst_len = builder->CreateStructGEP(NULL, dst, 1, "string_len");
-    Value* null_len = codegen->GetIntConstant(TYPE_INT, fvn_seed);
+    Value* null_len = codegen->GetIntConstant(TYPE_INT, fnv_seed);
     Value* null_ptr = builder->CreateIntToPtr(null_len, codegen->ptr_type());
     builder->CreateStore(null_ptr, dst_ptr);
     builder->CreateStore(null_len, dst_len);
   } else {
     Value* null_value = NULL;
-    // Get a type specific representation of fvn_seed
+    int byte_size = type.GetByteSize();
+    // Get a type specific representation of fnv_seed
     switch (type.type) {
       case TYPE_BOOLEAN:
         // In results, booleans are stored as 1 byte
         dst = builder->CreateBitCast(dst, codegen->ptr_type());
-        null_value = codegen->GetIntConstant(TYPE_TINYINT, fvn_seed);
+        null_value = codegen->GetIntConstant(TYPE_TINYINT, fnv_seed);
         break;
+      case TYPE_TIMESTAMP: {
+        // Cast 'dst' to 'i128*'
+        DCHECK_EQ(byte_size, 16);
+        PointerType* fnv_seed_ptr_type =
+            codegen->GetPtrType(Type::getIntNTy(codegen->context(), byte_size * 8));
+        dst = builder->CreateBitCast(dst, fnv_seed_ptr_type);
+        null_value = codegen->GetIntConstant(byte_size, fnv_seed, fnv_seed);
+        break;
+      }
       case TYPE_TINYINT:
       case TYPE_SMALLINT:
       case TYPE_INT:
       case TYPE_BIGINT:
       case TYPE_DECIMAL:
-        null_value = codegen->GetIntConstant(type.GetByteSize(), fvn_seed);
+        null_value = codegen->GetIntConstant(byte_size, fnv_seed, fnv_seed);
         break;
       case TYPE_FLOAT: {
         // Don't care about the value, just the bit pattern
-        float fvn_seed_float = *reinterpret_cast<float*>(&fvn_seed);
-        null_value = ConstantFP::get(codegen->context(), APFloat(fvn_seed_float));
+        float fnv_seed_float = *reinterpret_cast<float*>(&fnv_seed);
+        null_value = ConstantFP::get(codegen->context(), APFloat(fnv_seed_float));
         break;
       }
       case TYPE_DOUBLE: {
         // Don't care about the value, just the bit pattern
-        double fvn_seed_double = *reinterpret_cast<double*>(&fvn_seed);
-        null_value = ConstantFP::get(codegen->context(), APFloat(fvn_seed_double));
+        double fnv_seed_double = *reinterpret_cast<double*>(&fnv_seed);
+        null_value = ConstantFP::get(codegen->context(), APFloat(fnv_seed_double));
         break;
       }
       default:
@@ -685,13 +695,11 @@ static void CodegenAssignNullValue(LlvmCodeGen* codegen,
 // becomes the start of the next block for codegen (either the next expr or just the
 // end of the function).
 Status HashTableCtx::CodegenEvalRow(LlvmCodeGen* codegen, bool build, Function** fn) {
-  // TODO: CodegenAssignNullValue() can't handle TYPE_TIMESTAMP or TYPE_DECIMAL yet
   const vector<ExprContext*>& ctxs = build ? build_expr_ctxs_ : probe_expr_ctxs_;
   for (int i = 0; i < ctxs.size(); ++i) {
-    PrimitiveType type = ctxs[i]->root()->type().type;
-    if (type == TYPE_TIMESTAMP || type == TYPE_CHAR) {
-      return Status(Substitute("HashTableCtx::CodegenEvalRow(): type $0 NYI",
-          TypeToString(type)));
+    // Disable codegen for CHAR
+    if (ctxs[i]->root()->type().type == TYPE_CHAR) {
+      return Status("HashTableCtx::CodegenEvalRow(): CHAR NYI");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/exec/old-hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/old-hash-table.cc b/be/src/exec/old-hash-table.cc
index db89782..f51bc74 100644
--- a/be/src/exec/old-hash-table.cc
+++ b/be/src/exec/old-hash-table.cc
@@ -184,41 +184,52 @@ int OldHashTable::AddBloomFilters() {
 // we'll pick a more random value.
 static void CodegenAssignNullValue(LlvmCodeGen* codegen,
     LlvmCodeGen::LlvmBuilder* builder, Value* dst, const ColumnType& type) {
-  int64_t fvn_seed = HashUtil::FNV_SEED;
+  uint64_t fnv_seed = HashUtil::FNV_SEED;
 
   if (type.type == TYPE_STRING || type.type == TYPE_VARCHAR) {
     Value* dst_ptr = builder->CreateStructGEP(NULL, dst, 0, "string_ptr");
     Value* dst_len = builder->CreateStructGEP(NULL, dst, 1, "string_len");
-    Value* null_len = codegen->GetIntConstant(TYPE_INT, fvn_seed);
+    Value* null_len = codegen->GetIntConstant(TYPE_INT, fnv_seed);
     Value* null_ptr = builder->CreateIntToPtr(null_len, codegen->ptr_type());
     builder->CreateStore(null_ptr, dst_ptr);
     builder->CreateStore(null_len, dst_len);
     return;
   } else {
     Value* null_value = NULL;
-    // Get a type specific representation of fvn_seed
+    int byte_size = type.GetByteSize();
+    // Get a type specific representation of fnv_seed
     switch (type.type) {
       case TYPE_BOOLEAN:
         // In results, booleans are stored as 1 byte
         dst = builder->CreateBitCast(dst, codegen->ptr_type());
-        null_value = codegen->GetIntConstant(TYPE_TINYINT, fvn_seed);
+        null_value = codegen->GetIntConstant(TYPE_TINYINT, fnv_seed);
         break;
+      case TYPE_TIMESTAMP: {
+        // Cast 'dst' to 'i128*'
+        DCHECK_EQ(byte_size, 16);
+        PointerType* fnv_seed_ptr_type =
+            codegen->GetPtrType(Type::getIntNTy(codegen->context(), byte_size * 8));
+        dst = builder->CreateBitCast(dst, fnv_seed_ptr_type);
+        null_value = codegen->GetIntConstant(byte_size, fnv_seed, fnv_seed);
+        break;
+      }
       case TYPE_TINYINT:
       case TYPE_SMALLINT:
       case TYPE_INT:
       case TYPE_BIGINT:
-        null_value = codegen->GetIntConstant(type.type, fvn_seed);
+      case TYPE_DECIMAL:
+        null_value = codegen->GetIntConstant(byte_size, fnv_seed, fnv_seed);
         break;
       case TYPE_FLOAT: {
         // Don't care about the value, just the bit pattern
-        float fvn_seed_float = *reinterpret_cast<float*>(&fvn_seed);
-        null_value = ConstantFP::get(codegen->context(), APFloat(fvn_seed_float));
+        float fnv_seed_float = *reinterpret_cast<float*>(&fnv_seed);
+        null_value = ConstantFP::get(codegen->context(), APFloat(fnv_seed_float));
         break;
       }
       case TYPE_DOUBLE: {
         // Don't care about the value, just the bit pattern
-        double fvn_seed_double = *reinterpret_cast<double*>(&fvn_seed);
-        null_value = ConstantFP::get(codegen->context(), APFloat(fvn_seed_double));
+        double fnv_seed_double = *reinterpret_cast<double*>(&fnv_seed);
+        null_value = ConstantFP::get(codegen->context(), APFloat(fnv_seed_double));
         break;
       }
       default:
@@ -256,11 +267,10 @@ static void CodegenAssignNullValue(LlvmCodeGen* codegen,
 // becomes the start of the next block for codegen (either the next expr or just the
 // end of the function).
 Function* OldHashTable::CodegenEvalTupleRow(LlvmCodeGen* codegen, bool build) {
-  // TODO: CodegenAssignNullValue() can't handle TYPE_TIMESTAMP or TYPE_DECIMAL yet
   const vector<ExprContext*>& ctxs = build ? build_expr_ctxs_ : probe_expr_ctxs_;
   for (int i = 0; i < ctxs.size(); ++i) {
     PrimitiveType type = ctxs[i]->root()->type().type;
-    if (type == TYPE_TIMESTAMP || type == TYPE_DECIMAL || type == TYPE_CHAR) return NULL;
+    if (type == TYPE_CHAR) return NULL;
   }
 
   // Get types to generate function prototype

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/be/src/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 33dd02b..9a5c4a4 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -82,6 +82,10 @@ class BitUtil {
     return value & ~(factor - 1);
   }
 
+  constexpr static inline bool IsPowerOf2(int64_t value) {
+    return (value & (value - 1)) == 0;
+  }
+
   /// Specialized round up and down functions for frequently used factors,
   /// like 8 (bits->bytes), 32 (bits->i32), and 64 (bits->i64).
   /// Returns the rounded up number of bytes that fit the number of bits.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/13455b5a/testdata/workloads/functional-query/queries/QueryTest/joins.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/joins.test b/testdata/workloads/functional-query/queries/QueryTest/joins.test
index ebb6287..db915df 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/joins.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/joins.test
@@ -720,3 +720,13 @@ on extract(minute from t1.timestamp_col) = extract(hour from t2.timestamp_col);
 ---- TYPES
 BIGINT
 ====
+---- QUERY
+# Regression for IMPALA-3884. Exercise HashTableCtx::AssignNullValue() for
+# 128-bit TimestampValue.
+select count(*) from functional.alltypes t1 right outer join functional.decimal_tbl t2 on
+t1.timestamp_col = cast(t2.d4 as TIMESTAMP);
+---- RESULTS
+5
+---- TYPES
+BIGINT
+====


[22/33] 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/hadoop-next
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
 


[24/33] incubator-impala git commit: IMPALA-4285/IMPALA-4286: Fixes for Parquet scanner with MT_DOP > 0.

Posted by ta...@apache.org.
IMPALA-4285/IMPALA-4286: Fixes for Parquet scanner with MT_DOP > 0.

IMPALA-4258: The problem was that there was a reference to
HdfsScanner::batch_ hidden inside WriteEmptyTuples(). The batch_
reference is NULL when the scanner is run with MT_DOP > 1.

IMPALA-4286: When there are no scan ranges HdfsScanNodeBase::Open()
exits early without initializing the reader context. This lead to
a DCHECK in IoMgr::GetNextRange() called from HdfsScanNodeMt.
The fix is to remove that unnecessary short-circuit Open().

I combined these two bugfixes because the new basic test covers
both cases.

Testing: Added a new test_mt_dop.py test. A private code/hdfs
run passed.

Change-Id: I79c0f6fd2aeb4bc6fa5f87219a485194fef2db1b
Reviewed-on: http://gerrit.cloudera.org:8080/4767
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/ff6b450a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/ff6b450a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/ff6b450a

Branch: refs/heads/hadoop-next
Commit: ff6b450ad380ce840e18875a89d9cf98058277a3
Parents: 51268c0
Author: Alex Behm <al...@cloudera.com>
Authored: Wed Oct 19 23:27:14 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Oct 22 10:24:24 2016 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-avro-scanner.cc                |  2 +-
 be/src/exec/hdfs-parquet-scanner.cc             |  2 +-
 be/src/exec/hdfs-rcfile-scanner.cc              |  2 +-
 be/src/exec/hdfs-scan-node-base.cc              |  2 -
 be/src/exec/hdfs-scanner.cc                     | 64 ++------------------
 be/src/exec/hdfs-scanner.h                      | 14 ++---
 be/src/exec/hdfs-sequence-scanner.cc            |  4 +-
 be/src/exec/hdfs-text-scanner.cc                |  2 +-
 .../queries/QueryTest/mt-dop.test               |  9 +++
 tests/query_test/test_mt_dop.py                 | 47 ++++++++++++++
 10 files changed, 73 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-avro-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.cc b/be/src/exec/hdfs-avro-scanner.cc
index 88d6d3a..91a9d03 100644
--- a/be/src/exec/hdfs-avro-scanner.cc
+++ b/be/src/exec/hdfs-avro-scanner.cc
@@ -538,7 +538,7 @@ Status HdfsAvroScanner::ProcessRange() {
       int num_to_commit;
       if (scan_node_->materialized_slots().empty()) {
         // No slots to materialize (e.g. count(*)), no need to decode data
-        num_to_commit = WriteEmptyTuples(context_, tuple_row, max_tuples);
+        num_to_commit = WriteTemplateTuples(tuple_row, max_tuples);
       } else {
         if (codegend_decode_avro_data_ != NULL) {
           num_to_commit = codegend_decode_avro_data_(this, max_tuples, pool, &data,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index e91a7ec..542f4cc 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -339,7 +339,7 @@ Status HdfsParquetScanner::GetNextInternal(RowBatch* row_batch) {
     int rows_remaining = file_metadata_.num_rows - row_group_rows_read_;
     int max_tuples = min(row_batch->capacity(), rows_remaining);
     TupleRow* current_row = row_batch->GetRow(row_batch->AddRow());
-    int num_to_commit = WriteEmptyTuples(context_, current_row, max_tuples);
+    int num_to_commit = WriteTemplateTuples(current_row, max_tuples);
     Status status = CommitRows(row_batch, num_to_commit);
     assemble_rows_timer_.Stop();
     RETURN_IF_ERROR(status);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-rcfile-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-rcfile-scanner.cc b/be/src/exec/hdfs-rcfile-scanner.cc
index f43b2aa..012a424 100644
--- a/be/src/exec/hdfs-rcfile-scanner.cc
+++ b/be/src/exec/hdfs-rcfile-scanner.cc
@@ -485,7 +485,7 @@ Status HdfsRCFileScanner::ProcessRange() {
         // If there are no materialized slots (e.g. count(*) or just partition cols)
         // we can shortcircuit the parse loop
         row_pos_ += max_tuples;
-        int num_to_commit = WriteEmptyTuples(context_, current_row, max_tuples);
+        int num_to_commit = WriteTemplateTuples(current_row, max_tuples);
         COUNTER_ADD(scan_node_->rows_read_counter(), max_tuples);
         RETURN_IF_ERROR(CommitRows(num_to_commit));
         continue;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index cf6708c..957338d 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -351,8 +351,6 @@ void HdfsScanNodeBase::Codegen(RuntimeState* state) {
 Status HdfsScanNodeBase::Open(RuntimeState* state) {
   RETURN_IF_ERROR(ExecNode::Open(state));
 
-  if (file_descs_.empty()) return Status::OK();
-
   // Open collection conjuncts
   for (const auto& entry: conjuncts_map_) {
     // conjuncts_ are already opened in ExecNode::Open()

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index 0b6e8c5..3885522 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -211,66 +211,14 @@ Status HdfsScanner::CommitRows(int num_rows) {
   return Status::OK();
 }
 
-// In this code path, no slots were materialized from the input files.  The only
-// slots are from partition keys.  This lets us simplify writing out the batches.
-//   1. template_tuple_ is the complete tuple.
-//   2. Eval conjuncts against the tuple.
-//   3. If it passes, stamp out 'num_tuples' copies of it into the row_batch.
-int HdfsScanner::WriteEmptyTuples(RowBatch* row_batch, int num_tuples) {
-  DCHECK_GT(num_tuples, 0);
-
-  if (template_tuple_ == NULL) {
-    // No slots from partitions keys or slots.  This is count(*).  Just add the
-    // number of rows to the batch.
-    row_batch->AddRows(num_tuples);
-    row_batch->CommitRows(num_tuples);
-  } else {
-    // Make a row and evaluate the row
-    int row_idx = row_batch->AddRow();
-
-    TupleRow* current_row = row_batch->GetRow(row_idx);
-    current_row->SetTuple(scan_node_->tuple_idx(), template_tuple_);
-    if (!EvalConjuncts(current_row)) return 0;
-    // Add first tuple
-    row_batch->CommitLastRow();
-    --num_tuples;
-
-    DCHECK_LE(num_tuples, row_batch->capacity() - row_batch->num_rows());
-
-    for (int n = 0; n < num_tuples; ++n) {
-      DCHECK(!row_batch->AtCapacity());
-      TupleRow* current_row = row_batch->GetRow(row_batch->AddRow());
-      current_row->SetTuple(scan_node_->tuple_idx(), template_tuple_);
-      row_batch->CommitLastRow();
-    }
-  }
-  return num_tuples;
-}
-
-// In this code path, no slots were materialized from the input files.  The only
-// slots are from partition keys.  This lets us simplify writing out the batches.
-//   1. template_tuple_ is the complete tuple.
-//   2. Eval conjuncts against the tuple.
-//   3. If it passes, stamp out 'num_tuples' copies of it into the row_batch.
-int HdfsScanner::WriteEmptyTuples(ScannerContext* context,
-    TupleRow* row, int num_tuples) {
+int HdfsScanner::WriteTemplateTuples(TupleRow* row, int num_tuples) {
   DCHECK_GE(num_tuples, 0);
-  if (num_tuples == 0) return 0;
-
-  if (template_tuple_ == NULL) {
-    // Must be conjuncts on constant exprs.
-    if (!EvalConjuncts(row)) return 0;
-    return num_tuples;
-  } else {
-    row->SetTuple(scan_node_->tuple_idx(), template_tuple_);
-    if (!EvalConjuncts(row)) return 0;
-    row = next_row(row);
+  DCHECK_EQ(scan_node_->tuple_idx(), 0);
+  DCHECK_EQ(scanner_conjunct_ctxs_->size(), 0);
+  if (num_tuples == 0 || template_tuple_ == NULL) return num_tuples;
 
-    for (int n = 1; n < num_tuples; ++n) {
-      row->SetTuple(scan_node_->tuple_idx(), template_tuple_);
-      row = next_row(row);
-    }
-  }
+  Tuple** row_tuple = reinterpret_cast<Tuple**>(row);
+  for (int i = 0; i < num_tuples; ++i) row_tuple[i] = template_tuple_;
   return num_tuples;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.h b/be/src/exec/hdfs-scanner.h
index 4a4d366..71efd5a 100644
--- a/be/src/exec/hdfs-scanner.h
+++ b/be/src/exec/hdfs-scanner.h
@@ -347,14 +347,9 @@ class HdfsScanner {
                                    scanner_conjunct_ctxs_->size(), row);
   }
 
-  /// Utility method to write out tuples when there are no materialized
-  /// fields (e.g. select count(*) or only partition keys).
-  ///   num_tuples - Total number of tuples to write out.
-  /// Returns the number of tuples added to the row batch.
-  int WriteEmptyTuples(RowBatch* row_batch, int num_tuples);
-
-  /// Write empty tuples and commit them to the context object
-  int WriteEmptyTuples(ScannerContext* context, TupleRow* tuple_row, int num_tuples);
+  /// Sets 'num_tuples' template tuples in the batch that 'row' points to. Assumes the
+  /// 'tuple_row' only has a single tuple. Returns the number of tuples set.
+  int WriteTemplateTuples(TupleRow* row, int num_tuples);
 
   /// Processes batches of fields and writes them out to tuple_row_mem.
   /// - 'pool' mempool to allocate from for auxiliary tuple memory
@@ -455,9 +450,10 @@ class HdfsScanner {
     return reinterpret_cast<Tuple*>(mem + tuple_byte_size);
   }
 
+  /// Assumes the row only has a single tuple.
   inline TupleRow* next_row(TupleRow* r) const {
     uint8_t* mem = reinterpret_cast<uint8_t*>(r);
-    return reinterpret_cast<TupleRow*>(mem + batch_->row_byte_size());
+    return reinterpret_cast<TupleRow*>(mem + sizeof(Tuple*));
   }
 
   /// Simple wrapper around scanner_conjunct_ctxs_. Used in the codegen'd version of

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-sequence-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-sequence-scanner.cc b/be/src/exec/hdfs-sequence-scanner.cc
index fd552be..33be362 100644
--- a/be/src/exec/hdfs-sequence-scanner.cc
+++ b/be/src/exec/hdfs-sequence-scanner.cc
@@ -214,7 +214,7 @@ Status HdfsSequenceScanner::ProcessDecompressedBlock() {
 
   if (scan_node_->materialized_slots().empty()) {
     // Handle case where there are no slots to materialize (e.g. count(*))
-    num_to_process = WriteEmptyTuples(context_, tuple_row, num_to_process);
+    num_to_process = WriteTemplateTuples(tuple_row, num_to_process);
     COUNTER_ADD(scan_node_->rows_read_counter(), num_to_process);
     RETURN_IF_ERROR(CommitRows(num_to_process));
     return Status::OK();
@@ -334,7 +334,7 @@ Status HdfsSequenceScanner::ProcessRange() {
         RETURN_IF_ERROR(parse_status_);
       }
     } else {
-      add_row = WriteEmptyTuples(context_, tuple_row_mem, 1);
+      add_row = WriteTemplateTuples(tuple_row_mem, 1);
     }
 
     COUNTER_ADD(scan_node_->rows_read_counter(), 1);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/be/src/exec/hdfs-text-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-text-scanner.cc b/be/src/exec/hdfs-text-scanner.cc
index cc63408..0b048f4 100644
--- a/be/src/exec/hdfs-text-scanner.cc
+++ b/be/src/exec/hdfs-text-scanner.cc
@@ -400,7 +400,7 @@ Status HdfsTextScanner::ProcessRange(int* num_tuples, bool past_scan_range) {
       SCOPED_TIMER(scan_node_->materialize_tuple_timer());
       // If we are doing count(*) then we return tuples only containing partition keys
       boundary_row_.Clear();
-      num_tuples_materialized = WriteEmptyTuples(context_, tuple_row_mem, *num_tuples);
+      num_tuples_materialized = WriteTemplateTuples(tuple_row_mem, *num_tuples);
     }
 
     // Save contents that are split across buffers if we are going to return this column

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/testdata/workloads/functional-query/queries/QueryTest/mt-dop.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/mt-dop.test b/testdata/workloads/functional-query/queries/QueryTest/mt-dop.test
new file mode 100644
index 0000000..ac453ca
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/mt-dop.test
@@ -0,0 +1,9 @@
+====
+---- QUERY
+# IMPALA-4285: Test scan with no materialized slots.
+select count(*) from alltypes
+---- RESULTS
+7300
+---- TYPES
+BIGINT
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ff6b450a/tests/query_test/test_mt_dop.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_mt_dop.py b/tests/query_test/test_mt_dop.py
new file mode 100644
index 0000000..1cd6d31
--- /dev/null
+++ b/tests/query_test/test_mt_dop.py
@@ -0,0 +1,47 @@
+# 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.
+
+# Tests queries with the MT_DOP query option.
+
+import pytest
+
+from copy import deepcopy
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.test_vector import TestDimension
+from tests.common.test_vector import TestVector
+
+MT_DOP_VALUES = [1, 2, 8]
+
+class TestMtDop(ImpalaTestSuite):
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestMtDop, cls).add_test_dimensions()
+    cls.TestMatrix.add_dimension(TestDimension('mt_dop', *MT_DOP_VALUES))
+    # IMPALA-4332: The MT scheduler does not work for Kudu or HBase tables.
+    cls.TestMatrix.add_constraint(\
+        lambda v: v.get_value('table_format').file_format != 'hbase')
+    cls.TestMatrix.add_constraint(\
+        lambda v: v.get_value('table_format').file_format != 'kudu')
+
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  def test_mt_dop(self, vector):
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
+    self.run_test_case('QueryTest/mt-dop', new_vector)


[23/33] incubator-impala git commit: IMPALA-4120: Incorrect results with LEAD() analytic function

Posted by ta...@apache.org.
IMPALA-4120: Incorrect results with LEAD() analytic function

This change fixes a memory management problem with LEAD()/LAG()
analytic functions which led to incorrect result. In particular,
the update functions specified for these analytic functions only
make a shallow copy of StringVal (i.e. copying only the pointer
and the length of the string) without copying the string itself.
This may lead to problem if the string is created from some UDFs
which do local allocations whose buffer may be freed and reused
before the result tuple is copied out. This change fixes the problem
above by allocating a buffer at the Init() functions of these
analytic functions to track the intermediate value. In addition,
when the value is copied out in GetValue(), it will be copied into
the MemPool belonging to the AnalyticEvalNode and attached to the
outgoing row batches. This change also fixes a missing free of
local allocations in QueryMaintenance().

Change-Id: I85bb1745232d8dd383a6047c86019c6378ab571f
Reviewed-on: http://gerrit.cloudera.org:8080/4740
Reviewed-by: Michael Ho <kw...@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/51268c05
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/51268c05
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/51268c05

Branch: refs/heads/hadoop-next
Commit: 51268c053ffe41dc1aa9f1b250878113d4225258
Parents: 4808527
Author: Michael Ho <kw...@cloudera.com>
Authored: Mon Sep 26 16:04:27 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Oct 22 07:39:37 2016 +0000

----------------------------------------------------------------------
 be/src/exec/analytic-eval-node.cc               | 88 +++++++++++++-------
 be/src/exec/analytic-eval-node.h                | 28 ++++---
 be/src/exprs/agg-fn-evaluator.h                 | 27 +++---
 be/src/exprs/aggregate-functions-ir.cc          | 11 ++-
 be/src/udf/udf.cc                               |  3 +-
 .../org/apache/impala/catalog/BuiltinsDb.java   |  4 +-
 .../queries/QueryTest/analytic-fns.test         | 14 ++++
 7 files changed, 120 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/be/src/exec/analytic-eval-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/analytic-eval-node.cc b/be/src/exec/analytic-eval-node.cc
index 21a0805..06b0467 100644
--- a/be/src/exec/analytic-eval-node.cc
+++ b/be/src/exec/analytic-eval-node.cc
@@ -359,7 +359,7 @@ inline Status AnalyticEvalNode::AddRow(int64_t stream_idx, TupleRow* row) {
     }
   }
 
-  Status status = Status::OK();
+  Status status;
   // Buffer the entire input row to be returned later with the analytic eval results.
   if (UNLIKELY(!input_stream_->AddRow(row, &status))) {
     // AddRow returns false if an error occurs (available via status()) or there is
@@ -377,45 +377,62 @@ inline Status AnalyticEvalNode::AddRow(int64_t stream_idx, TupleRow* row) {
     }
   }
   DCHECK(status.ok());
-  return status;
+  return Status::OK();
 }
 
-void AnalyticEvalNode::AddResultTuple(int64_t stream_idx) {
+Status AnalyticEvalNode::AddResultTuple(int64_t stream_idx) {
   VLOG_ROW << id() << " AddResultTuple idx=" << stream_idx;
   DCHECK(curr_tuple_ != NULL);
-  Tuple* result_tuple = Tuple::Create(result_tuple_desc_->byte_size(),
-      curr_tuple_pool_.get());
+  MemPool* cur_tuple_pool = curr_tuple_pool_.get();
+  Tuple* result_tuple = Tuple::Create(result_tuple_desc_->byte_size(), cur_tuple_pool);
 
   AggFnEvaluator::GetValue(evaluators_, fn_ctxs_, curr_tuple_, result_tuple);
+  // Copy any string data in 'result_tuple' into 'cur_tuple_pool_'.
+  for (const SlotDescriptor* slot_desc : result_tuple_desc_->slots()) {
+    if (!slot_desc->type().IsVarLenStringType()) continue;
+    StringValue* sv = reinterpret_cast<StringValue*>(
+        result_tuple->GetSlot(slot_desc->tuple_offset()));
+    if (sv == NULL || sv->len == 0) continue;
+    char* new_ptr = reinterpret_cast<char*>(cur_tuple_pool->TryAllocate(sv->len));
+    if (UNLIKELY(new_ptr == NULL)) {
+      return cur_tuple_pool->mem_tracker()->MemLimitExceeded(NULL,
+          "Failed to allocate memory for analytic function's result.", sv->len);
+    }
+    memcpy(new_ptr, sv->ptr, sv->len);
+    sv->ptr = new_ptr;
+  }
+
   DCHECK_GT(stream_idx, last_result_idx_);
   result_tuples_.push_back(pair<int64_t, Tuple*>(stream_idx, result_tuple));
   last_result_idx_ = stream_idx;
   VLOG_ROW << id() << " Added result tuple, final state: " << DebugStateString(true);
+  return Status::OK();
 }
 
-inline void AnalyticEvalNode::TryAddResultTupleForPrevRow(bool next_partition,
+inline Status AnalyticEvalNode::TryAddResultTupleForPrevRow(bool next_partition,
     int64_t stream_idx, TupleRow* row) {
   // The analytic fns are finalized after the previous row if we found a new partition
   // or the window is a RANGE and the order by exprs changed. For ROWS windows we do not
   // need to compare the current row to the previous row.
   VLOG_ROW << id() << " TryAddResultTupleForPrevRow partition=" << next_partition
            << " idx=" << stream_idx;
-  if (fn_scope_ == ROWS) return;
-  if (next_partition || (fn_scope_ == RANGE && window_.__isset.window_end &&
-      !PrevRowCompare(order_by_eq_expr_ctx_))) {
-    AddResultTuple(stream_idx - 1);
+  if (fn_scope_ != ROWS && (next_partition || (fn_scope_ == RANGE &&
+      window_.__isset.window_end && !PrevRowCompare(order_by_eq_expr_ctx_)))) {
+    RETURN_IF_ERROR(AddResultTuple(stream_idx - 1));
   }
+  return Status::OK();
 }
 
-inline void AnalyticEvalNode::TryAddResultTupleForCurrRow(int64_t stream_idx,
+inline Status AnalyticEvalNode::TryAddResultTupleForCurrRow(int64_t stream_idx,
     TupleRow* row) {
   VLOG_ROW << id() << " TryAddResultTupleForCurrRow idx=" << stream_idx;
   // We only add results at this point for ROWS windows (unless unbounded following)
-  if (fn_scope_ != ROWS || !window_.__isset.window_end) return;
-
   // Nothing to add if the end offset is before the start of the partition.
-  if (stream_idx - rows_end_offset_ < curr_partition_idx_) return;
-  AddResultTuple(stream_idx - rows_end_offset_);
+  if (fn_scope_ == ROWS && window_.__isset.window_end &&
+      stream_idx - rows_end_offset_ >= curr_partition_idx_) {
+    RETURN_IF_ERROR(AddResultTuple(stream_idx - rows_end_offset_));
+  }
+  return Status::OK();
 }
 
 inline void AnalyticEvalNode::TryRemoveRowsBeforeWindow(int64_t stream_idx) {
@@ -435,21 +452,27 @@ inline void AnalyticEvalNode::TryRemoveRowsBeforeWindow(int64_t stream_idx) {
   window_tuples_.pop_front();
 }
 
-inline void AnalyticEvalNode::TryAddRemainingResults(int64_t partition_idx,
+inline Status AnalyticEvalNode::TryAddRemainingResults(int64_t partition_idx,
     int64_t prev_partition_idx) {
   DCHECK_LT(prev_partition_idx, partition_idx);
   // For PARTITION, RANGE, or ROWS with UNBOUNDED PRECEDING: add a result tuple for the
   // remaining rows in the partition that do not have an associated result tuple yet.
   if (fn_scope_ != ROWS || !window_.__isset.window_end) {
-    if (last_result_idx_ < partition_idx - 1) AddResultTuple(partition_idx - 1);
-    return;
+    if (last_result_idx_ < partition_idx - 1) {
+      RETURN_IF_ERROR(AddResultTuple(partition_idx - 1));
+    }
+    return Status::OK();
   }
 
   // lead() is re-written to a ROWS window with an end bound FOLLOWING. Any remaining
   // results need the default value (set by Init()). If this is the case, the start bound
   // is UNBOUNDED PRECEDING (DCHECK in Init()).
   for (int i = 0; i < evaluators_.size(); ++i) {
-    if (is_lead_fn_[i]) evaluators_[i]->Init(fn_ctxs_[i], curr_tuple_);
+    if (is_lead_fn_[i]) {
+      // Needs to call Finalize() to release resources.
+      evaluators_[i]->Finalize(fn_ctxs_[i], curr_tuple_, dummy_result_tuple_);
+      evaluators_[i]->Init(fn_ctxs_[i], curr_tuple_);
+    }
   }
 
   // If the start bound is not UNBOUNDED PRECEDING and there are still rows in the
@@ -470,13 +493,16 @@ inline void AnalyticEvalNode::TryAddRemainingResults(int64_t partition_idx,
       AggFnEvaluator::Remove(evaluators_, fn_ctxs_, remove_row, curr_tuple_);
       window_tuples_.pop_front();
     }
-    AddResultTuple(last_result_idx_ + 1);
+    RETURN_IF_ERROR(AddResultTuple(last_result_idx_ + 1));
   }
 
   // If there are still rows between the row with the last result (AddResultTuple() may
   // have updated last_result_idx_) and the partition boundary, add the current results
   // for the remaining rows with the same result tuple (curr_tuple_ is not modified).
-  if (last_result_idx_ < partition_idx - 1) AddResultTuple(partition_idx - 1);
+  if (last_result_idx_ < partition_idx - 1) {
+    RETURN_IF_ERROR(AddResultTuple(partition_idx - 1));
+  }
+  return Status::OK();
 }
 
 inline Status AnalyticEvalNode::InitNextPartition(RuntimeState* state,
@@ -523,7 +549,7 @@ inline Status AnalyticEvalNode::InitNextPartition(RuntimeState* state,
 
   if (fn_scope_ == ROWS && stream_idx > 0 && (!window_.__isset.window_end ||
         window_.window_end.type == TAnalyticWindowBoundaryType::FOLLOWING)) {
-    TryAddRemainingResults(stream_idx, prev_partition_stream_idx);
+    RETURN_IF_ERROR(TryAddRemainingResults(stream_idx, prev_partition_stream_idx));
   }
   window_tuples_.clear();
 
@@ -557,10 +583,10 @@ inline Status AnalyticEvalNode::InitNextPartition(RuntimeState* state,
       // 2) Insert the initial result tuple at first_val_null_offset_. This happens when
       //    the end bound was actually Y PRECEDING.
       if (first_val_null_offset_ != -1) {
-        AddResultTuple(curr_partition_idx_ + first_val_null_offset_ - 1);
+        RETURN_IF_ERROR(AddResultTuple(curr_partition_idx_ + first_val_null_offset_ - 1));
       }
     } else {
-      AddResultTuple(curr_partition_idx_ - rows_end_offset_ - 1);
+      RETURN_IF_ERROR(AddResultTuple(curr_partition_idx_ - rows_end_offset_ - 1));
     }
   }
   return Status::OK();
@@ -614,7 +640,7 @@ Status AnalyticEvalNode::ProcessChildBatch(RuntimeState* state) {
   if (UNLIKELY(stream_idx == 0 && curr_child_batch_->num_rows() > 0)) {
     TupleRow* row = curr_child_batch_->GetRow(0);
     RETURN_IF_ERROR(AddRow(0, row));
-    TryAddResultTupleForCurrRow(0, row);
+    RETURN_IF_ERROR(TryAddResultTupleForCurrRow(0, row));
     prev_input_row_ = row;
     ++batch_idx;
     ++stream_idx;
@@ -647,19 +673,19 @@ Status AnalyticEvalNode::ProcessChildBatch(RuntimeState* state) {
       // partition_by_eq_expr_ctx_ checks equality over the predicate exprs
       next_partition = !PrevRowCompare(partition_by_eq_expr_ctx_);
     }
-    TryAddResultTupleForPrevRow(next_partition, stream_idx, row);
+    RETURN_IF_ERROR(TryAddResultTupleForPrevRow(next_partition, stream_idx, row));
     if (next_partition) RETURN_IF_ERROR(InitNextPartition(state, stream_idx));
 
     // The evaluators_ are updated with the current row.
     RETURN_IF_ERROR(AddRow(stream_idx, row));
 
-    TryAddResultTupleForCurrRow(stream_idx, row);
+    RETURN_IF_ERROR(TryAddResultTupleForCurrRow(stream_idx, row));
     prev_input_row_ = row;
   }
 
   if (UNLIKELY(input_eos_ && stream_idx > curr_partition_idx_)) {
     // We need to add the results for the last row(s).
-    TryAddRemainingResults(stream_idx, curr_partition_idx_);
+    RETURN_IF_ERROR(TryAddRemainingResults(stream_idx, curr_partition_idx_));
   }
 
   // Transfer resources to prev_tuple_pool_ when enough resources have accumulated
@@ -846,7 +872,10 @@ void AnalyticEvalNode::Close(RuntimeState* state) {
     }
     evaluators_[i]->Close(state);
   }
-  for (int i = 0; i < fn_ctxs_.size(); ++i) fn_ctxs_[i]->impl()->Close();
+  for (int i = 0; i < fn_ctxs_.size(); ++i) {
+    fn_ctxs_[i]->impl()->FreeLocalAllocations();
+    fn_ctxs_[i]->impl()->Close();
+  }
 
   if (partition_by_eq_expr_ctx_ != NULL) partition_by_eq_expr_ctx_->Close(state);
   if (order_by_eq_expr_ctx_ != NULL) order_by_eq_expr_ctx_->Close(state);
@@ -878,6 +907,7 @@ Status AnalyticEvalNode::QueryMaintenance(RuntimeState* state) {
   for (int i = 0; i < evaluators_.size(); ++i) {
     ExprContext::FreeLocalAllocations(evaluators_[i]->input_expr_ctxs());
   }
+  ExprContext::FreeLocalAllocations(fn_ctxs_);
   return ExecNode::QueryMaintenance(state);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/be/src/exec/analytic-eval-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/analytic-eval-node.h b/be/src/exec/analytic-eval-node.h
index 68b5bbc..579ad5b 100644
--- a/be/src/exec/analytic-eval-node.h
+++ b/be/src/exec/analytic-eval-node.h
@@ -132,21 +132,24 @@ class AnalyticEvalNode : public ExecNode {
   Status AddRow(int64_t stream_idx, TupleRow* row);
 
   /// Determines if there is a window ending at the previous row, and if so, calls
-  /// AddResultTuple() with the index of the previous row in input_stream_. next_partition
-  /// indicates if the current row is the start of a new partition. stream_idx is the
-  /// index of the current input row from input_stream_.
-  void TryAddResultTupleForPrevRow(bool next_partition, int64_t stream_idx,
+  /// AddResultTuple() with the index of the previous row in 'input_stream_'.
+  /// 'next_partition' indicates if the current row is the start of a new partition.
+  /// 'stream_idx' is the index of the current input row from 'input_stream_'.
+  /// Returns an error when memory limit is exceeded.
+  Status TryAddResultTupleForPrevRow(bool next_partition, int64_t stream_idx,
       TupleRow* row);
 
   /// Determines if there is a window ending at the current row, and if so, calls
-  /// AddResultTuple() with the index of the current row in input_stream_. stream_idx is
-  /// the index of the current input row from input_stream_.
-  void TryAddResultTupleForCurrRow(int64_t stream_idx, TupleRow* row);
+  /// AddResultTuple() with the index of the current row in 'input_stream_'.
+  /// 'stream_idx' is the index of the current input row from 'input_stream_'.
+  /// Returns an error when memory limit is exceeded.
+  Status TryAddResultTupleForCurrRow(int64_t stream_idx, TupleRow* row);
 
   /// Adds additional result tuples at the end of a partition, e.g. if the end bound is
   /// FOLLOWING. partition_idx is the index into input_stream_ of the new partition,
-  /// prev_partition_idx is the index of the previous partition.
-  void TryAddRemainingResults(int64_t partition_idx, int64_t prev_partition_idx);
+  /// 'prev_partition_idx' is the index of the previous partition.
+  /// Returns an error when memory limit is exceeded.
+  Status TryAddRemainingResults(int64_t partition_idx, int64_t prev_partition_idx);
 
   /// Removes rows from curr_tuple_ (by calling AggFnEvaluator::Remove()) that are no
   /// longer in the window (i.e. they are before the window start boundary). stream_idx
@@ -159,9 +162,10 @@ class AnalyticEvalNode : public ExecNode {
   Status InitNextPartition(RuntimeState* state, int64_t stream_idx);
 
   /// Produces a result tuple with analytic function results by calling GetValue() or
-  /// Finalize() for curr_tuple_ on the evaluators_. The result tuple is stored in
-  /// result_tuples_ with the index into input_stream_ specified by stream_idx.
-  void AddResultTuple(int64_t stream_idx);
+  /// Finalize() for 'curr_tuple_' on the 'evaluators_'. The result tuple is stored in
+  /// 'result_tuples_' with the index into 'input_stream_' specified by 'stream_idx'.
+  /// Returns an error when memory limit is exceeded.
+  Status AddResultTuple(int64_t stream_idx);
 
   /// Gets the number of rows that are ready to be returned by subsequent calls to
   /// GetNextOutputBatch().

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/be/src/exprs/agg-fn-evaluator.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/agg-fn-evaluator.h b/be/src/exprs/agg-fn-evaluator.h
index cde969c..b3ecda0 100644
--- a/be/src/exprs/agg-fn-evaluator.h
+++ b/be/src/exprs/agg-fn-evaluator.h
@@ -147,9 +147,11 @@ class AggFnEvaluator {
   void Finalize(FunctionContext* agg_fn_ctx, Tuple* src, Tuple* dst);
 
   /// Puts the finalized value from Tuple* src in Tuple* dst just as Finalize() does.
-  /// However, unlike Finalize(), GetValue() does not clean up state in src. GetValue()
-  /// can be called repeatedly with the same src. Only used internally for analytic fn
-  /// builtins.
+  /// However, unlike Finalize(), GetValue() does not clean up state in src.
+  /// GetValue() can be called repeatedly with the same src. Only used internally for
+  /// analytic fn builtins. Note that StringVal result is from local allocation (which
+  /// will be freed in the next QueryMaintenance()) so it needs to be copied out if it
+  /// needs to survive beyond QueryMaintenance() (e.g. if 'dst' lives in a row batch).
   void GetValue(FunctionContext* agg_fn_ctx, Tuple* src, Tuple* dst);
 
   /// Helper functions for calling the above functions on many evaluators.
@@ -229,10 +231,13 @@ class AggFnEvaluator {
   /// fn must be a function that implement's the UDA Update() signature.
   void Update(FunctionContext* agg_fn_ctx, const TupleRow* row, Tuple* dst, void* fn);
 
-  /// Sets up the arguments to call fn. This converts from the agg-expr signature,
+  /// Sets up the arguments to call 'fn'. This converts from the agg-expr signature,
   /// taking TupleRow to the UDA signature taking AnvVals. Writes the serialize/finalize
-  /// result to the given destination slot/tuple. The fn can be NULL to indicate the src
-  /// value should simply be written into the destination.
+  /// result to the given destination slot/tuple. 'fn' can be NULL to indicate the src
+  /// value should simply be written into the destination. Note that StringVal result is
+  /// from local allocation (which will be freed in the next QueryMaintenance()) so it
+  /// needs to be copied out if it needs to survive beyond QueryMaintenance() (e.g. if
+  /// 'dst' lives in a row batch).
   void SerializeOrFinalize(FunctionContext* agg_fn_ctx, Tuple* src,
       const SlotDescriptor* dst_slot_desc, Tuple* dst, void* fn);
 
@@ -265,7 +270,7 @@ inline void AggFnEvaluator::GetValue(
 }
 
 inline void AggFnEvaluator::Init(const std::vector<AggFnEvaluator*>& evaluators,
-      const std::vector<FunctionContext*>& fn_ctxs, Tuple* dst) {
+    const std::vector<FunctionContext*>& fn_ctxs, Tuple* dst) {
   DCHECK_EQ(evaluators.size(), fn_ctxs.size());
   for (int i = 0; i < evaluators.size(); ++i) {
     evaluators[i]->Init(fn_ctxs[i], dst);
@@ -279,28 +284,28 @@ inline void AggFnEvaluator::Add(const std::vector<AggFnEvaluator*>& evaluators,
   }
 }
 inline void AggFnEvaluator::Remove(const std::vector<AggFnEvaluator*>& evaluators,
-      const std::vector<FunctionContext*>& fn_ctxs, const TupleRow* src, Tuple* dst) {
+    const std::vector<FunctionContext*>& fn_ctxs, const TupleRow* src, Tuple* dst) {
   DCHECK_EQ(evaluators.size(), fn_ctxs.size());
   for (int i = 0; i < evaluators.size(); ++i) {
     evaluators[i]->Remove(fn_ctxs[i], src, dst);
   }
 }
 inline void AggFnEvaluator::Serialize(const std::vector<AggFnEvaluator*>& evaluators,
-      const std::vector<FunctionContext*>& fn_ctxs, Tuple* dst) {
+    const std::vector<FunctionContext*>& fn_ctxs, Tuple* dst) {
   DCHECK_EQ(evaluators.size(), fn_ctxs.size());
   for (int i = 0; i < evaluators.size(); ++i) {
     evaluators[i]->Serialize(fn_ctxs[i], dst);
   }
 }
 inline void AggFnEvaluator::GetValue(const std::vector<AggFnEvaluator*>& evaluators,
-      const std::vector<FunctionContext*>& fn_ctxs, Tuple* src, Tuple* dst) {
+    const std::vector<FunctionContext*>& fn_ctxs, Tuple* src, Tuple* dst) {
   DCHECK_EQ(evaluators.size(), fn_ctxs.size());
   for (int i = 0; i < evaluators.size(); ++i) {
     evaluators[i]->GetValue(fn_ctxs[i], src, dst);
   }
 }
 inline void AggFnEvaluator::Finalize(const std::vector<AggFnEvaluator*>& evaluators,
-      const std::vector<FunctionContext*>& fn_ctxs, Tuple* src, Tuple* dst) {
+    const std::vector<FunctionContext*>& fn_ctxs, Tuple* src, Tuple* dst) {
   DCHECK_EQ(evaluators.size(), fn_ctxs.size());
   for (int i = 0; i < evaluators.size(); ++i) {
     evaluators[i]->Finalize(fn_ctxs[i], src, dst);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/be/src/exprs/aggregate-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/aggregate-functions-ir.cc b/be/src/exprs/aggregate-functions-ir.cc
index 44b72a3..dd81005 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -1583,10 +1583,19 @@ void AggregateFunctions::OffsetFnInit(FunctionContext* ctx, T* dst) {
   *dst = *static_cast<T*>(ctx->GetConstantArg(2));
 }
 
+template <>
+void AggregateFunctions::OffsetFnInit(FunctionContext* ctx, StringVal* dst) {
+  DCHECK_EQ(ctx->GetNumArgs(), 3);
+  DCHECK(ctx->IsArgConstant(1));
+  DCHECK(ctx->IsArgConstant(2));
+  DCHECK_EQ(ctx->GetArgType(0)->type, ctx->GetArgType(2)->type);
+  CopyStringVal(ctx, *static_cast<StringVal*>(ctx->GetConstantArg(2)), dst);
+}
+
 template <typename T>
 void AggregateFunctions::OffsetFnUpdate(FunctionContext* ctx, const T& src,
     const BigIntVal&, const T& default_value, T* dst) {
-  *dst = src;
+  UpdateVal(ctx, src, dst);
 }
 
 // Stamp out the templates for the types we need.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/be/src/udf/udf.cc
----------------------------------------------------------------------
diff --git a/be/src/udf/udf.cc b/be/src/udf/udf.cc
index 13634dc..4ad8d8b 100644
--- a/be/src/udf/udf.cc
+++ b/be/src/udf/udf.cc
@@ -194,7 +194,8 @@ void FunctionContextImpl::Close() {
   stringstream error_ss;
   if (!debug_) {
     if (pool_->net_allocations() > 0) {
-      error_ss << "Memory leaked via FunctionContext::Allocate()";
+      error_ss << "Memory leaked via FunctionContext::Allocate() "
+               << "or FunctionContext::AllocateLocal()";
     } else if (pool_->net_allocations() < 0) {
       error_ss << "FunctionContext::Free() called on buffer that was already freed or "
                   "was not allocated.";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
index fb4accc..4fcecb2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
@@ -1035,7 +1035,9 @@ public class BuiltinsDb extends Db {
           db, "lead", Lists.newArrayList(t, Type.BIGINT, t), t, t,
           prefix + OFFSET_FN_INIT_SYMBOL.get(t),
           prefix + OFFSET_FN_UPDATE_SYMBOL.get(t),
-          null, null, null));
+          null,
+          t == Type.STRING ? stringValGetValue : null,
+          t == Type.STRING ? stringValSerializeOrFinalize : null));
 
       // lead() and lag() the default offset and the default value should be
       // rewritten to call the overrides that take all parameters.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51268c05/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
index 93431bf..62c9a67 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
@@ -1893,3 +1893,17 @@ DECIMAL, DECIMAL
 12345,1234
 132842,1234
 ====
+---- QUERY
+# Regression test for IMPALA-4120: Invoke UDFs which do local allocations and verifies
+# that the results are copied out.
+select count(*) from (
+select
+  from_unixtime(lead(bigint_col, 1) over (order by id), 'yyyyMMddHH:mm:ss') as a,
+  lead(from_unixtime(bigint_col, 'yyyyMMddHH:mm:ss'), 1) over (order by id) AS b
+from functional.alltypes) x
+where x.a = x.b
+---- TYPES
+BIGINT
+---- RESULTS
+7299
+====


[04/33] incubator-impala git commit: IMPALA-4230: ASF policy issues from 2.7.0 rc3.

Posted by ta...@apache.org.
IMPALA-4230: ASF policy issues from 2.7.0 rc3.

In our IPMC vote to release 2.7.0 rc3, Justing Mclean pointed out a
number of issues of compliance with ASF policy. He asked:

1. "Please place build instruction and supported platforms in the
README. The wiki may change over time and that may make it difficult
to build older versions."

2. Remove binary file llvm-ir/test-loop.bc

3. Add be/src/gutil/valgrind.h,
shell/ext-py/sqlparse-0.1.14/sqlparse/pipeline.py and
cmake_modules/FindJNI.cmake, normalize.css (embedded in bootstrap.css)
to LICENSE.txt

4. Fix be/src/thirdparty/squeasel/squeasel* in LICENSE.txt

5. Remove outdated copyright lines from HBase (see
https://issues.apache.org/jira/browse/HBASE-3870)

6. Remove duplicate jquery notice from LICENSE.txt

Change-Id: I30ff77d7ac28ce67511c200764fba19ae69922e0
Reviewed-on: http://gerrit.cloudera.org:8080/4582
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/5a2c50a1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/5a2c50a1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/5a2c50a1

Branch: refs/heads/hadoop-next
Commit: 5a2c50a1633d8bc2d848f609e867b324c24fe06c
Parents: ef5d0c3
Author: Jim Apple <jb...@cloudera.com>
Authored: Fri Sep 30 14:29:45 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Oct 19 23:59:02 2016 +0000

----------------------------------------------------------------------
 LICENSE.txt                                     | 130 ++++++++++++++++++-
 README.md                                       |  10 +-
 be/src/codegen/CMakeLists.txt                   |   7 +
 bin/rat_exclude_files.txt                       |   1 -
 fe/src/test/resources/hbase-site.xml.template   |   2 -
 llvm-ir/test-loop.bc                            | Bin 1828 -> 0 bytes
 .../common/etc/hadoop/conf/hadoop-policy.xml    |   2 -
 .../etc/hadoop/conf/log4j.properties.tmpl       |   2 -
 8 files changed, 144 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index eb5990d..4cda0f2 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -251,6 +251,47 @@ src/be/gutil/utf: licensed under the following terms:
 
 --------------------------------------------------------------------------------
 
+src/be/gutil/valgrind.h: licensed under the following terms:
+
+   This file is part of Valgrind, a dynamic binary instrumentation
+   framework.
+
+   Copyright (C) 2000-2008 Julian Seward.  All rights reserved.
+
+   Redistribution and use in source and binary forms, with or without
+   modification, are permitted provided that the following conditions
+   are met:
+
+   1. Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+   2. The origin of this software must not be misrepresented; you must
+      not claim that you wrote the original software.  If you use this
+      software in a product, an acknowledgment in the product
+      documentation would be appreciated but is not required.
+
+   3. Altered source versions must be plainly marked as such, and must
+      not be misrepresented as being the original software.
+
+   4. The name of the author may not be used to endorse or promote
+      products derived from this software without specific prior written
+      permission.
+
+   THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS
+   OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+   WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+   ARE DISCLAIMED.  IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY
+   DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+   DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE
+   GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+   INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+   WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+   NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+   SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+--------------------------------------------------------------------------------
+
 www/d3.v3.[min.]js: BSD 3-clause license
 
    Copyright (c) 2012, Michael Bostock
@@ -458,6 +499,31 @@ tests/comparison/leopard/static/fonts/glyphicons-halflings*: MIT license
 
 --------------------------------------------------------------------------------
 
+normalize.css embedded in www/bootstrap/css/bootstrap.css: MIT
+normalize.css embedded in tests/comparison/leopard/static/css/bootstrap.css: MIT
+
+Copyright (c) Nicolas Gallagher and Jonathan Neal
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+--------------------------------------------------------------------------------
+
 www/highlight: 3-clause BSD
 tests/comparison/leopard/static/css/hljs.css: 3-clause BSD
 tests/comparison/leopard/static/js/highlight.pack.js: 3-clause BSD
@@ -521,7 +587,7 @@ shell/ext-py/prettytable-0.7.1: 3-clause BSD
 
 --------------------------------------------------------------------------------
 
-shell/ext-py/sqlparse-0.1.14: 3-clause BSD
+shell/ext-py/sqlparse-0.1.14 (most parts): 3-clause BSD
 
   Copyright (c) 2009, Andi Albrecht <al...@gmail.com>
 
@@ -573,6 +639,34 @@ THE SOFTWARE.
 
 --------------------------------------------------------------------------------
 
+shell/ext-py/sqlparse-0.1.14/sqlparse/pipeline.py:
+
+Copyright (C) 2011 Jesus Leganes "piranna", piranna@gmail.com
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are
+permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this list of
+conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice, this list of
+conditions and the following disclaimer in the documentation and/or other materials
+provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR
+TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
 be/src/thirdparty/mustache: Apache 2.0 license
 be/src/expr/hll-bias.h: Apache 2.0 license
 shell/ext-py/sasl-0.1.1: Apache 2.0 license
@@ -603,4 +697,36 @@ cmake_modules/FindGTest.cmake: MIT license
 
 --------------------------------------------------------------------------------
 
-www/jquery/jquery-1.12.4.min.js: MIT license
+cmake_modules/FindJNI.cmake: 3-clause BSD
+
+ Copyright 2001-2009 Kitware, Inc.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions
+ are met:
+
+ * Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+
+ * Neither the names of Kitware, Inc., the Insight Software Consortium,
+   nor the names of their contributors may be used to endorse or promote
+   products derived from this software without specific prior written
+   permission.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index b7778e9..b90e7c7 100644
--- a/README.md
+++ b/README.md
@@ -23,4 +23,12 @@ visit the [Impala homepage](https://impala.apache.org).
 
 If you are interested in contributing to Impala as a developer, or learning more about
 Impala's internals and architecture, visit the
-[Impala wiki](https://cwiki.apache.org/confluence/display/IMPALA/Impala+Home).
\ No newline at end of file
+[Impala wiki](https://cwiki.apache.org/confluence/display/IMPALA/Impala+Home).
+
+## Supported Platforms
+
+Impala only supports Linux at the moment.
+
+## Build Instructions
+
+./buildall.sh -notests
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/be/src/codegen/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/codegen/CMakeLists.txt b/be/src/codegen/CMakeLists.txt
index 8ae44e0..d002c68 100644
--- a/be/src/codegen/CMakeLists.txt
+++ b/be/src/codegen/CMakeLists.txt
@@ -99,5 +99,12 @@ add_custom_command(
   DEPENDS ${IR_NO_SSE_OUTPUT_FILE}
 )
 
+# Run the clang compiler to generate BC for llvm-codegen-test
+add_custom_target(test-loop.bc
+  COMMAND ${LLVM_CLANG_EXECUTABLE} ${CLANG_IR_CXX_FLAGS} ${CLANG_INCLUDE_FLAGS} ${CMAKE_SOURCE_DIR}/testdata/llvm/test-loop.cc -o ${CMAKE_SOURCE_DIR}/llvm-ir/test-loop.bc
+  SOURCES ${CMAKE_SOURCE_DIR}/testdata/llvm/test-loop.cc
+)
+
 ADD_BE_TEST(llvm-codegen-test)
+add_dependencies(llvm-codegen-test test-loop.bc)
 ADD_BE_TEST(instruction-counter-test)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/bin/rat_exclude_files.txt
----------------------------------------------------------------------
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 2a07b68..a27355e 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -61,7 +61,6 @@ be/src/testutil/*.pem
 fe/src/test/resources/*.xml
 fe/src/test/resources/hbase-jaas-client.conf.template
 fe/src/test/resources/hbase-jaas-server.conf.template
-llvm-ir/test-loop.bc
 testdata/AllTypesError/*.txt
 testdata/AllTypesErrorNoNulls/*.txt
 *.avsc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/fe/src/test/resources/hbase-site.xml.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/hbase-site.xml.template b/fe/src/test/resources/hbase-site.xml.template
index 7a72924..e17c0b8 100644
--- a/fe/src/test/resources/hbase-site.xml.template
+++ b/fe/src/test/resources/hbase-site.xml.template
@@ -2,8 +2,6 @@
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
 /**
- * Copyright 2010 The Apache Software Foundation
- *
  * 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

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/llvm-ir/test-loop.bc
----------------------------------------------------------------------
diff --git a/llvm-ir/test-loop.bc b/llvm-ir/test-loop.bc
deleted file mode 100644
index f7b50c1..0000000
Binary files a/llvm-ir/test-loop.bc and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/testdata/cluster/node_templates/common/etc/hadoop/conf/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/common/etc/hadoop/conf/hadoop-policy.xml b/testdata/cluster/node_templates/common/etc/hadoop/conf/hadoop-policy.xml
index cc83bc5..db85124 100644
--- a/testdata/cluster/node_templates/common/etc/hadoop/conf/hadoop-policy.xml
+++ b/testdata/cluster/node_templates/common/etc/hadoop/conf/hadoop-policy.xml
@@ -2,8 +2,6 @@
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
 
- Copyright 2011 The Apache Software Foundation
-
  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

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5a2c50a1/testdata/cluster/node_templates/common/etc/hadoop/conf/log4j.properties.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/common/etc/hadoop/conf/log4j.properties.tmpl b/testdata/cluster/node_templates/common/etc/hadoop/conf/log4j.properties.tmpl
index c9bd0ff..aa477d7 100644
--- a/testdata/cluster/node_templates/common/etc/hadoop/conf/log4j.properties.tmpl
+++ b/testdata/cluster/node_templates/common/etc/hadoop/conf/log4j.properties.tmpl
@@ -1,5 +1,3 @@
-# Copyright 2011 The Apache Software Foundation
-#
 # 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


[13/33] 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"



[11/33] 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/hadoop-next
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