You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bo...@apache.org on 2018/11/19 10:45:08 UTC

[01/33] impala git commit: IMPALA-7103: [DOCS] Document the ALLOW_ERASURE_CODED_FILES query option

Repository: impala
Updated Branches:
  refs/heads/branch-3.1.0 e3a702707 -> f79f2d4ad


IMPALA-7103: [DOCS] Document the ALLOW_ERASURE_CODED_FILES query option

Change-Id: I63b54031b725e528196d19eac9ddf36a19c43e28
Reviewed-on: http://gerrit.cloudera.org:8080/11855
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: cb9db5363ee4cd8c1941350162169e763c3dec20
Parents: 8e9298c
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Thu Nov 1 19:16:50 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 docs/impala.ditamap                             |  1 +
 .../topics/impala_allow_erasure_coded_files.xml | 67 ++++++++++++++++++++
 2 files changed, 68 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/cb9db536/docs/impala.ditamap
----------------------------------------------------------------------
diff --git a/docs/impala.ditamap b/docs/impala.ditamap
index d1d09cc..051b838 100644
--- a/docs/impala.ditamap
+++ b/docs/impala.ditamap
@@ -173,6 +173,7 @@ under the License.
       <topicref href="topics/impala_set.xml">
         <topicref href="topics/impala_query_options.xml">
           <topicref href="topics/impala_abort_on_error.xml"/>
+          <topicref href="topics/impala_allow_erasure_coded_files.xml"/>
           <topicref href="topics/impala_allow_unsupported_formats.xml"/>
           <topicref href="topics/impala_appx_count_distinct.xml"/>
           <topicref href="topics/impala_batch_size.xml"/>

http://git-wip-us.apache.org/repos/asf/impala/blob/cb9db536/docs/topics/impala_allow_erasure_coded_files.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_allow_erasure_coded_files.xml b/docs/topics/impala_allow_erasure_coded_files.xml
new file mode 100644
index 0000000..b3089ee
--- /dev/null
+++ b/docs/topics/impala_allow_erasure_coded_files.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
+<concept id="allow_erasure_coded_files">
+
+  <title>ALLOW_ERASURE_CODED_FILES Query Option</title>
+
+  <titlealts audience="PDF">
+
+    <navtitle>ALLOW_ERASURE_CODED_FILES</navtitle>
+
+  </titlealts>
+
+  <prolog>
+    <metadata>
+      <data name="Category" value="Impala"/>
+      <data name="Category" value="Impala Query Options"/>
+      <data name="Category" value="Deprecated Features"/>
+    </metadata>
+  </prolog>
+
+  <conbody>
+
+    <p>
+      Use the <codeph>ALLOW_ERASURE_CODED_FILES</codeph> query option to enable or disable the
+      support of erasure coded files in Impala. Until Impala is fully tested and certified with
+      erasure coded files, this query option is set to <codeph>FALSE</codeph> by default.
+    </p>
+
+    <p>
+      When the <codeph>ALLOW_ERASURE_CODED_FILES</codeph> query option is set to
+      <codeph>FALSE</codeph>, Impala returns an error when a query requires scanning an erasure
+      coded file.
+    </p>
+
+    <p>
+      <b>Type:</b> <codeph>BOOLEAN</codeph>
+    </p>
+
+    <p>
+      <b>Default:</b> <codeph>FALSE</codeph>
+    </p>
+
+    <p>
+      <b>Added in:</b> <keyword keyref="impala31"/>
+    </p>
+
+  </conbody>
+
+</concept>


[05/33] impala git commit: IMPALA-6661 Make NaN values equal for grouping purposes.

Posted by bo...@apache.org.
IMPALA-6661 Make NaN values equal for grouping purposes.

Similar to the treatment of NULLs, we want to consider NaN values
as equal when grouping.

- When detecting a NaN in a set of row values, the NaN value must
  be converted to a canonical value - so that all NaN values have
  the same bit-pattern for hashing purposes.

- When doing equality evaluation, floating point types must have
  additional logic to consider NaN values as equal.

- Existing logic for handling NULLs in this way is appropriate for
  triggering this behavior for NaN values.

- Relabel "force null equality" as "inclusive equality" to expand
  the scope of the concept to a more generic form that includes NaN.

Change-Id: I996c4a2e1934fd887046ed0c55457b7285375086
Reviewed-on: http://gerrit.cloudera.org:8080/11535
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Michael Ho <kw...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 6684fe1900d1c5655f449cf551aa798c1cad3c17
Parents: cb2574b
Author: Michal Ostrowski <mo...@cloudera.com>
Authored: Mon Sep 24 18:58:23 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/codegen/codegen-anyval.cc                | 42 ++++++++++++++++++--
 be/src/codegen/codegen-anyval.h                 | 21 +++++++++-
 be/src/exec/hash-table.cc                       | 35 +++++++++++-----
 be/src/exec/hash-table.h                        | 35 ++++++++--------
 be/src/exec/hash-table.inline.h                 |  4 +-
 be/src/runtime/raw-value.cc                     |  2 +
 be/src/runtime/raw-value.h                      | 13 ++++++
 be/src/runtime/raw-value.inline.h               | 25 ++++++++++++
 .../queries/QueryTest/aggregation.test          | 23 +++++++++++
 .../queries/QueryTest/exprs.test                | 10 +++++
 .../queries/QueryTest/joins.test                | 27 +++++++++++++
 11 files changed, 202 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/codegen/codegen-anyval.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.cc b/be/src/codegen/codegen-anyval.cc
index 3e2eb1e..a44eccc 100644
--- a/be/src/codegen/codegen-anyval.cc
+++ b/be/src/codegen/codegen-anyval.cc
@@ -18,7 +18,7 @@
 #include "codegen/codegen-anyval.h"
 
 #include "codegen/codegen-util.h"
-
+#include "runtime/raw-value.h"
 #include "common/names.h"
 
 using namespace impala;
@@ -466,6 +466,29 @@ void CodegenAnyVal::SetDate(llvm::Value* date) {
   value_ = builder_->CreateInsertValue(value_, v, 0, name_);
 }
 
+void CodegenAnyVal::ConvertToCanonicalForm() {
+  // Convert the value to a bit pattern that is unambiguous.
+  // Specifically, for floating point type values, NaN values are converted to
+  // the same bit pattern.
+  switch(type_.type) {
+    case TYPE_FLOAT:
+    case TYPE_DOUBLE: {
+      llvm::Value* raw = GetVal();
+      llvm::Value* canonical_val;
+      if (type_.type == TYPE_FLOAT) {
+        canonical_val = llvm::ConstantFP::getNaN(codegen_->float_type());
+      } else {
+        canonical_val = llvm::ConstantFP::getNaN(codegen_->double_type());
+      }
+      llvm::Value* is_nan = builder_->CreateFCmpUNO(raw, raw, "cmp_nan");
+      SetVal(builder_->CreateSelect(is_nan, canonical_val, raw));
+      break;
+    }
+    default:
+      ;
+  }
+}
+
 llvm::Value* CodegenAnyVal::GetLoweredPtr(const string& name) const {
   llvm::Value* lowered_ptr =
       codegen_->CreateEntryBlockAlloca(*builder_, value_->getType(), name.c_str());
@@ -702,7 +725,8 @@ llvm::Value* CodegenAnyVal::Eq(CodegenAnyVal* other) {
   }
 }
 
-llvm::Value* CodegenAnyVal::EqToNativePtr(llvm::Value* native_ptr) {
+llvm::Value* CodegenAnyVal::EqToNativePtr(llvm::Value* native_ptr,
+    bool inclusive_equality) {
   llvm::Value* val = NULL;
   if (!type_.IsStringType()) {
      val = builder_->CreateLoad(native_ptr);
@@ -718,9 +742,19 @@ llvm::Value* CodegenAnyVal::EqToNativePtr(llvm::Value* native_ptr) {
     case TYPE_DECIMAL:
       return builder_->CreateICmpEQ(GetVal(), val, "cmp_raw");
     case TYPE_FLOAT:
-    case TYPE_DOUBLE:
+    case TYPE_DOUBLE:{
       // Use the ordering version "OEQ" to ensure that 'nan' != 'nan'.
-      return builder_->CreateFCmpOEQ(GetVal(), val, "cmp_raw");
+      llvm::Value* local_val = GetVal();
+      llvm::Value* cmp_raw = builder_->CreateFCmpOEQ(local_val, val, "cmp_raw");
+      if (!inclusive_equality) return cmp_raw;
+
+      // Mirror logic in HashTableCtx::Equals - IMPALA-6661
+      llvm::Value* local_is_nan = builder_->CreateFCmpUNO(local_val,
+          local_val, "local_val_is_nan");
+      llvm::Value* val_is_nan = builder_->CreateFCmpUNO(val, val, "val_is_nan");
+      llvm::Value* both_nan = builder_->CreateAnd(local_is_nan, val_is_nan);
+      return builder_->CreateOr(cmp_raw, both_nan, "cmp_raw_with_nan");
+    }
     case TYPE_STRING:
     case TYPE_VARCHAR:
     case TYPE_FIXED_UDA_INTERMEDIATE: {

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/codegen/codegen-anyval.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/codegen-anyval.h b/be/src/codegen/codegen-anyval.h
index 2bc800e..bf98fd9 100644
--- a/be/src/codegen/codegen-anyval.h
+++ b/be/src/codegen/codegen-anyval.h
@@ -226,14 +226,31 @@ class CodegenAnyVal {
   void WriteToSlot(const SlotDescriptor& slot_desc, llvm::Value* tuple,
       llvm::Value* pool_val, llvm::BasicBlock* insert_before = nullptr);
 
+  /// Rewrites the bit values of a value in a canonical form.
+  /// Floating point values may be "NaN".  Nominally, NaN != NaN, but
+  /// for grouping purposes we want that to not be the case.
+  /// Therefore all NaN values need to be converted into a consistent
+  /// form where all bits are the same.  This method will do that -
+  /// ensure that all NaN values have the same bit pattern.
+  ///
+  /// Generically speaking, a canonical form of a value ensures that
+  /// all ambiguity is removed from a value's bit settings -- if there
+  /// are bits that can be freely changed without changing the logical
+  /// value of the value. (Currently this only has an impact for NaN
+  /// float and double values.)
+  void ConvertToCanonicalForm();
+
   /// Returns the i1 result of this == other. this and other must be non-null.
   llvm::Value* Eq(CodegenAnyVal* other);
 
   /// Compares this *Val to the value of 'native_ptr'. 'native_ptr' should be a pointer to
   /// a native type, e.g. StringValue, or TimestampValue. This *Val should match
   /// 'native_ptr's type (e.g. if this is an IntVal, 'native_ptr' should have type i32*).
-  /// Returns the i1 result of the equality comparison.
-  llvm::Value* EqToNativePtr(llvm::Value* native_ptr);
+  /// Returns the i1 result of the equality comparison. "inclusive_equality" means that
+  /// the scope of equality will be expanded to include considering as equal scenarios
+  /// that would otherwise resolve to not-equal, such as a comparison of floating-point
+  /// "NaN" values.
+  llvm::Value* EqToNativePtr(llvm::Value* native_ptr, bool inclusive_equality = false);
 
   /// Returns the i32 result of comparing this value to 'other' (similar to
   /// RawValue::Compare()). This and 'other' must be non-null. Return value is < 0 if

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/exec/hash-table.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index e315fd1..576deca 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -185,7 +185,7 @@ bool HashTableCtx::EvalRow(const TupleRow* row,
   bool has_null = false;
   for (int i = 0; i < evals.size(); ++i) {
     void* loc = expr_values_cache_.ExprValuePtr(expr_values, i);
-    void* val = evals[i]->GetValue(row);
+    const void* val = evals[i]->GetValue(row);
     if (val == NULL) {
       // If the table doesn't store nulls, no reason to keep evaluating
       if (!stores_nulls_) return true;
@@ -195,8 +195,12 @@ bool HashTableCtx::EvalRow(const TupleRow* row,
     } else {
       expr_values_null[i] = false;
     }
-    DCHECK_LE(build_exprs_[i]->type().GetSlotSize(), sizeof(NULL_VALUE));
-    RawValue::Write(val, loc, build_exprs_[i]->type(), NULL);
+    const ColumnType& expr_type = build_exprs_[i]->type();
+    DCHECK_LE(expr_type.GetSlotSize(), sizeof(NULL_VALUE));
+    if (RawValue::IsNaN(val, expr_type)) {
+      val = RawValue::CanonicalNaNValue(expr_type);
+    }
+    RawValue::Write(val, loc, expr_type, NULL);
   }
   return has_null;
 }
@@ -230,13 +234,13 @@ uint32_t HashTableCtx::HashVariableLenRow(const uint8_t* expr_values,
   return hash;
 }
 
-template <bool FORCE_NULL_EQUALITY>
+template <bool INCLUSIVE_EQUALITY>
 bool HashTableCtx::Equals(const TupleRow* build_row, const uint8_t* expr_values,
     const uint8_t* expr_values_null) const noexcept {
   for (int i = 0; i < build_expr_evals_.size(); ++i) {
     void* val = build_expr_evals_[i]->GetValue(build_row);
     if (val == NULL) {
-      if (!(FORCE_NULL_EQUALITY || finds_nulls_[i])) return false;
+      if (!(INCLUSIVE_EQUALITY || finds_nulls_[i])) return false;
       if (!expr_values_null[i]) return false;
       continue;
     } else {
@@ -245,7 +249,16 @@ bool HashTableCtx::Equals(const TupleRow* build_row, const uint8_t* expr_values,
 
     const void* loc = expr_values_cache_.ExprValuePtr(expr_values, i);
     DCHECK(build_exprs_[i] == &build_expr_evals_[i]->root());
-    if (!RawValue::Eq(loc, val, build_exprs_[i]->type())) return false;
+
+    const ColumnType& expr_type = build_exprs_[i]->type();
+    if (!RawValue::Eq(loc, val, expr_type)) {
+      if (INCLUSIVE_EQUALITY) {
+        bool val_is_nan = RawValue::IsNaN(val, expr_type);
+        bool local_is_nan = RawValue::IsNaN(loc, expr_type);
+        if (val_is_nan && local_is_nan) continue;
+      }
+      return false;
+    }
   }
   return true;
 }
@@ -786,6 +799,9 @@ Status HashTableCtx::CodegenEvalRow(
 
     // Not null block
     builder.SetInsertPoint(not_null_block);
+
+    result.ConvertToCanonicalForm();
+
     result.StoreToNativePtr(llvm_loc);
     builder.CreateBr(continue_block);
 
@@ -1059,7 +1075,7 @@ Status HashTableCtx::CodegenHashRow(
 //   br i1 %cmp_raw10, label %continue3, label %false_block
 // }
 Status HashTableCtx::CodegenEquals(
-    LlvmCodeGen* codegen, bool force_null_equality, llvm::Function** fn) {
+    LlvmCodeGen* codegen, bool inclusive_equality, llvm::Function** fn) {
   for (int i = 0; i < build_exprs_.size(); ++i) {
     // Disable codegen for CHAR
     if (build_exprs_[i]->type().type == TYPE_CHAR) {
@@ -1123,13 +1139,14 @@ Status HashTableCtx::CodegenEquals(
 
     // We consider null values equal if we are comparing build rows or if the join
     // predicate is <=>
-    if (force_null_equality || finds_nulls_[i]) {
+    if (inclusive_equality || finds_nulls_[i]) {
       llvm::Value* llvm_null_byte_loc = builder.CreateInBoundsGEP(
           NULL, expr_values_null, codegen->GetI32Constant(i), "null_byte_loc");
       llvm::Value* null_byte = builder.CreateLoad(llvm_null_byte_loc);
       row_is_null =
           builder.CreateICmpNE(null_byte, codegen->GetI8Constant(0));
     }
+    if (inclusive_equality) result.ConvertToCanonicalForm();
 
     // Get llvm value for row_val from 'expr_values'
     int offset = expr_values_cache_.expr_values_offsets(i);
@@ -1154,7 +1171,7 @@ Status HashTableCtx::CodegenEquals(
       builder.SetInsertPoint(cmp_block);
     }
     // Check result == row_val
-    llvm::Value* is_equal = result.EqToNativePtr(row_val);
+    llvm::Value* is_equal = result.EqToNativePtr(row_val, inclusive_equality);
     builder.CreateCondBr(is_equal, continue_block, false_block);
 
     builder.SetInsertPoint(continue_block);

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/exec/hash-table.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index c57b7a5..e98bdfd 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -165,10 +165,10 @@ class HashTableCtx {
   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(LlvmCodeGen* codegen, bool force_null_equality,
+  /// matches HashTable::Equals(). 'inclusive_equality' is true if the generated
+  /// equality function should treat all NULLs as equal and all NaNs as equal.
+  /// See the template parameter to HashTable::Equals().
+  Status CodegenEquals(LlvmCodeGen* codegen, bool inclusive_equality,
       llvm::Function** fn);
 
   /// Codegen for hashing expr values. Function prototype matches HashRow identically.
@@ -431,11 +431,11 @@ class HashTableCtx {
   /// Wrapper function for calling correct HashUtil function in non-codegen'd case.
   uint32_t Hash(const void* input, int len, uint32_t hash) const;
 
-  /// Evaluate 'row' over build exprs, storing values into 'expr_values' and nullness into
-  /// 'expr_values_null'. This will be replaced by codegen. We do not want this function
-  /// inlined when cross compiled because we need to be able to differentiate between
-  /// EvalBuildRow and EvalProbeRow by name and the build/probe exprs are baked into the
-  /// codegen'd function.
+  /// Evaluate 'row' over build exprs, storing values into 'expr_values' and nullness
+  /// into 'expr_values_null'. This will be replaced by codegen. We do not want this
+  /// function inlined when cross compiled because we need to be able to differentiate
+  /// between EvalBuildRow and EvalProbeRow by name and the build/probe exprs are baked
+  /// into the codegen'd function.
   bool IR_NO_INLINE EvalBuildRow(
       const TupleRow* row, uint8_t* expr_values, uint8_t* expr_values_null) noexcept {
     return EvalRow(row, build_expr_evals_, expr_values, expr_values_null);
@@ -460,18 +460,17 @@ class HashTableCtx {
       uint8_t* expr_values, uint8_t* expr_values_null) noexcept;
 
   /// Returns true if the values of build_exprs evaluated over 'build_row' equal the
-  /// values in 'expr_values' with nullness 'expr_values_null'. FORCE_NULL_EQUALITY is
-  /// true if all nulls should be treated as equal, regardless of the values of
-  /// 'finds_nulls_'. This will be replaced by codegen.
-  template <bool FORCE_NULL_EQUALITY>
+  /// values in 'expr_values' with nullness 'expr_values_null'. INCLUSIVE_EQUALITY
+  /// means "NULL==NULL" and "NaN==NaN". This will be replaced by codegen.
+  template <bool INCLUSIVE_EQUALITY>
   bool IR_NO_INLINE Equals(const TupleRow* build_row, const uint8_t* expr_values,
       const uint8_t* expr_values_null) const noexcept;
 
   /// Helper function that calls Equals() with the current row. Always inlined so that
   /// it does not appear in cross-compiled IR.
-  template <bool FORCE_NULL_EQUALITY>
+  template <bool INCLUSIVE_EQUALITY>
   bool ALWAYS_INLINE Equals(const TupleRow* build_row) const {
-    return Equals<FORCE_NULL_EQUALITY>(build_row, expr_values_cache_.cur_expr_values(),
+    return Equals<INCLUSIVE_EQUALITY>(build_row, expr_values_cache_.cur_expr_values(),
         expr_values_cache_.cur_expr_values_null());
   }
 
@@ -848,14 +847,14 @@ class HashTable {
   /// this function. The values of the expression values cache in 'ht_ctx' will be
   /// used to probe the hash table.
   ///
-  /// 'FORCE_NULL_EQUALITY' is true if NULLs should always be considered equal when
-  /// comparing two rows.
+  /// 'INCLUSIVE_EQUALITY' is true if NULLs and NaNs should always be
+  /// considered equal when comparing two rows.
   ///
   /// 'hash' is the hash computed by EvalAndHashBuild() or EvalAndHashProbe().
   /// 'found' indicates that a bucket that contains an equal row is found.
   ///
   /// There are wrappers of this function that perform the Find and Insert logic.
-  template <bool FORCE_NULL_EQUALITY>
+  template <bool INCLUSIVE_EQUALITY>
   int64_t IR_ALWAYS_INLINE Probe(Bucket* buckets, int64_t num_buckets,
       HashTableCtx* ht_ctx, uint32_t hash, bool* found);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/exec/hash-table.inline.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-table.inline.h b/be/src/exec/hash-table.inline.h
index 205abe7..2bcecef 100644
--- a/be/src/exec/hash-table.inline.h
+++ b/be/src/exec/hash-table.inline.h
@@ -48,7 +48,7 @@ inline void HashTableCtx::ExprValuesCache::NextRow() {
   DCHECK_LE(cur_expr_values_hash_ - expr_values_hash_array_.get(), capacity_);
 }
 
-template <bool FORCE_NULL_EQUALITY>
+template <bool INCLUSIVE_EQUALITY>
 inline int64_t HashTable::Probe(Bucket* buckets, int64_t num_buckets,
     HashTableCtx* ht_ctx, uint32_t hash, bool* found) {
   DCHECK(buckets != NULL);
@@ -66,7 +66,7 @@ inline int64_t HashTable::Probe(Bucket* buckets, int64_t num_buckets,
     if (LIKELY(!bucket->filled)) return bucket_idx;
     if (hash == bucket->hash) {
       if (ht_ctx != NULL &&
-          ht_ctx->Equals<FORCE_NULL_EQUALITY>(GetRow(bucket, ht_ctx->scratch_row_))) {
+          ht_ctx->Equals<INCLUSIVE_EQUALITY>(GetRow(bucket, ht_ctx->scratch_row_))) {
         *found = true;
         return bucket_idx;
       }

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/runtime/raw-value.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/raw-value.cc b/be/src/runtime/raw-value.cc
index b5de7bc..f15986b 100644
--- a/be/src/runtime/raw-value.cc
+++ b/be/src/runtime/raw-value.cc
@@ -29,6 +29,8 @@
 namespace impala {
 
 const int RawValue::ASCII_PRECISION = 16; // print 16 digits for double/float
+const double RawValue::CANONICAL_DOUBLE_NAN = nan("");
+const float RawValue::CANONICAL_FLOAT_NAN = nanf("");
 
 void RawValue::PrintValueAsBytes(const void* value, const ColumnType& type,
                                  stringstream* stream) {

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/runtime/raw-value.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/raw-value.h b/be/src/runtime/raw-value.h
index 6ee9191..7bf52a5 100644
--- a/be/src/runtime/raw-value.h
+++ b/be/src/runtime/raw-value.h
@@ -37,6 +37,12 @@ class RawValue {
   /// Ascii output precision for double/float
   static const int ASCII_PRECISION;
 
+  /// Single NaN values to ensure all NaN values can be assigned one bit pattern
+  /// that will always compare and hash the same way.  Allows for all NaN values
+  /// to be put into the same "group by" bucket.
+  static const double CANONICAL_DOUBLE_NAN;
+  static const float CANONICAL_FLOAT_NAN;
+
   /// Convert 'value' into ascii and write to 'stream'. NULL turns into "NULL". 'scale'
   /// determines how many digits after the decimal are printed for floating point numbers,
   /// -1 indicates to use the stream's current formatting.
@@ -97,6 +103,13 @@ class RawValue {
   /// This is more performant than Compare() == 0 for string equality, mostly because of
   /// the length comparison check.
   static inline bool Eq(const void* v1, const void* v2, const ColumnType& type);
+
+  /// Returns true if val/type correspond to a NaN floating point value.
+  static inline bool IsNaN(const void* val, const ColumnType& type);
+
+  /// Returns a canonical NaN value for a floating point type
+  /// (which will always have the same bit-pattern to maintain consistency in hashing).
+  static inline const void* CanonicalNaNValue(const ColumnType& type);
 };
 
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/be/src/runtime/raw-value.inline.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/raw-value.inline.h b/be/src/runtime/raw-value.inline.h
index 03c3699..9206cf4 100644
--- a/be/src/runtime/raw-value.inline.h
+++ b/be/src/runtime/raw-value.inline.h
@@ -39,6 +39,29 @@ namespace impala {
 static const uint32_t HASH_VAL_NULL = 0x58081667;
 static const uint32_t HASH_VAL_EMPTY = 0x7dca7eee;
 
+inline bool RawValue::IsNaN(const void* val, const ColumnType& type) {
+  switch(type.type) {
+  case TYPE_FLOAT:
+    return std::isnan(*reinterpret_cast<const float*>(val));
+  case TYPE_DOUBLE:
+    return std::isnan(*reinterpret_cast<const double*>(val));
+  default:
+    return false;
+  }
+}
+
+inline const void* RawValue::CanonicalNaNValue(const ColumnType& type) {
+  switch(type.type) {
+  case TYPE_FLOAT:
+    return &CANONICAL_FLOAT_NAN;
+  case TYPE_DOUBLE:
+    return &CANONICAL_DOUBLE_NAN;
+  default:
+    DCHECK(false);
+    return nullptr;
+  }
+}
+
 inline bool RawValue::Eq(const void* v1, const void* v2, const ColumnType& type) {
   const StringValue* string_value1;
   const StringValue* string_value2;
@@ -144,6 +167,7 @@ inline uint32_t RawValue::GetHashValueNonNull<float>(const float* v,
     const ColumnType& type, uint32_t seed) {
   DCHECK_EQ(type.type, TYPE_FLOAT);
   DCHECK(v != NULL);
+  if (std::isnan(*v)) v = &RawValue::CANONICAL_FLOAT_NAN;
   return HashUtil::MurmurHash2_64(v, 4, seed);
 }
 
@@ -152,6 +176,7 @@ inline uint32_t RawValue::GetHashValueNonNull<double>(const double* v,
     const ColumnType& type, uint32_t seed) {
   DCHECK_EQ(type.type, TYPE_DOUBLE);
   DCHECK(v != NULL);
+  if (std::isnan(*v)) v = &RawValue::CANONICAL_DOUBLE_NAN;
   return HashUtil::MurmurHash2_64(v, 8, seed);
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/testdata/workloads/functional-query/queries/QueryTest/aggregation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/aggregation.test b/testdata/workloads/functional-query/queries/QueryTest/aggregation.test
index 7aba138..69372f8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/aggregation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/aggregation.test
@@ -1378,3 +1378,26 @@ having extract(hour from timestamp_col) = int_col
 TIMESTAMP,BIGINT
 ---- RESULTS
 ====
+---- QUERY
+# GROUP BY of NaN values aggregates NaN's as one grouping
+select count(*), sqrt(0.5-x) as Z
+from (VALUES((1.6 x, 2 y), (3.2, 4), (5.4,6))) T
+group by Z
+---- RESULTS
+3, NaN
+---- TYPES
+bigint, double
+====
+---- QUERY
+# GROUP BY of NaN values aggregates NaN's as one grouping
+select count(*), cast(sqrt(0.4-x) as FLOAT) as P, cast(sqrt(1.5-y) as FLOAT) as Q
+from (VALUES((1.6 x, 1.6 y, 0 z), (0.5, 0.5, 0), (5.4, 6, 0),
+              (0.5, 0.5, 0), (0.5, 0.5, 0), (-0.6, 0.5, 0))) T
+group by P, Q order by P, Q
+---- RESULTS
+2, NaN, NaN
+3, NaN, 1
+1, 1, 1
+---- TYPES
+bigint, float, float
+====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/testdata/workloads/functional-query/queries/QueryTest/exprs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/exprs.test b/testdata/workloads/functional-query/queries/QueryTest/exprs.test
index 9187b6a..8a25b85 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/exprs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/exprs.test
@@ -3071,3 +3071,13 @@ where cast(get_json_object(t.json, '$.a.b.c') as int) > 1
 ---- TYPES
 BIGINT
 ====
+---- QUERY
+# IMPALA-6661 - NaN should not evaluate as the same as any other NaN via <=>
+WITH W AS (SELECT T.*, CAST(SQRT(X) AS FLOAT)  P, CAST(SQRT(Y) AS FLOAT) Q
+FROM (VALUES((CAST(-1.0 AS FLOAT) X, CAST(-1.0 AS FLOAT) Y), (-1.0, 0), (0, -1.0), (0, 0))) T )
+SELECT * FROM W WHERE W.Q<=>W.P
+---- RESULTS
+0,0,0,0
+---- TYPES
+FLOAT, FLOAT, FLOAT, FLOAT
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/6684fe19/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 643c3b1..bbead6c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/joins.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/joins.test
@@ -795,3 +795,30 @@ where sqrt(-a.id) = b.float_col / b.double_col
 ---- TYPES
 INT
 ====
+---- QUERY
+# Test than 'nan' != 'nan' when joining.
+with t1 as (select cast(sqrt(0.5-x) as FLOAT) c1, y c2 from
+     (VALUES((1.6 x, 0 y), (3.2, 1), (5.4,2), (0.5, 3), (0.5, 4), (-0.5, 5))) XX),
+     t2 as (select * from t1)
+select t1.c1,t2.c1 from t1 right outer join t2 on t1.c1 = t2.c1;
+---- RESULTS
+0,0
+0,0
+0,0
+0,0
+1,1
+NULL,NaN
+NULL,NaN
+NULL,NaN
+---- TYPES
+FLOAT,FLOAT
+====
+---- QUERY
+# Test that NaN <=> NaN in joins
+select t1.float_col as v
+    from functional.alltypessmall t1, functional.alltypessmall t2
+    where sqrt(0.5-t1.float_col) <=> sqrt(0.5-t2.float_col) and t1.float_col > 0.5
+---- RESULTS
+---- TYPES
+FLOAT
+====


[04/33] impala git commit: IMPALA-7794: Rewrite flaky ownership authorization tests

Posted by bo...@apache.org.
IMPALA-7794: Rewrite flaky ownership authorization tests

This patch rewrites the ownership authorization tests to not depend on
delay and timeout, which can help to avoid the flakiness. The patch also
refactors some tests to reuse Sentry and Impala instances without having
to restart them, which can speed up the tests. To keep the same test
coverage, no tests were removed.

Testing:
- Ran all authorization E2E tests multiple times.

Change-Id: Ic98f8dbec41360261fd0339d835f3ce6b504ee29
Reviewed-on: http://gerrit.cloudera.org:8080/11837
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 76160838899fefc4955e30332a19732af6ecee6d
Parents: 6684fe1
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Tue Oct 30 19:48:54 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 tests/authorization/test_grant_revoke.py     | 247 +++++---------
 tests/authorization/test_owner_privileges.py | 396 ++++++++++------------
 tests/common/sentry_cache_test_suite.py      |  66 ++--
 3 files changed, 290 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/76160838/tests/authorization/test_grant_revoke.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_grant_revoke.py b/tests/authorization/test_grant_revoke.py
index 949887b..f7299bf 100644
--- a/tests/authorization/test_grant_revoke.py
+++ b/tests/authorization/test_grant_revoke.py
@@ -16,14 +16,9 @@
 # under the License.
 #
 # Client tests for SQL statement authorization
-# In these tests we run all the tests twice. Once with just using cache, hence the
-# long sentry poll, and once by ensuring refreshes happen from Sentry.
 
 import grp
 import pytest
-import os
-import sys
-import subprocess
 from getpass import getuser
 from os import getenv
 from time import sleep
@@ -36,16 +31,8 @@ from tests.verifiers.metric_verifier import MetricVerifier
 SENTRY_CONFIG_FILE = getenv('IMPALA_HOME') + \
     '/fe/src/test/resources/sentry-site.xml'
 
-# The polling frequency used by catalogd to refresh Sentry privileges.
-# The long polling is so we can check updates just for the cache.
-# The other polling is so we can get the updates without having to wait.
-SENTRY_POLLING_FREQUENCY_S = 1
-SENTRY_LONG_POLLING_FREQUENCY_S = 120
-# The timeout, in seconds, when waiting for a refresh of Sentry privileges.
-# This is also used as a delay before executing a statement. The difference between
-# the two usages is one is used to check results and once successful can return before
-# the time is up.  The other is to enforce a delay with no short circuit.
-SENTRY_REFRESH_TIMEOUT_S = SENTRY_POLLING_FREQUENCY_S * 3
+# Sentry long polling frequency to make Sentry refresh not run.
+SENTRY_LONG_POLLING_FREQUENCY_S = 3600
 
 
 class TestGrantRevoke(SentryCacheTestSuite):
@@ -118,155 +105,99 @@ class TestGrantRevoke(SentryCacheTestSuite):
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(
       impalad_args="--server_name=server1",
-      catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE +
-      " --sentry_catalog_polling_frequency_s=" + str(SENTRY_POLLING_FREQUENCY_S),
+      catalogd_args="--sentry_config={0} --sentry_catalog_polling_frequency_s={1}"
+                    .format(SENTRY_CONFIG_FILE, SENTRY_LONG_POLLING_FREQUENCY_S),
       sentry_config=SENTRY_CONFIG_FILE)
   def test_grant_revoke_with_sentry(self, vector):
-    self.__execute_with_grant_option_tests(TestObject(TestObject.SERVER), "all",
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.DATABASE,
-        "grant_rev_db"), "all", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.TABLE,
-        "grant_rev_db.tbl1"), "all", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.VIEW,
-        "grant_rev_db.tbl1"), "all", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.SERVER), "select",
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.DATABASE,
-        "grant_rev_db"), "select", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.TABLE,
-        "grant_rev_db.tbl1"), "select", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_with_grant_option_tests(TestObject(TestObject.VIEW,
-        "grant_rev_db.tbl1"), "select", sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-
-  @pytest.mark.execute_serially
-  @SentryCacheTestSuite.with_args(
-      impalad_args="--server_name=server1",
-      catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE +
-      " --sentry_catalog_polling_frequency_s=" + str(SENTRY_LONG_POLLING_FREQUENCY_S),
-      sentry_config=SENTRY_CONFIG_FILE)
-  def test_grant_revoke_with_sentry_long_poll(self, vector):
-    self.__execute_with_grant_option_tests(TestObject(TestObject.SERVER), "all")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.DATABASE,
-        "grant_rev_db"), "all")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.TABLE,
-        "grant_rev_db.tbl1"), "all")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.VIEW,
-        "grant_rev_db.tbl1"), "all")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.SERVER), "select")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.DATABASE,
-        "grant_rev_db"), "select")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.TABLE,
-        "grant_rev_db.tbl1"), "select")
-    self.__execute_with_grant_option_tests(TestObject(TestObject.VIEW,
-        "grant_rev_db.tbl1"), "select")
-
-  @pytest.mark.execute_serially
-  @SentryCacheTestSuite.with_args(
-    impalad_args="--server_name=server1",
-    catalogd_args="--sentry_config=%s " % SENTRY_CONFIG_FILE +
-                  "--sentry_catalog_polling_frequency_s=%d" % SENTRY_POLLING_FREQUENCY_S,
-    sentry_config=SENTRY_CONFIG_FILE)
-  def test_grant_revoke_with_sentry_refresh(self):
-    group_name = grp.getgrnam(getuser()).gr_name
-    role_name = "grant_revoke_test_sentry_refresh"
-    try:
-      self.client.execute("create role %s" % role_name)
-      self.client.execute("grant role %s to group `%s`" % (role_name, group_name))
-      # Add select and insert into the catalog.
-      self.client.execute("grant select on table functional.alltypes to %s" % role_name)
-      self.client.execute("grant insert on table functional.alltypes to %s" % role_name)
-      # Grant alter privilege and revoke insert privilege using Sentry CLI.
-      subprocess.check_call(
-        ["/bin/bash", "-c", "%s/bin/sentryShell "
-         "--conf %s/sentry-site.xml -gpr "
-         "-p 'server=server1->db=functional->table=alltypes->action=alter' "
-         "-r %s" % (os.getenv("SENTRY_HOME"), os.getenv("SENTRY_CONF_DIR"), role_name)],
-        stdout=sys.stdout, stderr=sys.stderr)
-      subprocess.check_call(
-          ["/bin/bash", "-c", "%s/bin/sentryShell "
-           "--conf %s/sentry-site.xml -rpr "
-           "-p 'server=server1->db=functional->table=alltypes->action=insert' "
-           "-r %s" % (os.getenv("SENTRY_HOME"), os.getenv("SENTRY_CONF_DIR"), role_name)],
-          stdout=sys.stdout, stderr=sys.stderr)
-      # Wait for the catalog update with Sentry refresh.
-      sleep(SENTRY_POLLING_FREQUENCY_S + 3)
-      # Ensure alter privilege was granted and insert privilege was revoked.
-      result = self.execute_query_expect_success(self.client, "show grant role %s" %
-                                                 role_name)
-      assert len(result.data) == 2
-      assert any('select' in x for x in result.data)
-      assert any('alter' in x for x in result.data)
-    finally:
-      self.client.execute("drop role %s" % role_name)
-
-  def __execute_with_grant_option_tests(self, test_obj, privilege,
-      sentry_refresh_timeout_s=0):
+    """Tests grant and revoke for all objects. In these tests, we run tests twice. One
+    with just using cache, hence the long sentry poll, and another one by ensuring
+    refreshes happen from Sentry."""
+    for invalidate in [True, False]:
+      for priv in ["all", "select"]:
+        self._execute_with_grant_option_tests(TestObject(TestObject.SERVER), priv,
+                                              invalidate)
+        self._execute_with_grant_option_tests(TestObject(TestObject.DATABASE,
+                                                         "grant_rev_db"), priv,
+                                              invalidate)
+        self._execute_with_grant_option_tests(TestObject(TestObject.TABLE,
+                                                         "grant_rev_db.tbl1"), priv,
+                                              invalidate)
+        self._execute_with_grant_option_tests(TestObject(TestObject.VIEW,
+                                                         "grant_rev_db.tbl1"), priv,
+                                              invalidate)
+
+  def _execute_with_grant_option_tests(self, test_obj, privilege, invalidate_metadata):
     """
-    Executes grant/revoke tests with grant option
+    Executes grant/revoke tests with grant option.
     """
-    # Do some initial setup only for this test.
-    group_name = grp.getgrnam(getuser()).gr_name
-    try:
-      self.client.execute("create role grant_revoke_test_admin")
-    except Exception:
-      # Ignore this as it was already created on the last run.
-      pass
-    self.client.execute("grant all on server to grant_revoke_test_admin")
-    self.client.execute("grant role grant_revoke_test_admin to group `%s`" % group_name)
-    self.client.execute("create role grant_revoke_test_role")
-    if test_obj.obj_type != TestObject.SERVER:
-      self.user_query(self.client, "create %s if not exists %s %s %s"
-          % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
-          test_obj.view_select), user="root")
-
-    # Grant a basic privilege
-    self.user_query(self.client, "grant %s on %s %s to role grant_revoke_test_role"
-        % (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
-
-    # Ensure role has privilege.
-    self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
-        test_obj, timeout_sec=sentry_refresh_timeout_s)
-
-    # Try with grant option on existing privilege.
-    test_obj.grant = True
-    self.user_query(self.client, "grant %s on %s %s " % (privilege, test_obj.grant_name,
-        test_obj.obj_name) + " to role grant_revoke_test_role with grant option",
-        user="root")
-    # Ensure role has updated privilege.
-    self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
-        test_obj, timeout_sec=sentry_refresh_timeout_s)
-
-    # Revoke the grant option
-    self.user_query(self.client,
-        "revoke grant option for %s on %s %s from role grant_revoke_test_role"
-        % (privilege, test_obj.grant_name, test_obj.obj_name))
-    # Ensure role has updated privilege.
-    test_obj.grant = False
-    self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
-        test_obj, delay_s=sentry_refresh_timeout_s)
-
-    # Add the grant option back, then add a regular privilege
-    self.user_query(self.client, "grant %s on %s %s to role grant_revoke_test_role"
-        % (privilege, test_obj.grant_name, test_obj.obj_name) + " with grant option",
-        user="root")
-    self.user_query(self.client, "grant %s on %s %s to role grant_revoke_test_role"
-        % (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
-    test_obj.grant = True
-    self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
-        test_obj, timeout_sec=sentry_refresh_timeout_s)
+    def setup():
+      group_name = grp.getgrnam(getuser()).gr_name
+      try:
+        self.client.execute("create role grant_revoke_test_admin")
+      except Exception:
+        # Ignore this as it was already created on the last run.
+        pass
+      self.client.execute("grant all on server to grant_revoke_test_admin")
+      self.client.execute("grant role grant_revoke_test_admin to group `%s`" % group_name)
+      self.client.execute("create role grant_revoke_test_role")
 
-    # Revoke the privilege
-    self.user_query(self.client, "revoke %s on %s %s from role grant_revoke_test_role"
-        % (privilege, test_obj.grant_name, test_obj.obj_name))
-    result = self.user_query(self.client, "show grant role grant_revoke_test_role",
-        delay_s=sentry_refresh_timeout_s)
-    assert len(result.data) == 0
+    def cleanup():
+      # Reset the grant value
+      test_obj.grant = False
+      # Remove the role
+      self.client.execute("drop role grant_revoke_test_role")
 
-    # Reset the grant value
-    test_obj.grant = False
-    # Remove the role
-    self.client.execute("drop role grant_revoke_test_role")
+    try:
+      setup()
+      if test_obj.obj_type != TestObject.SERVER:
+        self.user_query(self.client, "create %s if not exists %s %s %s" %
+                        (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                         test_obj.view_select), user="root")
+
+      # Grant a basic privilege
+      self.user_query(self.client, "grant %s on %s %s to role grant_revoke_test_role" %
+                      (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
+
+      # Ensure role has privilege.
+      self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
+                               test_obj, invalidate_metadata=invalidate_metadata)
+
+      # Try with grant option on existing privilege.
+      test_obj.grant = True
+      self.user_query(self.client,
+                      "grant %s on %s %s to role grant_revoke_test_role with grant option"
+                      % (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
+      # Ensure role has updated privilege.
+      self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
+                               test_obj, invalidate_metadata=invalidate_metadata)
+
+      # Revoke the grant option
+      self.user_query(self.client, "revoke grant option for %s on %s %s from role "
+                                   "grant_revoke_test_role" % (privilege,
+                                                               test_obj.grant_name,
+                                                               test_obj.obj_name))
+      # Ensure role has updated privilege.
+      test_obj.grant = False
+      self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
+                               test_obj, invalidate_metadata=invalidate_metadata)
+
+      # Add the grant option back, then add a regular privilege
+      self.user_query(self.client,
+                      "grant %s on %s %s to role grant_revoke_test_role with grant option"
+                      % (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
+      self.user_query(self.client, "grant %s on %s %s to role grant_revoke_test_role" %
+                      (privilege, test_obj.grant_name, test_obj.obj_name), user="root")
+      test_obj.grant = True
+      self.validate_privileges(self.client, "show grant role grant_revoke_test_role",
+                               test_obj, invalidate_metadata=invalidate_metadata)
+
+      # Revoke the privilege
+      self.user_query(self.client, "revoke %s on %s %s from role grant_revoke_test_role" %
+                      (privilege, test_obj.grant_name, test_obj.obj_name))
+      result = self.user_query(self.client, "show grant role grant_revoke_test_role")
+      assert len(result.data) == 0
+    finally:
+      cleanup()
 
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(

http://git-wip-us.apache.org/repos/asf/impala/blob/76160838/tests/authorization/test_owner_privileges.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_owner_privileges.py b/tests/authorization/test_owner_privileges.py
index 1cbce98..26e7a97 100644
--- a/tests/authorization/test_owner_privileges.py
+++ b/tests/authorization/test_owner_privileges.py
@@ -15,29 +15,18 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-# Client tests to ensure object ownership functionality
-#
-# The tests uses two clients, one for oo_user1 and one for 'oo_user2'. oo_user2 is
-# used as a user to transfer ownership to. Additionally, in these tests we run all
-# the tests twice. Once with just using cache, hence the long sentry poll, and once
-# by ensuring refreshes happen from Sentry.
+# Client tests to ensure object ownership functionality.
 
 import grp
 import pytest
 from getpass import getuser
 from os import getenv
-from time import sleep, time
 
 from tests.common.sentry_cache_test_suite import SentryCacheTestSuite, TestObject
 from tests.common.test_dimensions import create_uncompressed_text_dimension
 
-# The polling frequency used by catalogd to refresh Sentry privileges.
-# The long polling is so we can check updates just for the cache.
-# The other polling is so we can get the updates without having to wait.
-SENTRY_LONG_POLLING_FREQUENCY_S = 60
-SENTRY_POLLING_FREQUENCY_S = 1
-# The timeout, in seconds, when waiting for a refresh of Sentry privileges.
-SENTRY_REFRESH_TIMEOUT_S = SENTRY_POLLING_FREQUENCY_S * 2
+# Sentry long polling frequency to make Sentry refresh not run.
+SENTRY_LONG_POLLING_FREQUENCY_S = 3600
 
 SENTRY_CONFIG_DIR = getenv('IMPALA_HOME') + '/fe/src/test/resources/'
 SENTRY_BASE_LOG_DIR = getenv('IMPALA_CLUSTER_LOGS_DIR') + "/sentry"
@@ -58,7 +47,14 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
 
   def setup_method(self, method):
     super(TestOwnerPrivileges, self).setup_method(method)
-    self._test_cleanup()
+    self._setup_admin()
+
+  def teardown_method(self, method):
+    self._cleanup_admin()
+    super(TestOwnerPrivileges, self).teardown_method(method)
+
+  def _setup_ownership_test(self):
+    self._cleanup_ownership_test()
     # Base roles for enabling tests.
     self.execute_query("create role owner_priv_test_oo_user1")
     # Role for verifying grant.
@@ -70,10 +66,18 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
     self.execute_query("grant create on server to owner_priv_test_oo_user1")
     self.execute_query("grant select on database functional to owner_priv_test_oo_user1")
 
-  def teardown_method(self, method):
-    self._test_cleanup()
-    self.execute_query("drop role owner_priv_admin")
-    super(TestOwnerPrivileges, self).teardown_method(method)
+  def _cleanup_ownership_test(self):
+    # Clean up the test artifacts.
+    try:
+      self.cleanup_db("owner_priv_db", sync_ddl=0)
+    except Exception:
+      # Ignore this if we can't show tables.
+      pass
+
+    # Clean up any old roles created by this test
+    for role_name in self.execute_query("show roles").data:
+      if "owner_priv_test" in role_name:
+        self.execute_query("drop role %s" % role_name)
 
   @staticmethod
   def count_user_privileges(result):
@@ -90,11 +94,12 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
         total += 1
     return total
 
-  def _validate_user_privilege_count(self, client, query, user, delay_s, count):
-    result = self.user_query(client, query, user=user, delay_s=delay_s)
-    return self.count_user_privileges(result) == count
+  def _validate_no_user_privileges(self, client, user, invalidate_metadata):
+    if invalidate_metadata: self.execute_query("invalidate metadata")
+    result = self.user_query(client, "show grant user %s" % user, user=user)
+    return TestOwnerPrivileges.count_user_privileges(result) == 0
 
-  def _test_cleanup(self):
+  def _setup_admin(self):
     # Admin for manipulation and cleaning up.
     try:
       self.execute_query("drop role owner_priv_admin")
@@ -105,38 +110,9 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
     self.execute_query("grant all on server to owner_priv_admin with grant option")
     group_name = grp.getgrnam(getuser()).gr_name
     self.execute_query("grant role owner_priv_admin to group `%s`" % group_name)
-    # Clean up the test artifacts.
-    try:
-      self.cleanup_db("owner_priv_db", sync_ddl=0)
-    except Exception:
-      # Ignore this if we can't show tables.
-      pass
-
-    # Clean up any old roles created by this test
-    for role_name in self.execute_query("show roles").data:
-      if "owner_priv_test" in role_name:
-        self.execute_query("drop role %s" % role_name)
 
-  @pytest.mark.execute_serially
-  @SentryCacheTestSuite.with_args(
-      impalad_args="--server_name=server1 --sentry_config={0} "
-                   "--authorization_policy_provider_class="
-                   "org.apache.impala.service.CustomClusterResourceAuthorizationProvider "
-                   .format(SENTRY_CONFIG_FILE_OO),
-      catalogd_args="--sentry_config={0} --authorization_policy_provider_class="
-                    "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                    " --sentry_catalog_polling_frequency_s={1}"
-                    .format(SENTRY_CONFIG_FILE_OO, str(SENTRY_LONG_POLLING_FREQUENCY_S)),
-      sentry_config=SENTRY_CONFIG_FILE_OO,
-      sentry_log_dir="{0}/test_owner_privileges_with_grant_log_poll"
-                     .format(SENTRY_BASE_LOG_DIR))
-  def test_owner_privileges_with_grant_long_poll(self, vector, unique_database):
-    self.__execute_owner_privilege_tests(TestObject(TestObject.DATABASE, "owner_priv_db",
-        grant=True))
-    self.__execute_owner_privilege_tests(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl", grant=True))
-    self.__execute_owner_privilege_tests(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view", grant=True))
+  def _cleanup_admin(self):
+    self.execute_query("drop role owner_priv_admin")
 
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(
@@ -147,98 +123,118 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
       catalogd_args="--sentry_config={0} --sentry_catalog_polling_frequency_s={1} "
                     "--authorization_policy_provider_class="
                     "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                    .format(SENTRY_CONFIG_FILE_OO, str(SENTRY_POLLING_FREQUENCY_S)),
+                    .format(SENTRY_CONFIG_FILE_OO, SENTRY_LONG_POLLING_FREQUENCY_S),
       sentry_config=SENTRY_CONFIG_FILE_OO,
-      sentry_log_dir="{0}/test_owner_privileges_with_grant"
-                     .format(SENTRY_BASE_LOG_DIR))
+      sentry_log_dir="{0}/test_owner_privileges_with_grant".format(SENTRY_BASE_LOG_DIR))
   def test_owner_privileges_with_grant(self, vector, unique_database):
-    self.__execute_owner_privilege_tests(TestObject(TestObject.DATABASE, "owner_priv_db",
-        grant=True), sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl", grant=True),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view", grant=True),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-
-  def __execute_owner_privilege_tests(self, test_obj, sentry_refresh_timeout_s=0):
+    """Tests owner privileges with grant on database, table, and view.
+    - invalidate_metadata=True: With Sentry refresh to make sure privileges are really
+                                stored in Sentry.
+    - invalidate_metadata=False: No Sentry refresh to make sure user can use owner
+                                 privileges right away without a Sentry refresh."""
+    for invalidate in [True, False]:
+      try:
+        self._setup_ownership_test()
+        self._execute_owner_privilege_tests(TestObject(TestObject.DATABASE,
+                                                       "owner_priv_db",
+                                                       grant=True),
+                                            invalidate_metadata=invalidate)
+        self._execute_owner_privilege_tests(TestObject(TestObject.TABLE,
+                                                       unique_database +
+                                                       ".owner_priv_tbl",
+                                                       grant=True),
+                                            invalidate_metadata=invalidate)
+        self._execute_owner_privilege_tests(TestObject(TestObject.VIEW,
+                                                       unique_database +
+                                                       ".owner_priv_view",
+                                                       grant=True),
+                                            invalidate_metadata=invalidate)
+      finally:
+        self._cleanup_ownership_test()
+
+  def _execute_owner_privilege_tests(self, test_obj, invalidate_metadata):
     """
     Executes all the statements required to validate owner privileges work correctly
     for a specific database, table, or view.
     """
     # Create object and ensure oo_user1 gets owner privileges.
     self.oo_user1_impalad_client = self.create_impala_client()
+    # oo_user2 is only used for transferring ownership.
     self.oo_user2_impalad_client = self.create_impala_client()
-    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s"
-        % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
-        test_obj.view_select), user="oo_user1")
+    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                     test_obj.view_select), user="oo_user1")
     self.validate_privileges(self.oo_user1_impalad_client, "show grant user oo_user1",
-        test_obj, sentry_refresh_timeout_s, "oo_user1")
+                             test_obj, user="oo_user1",
+                             invalidate_metadata=invalidate_metadata)
 
     # Ensure grant works.
     self.user_query(self.oo_user1_impalad_client,
-        "grant all on %s %s to role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1")
+                    "grant all on %s %s to role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1")
     self.user_query(self.oo_user1_impalad_client,
-        "revoke all on %s %s from role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1")
+                    "revoke all on %s %s from role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1")
 
     # Change the database owner and ensure oo_user1 does not have owner privileges.
-    # Use a delay to avoid cache consistency issue that could occur after create.
-    self.user_query(self.oo_user1_impalad_client, "alter %s %s set owner user oo_user2"
-        % (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
-        delay_s=sentry_refresh_timeout_s)
-    assert self._validate_user_privilege_count(self.oo_user1_impalad_client,
-        "show grant user oo_user1", "oo_user1", sentry_refresh_timeout_s, 0)
+    self.user_query(self.oo_user1_impalad_client, "alter %s %s set owner user oo_user2" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1")
+    assert self._validate_no_user_privileges(self.oo_user1_impalad_client,
+                                               user="oo_user1",
+                                               invalidate_metadata=invalidate_metadata)
 
     # Ensure oo_user1 cannot drop database after owner change.
     # Use a delay to avoid cache consistency issue that could occur after alter.
-    self.user_query(self.oo_user1_impalad_client, "drop %s %s" % (test_obj.obj_type,
-        test_obj.obj_name), user="oo_user1", delay_s=sentry_refresh_timeout_s,
-        error_msg="does not have privileges to execute 'DROP'")
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute 'DROP'")
 
     # oo_user2 should have privileges for object now.
     self.validate_privileges(self.oo_user2_impalad_client, "show grant user oo_user2",
-        test_obj, sentry_refresh_timeout_s, "oo_user2")
+                             test_obj, user="oo_user2",
+                             invalidate_metadata=invalidate_metadata)
 
     # Change the owner to a role and ensure oo_user2 doesn't have privileges.
     # Set the owner back to oo_user1 since for views, oo_user2 doesn't have select
     # privileges on the underlying table.
-    self.execute_query("alter %s %s set owner user oo_user1" % (test_obj.obj_type,
-        test_obj.obj_name))
-    assert self._validate_user_privilege_count(self.oo_user2_impalad_client,
-        "show grant user oo_user2", "oo_user2", sentry_refresh_timeout_s, 0)
-    # Use a delay to avoid cache consistency issue that could occur after alter.
+    self.execute_query("alter %s %s set owner user oo_user1" %
+                       (test_obj.obj_type, test_obj.obj_name),
+                       query_options={"sync_ddl": 1})
+    assert self._validate_no_user_privileges(self.oo_user2_impalad_client,
+                                               user="oo_user2",
+                                               invalidate_metadata=invalidate_metadata)
     self.user_query(self.oo_user1_impalad_client,
-        "alter %s %s set owner role owner_priv_test_owner_role"
-        % (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
-        delay_s=sentry_refresh_timeout_s)
+                    "alter %s %s set owner role owner_priv_test_owner_role" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1")
     # Ensure oo_user1 does not have user privileges.
-    assert self._validate_user_privilege_count(self.oo_user1_impalad_client,
-        "show grant user oo_user1", "oo_user1", sentry_refresh_timeout_s, 0)
+    assert self._validate_no_user_privileges(self.oo_user1_impalad_client,
+                                               user="oo_user1",
+                                               invalidate_metadata=invalidate_metadata)
 
     # Ensure role has owner privileges.
     self.validate_privileges(self.oo_user1_impalad_client,
-        "show grant role owner_priv_test_owner_role", test_obj, sentry_refresh_timeout_s,
-        "oo_user1")
+                             "show grant role owner_priv_test_owner_role", test_obj,
+                             user="oo_user1", invalidate_metadata=invalidate_metadata)
 
     # Drop the object and ensure no role privileges.
     # Use a delay to avoid cache consistency issue that could occur after alter.
-    self.user_query(self.oo_user1_impalad_client, "drop %s %s " % (test_obj.obj_type,
-        test_obj.obj_name), user="oo_user1", delay_s=sentry_refresh_timeout_s)
-    assert self._validate_user_privilege_count(self.oo_user1_impalad_client,
-        "show grant user oo_user1", "oo_user1", sentry_refresh_timeout_s, 0)
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s " %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1")
+    assert self._validate_no_user_privileges(self.oo_user1_impalad_client,
+                                             user="oo_user1",
+                                             invalidate_metadata=invalidate_metadata)
 
     # Ensure user privileges are gone after drop.
     # Use a delay to avoid cache consistency issue that could occur after drop.
-    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s"
-        % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
-        test_obj.view_select), user="oo_user1", delay_s=sentry_refresh_timeout_s)
+    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                     test_obj.view_select), user="oo_user1")
     # Use a delay to avoid cache consistency issue that could occur after create.
-    self.user_query(self.oo_user1_impalad_client, "drop %s %s " % (test_obj.obj_type,
-        test_obj.obj_name), user="oo_user1", delay_s=sentry_refresh_timeout_s)
-    assert self._validate_user_privilege_count(self.oo_user1_impalad_client,
-        "show grant user oo_user1", "oo_user1", sentry_refresh_timeout_s, 0)
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s " %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1")
+    assert self._validate_no_user_privileges(self.oo_user1_impalad_client,
+                                             user="oo_user1",
+                                             invalidate_metadata=invalidate_metadata)
 
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(
@@ -248,44 +244,25 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
                    .format(SENTRY_CONFIG_FILE_NO_OO),
       catalogd_args="--sentry_config={0} --authorization_policy_provider_class="
                     "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                    " --sentry_catalog_polling_frequency_s={1}"
-                    .format(SENTRY_CONFIG_FILE_NO_OO,
-                    str(SENTRY_LONG_POLLING_FREQUENCY_S)),
-      sentry_config=SENTRY_CONFIG_FILE_NO_OO,
-      sentry_log_dir="{0}/test_owner_privileges_disabled_log_poll"
-                     .format(SENTRY_BASE_LOG_DIR))
-  def test_owner_privileges_disabled_long_poll(self, vector, unique_database):
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.DATABASE,
-        "owner_priv_db"))
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl"))
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view"))
-
-  @pytest.mark.execute_serially
-  @SentryCacheTestSuite.with_args(
-      impalad_args="--server_name=server1 --sentry_config={0} "
-                   "--authorization_policy_provider_class="
-                   "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                   .format(SENTRY_CONFIG_FILE_NO_OO),
-      catalogd_args="--sentry_config={0} --sentry_catalog_polling_frequency_s={1} "
-                    "--authorization_policy_provider_class="
-                    "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                    .format(SENTRY_CONFIG_FILE_NO_OO, str(SENTRY_POLLING_FREQUENCY_S)),
+                    .format(SENTRY_CONFIG_FILE_NO_OO),
       sentry_config=SENTRY_CONFIG_FILE_NO_OO,
-      sentry_log_dir="{0}/test_owner_privileges_disabled"
-                     .format(SENTRY_BASE_LOG_DIR))
+      sentry_log_dir="{0}/test_owner_privileges_disabled".format(SENTRY_BASE_LOG_DIR))
   def test_owner_privileges_disabled(self, vector, unique_database):
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.DATABASE,
-        "owner_priv_db"), sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl"),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests_no_oo(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view"),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-
-  def __execute_owner_privilege_tests_no_oo(self, test_obj, sentry_refresh_timeout_s=0):
+    """Tests that there should not be owner privileges."""
+    try:
+      self._setup_ownership_test()
+      self._execute_owner_privilege_tests_no_oo(TestObject(TestObject.DATABASE,
+                                                           "owner_priv_db"))
+      self._execute_owner_privilege_tests_no_oo(TestObject(TestObject.TABLE,
+                                                           unique_database +
+                                                           ".owner_priv_tbl"))
+      self._execute_owner_privilege_tests_no_oo(TestObject(TestObject.VIEW,
+                                                           unique_database +
+                                                           ".owner_priv_view"))
+    finally:
+      self._cleanup_ownership_test()
+
+  def _execute_owner_privilege_tests_no_oo(self, test_obj):
     """
     Executes all the statements required to validate owner privileges work correctly
     for a specific database, table, or view.
@@ -293,53 +270,30 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
     # Create object and ensure oo_user1 gets owner privileges.
     self.oo_user1_impalad_client = self.create_impala_client()
     self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s"
-        % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
-        test_obj.view_select), user="oo_user1")
+                    % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                       test_obj.view_select), user="oo_user1")
 
     # Ensure grant doesn't work.
     self.user_query(self.oo_user1_impalad_client,
-        "grant all on %s %s to role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges to execute: GRANT_PRIVILEGE")
+                    "grant all on %s %s to role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute: GRANT_PRIVILEGE")
 
     self.user_query(self.oo_user1_impalad_client,
-        "revoke all on %s %s from role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges to execute: REVOKE_PRIVILEGE")
+                    "revoke all on %s %s from role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute: REVOKE_PRIVILEGE")
 
     # Ensure changing the database owner doesn't work.
-    self.user_query(self.oo_user1_impalad_client, "alter %s %s set owner user oo_user2"
-        % (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges with 'GRANT OPTION'")
+    self.user_query(self.oo_user1_impalad_client,
+                    "alter %s %s set owner user oo_user2" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges with 'GRANT OPTION'")
 
     # Ensure oo_user1 cannot drop database.
-    # Use a delay to avoid cache consistency issue that could occur after alter.
-    self.user_query(self.oo_user1_impalad_client, "drop %s %s" % (test_obj.obj_type,
-        test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges to execute 'DROP'",
-        delay_s=sentry_refresh_timeout_s)
-
-  @pytest.mark.execute_serially
-  @SentryCacheTestSuite.with_args(
-      impalad_args="--server_name=server1 --sentry_config={0} "
-                   "--authorization_policy_provider_class="
-                   "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                   .format(SENTRY_CONFIG_FILE_OO_NOGRANT),
-      catalogd_args="--sentry_config={0} --authorization_policy_provider_class="
-                    "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                    " --sentry_catalog_polling_frequency_s={1}"
-                    .format(SENTRY_CONFIG_FILE_OO_NOGRANT,
-                    str(SENTRY_LONG_POLLING_FREQUENCY_S)),
-      sentry_config=SENTRY_CONFIG_FILE_OO_NOGRANT,
-      sentry_log_dir="{0}/test_owner_privileges_without_grant_log_poll"
-                     .format(SENTRY_BASE_LOG_DIR))
-  def test_owner_privileges_without_grant_long_poll(self, vector, unique_database):
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.DATABASE,
-        "owner_priv_db"))
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl"))
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view"))
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute 'DROP'")
 
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(
@@ -351,56 +305,68 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
                     "--authorization_policy_provider_class="
                     "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
                     .format(SENTRY_CONFIG_FILE_OO_NOGRANT,
-                    str(SENTRY_POLLING_FREQUENCY_S)),
+                            SENTRY_LONG_POLLING_FREQUENCY_S),
       sentry_config=SENTRY_CONFIG_FILE_OO_NOGRANT,
       sentry_log_dir="{0}/test_owner_privileges_without_grant"
                      .format(SENTRY_BASE_LOG_DIR))
   def test_owner_privileges_without_grant(self, vector, unique_database):
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.DATABASE,
-        "owner_priv_db"), sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.TABLE,
-        unique_database + ".owner_priv_tbl"),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-    self.__execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.VIEW,
-        unique_database + ".owner_priv_view"),
-        sentry_refresh_timeout_s=SENTRY_REFRESH_TIMEOUT_S)
-
-  def __execute_owner_privilege_tests_oo_nogrant(self, test_obj,
-      sentry_refresh_timeout_s=0):
+    """Tests owner privileges without grant on database, table, and view.
+    - invalidate_metadata=True: With Sentry refresh to make sure privileges are really
+                                stored in Sentry.
+    - invalidate_metadata=False: No Sentry refresh to make sure user can use owner
+                                 privileges right away without a Sentry refresh."""
+    for invalidate in [True, False]:
+      try:
+        self._setup_ownership_test()
+        self._execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.DATABASE,
+                                                                  "owner_priv_db"),
+                                                       invalidate_metadata=invalidate)
+        self._execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.TABLE,
+                                                                  unique_database +
+                                                                  ".owner_priv_tbl"),
+                                                       invalidate_metadata=invalidate)
+        self._execute_owner_privilege_tests_oo_nogrant(TestObject(TestObject.VIEW,
+                                                                  unique_database +
+                                                                  ".owner_priv_view"),
+                                                       invalidate_metadata=invalidate)
+      finally:
+        self._cleanup_ownership_test()
+
+  def _execute_owner_privilege_tests_oo_nogrant(self, test_obj, invalidate_metadata):
     """
     Executes all the statements required to validate owner privileges work correctly
     for a specific database, table, or view.
     """
     # Create object and ensure oo_user1 gets owner privileges.
     self.oo_user1_impalad_client = self.create_impala_client()
-    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s"
-        % (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
-        test_obj.view_select), user="oo_user1")
+    self.user_query(self.oo_user1_impalad_client, "create %s if not exists %s %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                     test_obj.view_select), user="oo_user1")
     self.validate_privileges(self.oo_user1_impalad_client, "show grant user oo_user1",
-        test_obj, sentry_refresh_timeout_s, "oo_user1")
+                             test_obj, user="oo_user1",
+                             invalidate_metadata=invalidate_metadata)
 
     # Ensure grant doesn't work.
     self.user_query(self.oo_user1_impalad_client,
-        "grant all on %s %s to role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges to execute: GRANT_PRIVILEGE")
+                    "grant all on %s %s to role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute: GRANT_PRIVILEGE")
 
     self.user_query(self.oo_user1_impalad_client,
-        "revoke all on %s %s from role owner_priv_test_all_role"
-        % (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
-        error_msg="does not have privileges to execute: REVOKE_PRIVILEGE")
+                    "revoke all on %s %s from role owner_priv_test_all_role" %
+                    (test_obj.grant_name, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges to execute: REVOKE_PRIVILEGE")
 
-    # Use a delay to avoid cache consistency issue that could occur after create.
-    self.user_query(self.oo_user1_impalad_client, "alter %s %s set owner user oo_user2"
-        % (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
-        delay_s=sentry_refresh_timeout_s,
-        error_msg="does not have privileges with 'GRANT OPTION'")
+    self.user_query(self.oo_user1_impalad_client, "alter %s %s set owner user oo_user2" %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1",
+                    error_msg="does not have privileges with 'GRANT OPTION'")
 
     # Use a delay to avoid cache consistency issue that could occur after alter.
-    self.user_query(self.oo_user1_impalad_client, "drop %s %s " % (test_obj.obj_type,
-        test_obj.obj_name), user="oo_user1", delay_s=sentry_refresh_timeout_s)
-    assert self._validate_user_privilege_count(self.oo_user1_impalad_client,
-        "show grant user oo_user1", "oo_user1", sentry_refresh_timeout_s, 0)
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s " %
+                    (test_obj.obj_type, test_obj.obj_name), user="oo_user1")
+    assert self._validate_no_user_privileges(self.oo_user1_impalad_client,
+                                             user="oo_user1",
+                                             invalidate_metadata=invalidate_metadata)
 
   @pytest.mark.execute_serially
   @SentryCacheTestSuite.with_args(
@@ -408,10 +374,10 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
                  "--authorization_policy_provider_class="
                  "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
                  .format(SENTRY_CONFIG_FILE_OO),
-    catalogd_args="--sentry_config={0} --sentry_catalog_polling_frequency_s={1} "
+    catalogd_args="--sentry_config={0} "
                   "--authorization_policy_provider_class="
                   "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
-                  .format(SENTRY_CONFIG_FILE_OO, str(SENTRY_POLLING_FREQUENCY_S)),
+                  .format(SENTRY_CONFIG_FILE_OO),
     sentry_config=SENTRY_CONFIG_FILE_OO,
     sentry_log_dir="{0}/test_owner_privileges_different_cases"
                    .format(SENTRY_BASE_LOG_DIR))

http://git-wip-us.apache.org/repos/asf/impala/blob/76160838/tests/common/sentry_cache_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/sentry_cache_test_suite.py b/tests/common/sentry_cache_test_suite.py
index 1a8aa24..4aca3be 100644
--- a/tests/common/sentry_cache_test_suite.py
+++ b/tests/common/sentry_cache_test_suite.py
@@ -17,20 +17,7 @@
 #
 # Base class for test that need to run with both just privilege
 # cache as well as Sentry privilege refresh.
-# There are two variables (delay_s timeout) used in these methods.
-# The first is the timeout when validating privileges. This is
-# needed to ensure that the privileges returned have been updated
-# from Sentry. The second is the delay_s before executing a query.
-# This is needed to ensure Sentry has been updated before running
-# the query. The reason for both is because the timeout can
-# be short circuited upon successful results. Using the delay_s
-# for every query and test would add significant time. As an
-# example, if a revoke is called, the expectation is the privilege
-# would not be in the result. If the cache is updated correctly,
-# but Sentry was not, using the timeout check, we would miss that
-# Sentry was not updated correctly.
 
-from time import sleep, time
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 
 
@@ -40,12 +27,10 @@ class SentryCacheTestSuite(CustomClusterTestSuite):
     return val.lower() == 'true'
 
   @staticmethod
-  def __check_privileges(result, test_obj, null_create_date=True, show_user=False):
+  def check_privileges(result, test_obj, null_create_date, show_user):
     """
-    This method validates privileges. Most validations are assertions, but for
-    null_create_date, we just return False to indicate the privilege cannot
-    be validated because it has not been refreshed from Sentry yet. If the query is
-    for "show grant user" then we need to account for the two extra columns.
+    This method validates privileges. If the query is for "show grant user" then we need
+    to account for the two extra columns.
     """
     # results should have the following columns for offset 0
     # scope, database, table, column, uri, privilege, grant_option, create_time
@@ -69,41 +54,30 @@ class SentryCacheTestSuite(CustomClusterTestSuite):
         return False
     return True
 
-  def validate_privileges(self, client, query, test_obj, timeout_sec=None, user=None,
-      delay_s=0):
-    """Validate privileges. If timeout_sec is > 0 then retry until create_date is not null
-    or the timeout_sec is reached. If delay_s is > 0 then wait that long before running.
+  def validate_privileges(self, client, query, test_obj, user=None,
+                          invalidate_metadata=False):
+    """Validate privileges. When invalidate_metadata is set to True, this function
+    will call "invalidate metadata" to ensure the privileges get refreshed from Sentry.
     """
     show_user = True if 'show grant user' in query else False
-    if delay_s > 0:
-      sleep(delay_s)
-    if timeout_sec is None or timeout_sec <= 0:
-      self.__check_privileges(self.execute_query_expect_success(client, query,
-          user=user), test_obj, show_user=show_user)
-    else:
-      start_time = time()
-      while time() - start_time < timeout_sec:
-        result = self.execute_query_expect_success(client, query, user=user)
-        success = self.__check_privileges(result, test_obj, null_create_date=False,
-            show_user=show_user)
-        if success:
-          return True
-        sleep(1)
-      return False
+    if invalidate_metadata: self.execute_query('invalidate metadata')
+    result = self.execute_query_expect_success(client, query, user=user)
+    return SentryCacheTestSuite.check_privileges(result, test_obj,
+                                                 null_create_date=not invalidate_metadata,
+                                                 show_user=show_user)
 
-  def user_query(self, client, query, user=None, delay_s=0, error_msg=None):
+  def user_query(self, client, query, user=None, error_msg=None):
     """
-    Executes a query with the root user client. If delay_s is > 0 then wait before
-    running the query. This is used to wait for Sentry refresh. If error_msg is
-    set, then expect a failure. Returns None when there is no error_msg.
+    Executes a query with the specified user client. If error_msg is set, then expect a
+    failure. Returns None when there is no error_msg.
     """
-    if delay_s > 0:
-      sleep(delay_s)
     if error_msg is not None:
-      e = self.execute_query_expect_failure(client, query, user=user)
-      self.verify_exceptions(error_msg, str(e))
+      e = self.execute_query_expect_failure(client, query, query_options={"sync_ddl": 1},
+                                            user=user)
+      SentryCacheTestSuite.verify_exceptions(error_msg, str(e))
       return None
-    return self.execute_query_expect_success(client, query, user=user)
+    return self.execute_query_expect_success(client, query, query_options={"sync_ddl": 1},
+                                             user=user)
 
   @staticmethod
   def verify_exceptions(expected_str, actual_str):


[18/33] impala git commit: [DOCS] A typo fixed in impala_live_summary

Posted by bo...@apache.org.
[DOCS] A typo fixed in impala_live_summary

Change-Id: Idb954199955d3af4271b5dc61c284e56536fe649
Reviewed-on: http://gerrit.cloudera.org:8080/11904
Reviewed-by: Alex Rodoni <ar...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: ac2b7c999a4cea9bf9d2b2467f8c7f8d4629fc03
Parents: cb31202
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Wed Nov 7 13:30:17 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 docs/topics/impala_live_summary.xml | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/ac2b7c99/docs/topics/impala_live_summary.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_live_summary.xml b/docs/topics/impala_live_summary.xml
index 10ecae3..087fce3 100644
--- a/docs/topics/impala_live_summary.xml
+++ b/docs/topics/impala_live_summary.xml
@@ -46,8 +46,6 @@ under the License.
     <p>Starting in <keyword keyref="impala31"/>, the summary output also
       includes the queuing status consisting of whether the query was queued and
       what was the latest queuing reason.</p>
-    <p>the queuing status, whether the query was queued and what was the latest
-      queuing reason.</p>
     <p><b>Type:</b>
       <codeph>Boolean</codeph></p>
     <p><b>Default:</b>


[27/33] impala git commit: IMPALA-7818: Standardize use of Expr predicates

Posted by bo...@apache.org.
IMPALA-7818: Standardize use of Expr predicates

The Expr node provide two kinds of queries about node categories:
predicates and isMumble() functions. This change standardizes two
existing methods to use predicates instead.

First, the existing isLiteral() method is replaced by a new
IS_LITERAL predicate.

The key purpose is to clean up the confusing use of the existing
isNullLiteral() method, which actually is a check for either a null
literal or a cast of a null. To make this clearer, replaced this
method with a new IS_NULL_VALUE() predicate.

Added a new IS_NULL_LITERAL predicate for the case when a node must be
exactly the NULL literal, not a cast to NULL. Replaced instances of
foo instanceof NullLiteral with a use of the new IS_NULL_LITERAL
predicate. (This revealed bugs which will be addressed separately.)

Added an IS_NON_NULL_LITERAL to replace the two-method idiom used in
several places.

Tests: No functional change. Reran all tests to ensure nothing changed.

Change-Id: I09a089c0f2484246e5c6444b78990fa9260c036a
Reviewed-on: http://gerrit.cloudera.org:8080/11887
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 8dfbe3c82d4acfae60d58636af041ac13708ef1e
Parents: ccabc49
Author: Paul Rogers <pr...@cloudera.com>
Authored: Tue Nov 6 10:34:30 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:49 2018 +0100

----------------------------------------------------------------------
 fe/src/main/cup/sql-parser.cup                  |  2 +-
 .../org/apache/impala/analysis/Analyzer.java    |  2 +-
 .../org/apache/impala/analysis/CaseExpr.java    |  7 +-
 .../org/apache/impala/analysis/CastExpr.java    |  2 +-
 .../org/apache/impala/analysis/ColumnDef.java   |  2 +-
 .../java/org/apache/impala/analysis/Expr.java   | 90 ++++++++++++++------
 .../impala/analysis/FunctionCallExpr.java       |  2 +-
 .../apache/impala/analysis/LikePredicate.java   |  4 +-
 .../org/apache/impala/analysis/LiteralExpr.java |  6 +-
 .../impala/analysis/PartitionKeyValue.java      |  5 +-
 .../apache/impala/analysis/PartitionSet.java    |  2 +-
 .../apache/impala/analysis/PartitionSpec.java   |  2 +-
 .../apache/impala/analysis/RangePartition.java  |  2 +-
 .../org/apache/impala/analysis/SelectStmt.java  |  2 +-
 .../impala/analysis/TupleIsNullPredicate.java   |  2 +-
 .../apache/impala/catalog/FeCatalogUtils.java   |  2 +-
 .../org/apache/impala/catalog/FeFsTable.java    |  3 +-
 .../org/apache/impala/catalog/HdfsTable.java    |  5 +-
 .../impala/catalog/local/LocalFsTable.java      |  4 +-
 .../impala/planner/HdfsPartitionPruner.java     | 14 +--
 .../org/apache/impala/planner/HdfsScanNode.java |  6 +-
 .../org/apache/impala/planner/KuduScanNode.java |  6 +-
 .../impala/rewrite/FoldConstantsRule.java       |  6 +-
 .../impala/rewrite/NormalizeCountStarRule.java  |  4 +-
 .../rewrite/RemoveRedundantStringCast.java      |  4 +-
 .../rewrite/SimplifyConditionalsRule.java       | 37 ++++----
 .../org/apache/impala/service/FeSupport.java    |  2 +-
 .../impala/analysis/ExprRewriterTest.java       |  4 +-
 .../impala/catalog/local/LocalCatalogTest.java  |  3 +-
 29 files changed, 138 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 ebe653a..50418d0 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -2916,7 +2916,7 @@ sign_chain_expr ::=
     // integrate signs into literals
     // integer literals require analysis to set their type, so the instance check below
     // is not equivalent to e.getType().isNumericType()
-    if (e.isLiteral() && e instanceof NumericLiteral) {
+    if (e instanceof NumericLiteral) {
       ((LiteralExpr)e).swapSign();
       RESULT = e;
     } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index 0672cfb..ca5ac72 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -1165,7 +1165,7 @@ public class Analyzer {
   public void createAuxEqPredicate(Expr lhs, Expr rhs) {
     // Check the expr type as well as the class because  NullLiteral could have been
     // implicitly cast to a type different than NULL.
-    if (lhs instanceof NullLiteral || rhs instanceof NullLiteral ||
+    if (Expr.IS_NULL_LITERAL.apply(lhs) || Expr.IS_NULL_LITERAL.apply(rhs) ||
         lhs.getType().isNull() || rhs.getType().isNull()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java b/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
index 196c5cd..7158f8d 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
@@ -32,7 +32,6 @@ import org.apache.impala.thrift.TExprNode;
 import org.apache.impala.thrift.TExprNodeType;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -127,8 +126,8 @@ public class CaseExpr extends Expr {
     // Add the key_expr/val_expr pairs
     while (childIdx + 2 <= decodeExpr.getChildren().size()) {
       Expr candidate = decodeExpr.getChild(childIdx++);
-      if (candidate.isLiteral()) {
-        if (candidate.isNullLiteral()) {
+      if (IS_LITERAL.apply(candidate)) {
+        if (IS_NULL_VALUE.apply(candidate)) {
           // An example case is DECODE(foo, NULL, bar), since NULLs are considered
           // equal, this becomes CASE WHEN foo IS NULL THEN bar END.
           children_.add(encodedIsNull.clone());
@@ -402,7 +401,7 @@ public class CaseExpr extends Expr {
       Expr outputExpr = children_.get(i);
 
       if (outputExpr.isConstant()) {
-        if (outputExpr.isLiteral()) {
+        if (IS_LITERAL.apply(outputExpr)) {
           LiteralExpr outputLiteral = (LiteralExpr) outputExpr;
           if (constLiteralSet.add(outputLiteral)) ++numOutputConstants;
         } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CastExpr.java b/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
index a674f4a..e76a558 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CastExpr.java
@@ -248,7 +248,7 @@ public class CastExpr extends Expr {
 
     // Ensure child has non-null type (even if it's a null literal). This is required
     // for the UDF interface.
-    if (children_.get(0) instanceof NullLiteral) {
+    if (Expr.IS_NULL_LITERAL.apply(children_.get(0))) {
       NullLiteral nullChild = (NullLiteral)(children_.get(0));
       nullChild.uncheckedCastTo(type_);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 24f6029..fa158e3 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
@@ -269,7 +269,7 @@ public class ColumnDef {
         e.analyze(analyzer);
         defaultValLiteral = LiteralExpr.create(e, analyzer.getQueryCtx());
         Preconditions.checkNotNull(defaultValLiteral);
-        if (defaultValLiteral.isNullLiteral()) {
+        if (Expr.IS_NULL_VALUE.apply(defaultValLiteral)) {
           throw new AnalysisException(String.format("String %s cannot be cast " +
               "to a TIMESTAMP literal.", defaultValue_.toSql()));
         }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 fdf639e..ac5c394 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -208,7 +208,7 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     public boolean apply(Expr arg) {
       return arg instanceof BinaryPredicate
           && ((BinaryPredicate) arg).getOp() == Operator.EQ
-          && (((BinaryPredicate) arg).getChild(1).isLiteral());
+          && IS_LITERAL.apply(((BinaryPredicate) arg).getChild(1));
     }
   };
 
@@ -231,6 +231,65 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
         }
       };
 
+  /**
+   * @return true if the expression is a literal.
+   */
+  public final static com.google.common.base.Predicate<Expr> IS_LITERAL =
+    new com.google.common.base.Predicate<Expr>() {
+      @Override
+      public boolean apply(Expr arg) {
+        return arg instanceof LiteralExpr;
+      }
+    };
+
+  /**
+   * @return true if the expression is a null literal.
+   */
+  public final static com.google.common.base.Predicate<Expr> IS_NULL_LITERAL =
+    new com.google.common.base.Predicate<Expr>() {
+      @Override
+      public boolean apply(Expr arg) {
+        return arg instanceof NullLiteral;
+      }
+    };
+
+  /**
+   * @return true if the expression is a literal value other than NULL.
+   */
+  public final static com.google.common.base.Predicate<Expr> IS_NON_NULL_LITERAL =
+    new com.google.common.base.Predicate<Expr>() {
+      @Override
+      public boolean apply(Expr arg) {
+        return IS_LITERAL.apply(arg) && !IS_NULL_LITERAL.apply(arg);
+      }
+    };
+
+  /**
+   * @return true if the expression is a null literal, or a
+   * cast of a null (as created by the ConstantFoldingRule.)
+   */
+  public final static com.google.common.base.Predicate<Expr> IS_NULL_VALUE =
+    new com.google.common.base.Predicate<Expr>() {
+      @Override
+      public boolean apply(Expr arg) {
+        if (arg instanceof NullLiteral) return true;
+        if (! (arg instanceof CastExpr)) return false;
+        return IS_NULL_VALUE.apply(((CastExpr) arg).getChild(0));
+      }
+    };
+
+  /**
+   * @return true if the expression is a  literal, or a
+   * cast of a null (as created by the ConstantFoldingRule.)
+   */
+  public final static com.google.common.base.Predicate<Expr> IS_LITERAL_VALUE =
+    new com.google.common.base.Predicate<Expr>() {
+      @Override
+      public boolean apply(Expr arg) {
+        return IS_LITERAL.apply(arg) || IS_NULL_VALUE.apply(arg);
+      }
+    };
+
   // id that's unique across the entire query statement and is assigned by
   // Analyzer.registerConjuncts(); only assigned for the top-level terms of a
   // conjunction, and therefore null for most Exprs
@@ -638,7 +697,8 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
       // Hack to ensure BE never sees TYPE_NULL. If an expr makes it this far without
       // being cast to a non-NULL type, the type doesn't matter and we can cast it
       // arbitrarily.
-      Preconditions.checkState(this instanceof NullLiteral || this instanceof SlotRef);
+      Preconditions.checkState(IS_NULL_LITERAL.apply(this) ||
+          this instanceof SlotRef);
       return NullLiteral.create(ScalarType.BOOLEAN).treeToThrift();
     }
     TExpr result = new TExpr();
@@ -1106,13 +1166,13 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
           // Remove constant boolean literal expressions.  N.B. - we may have
           // expressions determined to be constant which can not yet be discarded
           // because they can't be evaluated if expr rewriting is turned off.
-          if (rewritten instanceof NullLiteral ||
-              Expr.IS_FALSE_LITERAL.apply(rewritten)) {
+          if (IS_NULL_LITERAL.apply(rewritten) ||
+              IS_FALSE_LITERAL.apply(rewritten)) {
             conjuncts.clear();
             conjuncts.add(rewritten);
             return false;
           }
-          if (Expr.IS_TRUE_LITERAL.apply(rewritten)) {
+          if (IS_TRUE_LITERAL.apply(rewritten)) {
             pruned++;
             conjuncts.remove(index);
           }
@@ -1183,13 +1243,6 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
   }
 
   /**
-   * @return true if this is an instance of LiteralExpr
-   */
-  public boolean isLiteral() {
-    return this instanceof LiteralExpr;
-  }
-
-  /**
    * Returns true if this expression should be treated as constant. I.e. if the frontend
    * and backend should assume that two evaluations of the expression within a query will
    * return the same value. Examples of constant expressions include:
@@ -1220,17 +1273,6 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
   }
 
   /**
-   * @return true if this expr is either a null literal or a cast from
-   * a null literal.
-   */
-  public boolean isNullLiteral() {
-    if (this instanceof NullLiteral) return true;
-    if (!(this instanceof CastExpr)) return false;
-    Preconditions.checkState(children_.size() == 1);
-    return children_.get(0).isNullLiteral();
-  }
-
-  /**
    * Return true if this expr is a scalar subquery.
    */
   public boolean isScalarSubquery() {
@@ -1404,7 +1446,7 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
    */
   public static Expr pushNegationToOperands(Expr root) {
     Preconditions.checkNotNull(root);
-    if (Expr.IS_NOT_PREDICATE.apply(root)) {
+    if (IS_NOT_PREDICATE.apply(root)) {
       try {
         // Make sure we call function 'negate' only on classes that support it,
         // otherwise we may recurse infinitely.

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
index 2716199..6fd3e97 100644
--- a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
@@ -429,7 +429,7 @@ public class FunctionCallExpr extends Expr {
         // The second argument to these functions is the desired scale, otherwise
         // the default is 0.
         Preconditions.checkState(children_.size() == 2);
-        if (children_.get(1).isNullLiteral()) {
+        if (IS_NULL_VALUE.apply(children_.get(1))) {
           throw new AnalysisException(fnName_.getFunction() +
               "() cannot be called with a NULL second argument.");
         }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java b/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
index 3830c95..181c7f7 100644
--- a/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
@@ -129,7 +129,7 @@ public class LikePredicate extends Predicate {
     Preconditions.checkState(fn_ != null);
     Preconditions.checkState(fn_.getReturnType().isBoolean());
 
-    if (getChild(1).isLiteral() && !getChild(1).isNullLiteral()
+    if (Expr.IS_NON_NULL_LITERAL.apply(getChild(1))
         && (op_ == Operator.RLIKE || op_ == Operator.REGEXP || op_ == Operator.IREGEXP)) {
       // let's make sure the pattern works
       // TODO: this checks that it's a Java-supported regex, but the syntax supported
@@ -147,7 +147,7 @@ public class LikePredicate extends Predicate {
   @Override
   protected float computeEvalCost() {
     if (!hasChildCosts()) return UNKNOWN_COST;
-    if (getChild(1).isLiteral() && !getChild(1).isNullLiteral() &&
+    if (Expr.IS_NON_NULL_LITERAL.apply(getChild(1)) &&
       Pattern.matches("[%_]*[^%_]*[%_]*",
           ((StringLiteral) getChild(1)).getValueWithOriginalEscapes())) {
       // This pattern only has wildcards as leading or trailing character,

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java b/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
index 2a5e502..cde3db3 100644
--- a/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
@@ -275,9 +275,9 @@ public abstract class LiteralExpr extends Expr implements Comparable<LiteralExpr
   // Order NullLiterals based on the SQL ORDER BY default behavior: NULLS LAST.
   @Override
   public int compareTo(LiteralExpr other) {
-    if (this instanceof NullLiteral && other instanceof NullLiteral) return 0;
-    if (this instanceof NullLiteral) return -1;
-    if (other instanceof NullLiteral) return 1;
+    if (Expr.IS_NULL_LITERAL.apply(this) && Expr.IS_NULL_LITERAL.apply(other)) return 0;
+    if (Expr.IS_NULL_LITERAL.apply(this)) return -1;
+    if (Expr.IS_NULL_LITERAL.apply(other)) return 1;
     if (getClass() != other.getClass()) return -1;
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/PartitionKeyValue.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PartitionKeyValue.java b/fe/src/main/java/org/apache/impala/analysis/PartitionKeyValue.java
index b35a567..8b3b556 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PartitionKeyValue.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PartitionKeyValue.java
@@ -69,7 +69,7 @@ public class PartitionKeyValue {
    */
   public String toPredicateSql() {
     String ident = ToSqlUtils.getIdentSql(colName_);
-    if (literalValue_ instanceof NullLiteral ||
+    if (Expr.IS_NULL_LITERAL.apply(literalValue_) ||
         literalValue_.getStringValue().isEmpty()) {
       return ident + " IS NULL";
     }
@@ -84,7 +84,8 @@ public class PartitionKeyValue {
   public static String getPartitionKeyValueString(LiteralExpr literalValue,
       String nullPartitionKeyValue) {
     Preconditions.checkNotNull(literalValue);
-    if (literalValue instanceof NullLiteral || literalValue.getStringValue().isEmpty()) {
+    if (Expr.IS_NULL_LITERAL.apply(literalValue) ||
+        literalValue.getStringValue().isEmpty()) {
       return nullPartitionKeyValue;
     }
     return literalValue.getStringValue();

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/PartitionSet.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PartitionSet.java b/fe/src/main/java/org/apache/impala/analysis/PartitionSet.java
index 46a5d7b..1aa30ac 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PartitionSet.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PartitionSet.java
@@ -152,7 +152,7 @@ public class PartitionSet extends PartitionSpecBase {
         if (bp.getOp() == Operator.EQ) {
           SlotRef leftChild =
               bp.getChild(0) instanceof SlotRef ? ((SlotRef) bp.getChild(0)) : null;
-          NullLiteral nullChild = bp.getChild(1) instanceof NullLiteral ?
+          NullLiteral nullChild = Expr.IS_NULL_LITERAL.apply(bp.getChild(1)) ?
               ((NullLiteral) bp.getChild(1)) : null;
           StringLiteral stringChild = bp.getChild(1) instanceof StringLiteral ?
               ((StringLiteral) bp.getChild(1)) : null;

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java b/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
index 1a82dca..ac0551d 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
@@ -88,7 +88,7 @@ public class PartitionSpec extends PartitionSpecBase {
         throw new AnalysisException(String.format(
             "Column '%s' is not a partition column in table: %s",
              pk.getColName(), tableName_));
-      } else if (pk.getValue() instanceof NullLiteral) {
+      } else if (Expr.IS_NULL_LITERAL.apply(pk.getValue())) {
         // No need for further analysis checks of this partition key value.
         continue;
       }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/RangePartition.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/RangePartition.java b/fe/src/main/java/org/apache/impala/analysis/RangePartition.java
index e0441f8..515bf24 100644
--- a/fe/src/main/java/org/apache/impala/analysis/RangePartition.java
+++ b/fe/src/main/java/org/apache/impala/analysis/RangePartition.java
@@ -182,7 +182,7 @@ public class RangePartition implements ParseNode {
       e.analyze(analyzer);
       literal = LiteralExpr.create(e, analyzer.getQueryCtx());
       Preconditions.checkNotNull(literal);
-      if (literal.isNullLiteral()) {
+      if (Expr.IS_NULL_VALUE.apply(literal)) {
         throw new AnalysisException(String.format("Range partition value %s cannot be " +
             "cast to target TIMESTAMP partitioning column.", value.toSql()));
       }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java b/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
index ce90cd3..8debd95 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
@@ -958,7 +958,7 @@ public class SelectStmt extends QueryStmt {
   private Expr rewriteCheckOrdinalResult(ExprRewriter rewriter, Expr expr)
       throws AnalysisException {
     Expr rewrittenExpr = rewriter.rewrite(expr, analyzer_);
-    if (rewrittenExpr.isLiteral() && rewrittenExpr.getType().isIntegerType()) {
+    if (Expr.IS_LITERAL.apply(rewrittenExpr) && rewrittenExpr.getType().isIntegerType()) {
       return expr;
     } else {
       return rewrittenExpr;

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/analysis/TupleIsNullPredicate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TupleIsNullPredicate.java b/fe/src/main/java/org/apache/impala/analysis/TupleIsNullPredicate.java
index 25d6c51..b77735e 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TupleIsNullPredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TupleIsNullPredicate.java
@@ -173,7 +173,7 @@ public class TupleIsNullPredicate extends Predicate {
       List<Expr> params = fnCallExpr.getParams().exprs();
       if (fnCallExpr.getFnName().getFunction().equals("if") &&
           params.get(0) instanceof TupleIsNullPredicate &&
-          params.get(1) instanceof NullLiteral) {
+          Expr.IS_NULL_LITERAL.apply(params.get(1))) {
         return unwrapExpr(params.get(2));
       }
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
index 8c865fe..c50f406 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
@@ -292,7 +292,7 @@ public abstract class FeCatalogUtils {
     for (int i = 0; i < partColSql.size(); ++i) {
       LiteralExpr partVal = part.getPartitionValues().get(i);
       String partValSql = partVal.toSql();
-      if (partVal instanceof NullLiteral || partValSql.isEmpty()) {
+      if (Expr.IS_NULL_LITERAL.apply(partVal) || partValSql.isEmpty()) {
         conjuncts.add(partColSql.get(i) + " IS NULL");
       } else {
         conjuncts.add(partColSql.get(i) + "=" + partValSql);

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
index e9ef929..0c861af 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.LiteralExpr;
 import org.apache.impala.analysis.NullLiteral;
 import org.apache.impala.analysis.PartitionKeyValue;
@@ -409,7 +410,7 @@ public interface FeFsTable extends FeTable {
         boolean matchFound = true;
         for (int i = 0; i < targetValues.size(); ++i) {
           String value;
-          if (partitionValues.get(i) instanceof NullLiteral) {
+          if (Expr.IS_NULL_LITERAL.apply(partitionValues.get(i))) {
             value = table.getNullPartitionKeyValue();
           } else {
             value = partitionValues.get(i).getStringValue();

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 1087e87..768e0bf 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.LiteralExpr;
 import org.apache.impala.analysis.NullLiteral;
 import org.apache.impala.analysis.NumericLiteral;
@@ -1047,7 +1048,7 @@ public class HdfsTable extends Table implements FeFsTable {
       ColumnStats stats = getColumns().get(i).getStats();
       LiteralExpr literal = partition.getPartitionValues().get(i);
       // Store partitions with null partition values separately
-      if (literal instanceof NullLiteral) {
+      if (Expr.IS_NULL_LITERAL.apply(literal)) {
         stats.setNumNulls(stats.getNumNulls() + 1);
         if (nullPartitionIds_.get(i).isEmpty()) {
           stats.setNumDistinctValues(stats.getNumDistinctValues() + 1);
@@ -1105,7 +1106,7 @@ public class HdfsTable extends Table implements FeFsTable {
       ColumnStats stats = getColumns().get(i).getStats();
       LiteralExpr literal = partition.getPartitionValues().get(i);
       // Check if this is a null literal.
-      if (literal instanceof NullLiteral) {
+      if (Expr.IS_NULL_LITERAL.apply(literal)) {
         nullPartitionIds_.get(i).remove(partitionId);
         stats.setNumNulls(stats.getNumNulls() - 1);
         if (nullPartitionIds_.get(i).isEmpty()) {

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
index 3ace234..5b16fbe 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.LiteralExpr;
-import org.apache.impala.analysis.NullLiteral;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogObject.ThriftObjectType;
 import org.apache.impala.catalog.Column;
@@ -442,7 +442,7 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
       List<LiteralExpr> vals = partition.getPartitionValues();
       for (int i = 0; i < getNumClusteringCols(); i++) {
         LiteralExpr val = vals.get(i);
-        if (val instanceof NullLiteral) {
+        if (Expr.IS_NULL_LITERAL.apply(val)) {
           nullParts.get(i).add(partition.getId());
           continue;
         }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
index 1584d44..f2723a6 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
@@ -193,7 +193,7 @@ public class HdfsPartitionPruner {
       SlotRef slot = bp.getBoundSlot();
       if (slot == null) return false;
       Expr bindingExpr = bp.getSlotBinding(slot.getSlotId());
-      if (bindingExpr == null || !bindingExpr.isLiteral()) return false;
+      if (bindingExpr == null || !Expr.IS_LITERAL.apply(bindingExpr)) return false;
       return true;
     } else if (expr instanceof CompoundPredicate) {
       boolean res = canEvalUsingPartitionMd(expr.getChild(0), analyzer);
@@ -217,7 +217,7 @@ public class HdfsPartitionPruner {
       SlotRef slot = ((InPredicate)expr).getBoundSlot();
       if (slot == null) return false;
       for (int i = 1; i < expr.getChildren().size(); ++i) {
-        if (!(expr.getChild(i).isLiteral())) return false;
+        if (!Expr.IS_LITERAL.apply(expr.getChild(i))) return false;
       }
       return true;
     }
@@ -233,7 +233,7 @@ public class HdfsPartitionPruner {
     Preconditions.checkNotNull(expr);
     Preconditions.checkState(expr instanceof BinaryPredicate);
     boolean isSlotOnLeft = true;
-    if (expr.getChild(0).isLiteral()) isSlotOnLeft = false;
+    if (Expr.IS_LITERAL.apply(expr.getChild(0))) isSlotOnLeft = false;
 
     // Get the operands
     BinaryPredicate bp = (BinaryPredicate)expr;
@@ -241,10 +241,10 @@ public class HdfsPartitionPruner {
     Preconditions.checkNotNull(slot);
     Expr bindingExpr = bp.getSlotBinding(slot.getSlotId());
     Preconditions.checkNotNull(bindingExpr);
-    Preconditions.checkState(bindingExpr.isLiteral());
+    Preconditions.checkState(Expr.IS_LITERAL.apply(bindingExpr));
     LiteralExpr literal = (LiteralExpr)bindingExpr;
     Operator op = bp.getOp();
-    if ((literal instanceof NullLiteral) && (op != Operator.NOT_DISTINCT)
+    if (Expr.IS_NULL_LITERAL.apply(literal) && (op != Operator.NOT_DISTINCT)
         && (op != Operator.DISTINCT_FROM)) {
       return Sets.newHashSet();
     }
@@ -260,7 +260,7 @@ public class HdfsPartitionPruner {
     // Compute the matching partition ids
     if (op == Operator.NOT_DISTINCT) {
       // Case: SlotRef <=> Literal
-      if (literal instanceof NullLiteral) {
+      if (Expr.IS_NULL_LITERAL.apply(literal)) {
         Set<Long> ids = tbl_.getNullPartitionIds(partitionPos);
         if (ids != null) matchingIds.addAll(ids);
         return matchingIds;
@@ -277,7 +277,7 @@ public class HdfsPartitionPruner {
     if (op == Operator.DISTINCT_FROM) {
       // Case: SlotRef IS DISTINCT FROM Literal
       matchingIds.addAll(tbl_.getPartitionIds());
-      if (literal instanceof NullLiteral) {
+      if (Expr.IS_NULL_LITERAL.apply(literal)) {
         Set<Long> nullIds = tbl_.getNullPartitionIds(partitionPos);
         matchingIds.removeAll(nullIds);
       } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 c1ff092..4338bbc 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -513,7 +513,7 @@ public class HdfsScanNode extends ScanNode {
     // Only constant exprs can be evaluated against parquet::Statistics. This includes
     // LiteralExpr, but can also be an expr like "1 + 2".
     if (!constExpr.isConstant()) return;
-    if (constExpr.isNullLiteral()) return;
+    if (Expr.IS_NULL_VALUE.apply(constExpr)) return;
 
     BinaryPredicate.Operator op = binaryPred.getOp();
     if (op == BinaryPredicate.Operator.LT || op == BinaryPredicate.Operator.LE ||
@@ -545,10 +545,10 @@ public class HdfsScanNode extends ScanNode {
       Expr child = children.get(i);
 
       // If any child is not a literal, then nothing can be done
-      if (!child.isLiteral()) return;
+      if (!Expr.IS_LITERAL.apply(child)) return;
       LiteralExpr literalChild = (LiteralExpr) child;
       // If any child is NULL, then there is not a valid min/max. Nothing can be done.
-      if (literalChild instanceof NullLiteral) return;
+      if (Expr.IS_NULL_LITERAL.apply(literalChild)) return;
 
       if (min == null || literalChild.compareTo(min) < 0) min = literalChild;
       if (max == null || literalChild.compareTo(max) > 0) max = literalChild;

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/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 fc1f371..348f45f 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -378,7 +378,7 @@ public class KuduScanNode extends ScanNode {
     LiteralExpr literal = (LiteralExpr) predicate.getChild(1);
 
     // Cannot push predicates with null literal values (KUDU-1595).
-    if (literal instanceof NullLiteral) return false;
+    if (Expr.IS_NULL_LITERAL.apply(literal)) return false;
 
     String colName = ((KuduColumn) ref.getDesc().getColumn()).getKuduName();
     ColumnSchema column = table.getSchema().getColumn(colName);
@@ -462,11 +462,11 @@ public class KuduScanNode extends ScanNode {
     // KuduPredicate takes a list of values as Objects.
     List<Object> values = Lists.newArrayList();
     for (int i = 1; i < predicate.getChildren().size(); ++i) {
-      if (!(predicate.getChild(i).isLiteral())) return false;
+      if (!Expr.IS_LITERAL.apply(predicate.getChild(i))) return false;
       LiteralExpr literal = (LiteralExpr) predicate.getChild(i);
 
       // Cannot push predicates with null literal values (KUDU-1595).
-      if (literal instanceof NullLiteral) return false;
+      if (Expr.IS_NULL_LITERAL.apply(literal)) return false;
 
       Object value = getKuduInListValue(analyzer, literal);
       if (value == null) return false;

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/rewrite/FoldConstantsRule.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/FoldConstantsRule.java b/fe/src/main/java/org/apache/impala/rewrite/FoldConstantsRule.java
index 3437531..dd0052f 100644
--- a/fe/src/main/java/org/apache/impala/rewrite/FoldConstantsRule.java
+++ b/fe/src/main/java/org/apache/impala/rewrite/FoldConstantsRule.java
@@ -48,14 +48,14 @@ public class FoldConstantsRule implements ExprRewriteRule {
     // Avoid calling Expr.isConstant() because that would lead to repeated traversals
     // of the Expr tree. Assumes the bottom-up application of this rule. Constant
     // children should have been folded at this point.
-    for (Expr child: expr.getChildren()) if (!child.isLiteral()) return expr;
-    if (expr.isLiteral() || !expr.isConstant()) return expr;
+    for (Expr child: expr.getChildren()) if (!Expr.IS_LITERAL.apply(child)) return expr;
+    if (Expr.IS_LITERAL.apply(expr) || !expr.isConstant()) return expr;
 
     // Do not constant fold cast(null as dataType) because we cannot preserve the
     // cast-to-types and that can lead to query failures, e.g., CTAS
     if (expr instanceof CastExpr) {
       CastExpr castExpr = (CastExpr) expr;
-      if (castExpr.getChild(0) instanceof NullLiteral) {
+      if (Expr.IS_NULL_LITERAL.apply(castExpr.getChild(0))) {
         return expr;
       }
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/rewrite/NormalizeCountStarRule.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/NormalizeCountStarRule.java b/fe/src/main/java/org/apache/impala/rewrite/NormalizeCountStarRule.java
index 2128211..f7d5c57 100644
--- a/fe/src/main/java/org/apache/impala/rewrite/NormalizeCountStarRule.java
+++ b/fe/src/main/java/org/apache/impala/rewrite/NormalizeCountStarRule.java
@@ -43,8 +43,8 @@ public class NormalizeCountStarRule implements ExprRewriteRule {
     if (origExpr.getParams().isDistinct()) return expr;
     if (origExpr.getParams().exprs().size() != 1) return expr;
     Expr child = origExpr.getChild(0);
-    if (!child.isLiteral()) return expr;
-    if (child.isNullLiteral()) return expr;
+    if (!Expr.IS_LITERAL.apply(child)) return expr;
+    if (Expr.IS_NULL_VALUE.apply(child)) return expr;
     FunctionCallExpr result = new FunctionCallExpr(new FunctionName("count"),
         FunctionParams.createStarParam());
     return result;

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/rewrite/RemoveRedundantStringCast.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/RemoveRedundantStringCast.java b/fe/src/main/java/org/apache/impala/rewrite/RemoveRedundantStringCast.java
index d305f27..97dcdbb 100644
--- a/fe/src/main/java/org/apache/impala/rewrite/RemoveRedundantStringCast.java
+++ b/fe/src/main/java/org/apache/impala/rewrite/RemoveRedundantStringCast.java
@@ -67,7 +67,7 @@ public class RemoveRedundantStringCast implements ExprRewriteRule {
         (expr.getChild(0).ignoreImplicitCast() instanceof CastExpr) &&
         expr.getChild(0).ignoreImplicitCast().getType().isStringType() &&
         expr.getChild(1).getType().isStringType() &&
-        expr.getChild(1).isLiteral();
+        Expr.IS_LITERAL.apply(expr.getChild(1));
 
     if (!isPotentiallyRedundantCast) return expr;
     // Ignore the implicit casts added during parsing.
@@ -83,7 +83,7 @@ public class RemoveRedundantStringCast implements ExprRewriteRule {
     // Need to trim() while comparing char(n) types as conversion might add trailing
     // spaces to the 'resultOfReverseCast'.
     if (resultOfReverseCast != null &&
-        !resultOfReverseCast.isNullLiteral() &&
+        !Expr.IS_NULL_VALUE.apply(resultOfReverseCast) &&
         resultOfReverseCast.getStringValue().trim()
             .equals(literalExpr.getStringValue().trim())) {
       return new BinaryPredicate(op, castExprChild,

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java b/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
index 1a887b1..2d0c2d9 100644
--- a/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
+++ b/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
@@ -92,15 +92,14 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
    */
   private Expr simplifyIfFunctionCallExpr(FunctionCallExpr expr) {
     Preconditions.checkState(expr.getChildren().size() == 3);
-    if (expr.getChild(0) instanceof BoolLiteral) {
-      if (((BoolLiteral) expr.getChild(0)).getValue()) {
-        // IF(TRUE)
-        return expr.getChild(1);
-      } else {
-        // IF(FALSE)
-        return expr.getChild(2);
-      }
-    } else if (expr.getChild(0) instanceof NullLiteral) {
+    Expr head = expr.getChild(0);
+    if (Expr.IS_TRUE_LITERAL.apply(head)) {
+      // IF(TRUE)
+      return expr.getChild(1);
+    } else if (Expr.IS_FALSE_LITERAL.apply(head)) {
+      // IF(FALSE)
+      return expr.getChild(2);
+    } else if (Expr.IS_NULL_LITERAL.apply(head)) {
       // IF(NULL)
       return expr.getChild(2);
     }
@@ -116,8 +115,8 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
   private Expr simplifyIfNullFunctionCallExpr(FunctionCallExpr expr) {
     Preconditions.checkState(expr.getChildren().size() == 2);
     Expr child0 = expr.getChild(0);
-    if (child0 instanceof NullLiteral) return expr.getChild(1);
-    if (child0.isLiteral()) return child0;
+    if (Expr.IS_NULL_LITERAL.apply(child0)) return expr.getChild(1);
+    if (Expr.IS_LITERAL.apply(child0)) return child0;
     return expr;
   }
 
@@ -132,8 +131,8 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
     for (int i = 0; i < numChildren; ++i) {
       Expr childExpr = expr.getChildren().get(i);
       // Skip leading nulls.
-      if (childExpr.isNullLiteral()) continue;
-      if ((i == numChildren - 1) || childExpr.isLiteral()) {
+      if (Expr.IS_NULL_VALUE.apply(childExpr)) continue;
+      if ((i == numChildren - 1) || Expr.IS_LITERAL.apply(childExpr)) {
         result = childExpr;
       } else if (i == 0) {
         result = expr;
@@ -178,7 +177,7 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
     if (!(leftChild instanceof BoolLiteral)) return expr;
 
     if (expr.getOp() == CompoundPredicate.Operator.AND) {
-      if (((BoolLiteral) leftChild).getValue()) {
+      if (Expr.IS_TRUE_LITERAL.apply(leftChild)) {
         // TRUE AND 'expr', so return 'expr'.
         return expr.getChild(1);
       } else {
@@ -186,7 +185,7 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
         return leftChild;
       }
     } else if (expr.getOp() == CompoundPredicate.Operator.OR) {
-      if (((BoolLiteral) leftChild).getValue()) {
+      if (Expr.IS_TRUE_LITERAL.apply(leftChild)) {
         // TRUE OR 'expr', so return TRUE.
         return leftChild;
       } else {
@@ -209,14 +208,14 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
   private Expr simplifyCaseExpr(CaseExpr expr, Analyzer analyzer)
       throws AnalysisException {
     Expr caseExpr = expr.hasCaseExpr() ? expr.getChild(0) : null;
-    if (expr.hasCaseExpr() && !caseExpr.isLiteral()) return expr;
+    if (expr.hasCaseExpr() && !Expr.IS_LITERAL.apply(caseExpr)) return expr;
 
     int numChildren = expr.getChildren().size();
     int loopStart = expr.hasCaseExpr() ? 1 : 0;
     // Check and return early if there's nothing that can be simplified.
     boolean canSimplify = false;
     for (int i = loopStart; i < numChildren - 1; i += 2) {
-      if (expr.getChild(i).isLiteral()) {
+      if (Expr.IS_LITERAL.apply(expr.getChild(i))) {
         canSimplify = true;
         break;
       }
@@ -230,11 +229,11 @@ public class SimplifyConditionalsRule implements ExprRewriteRule {
     Expr elseExpr = null;
     for (int i = loopStart; i < numChildren - 1; i += 2) {
       Expr child = expr.getChild(i);
-      if (child instanceof NullLiteral) continue;
+      if (Expr.IS_NULL_LITERAL.apply(child)) continue;
 
       Expr whenExpr;
       if (expr.hasCaseExpr()) {
-        if (child.isLiteral()) {
+        if (Expr.IS_LITERAL.apply(child)) {
           BinaryPredicate pred = new BinaryPredicate(
               BinaryPredicate.Operator.EQ, caseExpr, expr.getChild(i));
           pred.analyze(analyzer);

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/main/java/org/apache/impala/service/FeSupport.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/FeSupport.java b/fe/src/main/java/org/apache/impala/service/FeSupport.java
index 41cd2d9..e67bf9d 100644
--- a/fe/src/main/java/org/apache/impala/service/FeSupport.java
+++ b/fe/src/main/java/org/apache/impala/service/FeSupport.java
@@ -229,7 +229,7 @@ public class FeSupport {
       throws InternalException {
     // Shortcuts to avoid expensive BE evaluation.
     if (pred instanceof BoolLiteral) return ((BoolLiteral) pred).getValue();
-    if (pred instanceof NullLiteral) return false;
+    if (Expr.IS_NULL_LITERAL.apply(pred)) return false;
     Preconditions.checkState(pred.getType().isBoolean());
     TColumnValue val = EvalExprWithoutRow(pred, queryCtx);
     // Return false if pred evaluated to false or NULL. True otherwise.

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java b/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
index 20d4fd9..77595e4 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
@@ -45,7 +45,7 @@ public class ExprRewriterTest extends AnalyzerTest {
     @Override
     public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
       if (expr.contains(Subquery.class)) return expr;
-      if (Predicate.IS_TRUE_LITERAL.apply(expr)) return expr;
+      if (Expr.IS_TRUE_LITERAL.apply(expr)) return expr;
       return new BoolLiteral(true);
     }
 
@@ -60,7 +60,7 @@ public class ExprRewriterTest extends AnalyzerTest {
 
     @Override
     public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
-      if (Predicate.IS_TRUE_LITERAL.apply(expr)) return new BoolLiteral(false);
+      if (Expr.IS_TRUE_LITERAL.apply(expr)) return new BoolLiteral(false);
       return expr;
     }
     private TrueToFalseRule() {}

http://git-wip-us.apache.org/repos/asf/impala/blob/8dfbe3c8/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
index 859ab74..acef882 100644
--- a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ToSqlUtils;
 import org.apache.impala.catalog.CatalogTest;
 import org.apache.impala.catalog.ColumnStats;
@@ -142,7 +143,7 @@ public class LocalCatalogTest {
     assertEquals(1,  ids.size());
     FeFsPartition partition = FeCatalogUtils.loadPartition(
         t, Iterables.getOnlyElement(ids));
-    assertTrue(partition.getPartitionValue(dayCol).isNullLiteral());
+    assertTrue(Expr.IS_NULL_VALUE.apply(partition.getPartitionValue(dayCol)));
   }
 
   @Test


[20/33] impala git commit: IMPALA-7565: Add startup flag to set thrift connection setup thread pool size

Posted by bo...@apache.org.
IMPALA-7565: Add startup flag to set thrift connection setup thread
pool size

This patch adds a hidden experimental startup flag called
'accepted_cnxn_setup_thread_pool_size' which enables setting the size
of the thread pool used to process the internal post-accept, pre-setup
connection queue in each thrift server set up to service
Impala internal and external connections. The default is set to 1
which ensures that this change does not affect current behavior.

Testing:
Tested manually by adding a sleep and making sure other threads are
used.
Ran exhaustive tests with a pool size set to 10 successfully.

Change-Id: I31344321a5f9e840a399ccb0f963c0759e2ab234
Reviewed-on: http://gerrit.cloudera.org:8080/11873
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 0a7da0fc749aea0b73daa4ca12b1135b592a4de8
Parents: cb9db53
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Fri Nov 2 15:36:17 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 be/src/rpc/TAcceptQueueServer.cpp | 24 +++++++++++++++++-------
 1 file changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0a7da0fc/be/src/rpc/TAcceptQueueServer.cpp
----------------------------------------------------------------------
diff --git a/be/src/rpc/TAcceptQueueServer.cpp b/be/src/rpc/TAcceptQueueServer.cpp
index 0f89a7e..730901b 100644
--- a/be/src/rpc/TAcceptQueueServer.cpp
+++ b/be/src/rpc/TAcceptQueueServer.cpp
@@ -26,8 +26,14 @@
 #include "util/thread-pool.h"
 
 DEFINE_int32(accepted_cnxn_queue_depth, 10000,
-    "(Advanced) The size of the post-accept, pre-setup connection queue for Impala "
-    "internal connections");
+    "(Advanced) The size of the post-accept, pre-setup connection queue in each thrift "
+    "server set up to service Impala internal and external connections.");
+
+DEFINE_int32_hidden(accepted_cnxn_setup_thread_pool_size, 1,
+    "(Advanced) The size of the thread pool that is used to process the "
+    "post-accept, pre-setup connection queue in each thrift server set up to service "
+    "Impala internal and external connections. Warning: This is untested for values "
+    "greater than 1 which might exhibit unpredictable behavior and/or cause crashes.");
 
 namespace apache {
 namespace thrift {
@@ -209,13 +215,17 @@ void TAcceptQueueServer::serve() {
     eventHandler_->preServe();
   }
 
-  // Only using one thread here is sufficient for performance, and it avoids potential
-  // thread safety issues with the thrift code called in SetupConnection.
-  constexpr int CONNECTION_SETUP_POOL_SIZE = 1;
-
+  if (FLAGS_accepted_cnxn_setup_thread_pool_size > 1) {
+    LOG(WARNING) << "connection_setup_thread_pool_size is set to "
+                 << FLAGS_accepted_cnxn_setup_thread_pool_size
+                 << ". Values greater than 1 are untested and might exhibit "
+                    "unpredictable behavior and/or cause crashes.";
+  }
   // New - this is the thread pool used to process the internal accept queue.
+  // TODO: IMPALA-7565: Make sure the related thrift code is thread safe and subsequently
+  // enable multi-threading by default.
   ThreadPool<shared_ptr<TTransport>> connection_setup_pool("setup-server", "setup-worker",
-      CONNECTION_SETUP_POOL_SIZE, FLAGS_accepted_cnxn_queue_depth,
+      FLAGS_accepted_cnxn_setup_thread_pool_size, FLAGS_accepted_cnxn_queue_depth,
       [this](int tid, const shared_ptr<TTransport>& item) {
         this->SetupConnection(item);
       });


[06/33] impala git commit: IMPALA-5946, IMPALA-5956: add TPC-DS q31, q59, q89

Posted by bo...@apache.org.
IMPALA-5946,IMPALA-5956: add TPC-DS q31,q59,q89

Q31: the substitution variables didn't match the TPC-DS spec.
     After fixing this, the results match up to 4 digits of
     rounding (there is some error introduced in intermediate
     calculations).
Q59: the results match the reference results up to rounding.
Q89: the results match up to 5 digits of rounding.

I verified the matches by using a spreadsheet comparing reference
and actual results.
https://docs.google.com/spreadsheets/d/1MNEqkfYRRAd3xqY6m20tTHquqjtCThDaGdizzRAQ8co/edit?usp=sharing

*https://github.com/gregrahn/tpcds-kit/blob/master/specification/TPC-DS_v2.10.0.pdf
^https://github.com/gregrahn/tpcds-kit/tree/master/answer_sets

Change-Id: I49634e8f63066773c9c78dd5585a0ee69daf720a
Reviewed-on: http://gerrit.cloudera.org:8080/11845
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: c8c06dd43a7a215a5d6b654a087c2e82013efbc3
Parents: 7616083
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Oct 31 12:05:42 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 .../tpcds-unmodified/queries/tpcds-q31.test     |  69 +++++-
 .../tpcds-unmodified/queries/tpcds-q59.test     | 227 +++++++++++++++----
 .../tpcds-unmodified/queries/tpcds-q89.test     | 119 +++++++++-
 tests/query_test/test_tpcds_queries.py          |   9 +
 4 files changed, 358 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/c8c06dd4/testdata/workloads/tpcds-unmodified/queries/tpcds-q31.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds-unmodified/queries/tpcds-q31.test b/testdata/workloads/tpcds-unmodified/queries/tpcds-q31.test
index 07d519e..bc74dfa 100644
--- a/testdata/workloads/tpcds-unmodified/queries/tpcds-q31.test
+++ b/testdata/workloads/tpcds-unmodified/queries/tpcds-q31.test
@@ -29,30 +29,79 @@ with ss as
        ,ws ws3
  where
     ss1.d_qoy = 1
-    and ss1.d_year = 2002
+    and ss1.d_year = 2000
     and ss1.ca_county = ss2.ca_county
     and ss2.d_qoy = 2
-    and ss2.d_year = 2002
+    and ss2.d_year = 2000
  and ss2.ca_county = ss3.ca_county
     and ss3.d_qoy = 3
-    and ss3.d_year = 2002
+    and ss3.d_year = 2000
     and ss1.ca_county = ws1.ca_county
     and ws1.d_qoy = 1
-    and ws1.d_year = 2002
+    and ws1.d_year = 2000
     and ws1.ca_county = ws2.ca_county
     and ws2.d_qoy = 2
-    and ws2.d_year = 2002
+    and ws2.d_year = 2000
     and ws1.ca_county = ws3.ca_county
     and ws3.d_qoy = 3
-    and ws3.d_year =2002
+    and ws3.d_year =2000
     and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end
        > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
     and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
        > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
- order by ss1.d_year;
-
-
+ order by ss1.ca_county;
 ---- RESULTS
+'Atchison County',2000,0.800221,0.242964,11.944560,3.162954
+'Bacon County',2000,1.168826,0.394038,0.968800,0.510743
+'Bourbon County',2000,1.913115,0.981928,3.364802,1.380123
+'Boyd County',2000,1.086324,0.811507,1.168972,0.742057
+'Bradley County',2000,1.489632,0.575785,1.344390,0.998919
+'Buchanan County',2000,1.195667,0.746049,3.332904,2.239788
+'Carter County',2000,3.953739,1.151033,2.119285,1.844491
+'Cass County',2000,2.398720,1.190582,2.257169,0.847801
+'Corson County',2000,0.560309,0.175081,4.807409,3.227141
+'Crockett County',2000,1.637106,0.360468,2.134042,1.832451
+'Culpeper County',2000,0.661751,0.619014,1.659248,1.224169
+'Edmonson County',2000,0.732442,0.299753,1.602897,1.491246
+'Ferry County',2000,0.701373,0.341010,4.002919,2.603464
+'Fillmore County',2000,0.507769,0.349921,2.443162,1.301136
+'Forest County',2000,0.644745,0.342367,5.771027,1.881074
+'Gaston County',2000,0.763774,0.455496,3.955611,2.141572
+'Grant County',2000,0.693336,0.622870,1.788644,1.722197
+'Green County',2000,0.763667,0.321499,4.694152,4.207034
+'Harlan County',2000,1.670355,1.590115,2.471959,2.107294
+'Harris County',2000,2.337569,0.333126,2.417058,1.025794
+'Heard County',2000,4.102554,1.266947,3.500228,1.127804
+'Houston County',2000,2.045326,1.038983,1.965051,1.421193
+'Ingham County',2000,0.574390,0.384114,1.306517,0.992061
+'Lake County',2000,1.255077,0.745928,1.515147,1.265662
+'Lamar County',2000,0.749358,0.745617,4.269036,2.029584
+'Lincoln County',2000,1.019137,0.944845,2.335960,1.776928
+'Marion County',2000,1.158999,0.916511,2.445040,1.851073
+'Mercer County',2000,0.738342,0.601678,3.012450,2.724470
+'Meriwether County',2000,0.365748,0.300417,2.772270,0.787769
+'Miller County',2000,2.575745,1.318273,2.191968,0.982218
+'Mitchell County',2000,4.439233,1.161335,1.394036,1.256074
+'Mora County',2000,1.183261,0.635654,2.513120,0.918567
+'Nantucket County',2000,1.437897,0.722692,1.175488,0.962304
+'New Kent County',2000,0.602616,0.399062,2.869386,2.625895
+'Nicholas County',2000,2.165116,2.056274,6.021300,1.262577
+'Otero County',2000,2.754655,1.246299,2.976111,2.245835
+'Oxford County',2000,0.973143,0.757240,4.012687,1.640751
+'Perry County',2000,1.580780,0.764453,2.153370,1.802410
+'Prince William County',2000,3.376373,0.630787,1.706967,0.934324
+'Refugio County',2000,1.812976,0.586732,1.301984,1.269604
+'Rice County',2000,1.134698,0.733018,2.378149,1.986402
+'Richmond County',2000,1.571665,1.294015,2.309592,1.778016
+'Sheridan County',2000,1.386030,1.250657,1.575936,0.537886
+'Smith County',2000,0.636936,0.427882,5.744845,4.477958
+'Stark County',2000,7.338220,1.417589,1.863840,1.227366
+'Steele County',2000,1.377413,0.766513,1.247973,0.931195
+'Stone County',2000,1.900042,0.811920,3.699361,1.521662
+'Tooele County',2000,6.590303,0.768910,1.788637,0.340067
+'Vernon County',2000,0.974454,0.915946,1.368804,1.041712
+'Williamson County',2000,2.985102,0.391418,5.805965,4.396699
+'Wright County',2000,5.029335,1.970810,4.076529,1.966472
 ---- TYPES
-INT, INT, STRING, DECIMAL
+STRING, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/c8c06dd4/testdata/workloads/tpcds-unmodified/queries/tpcds-q59.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds-unmodified/queries/tpcds-q59.test b/testdata/workloads/tpcds-unmodified/queries/tpcds-q59.test
index 6a9b431..a67ba3f 100644
--- a/testdata/workloads/tpcds-unmodified/queries/tpcds-q59.test
+++ b/testdata/workloads/tpcds-unmodified/queries/tpcds-q59.test
@@ -1,50 +1,187 @@
 ====
 ---- QUERY: TPCDS-Q59
-
-with wss as
- (select d_week_seq,
-        ss_store_sk,
-        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
-        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
-        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
-        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
-        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
-        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
-        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) 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;
-
-
+with
+  wss as
+  (select
+    d_week_seq,
+    ss_store_sk,
+    sum(case when (d_day_name = 'Sunday') then ss_sales_price else null end) sun_sales,
+    sum(case when (d_day_name = 'Monday') then ss_sales_price else null end) mon_sales,
+    sum(case when (d_day_name = 'Tuesday') then ss_sales_price else null end) tue_sales,
+    sum(case when (d_day_name = 'Wednesday') then ss_sales_price else null end) wed_sales,
+    sum(case when (d_day_name = 'Thursday') then ss_sales_price else null end) thu_sales,
+    sum(case when (d_day_name = 'Friday') then ss_sales_price else null end) fri_sales,
+    sum(case when (d_day_name = 'Saturday') then ss_sales_price else null end) 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 1212 and 1212 + 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 1212 + 12 and 1212 + 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
 ---- RESULTS
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5271,1.362652,3.086396,0.322089,0.480424,1.141632,0.759841,1.845486
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5272,1.096896,0.758348,0.680007,0.987386,1.003437,1.333548,0.992338
+'able','AAAAAAAACAAAAAAA',5273,1.197285,1.027151,0.474952,0.632463,0.708915,1.492329,0.574006
+'able','AAAAAAAACAAAAAAA',5273,1.197285,1.027151,0.474952,0.632463,0.708915,1.492329,0.574006
 ---- TYPES
-INT, INT, STRING, DECIMAL
+STRING, STRING, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/c8c06dd4/testdata/workloads/tpcds-unmodified/queries/tpcds-q89.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpcds-unmodified/queries/tpcds-q89.test b/testdata/workloads/tpcds-unmodified/queries/tpcds-q89.test
index e665976..6fa8080 100644
--- a/testdata/workloads/tpcds-unmodified/queries/tpcds-q89.test
+++ b/testdata/workloads/tpcds-unmodified/queries/tpcds-q89.test
@@ -1,7 +1,6 @@
 ====
 ---- QUERY: TPCDS-Q89
-
-select  *
+select *
 from(
 select i_category, i_class, i_brand,
        s_store_name, s_company_name,
@@ -14,21 +13,119 @@ 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 ('Jewelry','Women','Home') and
-          i_class in ('mens watch','swimwear','kids')
+      d_year in (1999) and
+        ((i_category in ('Books','Electronics','Sports') and
+          i_class in ('computers','stereo','football')
          )
-      or (i_category in ('Men','Sports','Music') and
-          i_class in ('accessories','football','rock')
+      or (i_category in ('Men','Jewelry','Women') and
+          i_class in ('shirts','birdal','dresses')
         ))
 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;
-
-
+limit 100
 ---- RESULTS
+'Men','shirts','importoimporto #1','ese','Unknown',3,2070.65,5492.817500
+'Men','shirts','importoimporto #1','ought','Unknown',6,2152.15,5045.758333
+'Men','shirts','importoimporto #1','able','Unknown',2,2333.43,5194.818333
+'Men','shirts','importoimporto #1','ese','Unknown',6,2691.33,5492.817500
+'Men','shirts','importoimporto #1','ought','Unknown',2,2253.70,5045.758333
+'Women','dresses','amalgamalg #1','able','Unknown',6,1761.81,4507.240000
+'Men','shirts','importoimporto #1','ation','Unknown',3,2672.68,5410.916667
+'Men','shirts','importoimporto #1','ation','Unknown',5,2677.81,5410.916667
+'Men','shirts','importoimporto #1','eing','Unknown',5,2356.88,5074.076667
+'Men','shirts','importoimporto #1','ese','Unknown',4,2784.11,5492.817500
+'Women','dresses','amalgamalg #1','ought','Unknown',7,1926.67,4619.707500
+'Women','dresses','amalgamalg #1','ought','Unknown',1,2065.94,4619.707500
+'Men','shirts','importoimporto #1','ation','Unknown',6,2881.34,5410.916667
+'Men','shirts','importoimporto #1','able','Unknown',3,2674.91,5194.818333
+'Women','dresses','amalgamalg #1','ation','Unknown',2,2192.31,4690.890000
+'Men','shirts','importoimporto #1','bar','Unknown',5,2475.90,4973.509167
+'Women','dresses','amalgamalg #1','ese','Unknown',2,2146.52,4642.990833
+'Men','shirts','importoimporto #1','eing','Unknown',2,2588.02,5074.076667
+'Women','dresses','amalgamalg #1','bar','Unknown',7,2428.75,4914.118333
+'Women','dresses','amalgamalg #1','bar','Unknown',3,2451.51,4914.118333
+'Men','shirts','importoimporto #1','ation','Unknown',7,2951.67,5410.916667
+'Men','shirts','importoimporto #1','able','Unknown',1,2765.75,5194.818333
+'Women','dresses','amalgamalg #1','able','Unknown',2,2082.54,4507.240000
+'Men','shirts','importoimporto #1','ation','Unknown',4,2988.08,5410.916667
+'Men','shirts','importoimporto #1','ought','Unknown',7,2667.92,5045.758333
+'Women','dresses','amalgamalg #1','bar','Unknown',4,2549.27,4914.118333
+'Women','dresses','amalgamalg #1','ese','Unknown',7,2307.00,4642.990833
+'Women','dresses','amalgamalg #1','eing','Unknown',4,2204.15,4528.822500
+'Men','shirts','importoimporto #1','able','Unknown',7,2880.34,5194.818333
+'Men','shirts','importoimporto #1','ese','Unknown',5,3210.74,5492.817500
+'Women','dresses','amalgamalg #1','ation','Unknown',4,2418.39,4690.890000
+'Men','shirts','importoimporto #1','eing','Unknown',4,2833.40,5074.076667
+'Women','dresses','amalgamalg #1','bar','Unknown',2,2692.79,4914.118333
+'Men','shirts','importoimporto #1','eing','Unknown',6,2854.62,5074.076667
+'Women','dresses','amalgamalg #1','eing','Unknown',6,2314.71,4528.822500
+'Men','shirts','importoimporto #1','ought','Unknown',5,2834.94,5045.758333
+'Women','dresses','amalgamalg #1','ation','Unknown',5,2480.25,4690.890000
+'Women','dresses','amalgamalg #1','able','Unknown',7,2302.21,4507.240000
+'Women','dresses','amalgamalg #1','ation','Unknown',6,2501.15,4690.890000
+'Men','shirts','importoimporto #1','ese','Unknown',2,3307.78,5492.817500
+'Men','shirts','importoimporto #1','able','Unknown',5,3016.43,5194.818333
+'Men','shirts','importoimporto #1','able','Unknown',4,3040.23,5194.818333
+'Women','dresses','amalgamalg #1','eing','Unknown',5,2420.31,4528.822500
+'Women','dresses','amalgamalg #1','ese','Unknown',5,2535.80,4642.990833
+'Women','dresses','amalgamalg #1','ought','Unknown',2,2521.70,4619.707500
+'Men','shirts','importoimporto #1','bar','Unknown',3,2877.96,4973.509167
+'Women','dresses','amalgamalg #1','ese','Unknown',3,2577.15,4642.990833
+'Women','dresses','amalgamalg #1','ation','Unknown',7,2637.27,4690.890000
+'Men','shirts','importoimporto #1','ation','Unknown',1,3380.16,5410.916667
+'Women','dresses','amalgamalg #1','eing','Unknown',7,2524.32,4528.822500
+'Men','shirts','importoimporto #1','bar','Unknown',4,2986.70,4973.509167
+'Women','dresses','amalgamalg #1','ought','Unknown',4,2648.80,4619.707500
+'Men','shirts','importoimporto #1','bar','Unknown',7,3026.41,4973.509167
+'Women','dresses','amalgamalg #1','ation','Unknown',3,2751.11,4690.890000
+'Men','shirts','importoimporto #1','bar','Unknown',2,3073.45,4973.509167
+'Men','shirts','importoimporto #1','eing','Unknown',3,3179.96,5074.076667
+'Women','dresses','amalgamalg #1','ation','Unknown',1,2797.07,4690.890000
+'Women','dresses','amalgamalg #1','able','Unknown',1,2621.05,4507.240000
+'Women','dresses','amalgamalg #1','ese','Unknown',4,2808.36,4642.990833
+'Women','dresses','amalgamalg #2','bar','Unknown',6,680.47,2506.963333
+'Men','shirts','importoimporto #1','ation','Unknown',2,3591.65,5410.916667
+'Women','dresses','amalgamalg #1','able','Unknown',3,2706.15,4507.240000
+'Women','dresses','amalgamalg #2','eing','Unknown',2,773.08,2569.346667
+'Men','shirts','importoimporto #1','bar','Unknown',6,3199.08,4973.509167
+'Men','shirts','importoimporto #1','ese','Unknown',7,3754.33,5492.817500
+'Women','dresses','amalgamalg #1','bar','Unknown',5,3190.77,4914.118333
+'Women','dresses','amalgamalg #1','eing','Unknown',2,2841.88,4528.822500
+'Men','shirts','importoimporto #1','able','Unknown',6,3538.25,5194.818333
+'Women','dresses','amalgamalg #1','ese','Unknown',6,3006.94,4642.990833
+'Women','dresses','amalgamalg #1','bar','Unknown',1,3284.93,4914.118333
+'Women','dresses','amalgamalg #2','bar','Unknown',5,878.42,2506.963333
+'Women','dresses','amalgamalg #1','able','Unknown',5,2891.95,4507.240000
+'Men','shirts','importoimporto #1','eing','Unknown',7,3465.77,5074.076667
+'Women','dresses','amalgamalg #2','eing','Unknown',3,964.29,2569.346667
+'Women','dresses','amalgamalg #1','bar','Unknown',6,3324.49,4914.118333
+'Men','shirts','importoimporto #1','ought','Unknown',3,3467.09,5045.758333
+'Women','dresses','amalgamalg #1','ought','Unknown',5,3049.25,4619.707500
+'Women','dresses','amalgamalg #1','eing','Unknown',3,2978.07,4528.822500
+'Men','shirts','importoimporto #1','ought','Unknown',4,3544.97,5045.758333
+'Men','shirts','importoimporto #1','bar','Unknown',1,3490.62,4973.509167
+'Men','shirts','importoimporto #2','able','Unknown',2,628.63,2095.470833
+'Men','shirts','importoimporto #1','eing','Unknown',1,3628.21,5074.076667
+'Women','dresses','amalgamalg #1','able','Unknown',4,3082.17,4507.240000
+'Men','shirts','importoimporto #2','eing','Unknown',5,765.86,2137.656667
+'Women','dresses','amalgamalg #2','ation','Unknown',4,1064.70,2431.226667
+'Women','dresses','amalgamalg #2','ought','Unknown',3,991.89,2355.441667
+'Women','dresses','amalgamalg #2','eing','Unknown',5,1211.13,2569.346667
+'Women','dresses','amalgamalg #2','ation','Unknown',7,1078.41,2431.226667
+'Men','shirts','importoimporto #2','ation','Unknown',4,807.67,2141.060000
+'Women','dresses','amalgamalg #2','ation','Unknown',2,1102.89,2431.226667
+'Women','dresses','amalgamalg #2','eing','Unknown',6,1247.86,2569.346667
+'Women','dresses','amalgamalg #2','able','Unknown',4,1020.39,2337.278333
+'Men','shirts','importoimporto #2','able','Unknown',7,780.91,2095.470833
+'Women','dresses','amalgamalg #2','ese','Unknown',2,904.34,2200.820833
+'Women','dresses','amalgamalg #2','ought','Unknown',7,1061.87,2355.441667
+'Women','dresses','amalgamalg #2','able','Unknown',3,1054.94,2337.278333
+'Men','shirts','importoimporto #2','ought','Unknown',6,633.96,1908.056667
+'Men','shirts','importoimporto #2','ese','Unknown',6,853.37,2113.573333
+'Women','dresses','amalgamalg #2','able','Unknown',5,1084.94,2337.278333
+'Women','dresses','amalgamalg #2','eing','Unknown',4,1317.08,2569.346667
 ---- TYPES
-INT, INT, STRING, DECIMAL
+STRING, STRING, STRING, STRING, STRING, INT, DECIMAL, DECIMAL
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/c8c06dd4/tests/query_test/test_tpcds_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_tpcds_queries.py b/tests/query_test/test_tpcds_queries.py
index b371bf4..54d884f 100644
--- a/tests/query_test/test_tpcds_queries.py
+++ b/tests/query_test/test_tpcds_queries.py
@@ -548,8 +548,17 @@ class TestTpcdsUnmodified(ImpalaTestSuite):
     cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('exec_option')['batch_size'] == 0)
 
+  def test_tpcds_q31(self, vector):
+    self.run_test_case('tpcds-q31', vector)
+
   def test_tpcds_q35a(self, vector):
     self.run_test_case('tpcds-q35a', vector)
 
   def test_tpcds_q48(self, vector):
     self.run_test_case('tpcds-q48', vector)
+
+  def test_tpcds_q59(self, vector):
+    self.run_test_case('tpcds-q59', vector)
+
+  def test_tpcds_q89(self, vector):
+    self.run_test_case('tpcds-q89', vector)


[32/33] impala git commit: IMPALA-5031: Jenkins jobs should fail when UBSAN fails

Posted by bo...@apache.org.
IMPALA-5031: Jenkins jobs should fail when UBSAN fails

Before this patch, Jenkins builds do not fail or warn if UBSAN
produces an error. This patch introduces two new failure modes,
"error" and "death".

In "error", a junit error is generated, which causes a build to be
labeled "unstable".

In "death", a forked process constantly monitors the test logs and
kills the test run if a UBSAN error is found.

Change-Id: I783243458ac2765a97a1dd7dd40d458cc2e1d80b
Reviewed-on: http://gerrit.cloudera.org:8080/11813
Reviewed-by: Jim Apple <jb...@apache.org>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: f79f2d4ad0b8ea2f209785a9b2bec61703102881
Parents: 87c2ad0
Author: Jim Apple <jb...@apache.org>
Authored: Sat Oct 27 21:47:54 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:36 2018 +0100

----------------------------------------------------------------------
 bin/jenkins/all-tests.sh | 34 ++++++++++++++++++++++++++++++++++
 bin/jenkins/finalize.sh  |  9 +++++++++
 2 files changed, 43 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/f79f2d4a/bin/jenkins/all-tests.sh
----------------------------------------------------------------------
diff --git a/bin/jenkins/all-tests.sh b/bin/jenkins/all-tests.sh
index d29f383..b531804 100644
--- a/bin/jenkins/all-tests.sh
+++ b/bin/jenkins/all-tests.sh
@@ -26,6 +26,40 @@ cd "${IMPALA_HOME}"
 
 export IMPALA_MAVEN_OPTIONS="-U"
 
+# When UBSAN_FAIL is "death", the logs are monitored for UBSAN errors. Any errors will
+# then cause this script to exit.
+#
+# When UBSAN_FAIL is "error", monitoring is delayed until tests have finished running.
+#
+# Any other value ignores UBSAN errors.
+: ${UBSAN_FAIL:=error}
+export UBSAN_FAIL
+
+if test -v CMAKE_BUILD_TYPE && [[ "${CMAKE_BUILD_TYPE}" =~ 'UBSAN' ]] \
+    && [ "${UBSAN_FAIL}" = "death" ]
+then
+  export PID_TO_KILL="$(echo $$)"
+  mkdir -p "${IMPALA_HOME}/logs"
+
+  function killer {
+    while ! grep -rI ": runtime error: " "${IMPALA_HOME}/logs"
+    do
+      sleep 1
+      if ! test -e "/proc/$PID_TO_KILL"
+      then
+        return
+      fi
+    done
+    >&2 echo "Killing process $PID_TO_KILL because it invoked undefined behavior"
+    kill -9 $PID_TO_KILL
+  }
+
+  killer &
+  export KILLER_PID="$(echo $!)"
+  disown
+  trap "kill -i $KILLER_PID" EXIT
+fi
+
 source bin/bootstrap_development.sh
 
 RET_CODE=0

http://git-wip-us.apache.org/repos/asf/impala/blob/f79f2d4a/bin/jenkins/finalize.sh
----------------------------------------------------------------------
diff --git a/bin/jenkins/finalize.sh b/bin/jenkins/finalize.sh
index aaebb83..b7adfb4 100755
--- a/bin/jenkins/finalize.sh
+++ b/bin/jenkins/finalize.sh
@@ -21,6 +21,15 @@
 set -euo pipefail
 trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
 
+if test -v CMAKE_BUILD_TYPE && [[ "${CMAKE_BUILD_TYPE}" =~ 'UBSAN' ]] \
+    && [ "${UBSAN_FAIL}" = "error" ] \
+    && { grep -rI ": runtime error: " "${IMPALA_HOME}/logs" 2>&1 | sort | uniq \
+     | tee logs/ubsan.txt ; }
+then
+  "${IMPALA_HOME}"/bin/generate_junitxml.py --step UBSAN \
+      --stderr "${IMPALA_HOME}"/logs/ubsan.txt --error "Undefined C++ behavior"
+fi
+
 rm -rf "${IMPALA_HOME}"/logs_system
 mkdir -p "${IMPALA_HOME}"/logs_system
 dmesg > "${IMPALA_HOME}"/logs_system/dmesg


[16/33] impala git commit: IMPALA-7789: [DOCS] Admission status in Impala Shell

Posted by bo...@apache.org.
IMPALA-7789: [DOCS] Admission status in Impala Shell

Change-Id: I17d788eb716c6a2f7a144ee2d81bbe823f74d16a
Reviewed-on: http://gerrit.cloudera.org:8080/11895
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-by: Bikramjeet Vig <bi...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: cb3120295b5ad2fdcec28d99c1528d4f709f5076
Parents: 09dc763
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Tue Nov 6 17:24:31 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 docs/topics/impala_admission.xml     |  72 +++++++++----------
 docs/topics/impala_live_progress.xml |  26 +++----
 docs/topics/impala_live_summary.xml  | 115 +++++++++++++-----------------
 3 files changed, 100 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/cb312029/docs/topics/impala_admission.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_admission.xml b/docs/topics/impala_admission.xml
index 8b114eb..1dc1512 100644
--- a/docs/topics/impala_admission.xml
+++ b/docs/topics/impala_admission.xml
@@ -783,42 +783,15 @@ impala.admission-control.pool-queue-timeout-ms.<varname>queue_name</varname></ph
         </metadata>
       </prolog>
       <conbody>
-        <p>
-          To see how admission control works for particular queries, examine
-          the profile output for the query. This information is available
-          through the <codeph>PROFILE</codeph> statement in
-            <cmdname>impala-shell</cmdname> immediately after running a query in
-          the shell, on the <uicontrol>queries</uicontrol> page of the Impala
-          debug web UI, or in the Impala log file (basic information at log
-          level 1, more detailed information at log level 2). The profile output
-          contains details about the admission decision, such as whether the
-          query was queued or not and which resource pool it was assigned to. It
-          also includes the estimated and actual memory usage for the query, so
-          you can fine-tune the configuration for the memory limits of the
-          resource pools.
-        </p>
-        <p>
-          Remember that the limits imposed by admission control are
-            <q>soft</q> limits. The decentralized nature of this mechanism means
-          that each Impala node makes its own decisions about whether to allow
-          queries to run immediately or to queue them. These decisions rely on
-          information passed back and forth between nodes by the statestore
-          service. If a sudden surge in requests causes more queries than
-          anticipated to run concurrently, then throughput could decrease due to
-          queries spilling to disk or contending for resources; or queries could
-          be cancelled if they exceed the <codeph>MEM_LIMIT</codeph> setting
-          while running.
-        </p>
-        <!--
-      <p>
-        If you have trouble getting a query to run because its estimated memory usage is too high, you can override
-        the estimate by setting the <codeph>MEM_LIMIT</codeph> query option in <cmdname>impala-shell</cmdname>,
-        then issuing the query through the shell in the same session. The <codeph>MEM_LIMIT</codeph> value is
-        treated as the estimated amount of memory, overriding the estimate that Impala would generate based on
-        table and column statistics. This value is used only for making admission control decisions, and is not
-        pre-allocated by the query.
-      </p>
--->
+        <p> The limits imposed by admission control are de-centrally managed
+            <q>soft</q> limits. Each Impala coordinator node makes its own
+          decisions about whether to allow queries to run immediately or to
+          queue them. These decisions rely on information passed back and forth
+          between nodes by the StateStore service. If a sudden surge in requests
+          causes more queries than anticipated to run concurrently, then the
+          throughput could decrease due to queries spilling to disk or
+          contending for resources. Or queries could be cancelled if they exceed
+          the <codeph>MEM_LIMIT</codeph> setting while running. </p>
         <p>
           In <cmdname>impala-shell</cmdname>, you can also specify which
           resource pool to direct queries to by setting the
@@ -830,6 +803,33 @@ impala.admission-control.pool-queue-timeout-ms.<varname>queue_name</varname></ph
           with Sentry security. See <xref
             href="impala_authorization.xml#authorization"/> for details.
         </p>
+        <p> To see how admission control works for particular queries, examine
+          the profile output or the summary output for the query. <ul>
+            <li>Profile<p>The information is available through the
+                  <codeph>PROFILE</codeph> statement in
+                  <cmdname>impala-shell</cmdname> immediately after running a
+                query in the shell, on the <uicontrol>queries</uicontrol> page
+                of the Impala debug web UI, or in the Impala log file (basic
+                information at log level 1, more detailed information at log
+                level 2). </p><p>The profile output contains details about the
+                admission decision, such as whether the query was queued or not
+                and which resource pool it was assigned to. It also includes the
+                estimated and actual memory usage for the query, so you can
+                fine-tune the configuration for the memory limits of the
+                resource pools. </p></li>
+            <li>Summary<p>Starting in <keyword keyref="impala31"/>, the
+                information is available in <cmdname>impala-shell</cmdname> when
+                the <codeph>LIVE_PROGRESS</codeph> or
+                  <codeph>LIVE_SUMMARY</codeph> query option is set to
+                  <codeph>TRUE</codeph>.</p><p>You can also start an
+                  <codeph>impala-shell</codeph> session with the
+                  <codeph>--live_progress</codeph> or
+                  <codeph>--live_summary</codeph> flags to monitor all queries
+                in that <codeph>impala-shell</codeph> session.</p><p>The summary
+                output includes the queuing status consisting of whether the
+                query was queued and what was the latest queuing
+              reason.</p></li>
+          </ul></p>
         <p>
           For details about all the Fair Scheduler configuration settings, see
             <xref keyref="FairScheduler">Fair Scheduler Configuration</xref>, in

http://git-wip-us.apache.org/repos/asf/impala/blob/cb312029/docs/topics/impala_live_progress.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_live_progress.xml b/docs/topics/impala_live_progress.xml
index 0c91824..63297aa 100644
--- a/docs/topics/impala_live_progress.xml
+++ b/docs/topics/impala_live_progress.xml
@@ -37,19 +37,19 @@ under the License.
 
   <conbody>
 
-    <p rev="2.3.0">
-      <indexterm audience="hidden">LIVE_PROGRESS query option</indexterm>
-      For queries submitted through the <cmdname>impala-shell</cmdname> command,
-      displays an interactive progress bar showing roughly what percentage of
-      processing has been completed. When the query finishes, the progress bar is erased
-      from the <cmdname>impala-shell</cmdname> console output.
-    </p>
-
-    <p>
-    </p>
-
-    <p conref="../shared/impala_common.xml#common/type_boolean"/>
-    <p conref="../shared/impala_common.xml#common/default_false_0"/>
+    <p rev="2.3.0"> When the <codeph>LIVE_PROGRESS</codeph> query option is set
+      to <codeph>TRUE</codeph>, Impala displays an interactive progress bar
+      showing roughly what percentage of processing has been completed for
+      queries submitted through the <cmdname>impala-shell</cmdname> command.
+      When the query finishes, the progress bar is erased from the
+        <cmdname>impala-shell</cmdname> console output. </p>
+    <p>Starting in <keyword keyref="impala31"/>, the summary output also
+      includes the queuing status consisting of whether the query was queued and
+      what was the latest queuing reason.</p>
+    <p><b>Type:</b>
+      <codeph>Boolean</codeph></p>
+    <p><b>Default:</b>
+      <codeph>FALSE (0)</codeph></p>
 
     <p conref="../shared/impala_common.xml#common/command_line_blurb"/>
     <p>

http://git-wip-us.apache.org/repos/asf/impala/blob/cb312029/docs/topics/impala_live_summary.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_live_summary.xml b/docs/topics/impala_live_summary.xml
index 94733d2..10ecae3 100644
--- a/docs/topics/impala_live_summary.xml
+++ b/docs/topics/impala_live_summary.xml
@@ -36,71 +36,59 @@ under the License.
   </prolog>
 
   <conbody>
-
-    <p rev="2.3.0">
-      <indexterm audience="hidden">LIVE_SUMMARY query option</indexterm>
-      For queries submitted through the <cmdname>impala-shell</cmdname> command,
-      displays the same output as the <codeph>SUMMARY</codeph> command,
-      with the measurements updated in real time as the query progresses.
-      When the query finishes, the final <codeph>SUMMARY</codeph> output remains
-      visible in the <cmdname>impala-shell</cmdname> console output.
-    </p>
-
-    <p>
-    </p>
-
-    <p conref="../shared/impala_common.xml#common/type_boolean"/>
-    <p conref="../shared/impala_common.xml#common/default_false_0"/>
-
+    <p rev="2.3.0"> When the <codeph>LIVE_SUMMARY</codeph> query option is set
+      to <codeph>TRUE</codeph>, Impala displays the same output as the
+        <codeph>SUMMARY</codeph> command for queries submitted through the
+        <cmdname>impala-shell</cmdname> command, with the measurements updated
+      in real time as the query progresses. When the query finishes, the final
+        <codeph>SUMMARY</codeph> output remains visible in the
+        <cmdname>impala-shell</cmdname> console output. </p>
+    <p>Starting in <keyword keyref="impala31"/>, the summary output also
+      includes the queuing status consisting of whether the query was queued and
+      what was the latest queuing reason.</p>
+    <p>the queuing status, whether the query was queued and what was the latest
+      queuing reason.</p>
+    <p><b>Type:</b>
+      <codeph>Boolean</codeph></p>
+    <p><b>Default:</b>
+      <codeph>FALSE (0)</codeph></p>
     <p conref="../shared/impala_common.xml#common/command_line_blurb"/>
-    <p>
-      You can enable this query option within <cmdname>impala-shell</cmdname>
+    <p> You can enable this query option within <cmdname>impala-shell</cmdname>
       by starting the shell with the <codeph>--live_summary</codeph>
-      command-line option.
-      You can still turn this setting off and on again within the shell through the
-      <codeph>SET</codeph> command.
-    </p>
-
+      command-line option. You can still turn this setting off and on again
+      within the shell through the <codeph>SET</codeph> command. </p>
     <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>
-    <p>
-      The live summary output can be useful for evaluating long-running queries,
-      to evaluate which phase of execution takes up the most time, or if some hosts
-      take much longer than others for certain operations, dragging overall performance down.
-      By making the information available in real time, this feature lets you decide what
-      action to take even before you cancel a query that is taking much longer than normal.
-    </p>
-    <p>
-      For example, you might see the HDFS scan phase taking a long time, and therefore revisit
-      performance-related aspects of your schema design such as constructing a partitioned table,
-      switching to the Parquet file format, running the <codeph>COMPUTE STATS</codeph> statement
-      for the table, and so on.
-      Or you might see a wide variation between the average and maximum times for all hosts to
-      perform some phase of the query, and therefore investigate if one particular host
-      needed more memory or was experiencing a network problem.
-    </p>
+    <p> The live summary output can be useful for evaluating long-running
+      queries, to evaluate which phase of execution takes up the most time, or
+      if some hosts take much longer than others for certain operations,
+      dragging overall performance down. By making the information available in
+      real time, this feature lets you decide what action to take even before
+      you cancel a query that is taking much longer than normal. </p>
+    <p> For example, you might see the HDFS scan phase taking a long time, and
+      therefore revisit performance-related aspects of your schema design such
+      as constructing a partitioned table, switching to the Parquet file format,
+      running the <codeph>COMPUTE STATS</codeph> statement for the table, and so
+      on. Or you might see a wide variation between the average and maximum
+      times for all hosts to perform some phase of the query, and therefore
+      investigate if one particular host needed more memory or was experiencing
+      a network problem. </p>
     <p conref="../shared/impala_common.xml#common/live_reporting_details"/>
-    <p>
-      For a simple and concise way of tracking the progress of an interactive query, see
-      <xref href="impala_live_progress.xml#live_progress"/>.
-    </p>
-
+    <p> For a simple and concise way of tracking the progress of an interactive
+      query, see <xref href="impala_live_progress.xml#live_progress"/>. </p>
     <p conref="../shared/impala_common.xml#common/restrictions_blurb"/>
-    <p conref="../shared/impala_common.xml#common/impala_shell_progress_reports_compute_stats_caveat"/>
-    <p conref="../shared/impala_common.xml#common/impala_shell_progress_reports_shell_only_caveat"/>
-
+    <p
+      conref="../shared/impala_common.xml#common/impala_shell_progress_reports_compute_stats_caveat"/>
+    <p
+      conref="../shared/impala_common.xml#common/impala_shell_progress_reports_shell_only_caveat"/>
     <p conref="../shared/impala_common.xml#common/added_in_230"/>
-
     <p conref="../shared/impala_common.xml#common/example_blurb"/>
-
-    <p>
-      The following example shows a series of <codeph>LIVE_SUMMARY</codeph> reports that
-      are displayed during the course of a query, showing how the numbers increase to
-      show the progress of different phases of the distributed query. When you do the same
-      in <cmdname>impala-shell</cmdname>, only a single report is displayed at any one time,
-      with each update overwriting the previous numbers.
-    </p>
-
-<codeblock><![CDATA[[localhost:21000] > set live_summary=true;
+    <p> The following example shows a series of <codeph>LIVE_SUMMARY</codeph>
+      reports that are displayed during the course of a query, showing how the
+      numbers increase to show the progress of different phases of the
+      distributed query. When you do the same in
+      <cmdname>impala-shell</cmdname>, only a single report is displayed at any
+      one time, with each update overwriting the previous numbers. </p>
+    <codeblock><![CDATA[[localhost:21000] > set live_summary=true;
 LIVE_SUMMARY set to true
 [localhost:21000] > select count(*) from customer t1 cross join customer t2;
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+
@@ -140,9 +128,8 @@ LIVE_SUMMARY set to true
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+
 ]]>
 </codeblock>
-
-<!-- Keeping this sample output that illustrates a couple of glitches in the LIVE_SUMMARY display, hidden, to help filing JIRAs. -->
-<codeblock audience="hidden"><![CDATA[[
+    <!-- Keeping this sample output that illustrates a couple of glitches in the LIVE_SUMMARY display, hidden, to help filing JIRAs. -->
+    <codeblock audience="hidden"><![CDATA[[
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+
 | Operator            | #Hosts | Avg Time | Max Time | #Rows   | Est. #Rows | Peak Mem | Est. Peak Mem | Detail                |
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+
@@ -222,8 +209,8 @@ Query: select count(*) from customer t1 cross join customer t2
 | Operator            | #Hosts | Avg Time | Max Time | #Rows   | Est. #Rows | Peak Mem | Est. Peak Mem | Detail                |
 ]]>
 </codeblock>
-
-    <p conref="../shared/impala_common.xml#common/live_progress_live_summary_asciinema"/>
-
+    <p
+      conref="../shared/impala_common.xml#common/live_progress_live_summary_asciinema"
+    />
   </conbody>
 </concept>


[29/33] impala git commit: IMPALA-7840: add missing error to test_concurrent_schema_change

Posted by bo...@apache.org.
IMPALA-7840: add missing error to test_concurrent_schema_change

test_concurrent_schema_change runs a series of alters and inserts on
the same Kudu table concurrently to ensure that Impala can handle this
without crashing.

There is a list of expected error messages in the test. One possible
legitimate error is missing, causing the test to sometimes be flaky.

This patch adds that error message to the test.

Change-Id: Ibc01e1244e05be30343c05b5a0f8676f4d4603d2
Reviewed-on: http://gerrit.cloudera.org:8080/11913
Reviewed-by: Thomas Marshall <th...@cmu.edu>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 4a10096064b95bff4d25d2af3c393794ec4e328b
Parents: 1d412a0
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Thu Nov 8 22:44:01 2018 +0000
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:36 2018 +0100

----------------------------------------------------------------------
 tests/query_test/test_kudu.py | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4a100960/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index 005fa63..5936649 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -437,14 +437,18 @@ class TestKuduOperations(KuduTestSuite):
     for error in insert_thread.errors:
       msg = str(error)
       # The first two are AnalysisExceptions, the next two come from KuduTableSink::Open()
-      # if the schema has changed since analysis, the last comes from the Kudu server if
+      # if the schema has changed since analysis, the rest come from the Kudu server if
       # the schema changes between KuduTableSink::Open() and when the write ops are sent.
-      assert "has fewer columns (1) than the SELECT / VALUES clause returns (2)" in msg \
-        or "(type: TINYINT) is not compatible with column 'col1' (type: STRING)" in msg \
-        or "has fewer columns than expected." in msg \
-        or "Column col1 has unexpected type." in msg \
-        or "Client provided column col1[int64 NULLABLE] not present in tablet" in msg \
-        or "Client provided column col1 INT64 NULLABLE not present in tablet"
+      possible_errors = [
+        "has fewer columns (1) than the SELECT / VALUES clause returns (2)",
+        "(type: TINYINT) is not compatible with column 'col1' (type: STRING)",
+        "has fewer columns than expected.",
+        "Column col1 has unexpected type.",
+        "Client provided column col1[int64 NULLABLE] not present in tablet",
+        "Client provided column col1 INT64 NULLABLE not present in tablet",
+        "The column 'col1' must have type string NULLABLE found int64 NULLABLE"
+      ]
+      assert any(err in msg for err in possible_errors)
 
   def _retry_query(self, cursor, query, expected):
     retries = 0


[13/33] impala git commit: IMPALA-7586: fix predicate pushdown of escaped strings

Posted by bo...@apache.org.
IMPALA-7586: fix predicate pushdown of escaped strings

This fixes a class of bugs where the planner incorrectly uses the raw
string from the parser instead of the unescaped string. This occurs in
several places that push predicates down to the storage layer:
* Kudu scans
* HBase scans
* Data source scans

There are some more complex issues with escapes and the LIKE predicate
that are tracked separately by IMPALA-2422.

This also uncovered a different issue with RCFiles that is tracked by
IMPALA-7778 and is worked around by the tests added.

In order to make bugs like this more obvious in future, I renamed
getValue() to getValueWithOriginalEscapes().

Testing:
Added regression test that tests handling of backslash escapes on all
file formats. I did not add a regression test for the data source bug
since it seems to require some major modification of the data source
test infrastructure.

Change-Id: I53d6e20dd48ab6837ddd325db8a9d49ee04fed28
Reviewed-on: http://gerrit.cloudera.org:8080/11814
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: c124d26b546b40e1a2d98bfe396898ab563ccafc
Parents: e3a7027
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Oct 26 16:50:22 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 .../org/apache/impala/analysis/AdminFnStmt.java |  2 +-
 .../java/org/apache/impala/analysis/Expr.java   |  2 +-
 .../apache/impala/analysis/ExtractFromExpr.java |  6 +-
 .../apache/impala/analysis/LikePredicate.java   |  5 +-
 .../apache/impala/analysis/StringLiteral.java   |  8 ++-
 .../impala/planner/DataSourceScanNode.java      |  3 +-
 .../apache/impala/planner/HBaseScanNode.java    |  4 +-
 .../org/apache/impala/planner/KuduScanNode.java |  4 +-
 testdata/data/README                            |  3 +
 testdata/data/strings_with_quotes.csv           | 11 ++++
 .../functional/functional_schema_template.sql   | 28 +++++++++
 .../datasets/functional/schema_constraints.csv  |  1 +
 .../QueryTest/string-escaping-rcfile-bug.test   | 66 ++++++++++++++++++++
 .../queries/QueryTest/string-escaping.test      | 62 ++++++++++++++++++
 tests/query_test/test_scanners.py               |  8 +++
 15 files changed, 199 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/analysis/AdminFnStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AdminFnStmt.java b/fe/src/main/java/org/apache/impala/analysis/AdminFnStmt.java
index 2f2eb2e..1e1f022 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AdminFnStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AdminFnStmt.java
@@ -117,7 +117,7 @@ public class AdminFnStmt extends StatementBase {
         throw new AnalysisException(
             "Invalid backend, must be a string literal: " + backendExpr.toSql());
       }
-      backend_ = parseBackendAddress(((StringLiteral) backendExpr).getValue());
+      backend_ = parseBackendAddress(((StringLiteral) backendExpr).getUnescapedValue());
     }
     if (deadlineExpr != null) {
       deadlineSecs_ = deadlineExpr.evalToNonNegativeInteger(analyzer, "deadline");

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/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 912ec8a..fdf639e 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -1486,7 +1486,7 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
         return DEFAULT_AVG_STRING_LENGTH;
       }
     } else if (e instanceof StringLiteral) {
-      return ((StringLiteral) e).getValue().length();
+      return ((StringLiteral) e).getUnescapedValue().length();
     } else {
       // TODO(tmarshall): Extend this to support other string Exprs, such as
       // function calls that return string.

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
index 40020e0..9f46187 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
@@ -20,7 +20,6 @@ package org.apache.impala.analysis;
 import java.util.Set;
 
 import org.apache.impala.catalog.BuiltinsDb;
-import org.apache.impala.catalog.Catalog;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TExtractField;
@@ -81,7 +80,8 @@ public class ExtractFromExpr extends FunctionCallExpr {
 
     super.analyzeImpl(analyzer);
 
-    String extractFieldIdent = ((StringLiteral)children_.get(1)).getValue();
+    String extractFieldIdent =
+        ((StringLiteral)children_.get(1)).getValueWithOriginalEscapes();
     Preconditions.checkNotNull(extractFieldIdent);
     if (!EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase())) {
       throw new AnalysisException("Time unit '" + extractFieldIdent + "' in expression '"
@@ -102,7 +102,7 @@ public class ExtractFromExpr extends FunctionCallExpr {
     StringBuilder strBuilder = new StringBuilder();
     strBuilder.append(getFnName().toString().toUpperCase());
     strBuilder.append("(");
-    strBuilder.append(((StringLiteral)getChild(1)).getValue());
+    strBuilder.append(((StringLiteral)getChild(1)).getValueWithOriginalEscapes());
     strBuilder.append(" FROM ");
     strBuilder.append(getChild(0).toSql());
     strBuilder.append(")");

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java b/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
index bc10b2a..3830c95 100644
--- a/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/LikePredicate.java
@@ -135,7 +135,7 @@ public class LikePredicate extends Predicate {
       // TODO: this checks that it's a Java-supported regex, but the syntax supported
       // by the backend is Posix; add a call to the backend to check the re syntax
       try {
-        Pattern.compile(((StringLiteral) getChild(1)).getValue());
+        Pattern.compile(((StringLiteral) getChild(1)).getValueWithOriginalEscapes());
       } catch (PatternSyntaxException e) {
         throw new AnalysisException(
             "invalid regular expression in '" + this.toSql() + "'");
@@ -148,7 +148,8 @@ public class LikePredicate extends Predicate {
   protected float computeEvalCost() {
     if (!hasChildCosts()) return UNKNOWN_COST;
     if (getChild(1).isLiteral() && !getChild(1).isNullLiteral() &&
-      Pattern.matches("[%_]*[^%_]*[%_]*", ((StringLiteral) getChild(1)).getValue())) {
+      Pattern.matches("[%_]*[^%_]*[%_]*",
+          ((StringLiteral) getChild(1)).getValueWithOriginalEscapes())) {
       // This pattern only has wildcards as leading or trailing character,
       // so it is linear.
       return getChildCosts() +

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java b/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
index 5c51c45..12ad635 100644
--- a/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
+++ b/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
@@ -79,7 +79,11 @@ public class StringLiteral extends LiteralExpr {
     msg.string_literal = new TStringLiteral(val);
   }
 
-  public String getValue() { return value_; }
+  /**
+   * Returns the original value that the string literal was constructed with,
+   * without escaping or unescaping it.
+   */
+  public String getValueWithOriginalEscapes() { return value_; }
 
   public String getUnescapedValue() {
     // Unescape string exactly like Hive does. Hive's method assumes
@@ -126,7 +130,7 @@ public class StringLiteral extends LiteralExpr {
 
   @Override
   public String getStringValue() {
-    return value_;
+    return getValueWithOriginalEscapes();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
index a41630b..1ddb394 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -129,7 +129,8 @@ public class DataSourceScanNode extends ScanNode {
         return new TColumnValue().setDouble_val(
             ((NumericLiteral) expr).getDoubleValue());
       case STRING:
-        return new TColumnValue().setString_val(((StringLiteral) expr).getValue());
+        return new TColumnValue().setString_val(
+            ((StringLiteral) expr).getUnescapedValue());
       case DECIMAL:
       case DATE:
       case DATETIME:

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
index 06aca1f..13ecb6a 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -166,7 +166,7 @@ public class HBaseScanNode extends ScanNode {
             analyzer.getQueryCtx());
         if (val instanceof StringLiteral) {
           StringLiteral litVal = (StringLiteral) val;
-          startKey_ = convertToBytes(litVal.getStringValue(),
+          startKey_ = convertToBytes(litVal.getUnescapedValue(),
               !rowRange.getLowerBoundInclusive());
         } else {
           // lower bound is null.
@@ -182,7 +182,7 @@ public class HBaseScanNode extends ScanNode {
             analyzer.getQueryCtx());
         if (val instanceof StringLiteral) {
           StringLiteral litVal = (StringLiteral) val;
-          stopKey_ = convertToBytes(litVal.getStringValue(),
+          stopKey_ = convertToBytes(litVal.getUnescapedValue(),
               rowRange.getUpperBoundInclusive());
         } else {
           // lower bound is null.

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/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 56c7602..adeaa72 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -415,7 +415,7 @@ public class KuduScanNode extends ScanNode {
       case VARCHAR:
       case CHAR: {
         kuduPredicate = KuduPredicate.newComparisonPredicate(column, op,
-            ((StringLiteral)literal).getStringValue());
+            ((StringLiteral)literal).getUnescapedValue());
         break;
       }
       case TIMESTAMP: {
@@ -523,7 +523,7 @@ public class KuduScanNode extends ScanNode {
       case BIGINT: return ((NumericLiteral) e).getLongValue();
       case FLOAT: return (float) ((NumericLiteral) e).getDoubleValue();
       case DOUBLE: return ((NumericLiteral) e).getDoubleValue();
-      case STRING: return ((StringLiteral) e).getValue();
+      case STRING: return ((StringLiteral) e).getUnescapedValue();
       case TIMESTAMP: {
         try {
           // TODO: Simplify when Impala supports a 64-bit TIMESTAMP type.

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/testdata/data/README
----------------------------------------------------------------------
diff --git a/testdata/data/README b/testdata/data/README
index 599b009..db055c7 100644
--- a/testdata/data/README
+++ b/testdata/data/README
@@ -230,3 +230,6 @@ valid range [0..24H). Before the fix, select * returned these values:
 1970-01-01 00:00:00
 1970-01-01 23:59:59.999999999
 1970-01-01 24:00:00 (invalid - time of day should be less than a whole day)
+
+strings_with_quotes.csv:
+Various strings with quotes in them to reproduce bugs like IMPALA-7586.

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/testdata/data/strings_with_quotes.csv
----------------------------------------------------------------------
diff --git a/testdata/data/strings_with_quotes.csv b/testdata/data/strings_with_quotes.csv
new file mode 100644
index 0000000..0d82f03
--- /dev/null
+++ b/testdata/data/strings_with_quotes.csv
@@ -0,0 +1,11 @@
+",1
+"",2
+\\",3
+'',4
+',5
+foo',6
+'foo,7
+"foo",8
+"foo,9
+foo"bar,10
+foo\\"bar,11

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/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 cfd9030..0712b7c 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -2194,3 +2194,31 @@ CREATE TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} (i1 integer)
 STORED AS {file_format}
 TBLPROPERTIES('skip.header.line.count'='2');
 ====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+strings_with_quotes
+---- COLUMNS
+s string
+i int
+---- ROW_FORMAT
+delimited fields terminated by ','  escaped by '\\'
+---- LOAD
+LOAD DATA LOCAL INPATH '{impala_home}/testdata/data/strings_with_quotes.csv'
+OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
+---- DEPENDENT_LOAD
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
+SELECT s, i
+FROM {db_name}.{table_name};
+---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
+CREATE TABLE {db_name}{db_suffix}.{table_name} (
+  s string PRIMARY KEY,
+  i int
+)
+PARTITION BY HASH (s) PARTITIONS 3 STORED AS KUDU;
+---- DEPENDENT_LOAD_KUDU
+INSERT into TABLE {db_name}{db_suffix}.{table_name}
+SELECT s, i
+FROM {db_name}.{table_name};
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/testdata/datasets/functional/schema_constraints.csv
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index baf0306..e09ca36 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -193,6 +193,7 @@ table_name:nulltable, constraint:only, table_format:kudu/none/none
 table_name:nullescapedtable, constraint:only, table_format:kudu/none/none
 table_name:decimal_tbl, constraint:only, table_format:kudu/none/none
 table_name:decimal_tiny, constraint:only, table_format:kudu/none/none
+table_name:strings_with_quotes, constraint:only, table_format:kudu/none/none
 
 # Skipping header lines is only effective with text tables
 table_name:table_with_header, constraint:restrict_to, table_format:text/none/none

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/testdata/workloads/functional-query/queries/QueryTest/string-escaping-rcfile-bug.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/string-escaping-rcfile-bug.test b/testdata/workloads/functional-query/queries/QueryTest/string-escaping-rcfile-bug.test
new file mode 100644
index 0000000..d757480
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/string-escaping-rcfile-bug.test
@@ -0,0 +1,66 @@
+# This file contains the current output for queries against strings_with_quotes
+# for RCFile. The output is different because of a bug in the RCFile scanner.
+# This file can be removed once IMPALA-7778 is fixed.
+====
+---- QUERY
+# Check that all strings in the table are returned correctly.
+# IMPALA-7778: escapes are ignored so output is incorrect
+select s
+from strings_with_quotes
+---- RESULTS
+'"'
+'""'
+'\\\\"'
+''''''
+''''
+'foo'''
+'''foo'
+'"foo"'
+'"foo'
+'foo"bar'
+'foo\\\\"bar'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s = '"'
+---- RESULTS
+'"'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+# IMPALA-7778: escapes are ignored so output is incorrect
+select s
+from strings_with_quotes
+where s = '\\"'
+---- RESULTS
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s in ('"', 'foo"bar')
+---- RESULTS
+'"'
+'foo"bar'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+# IMPALA-7778: escapes are ignored so output is incorrect
+select s
+from strings_with_quotes
+where s in ('\\"', 'foo"bar')
+---- RESULTS
+'foo"bar'
+---- TYPES
+STRING
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/testdata/workloads/functional-query/queries/QueryTest/string-escaping.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/string-escaping.test b/testdata/workloads/functional-query/queries/QueryTest/string-escaping.test
new file mode 100644
index 0000000..a2f2c5d
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/string-escaping.test
@@ -0,0 +1,62 @@
+====
+---- QUERY
+# Check that all strings in the table are returned correctly.
+select s
+from strings_with_quotes
+---- RESULTS
+'"'
+'""'
+'\\"'
+''''''
+''''
+'foo'''
+'''foo'
+'"foo"'
+'"foo'
+'foo"bar'
+'foo\\"bar'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s = '"'
+---- RESULTS
+'"'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s = '\\"'
+---- RESULTS
+'\\"'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s in ('"', 'foo"bar')
+---- RESULTS
+'"'
+'foo"bar'
+---- TYPES
+STRING
+====
+---- QUERY
+# Regression test for IMPALA-7586: predicate pushed down with incorrect string escaping.
+select s
+from strings_with_quotes
+where s in ('\\"', 'foo"bar')
+---- RESULTS
+'\\"'
+'foo"bar'
+---- TYPES
+STRING
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/c124d26b/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 91f0449..f4ddad4 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -97,6 +97,14 @@ class TestScannersAllTableFormats(ImpalaTestSuite):
       pytest.skip()
     self.run_test_case('QueryTest/hdfs_scanner_profile', vector)
 
+  def test_string_escaping(self, vector):
+    """Test handling of string escape sequences."""
+    if vector.get_value('table_format').file_format == 'rc':
+      # IMPALA-7778: RCFile scanner incorrectly ignores escapes for now.
+      self.run_test_case('QueryTest/string-escaping-rcfile-bug', vector)
+    else:
+      self.run_test_case('QueryTest/string-escaping', vector)
+
 # Test all the scanners with a simple limit clause. The limit clause triggers
 # cancellation in the scanner code paths.
 class TestScannersAllTableFormatsWithLimit(ImpalaTestSuite):


[11/33] impala git commit: IMPALA-5031: Make UBSAN-friendly arithmetic generic

Posted by bo...@apache.org.
IMPALA-5031: Make UBSAN-friendly arithmetic generic

ArithmeticUtil::AsUnsigned() makes it possible to do arithmetic on
signed integers in a way that does not invoke undefined behavior, but
it only works on integers. This patch adds ArithmeticUtil::Compute(),
which dispatches (at compile time) to the normal arithmetic evaluation
method if the type of the values is a floating point type, but uses
AsUnsigned() if the type of the values is an integral type.

Change-Id: I73bec71e59c5a921003d0ebca52a1d4e49bbef66
Reviewed-on: http://gerrit.cloudera.org:8080/11810
Reviewed-by: Jim Apple <jb...@apache.org>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: e0642c9191e082fde4cc92ad754c9f643e20e248
Parents: 9bdb73b
Author: Jim Apple <jb...@apache.org>
Authored: Sat Oct 27 07:49:23 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/exprs/expr-test.cc     |  6 ++---
 be/src/util/arithmetic-util.h | 51 ++++++++++++++++++++++++++++++++++++++
 2 files changed, 54 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e0642c91/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index c926ff0..dd6f1b3 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -943,11 +943,11 @@ class ExprTest : public testing::Test {
     string a_str = LiteralToString<Result>(cast_a);
     string b_str = LiteralToString<Result>(cast_b);
     TestValue(a_str + " + " + b_str, expected_type,
-        static_cast<Result>(cast_a + cast_b));
+        ArithmeticUtil::Compute<std::plus>(cast_a, cast_b));
     TestValue(a_str + " - " + b_str, expected_type,
-        static_cast<Result>(cast_a - cast_b));
+         ArithmeticUtil::Compute<std::minus>(cast_a, cast_b));
     TestValue(a_str + " * " + b_str, expected_type,
-        static_cast<Result>(cast_a * cast_b));
+        ArithmeticUtil::Compute<std::multiplies>(cast_a, cast_b));
     TestValue(a_str + " / " + b_str, TYPE_DOUBLE,
         static_cast<double>(a) / static_cast<double>(b));
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/e0642c91/be/src/util/arithmetic-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/arithmetic-util.h b/be/src/util/arithmetic-util.h
index 93a0039..4ab60cd 100644
--- a/be/src/util/arithmetic-util.h
+++ b/be/src/util/arithmetic-util.h
@@ -99,6 +99,57 @@ class ArithmeticUtil {
     const auto a = ToUnsigned(x), b = ToUnsigned(y);
     return ToSigned(Operator<UnsignedType<T>>()(a, b));
   }
+
+  // Compute() is meant to be used like AsUnsigned(), but when the template context does
+  // not enforce that the type T is integral. For floating point types, it performs
+  // Operator (i.e. std::plus) without modification, and for integral types it calls
+  // AsUnsigned().
+  //
+  // It is needed because AsUnsigned<std::plus>(1.0, 2.0) does not compile, since
+  // UnsignedType<float> is not a valid type. In contrast, Compute<std::plus>(1.0, 2.0)
+  // does compile and performs the usual addition on 1.0 and 2.0 to produce 3.0.
+  template <template <typename> class Operator, typename T>
+  static T Compute(T x, T y) {
+    return OperateOn<T>::template Compute<Operator>(x, y);
+  }
+
+ private:
+  // Ring and OperateOn are used for compile-time dispatching on how Compute() should
+  // perform an arithmetic operation: as an unsigned integer operation, as a
+  // floating-point operation, or not at all.
+  //
+  // For example, OperatorOn<int>::Compute<std::plus> is really just an alias for
+  // AsUnsigned<std::plus, int>, while OperatorOn<float>::Compute<std::plus> is really
+  // just an alias for the usual addition operator on floats.
+  enum class Ring { INTEGER, FLOAT, NEITHER };
+
+  template <typename T,
+      Ring R = std::is_integral<T>::value ?
+          Ring::INTEGER :
+          (std::is_floating_point<T>::value ? Ring::FLOAT : Ring::NEITHER)>
+  struct OperateOn;
+};
+
+template <typename T>
+struct ArithmeticUtil::OperateOn<T, ArithmeticUtil::Ring::FLOAT> {
+  template <template <typename> class Operator>
+  static T Compute(T a, T b) {
+    return Operator<T>()(a, b);
+  }
+};
+
+template <typename T>
+struct ArithmeticUtil::OperateOn<T, ArithmeticUtil::Ring::INTEGER> {
+  template <template <typename> class Operator>
+  static T Compute(T x, T y) {
+    return AsUnsigned<Operator>(x, y);
+  }
+};
+
+template <typename T>
+struct ArithmeticUtil::OperateOn<T, ArithmeticUtil::Ring::NEITHER> {
+  template <template <typename> class Operator>
+  static T Compute(T x, T y) = delete;
 };
 
 class ArithmeticUtilTest {


[25/33] impala git commit: IMPALA-3652: Fix resource transfer in subplans with limits

Posted by bo...@apache.org.
IMPALA-3652: Fix resource transfer in subplans with limits

Impala assumes that when Reset() is called on an ExecNode, all of the
memory returned from that node by GetNext() has been attached to the
output RowBatch. In a query with a LIMIT on the subplan, such that
some nodes don't reach 'eos', this may not be the case.

The solution is to have Reset() take a RowBatch that any such memory
can be attached to. I examined all ExecNodes for resources being
transferred on 'eos' and added transferring of those resources in
Resst().

Testing:
- Added e2e tests that repro the issue for hash and nested loop joins.

Change-Id: I3968a379fcbb5d30fcec304995d3e44933dbbc77
Reviewed-on: http://gerrit.cloudera.org:8080/11852
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: ccabc491e20e566144f0b07739a65092c48a4953
Parents: ffdb8a1
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Wed Oct 24 22:16:21 2018 +0000
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:40 2018 +0100

----------------------------------------------------------------------
 be/src/exec/aggregation-node-base.cc            |  6 ++--
 be/src/exec/aggregation-node-base.h             |  2 +-
 be/src/exec/aggregator.h                        |  2 +-
 be/src/exec/analytic-eval-node.cc               | 16 +++++------
 be/src/exec/analytic-eval-node.h                |  2 +-
 be/src/exec/blocking-join-node.cc               |  5 ++++
 be/src/exec/blocking-join-node.h                |  3 ++
 be/src/exec/cardinality-check-node.cc           |  4 +--
 be/src/exec/cardinality-check-node.h            |  2 +-
 be/src/exec/data-source-scan-node.cc            |  2 +-
 be/src/exec/data-source-scan-node.h             |  2 +-
 be/src/exec/exchange-node.cc                    |  2 +-
 be/src/exec/exchange-node.h                     |  2 +-
 be/src/exec/exec-node.cc                        |  4 +--
 be/src/exec/exec-node.h                         |  5 ++--
 be/src/exec/grouping-aggregator.cc              |  7 ++++-
 be/src/exec/grouping-aggregator.h               |  2 +-
 be/src/exec/hbase-scan-node.cc                  |  2 +-
 be/src/exec/hbase-scan-node.h                   |  2 +-
 be/src/exec/hdfs-scan-node-base.cc              |  2 +-
 be/src/exec/hdfs-scan-node-base.h               |  3 +-
 be/src/exec/nested-loop-join-node.cc            |  4 +--
 be/src/exec/nested-loop-join-node.h             |  2 +-
 be/src/exec/non-grouping-aggregator.h           |  4 ++-
 be/src/exec/partial-sort-node.cc                |  4 +--
 be/src/exec/partial-sort-node.h                 |  2 +-
 be/src/exec/partitioned-hash-join-builder.cc    | 12 ++++----
 be/src/exec/partitioned-hash-join-builder.h     |  7 +++--
 be/src/exec/partitioned-hash-join-node.cc       | 28 +++++++++++-------
 be/src/exec/partitioned-hash-join-node.h        |  7 +++--
 be/src/exec/select-node.cc                      |  6 ++--
 be/src/exec/select-node.h                       |  2 +-
 be/src/exec/sort-node.cc                        |  4 +--
 be/src/exec/sort-node.h                         |  2 +-
 be/src/exec/streaming-aggregation-node.cc       |  2 +-
 be/src/exec/streaming-aggregation-node.h        |  2 +-
 be/src/exec/subplan-node.cc                     | 13 +++++----
 be/src/exec/subplan-node.h                      |  2 +-
 be/src/exec/topn-node.cc                        |  6 ++--
 be/src/exec/topn-node.h                         |  2 +-
 be/src/exec/union-node.cc                       |  4 +--
 be/src/exec/union-node.h                        |  2 +-
 be/src/exec/unnest-node.cc                      |  4 +--
 be/src/exec/unnest-node.h                       |  2 +-
 .../QueryTest/nested-types-tpch-limit.test      | 30 ++++++++++++++++++++
 tests/query_test/test_nested_types.py           |  5 ++++
 46 files changed, 149 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/aggregation-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node-base.cc b/be/src/exec/aggregation-node-base.cc
index 5de54be..cd6cfe3 100644
--- a/be/src/exec/aggregation-node-base.cc
+++ b/be/src/exec/aggregation-node-base.cc
@@ -73,10 +73,10 @@ void AggregationNodeBase::Codegen(RuntimeState* state) {
   for (auto& agg : aggs_) agg->Codegen(state);
 }
 
-Status AggregationNodeBase::Reset(RuntimeState* state) {
-  for (auto& agg : aggs_) RETURN_IF_ERROR(agg->Reset(state));
+Status AggregationNodeBase::Reset(RuntimeState* state, RowBatch* row_batch) {
+  for (auto& agg : aggs_) RETURN_IF_ERROR(agg->Reset(state, row_batch));
   curr_output_agg_idx_ = 0;
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 Status AggregationNodeBase::SplitMiniBatches(

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/aggregation-node-base.h
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node-base.h b/be/src/exec/aggregation-node-base.h
index 7dfab34..89508e0 100644
--- a/be/src/exec/aggregation-node-base.h
+++ b/be/src/exec/aggregation-node-base.h
@@ -35,7 +35,7 @@ class AggregationNodeBase : public ExecNode {
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override;
   virtual Status Prepare(RuntimeState* state) override;
   virtual void Codegen(RuntimeState* state) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
 
  protected:
   /// If true, the input to this node should be passed into each Aggregator in 'aggs_'.

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/aggregator.h
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregator.h b/be/src/exec/aggregator.h
index bd2f10e..7cb47c3 100644
--- a/be/src/exec/aggregator.h
+++ b/be/src/exec/aggregator.h
@@ -74,7 +74,7 @@ class Aggregator {
   virtual Status Open(RuntimeState* state) WARN_UNUSED_RESULT;
   virtual Status GetNext(
       RuntimeState* state, RowBatch* row_batch, bool* eos) WARN_UNUSED_RESULT = 0;
-  virtual Status Reset(RuntimeState* state) WARN_UNUSED_RESULT = 0;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) WARN_UNUSED_RESULT = 0;
   virtual void Close(RuntimeState* state);
 
   /// Adds all of the rows in 'batch' to the aggregation.

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 c7bcbda..0a4d601 100644
--- a/be/src/exec/analytic-eval-node.cc
+++ b/be/src/exec/analytic-eval-node.cc
@@ -810,7 +810,7 @@ Status AnalyticEvalNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool*
   return Status::OK();
 }
 
-Status AnalyticEvalNode::Reset(RuntimeState* state) {
+Status AnalyticEvalNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   result_tuples_.clear();
   window_tuples_.clear();
   last_result_idx_ = -1;
@@ -818,11 +818,9 @@ Status AnalyticEvalNode::Reset(RuntimeState* state) {
   prev_pool_last_result_idx_ = -1;
   prev_pool_last_window_idx_ = -1;
   input_eos_ = false;
-  // TODO: The Reset() contract allows calling Reset() even if eos has not been reached,
-  // but the analytic eval node currently does not support that. In practice, we only
-  // call Reset() after eos.
-  DCHECK_EQ(curr_tuple_pool_->total_allocated_bytes(), 0);
-  DCHECK_EQ(prev_tuple_pool_->total_allocated_bytes(), 0);
+  // Transfer the ownership of all row-backing resources.
+  row_batch->tuple_data_pool()->AcquireData(prev_tuple_pool_.get(), false);
+  row_batch->tuple_data_pool()->AcquireData(curr_tuple_pool_.get(), false);
   // Call Finalize() to clear evaluator allocations, but do not Close() them,
   // so we can keep evaluating them.
   if (curr_tuple_init_) {
@@ -831,12 +829,14 @@ Status AnalyticEvalNode::Reset(RuntimeState* state) {
   }
   // The following members will be re-created in Open().
   // input_stream_ should have been closed by last GetNext() call.
-  DCHECK(input_stream_ == nullptr || input_stream_->is_closed());
+  if (input_stream_ != nullptr && !input_stream_->is_closed()) {
+    input_stream_->Close(row_batch, RowBatch::FlushMode::FLUSH_RESOURCES);
+  }
   input_stream_.reset();
   prev_input_tuple_ = nullptr;
   prev_input_tuple_pool_->Clear();
   curr_child_batch_->Reset();
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void AnalyticEvalNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 0c185ae..3d295aa 100644
--- a/be/src/exec/analytic-eval-node.h
+++ b/be/src/exec/analytic-eval-node.h
@@ -70,7 +70,7 @@ class AnalyticEvalNode : public ExecNode {
   virtual Status Prepare(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  protected:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/blocking-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/blocking-join-node.cc b/be/src/exec/blocking-join-node.cc
index 88ca007..e9281c6 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -137,6 +137,11 @@ Status BlockingJoinNode::Prepare(RuntimeState* state) {
   return Status::OK();
 }
 
+Status BlockingJoinNode::Reset(RuntimeState* state, RowBatch* row_batch) {
+  probe_batch_->TransferResourceOwnership(row_batch);
+  return ExecNode::Reset(state, row_batch);
+}
+
 void BlockingJoinNode::Close(RuntimeState* state) {
   if (is_closed()) return;
   build_batch_.reset();

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/blocking-join-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/blocking-join-node.h b/be/src/exec/blocking-join-node.h
index 8198ad0..2f1c161 100644
--- a/be/src/exec/blocking-join-node.h
+++ b/be/src/exec/blocking-join-node.h
@@ -57,6 +57,9 @@ class BlockingJoinNode : public ExecNode {
   /// Calls ExecNode::Open() and initializes 'eos_' and 'probe_side_eos_'.
   virtual Status Open(RuntimeState* state);
 
+  /// Transfers resources from 'probe_batch_' to 'row_batch'.
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
+
   /// Subclasses should close any other structures and then call
   /// BlockingJoinNode::Close().
   virtual void Close(RuntimeState* state);

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/cardinality-check-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/cardinality-check-node.cc b/be/src/exec/cardinality-check-node.cc
index 76579dd..016b0ef 100644
--- a/be/src/exec/cardinality-check-node.cc
+++ b/be/src/exec/cardinality-check-node.cc
@@ -95,9 +95,9 @@ Status CardinalityCheckNode::GetNext(RuntimeState* state, RowBatch* output_row_b
   return Status::OK();
 }
 
-Status CardinalityCheckNode::Reset(RuntimeState* state) {
+Status CardinalityCheckNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   row_batch_->Reset();
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void CardinalityCheckNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/cardinality-check-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/cardinality-check-node.h b/be/src/exec/cardinality-check-node.h
index c71bd2b..d44efd1 100644
--- a/be/src/exec/cardinality-check-node.h
+++ b/be/src/exec/cardinality-check-node.h
@@ -40,7 +40,7 @@ class CardinalityCheckNode : public ExecNode {
   virtual Status Prepare(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
   virtual void Close(RuntimeState* state) override;
  private:
   /////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/data-source-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-source-scan-node.cc b/be/src/exec/data-source-scan-node.cc
index 10f0c7d..1f2a003 100644
--- a/be/src/exec/data-source-scan-node.cc
+++ b/be/src/exec/data-source-scan-node.cc
@@ -367,7 +367,7 @@ Status DataSourceScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, boo
   }
 }
 
-Status DataSourceScanNode::Reset(RuntimeState* state) {
+Status DataSourceScanNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "NYI";
   return Status("NYI");
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/data-source-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/data-source-scan-node.h b/be/src/exec/data-source-scan-node.h
index 065fcf3..e8450fe 100644
--- a/be/src/exec/data-source-scan-node.h
+++ b/be/src/exec/data-source-scan-node.h
@@ -55,7 +55,7 @@ class DataSourceScanNode : public ScanNode {
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
 
   /// NYI
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
 
   /// Close the scanner, and report errors.
   virtual void Close(RuntimeState* state) override;

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/exchange-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.cc b/be/src/exec/exchange-node.cc
index ca8b973..affd93c 100644
--- a/be/src/exec/exchange-node.cc
+++ b/be/src/exec/exchange-node.cc
@@ -127,7 +127,7 @@ Status ExchangeNode::Open(RuntimeState* state) {
   return Status::OK();
 }
 
-Status ExchangeNode::Reset(RuntimeState* state) {
+Status ExchangeNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "NYI";
   return Status("NYI");
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/exchange-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.h b/be/src/exec/exchange-node.h
index bdc52c6..e9f1d00 100644
--- a/be/src/exec/exchange-node.h
+++ b/be/src/exec/exchange-node.h
@@ -52,7 +52,7 @@ class ExchangeNode : public ExecNode {
   /// 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);
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
   /// the number of senders needs to be set after the c'tor, because it's not

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 3a606f5..d8495ac 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -145,10 +145,10 @@ Status ExecNode::Open(RuntimeState* state) {
   return ScalarExprEvaluator::Open(conjunct_evals_, state);
 }
 
-Status ExecNode::Reset(RuntimeState* state) {
+Status ExecNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   num_rows_returned_ = 0;
   for (int i = 0; i < children_.size(); ++i) {
-    RETURN_IF_ERROR(children_[i]->Reset(state));
+    RETURN_IF_ERROR(children_[i]->Reset(state, row_batch));
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/exec-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index a62ed6c..1bbb095 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -111,7 +111,8 @@ class ExecNode {
   /// Resets the stream of row batches to be retrieved by subsequent GetNext() calls.
   /// Clears all internal state, returning this node to the state it was in after calling
   /// Prepare() and before calling Open(). This function must not clear memory
-  /// still owned by this node that is backing rows returned in GetNext().
+  /// still owned by this node that is backing rows returned in GetNext(). 'row_batch' can
+  /// be used to transfer ownership of any such memory.
   /// Prepare() and Open() must have already been called before calling Reset().
   /// GetNext() may have optionally been called (not necessarily until eos).
   /// Close() must not have been called.
@@ -121,7 +122,7 @@ class ExecNode {
   /// implementation calls Reset() on children.
   /// Note that this function may be called many times (proportional to the input data),
   /// so should be fast.
-  virtual Status Reset(RuntimeState* state) WARN_UNUSED_RESULT;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) WARN_UNUSED_RESULT;
 
   /// Close() will get called for every exec node, regardless of what else is called and
   /// the status of these calls (i.e. Prepare() may never have been called, or

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/grouping-aggregator.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/grouping-aggregator.cc b/be/src/exec/grouping-aggregator.cc
index 9b38a0c..382a6ae 100644
--- a/be/src/exec/grouping-aggregator.cc
+++ b/be/src/exec/grouping-aggregator.cc
@@ -364,12 +364,17 @@ void GroupingAggregator::CleanupHashTbl(
   }
 }
 
-Status GroupingAggregator::Reset(RuntimeState* state) {
+Status GroupingAggregator::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(!is_streaming_preagg_) << "Cannot reset preaggregation";
   partition_eos_ = false;
   streaming_idx_ = 0;
   // Reset the HT and the partitions for this grouping agg.
   ht_ctx_->set_level(0);
+  if (output_partition_ != nullptr) {
+    // Attach all buffers referenced by previously-returned rows.
+    output_partition_->aggregated_row_stream->Close(
+        row_batch, RowBatch::FlushMode::FLUSH_RESOURCES);
+  }
   ClosePartitions();
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/grouping-aggregator.h
----------------------------------------------------------------------
diff --git a/be/src/exec/grouping-aggregator.h b/be/src/exec/grouping-aggregator.h
index 6f630d4..405d4ab 100644
--- a/be/src/exec/grouping-aggregator.h
+++ b/be/src/exec/grouping-aggregator.h
@@ -124,7 +124,7 @@ class GroupingAggregator : public Aggregator {
   virtual void Codegen(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
   virtual void Close(RuntimeState* state) override;
 
   virtual Status AddBatch(RuntimeState* state, RowBatch* batch) override;

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/hbase-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hbase-scan-node.cc b/be/src/exec/hbase-scan-node.cc
index 148be66..f379823 100644
--- a/be/src/exec/hbase-scan-node.cc
+++ b/be/src/exec/hbase-scan-node.cc
@@ -266,7 +266,7 @@ Status HBaseScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eo
   return Status::OK();
 }
 
-Status HBaseScanNode::Reset(RuntimeState* state) {
+Status HBaseScanNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "NYI";
   return Status("NYI");
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/hbase-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hbase-scan-node.h b/be/src/exec/hbase-scan-node.h
index dbe7ff1..bd9502e 100644
--- a/be/src/exec/hbase-scan-node.h
+++ b/be/src/exec/hbase-scan-node.h
@@ -51,7 +51,7 @@ class HBaseScanNode : public ScanNode {
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
 
   /// NYI
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
 
   /// Close the hbase_scanner_, and report errors.
   virtual void Close(RuntimeState* state) override;

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 4a39124..a78bdb2 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -403,7 +403,7 @@ Status HdfsScanNodeBase::Open(RuntimeState* state) {
   return Status::OK();
 }
 
-Status HdfsScanNodeBase::Reset(RuntimeState* state) {
+Status HdfsScanNodeBase::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "Internal error: Scan nodes should not appear in subplans.";
   return Status("Internal error: Scan nodes should not appear in subplans.");
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 0a5a328..4f94b11 100644
--- a/be/src/exec/hdfs-scan-node-base.h
+++ b/be/src/exec/hdfs-scan-node-base.h
@@ -152,7 +152,8 @@ class HdfsScanNodeBase : public ScanNode {
   virtual Status Prepare(RuntimeState* state) override WARN_UNUSED_RESULT;
   virtual void Codegen(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override WARN_UNUSED_RESULT;
-  virtual Status Reset(RuntimeState* state) override WARN_UNUSED_RESULT;
+  virtual Status Reset(
+      RuntimeState* state, RowBatch* row_batch) override WARN_UNUSED_RESULT;
   virtual void Close(RuntimeState* state) override;
 
   /// Returns true if this node uses separate threads for scanners that append RowBatches

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 e0a375b..1e1face 100644
--- a/be/src/exec/nested-loop-join-node.cc
+++ b/be/src/exec/nested-loop-join-node.cc
@@ -124,14 +124,14 @@ Status NestedLoopJoinNode::Prepare(RuntimeState* state) {
   return Status::OK();
 }
 
-Status NestedLoopJoinNode::Reset(RuntimeState* state) {
+Status NestedLoopJoinNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   builder_->Reset();
   build_batches_ = NULL;
   matched_probe_ = false;
   current_probe_row_ = NULL;
   probe_batch_pos_ = 0;
   process_unmatched_build_rows_ = false;
-  return BlockingJoinNode::Reset(state);
+  return BlockingJoinNode::Reset(state, row_batch);
 }
 
 void NestedLoopJoinNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/nested-loop-join-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/nested-loop-join-node.h b/be/src/exec/nested-loop-join-node.h
index c94abbf..6fc09dc 100644
--- a/be/src/exec/nested-loop-join-node.h
+++ b/be/src/exec/nested-loop-join-node.h
@@ -48,7 +48,7 @@ class NestedLoopJoinNode : public BlockingJoinNode {
   virtual Status Prepare(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  private:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/non-grouping-aggregator.h
----------------------------------------------------------------------
diff --git a/be/src/exec/non-grouping-aggregator.h b/be/src/exec/non-grouping-aggregator.h
index 49a663d..3f18a71 100644
--- a/be/src/exec/non-grouping-aggregator.h
+++ b/be/src/exec/non-grouping-aggregator.h
@@ -48,7 +48,9 @@ class NonGroupingAggregator : public Aggregator {
   virtual void Codegen(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override { return Status::OK(); }
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override {
+    return Status::OK();
+  }
   virtual void Close(RuntimeState* state) override;
 
   virtual Status AddBatch(RuntimeState* state, RowBatch* batch) override;

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/partial-sort-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partial-sort-node.cc b/be/src/exec/partial-sort-node.cc
index 7e3c4d7..81445de 100644
--- a/be/src/exec/partial-sort-node.cc
+++ b/be/src/exec/partial-sort-node.cc
@@ -141,9 +141,9 @@ Status PartialSortNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool*
   return Status::OK();
 }
 
-Status PartialSortNode::Reset(RuntimeState* state) {
+Status PartialSortNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "PartialSortNode cannot be part of a subplan.";
-  return ExecNode::Reset(state);
+  return Status("Cannot reset partial sort");
 }
 
 void PartialSortNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/partial-sort-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partial-sort-node.h b/be/src/exec/partial-sort-node.h
index 421df31..156d574 100644
--- a/be/src/exec/partial-sort-node.h
+++ b/be/src/exec/partial-sort-node.h
@@ -50,7 +50,7 @@ class PartialSortNode : public ExecNode {
   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);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  protected:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 7754c99..2232108 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -236,7 +236,7 @@ Status PhjBuilder::FlushFinal(RuntimeState* state) {
 
 void PhjBuilder::Close(RuntimeState* state) {
   if (closed_) return;
-  CloseAndDeletePartitions();
+  CloseAndDeletePartitions(nullptr);
   if (ht_ctx_ != nullptr) ht_ctx_->Close(state);
   ht_ctx_.reset();
   for (const FilterContext& ctx : filter_ctxs_) {
@@ -249,10 +249,10 @@ void PhjBuilder::Close(RuntimeState* state) {
   closed_ = true;
 }
 
-void PhjBuilder::Reset() {
+void PhjBuilder::Reset(RowBatch* row_batch) {
   expr_results_pool_->Clear();
   non_empty_build_ = false;
-  CloseAndDeletePartitions();
+  CloseAndDeletePartitions(row_batch);
 }
 
 Status PhjBuilder::CreateAndPreparePartition(int level, Partition** partition) {
@@ -441,14 +441,14 @@ vector<unique_ptr<BufferedTupleStream>> PhjBuilder::TransferProbeStreams() {
   return std::move(spilled_partition_probe_streams_);
 }
 
-void PhjBuilder::CloseAndDeletePartitions() {
+void PhjBuilder::CloseAndDeletePartitions(RowBatch* row_batch) {
   // Close all the partitions and clean up all references to them.
-  for (unique_ptr<Partition>& partition : all_partitions_) partition->Close(NULL);
+  for (unique_ptr<Partition>& partition : all_partitions_) partition->Close(row_batch);
   all_partitions_.clear();
   hash_partitions_.clear();
   null_aware_partition_ = NULL;
   for (unique_ptr<BufferedTupleStream>& stream : spilled_partition_probe_streams_) {
-    stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+    stream->Close(row_batch, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
   }
   spilled_partition_probe_streams_.clear();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 f6105fc..147504b 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -98,7 +98,7 @@ class PhjBuilder : public DataSink {
   /////////////////////////////////////////
 
   /// Reset the builder to the same state as it was in after calling Open().
-  void Reset();
+  void Reset(RowBatch* row_batch);
 
   /// Transfer ownership of the probe streams to the caller. One stream was allocated per
   /// spilled partition in FlushFinal(). The probe streams are empty but prepared for
@@ -333,8 +333,9 @@ class PhjBuilder : public DataSink {
   Status InitSpilledPartitionProbeStreams() WARN_UNUSED_RESULT;
 
   /// Calls Close() on every Partition, deletes them, and cleans up any pointers that
-  /// may reference them. Also cleans up 'spilled_partition_probe_streams_'.
-  void CloseAndDeletePartitions();
+  /// may reference them. Also cleans up 'spilled_partition_probe_streams_'. If
+  /// 'row_batch' if not NULL, transfers the ownership of all row-backing resources to it.
+  void CloseAndDeletePartitions(RowBatch* row_batch);
 
   /// For each filter in filters_, allocate a bloom_filter from the fragment-local
   /// RuntimeFilterBank and store it in runtime_filters_ to populate during the build

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 2b36990..2ef5c08 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -197,7 +197,7 @@ Status PartitionedHashJoinNode::AcquireResourcesForBuild(RuntimeState* state) {
   return Status::OK();
 }
 
-Status PartitionedHashJoinNode::Reset(RuntimeState* state) {
+Status PartitionedHashJoinNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
     null_probe_output_idx_ = -1;
     matched_null_probe_.clear();
@@ -205,35 +205,41 @@ Status PartitionedHashJoinNode::Reset(RuntimeState* state) {
   }
   state_ = PARTITIONING_BUILD;
   ht_ctx_->set_level(0);
-  CloseAndDeletePartitions();
-  builder_->Reset();
+  CloseAndDeletePartitions(row_batch);
+  builder_->Reset(row_batch);
   memset(hash_tbls_, 0, sizeof(HashTable*) * PARTITION_FANOUT);
+  if (output_unmatched_batch_ != nullptr) {
+    output_unmatched_batch_->TransferResourceOwnership(row_batch);
+  }
   output_unmatched_batch_.reset();
   output_unmatched_batch_iter_.reset();
-  return ExecNode::Reset(state);
+  return BlockingJoinNode::Reset(state, row_batch);
 }
 
-void PartitionedHashJoinNode::CloseAndDeletePartitions() {
+void PartitionedHashJoinNode::CloseAndDeletePartitions(RowBatch* row_batch) {
   // Close all the partitions and clean up all references to them.
   for (unique_ptr<ProbePartition>& partition : probe_hash_partitions_) {
-    if (partition != NULL) partition->Close(NULL);
+    if (partition != NULL) partition->Close(row_batch);
   }
   probe_hash_partitions_.clear();
   for (unique_ptr<ProbePartition>& partition : spilled_partitions_) {
-    partition->Close(NULL);
+    partition->Close(row_batch);
   }
   spilled_partitions_.clear();
   if (input_partition_ != NULL) {
-    input_partition_->Close(NULL);
+    input_partition_->Close(row_batch);
     input_partition_.reset();
   }
   if (null_aware_probe_partition_ != NULL) {
-    null_aware_probe_partition_->Close(NULL);
+    null_aware_probe_partition_->Close(row_batch);
     null_aware_probe_partition_.reset();
   }
+  for (PhjBuilder::Partition* partition : output_build_partitions_) {
+    partition->Close(row_batch);
+  }
   output_build_partitions_.clear();
   if (null_probe_rows_ != NULL) {
-    null_probe_rows_->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+    null_probe_rows_->Close(row_batch, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
     null_probe_rows_.reset();
   }
 }
@@ -245,7 +251,7 @@ void PartitionedHashJoinNode::Close(RuntimeState* state) {
   output_null_aware_probe_rows_running_ = false;
   output_unmatched_batch_.reset();
   output_unmatched_batch_iter_.reset();
-  CloseAndDeletePartitions();
+  CloseAndDeletePartitions(nullptr);
   if (builder_ != nullptr) builder_->Close(state);
   ScalarExprEvaluator::Close(other_join_conjunct_evals_, state);
   ScalarExpr::Close(build_exprs_);

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/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 a0c03ef..8ad4d4a 100644
--- a/be/src/exec/partitioned-hash-join-node.h
+++ b/be/src/exec/partitioned-hash-join-node.h
@@ -107,7 +107,7 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   virtual void Codegen(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
   virtual void Close(RuntimeState* state) override;
 
  protected:
@@ -383,8 +383,9 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   Status NullAwareAntiJoinError(BufferedTupleStream* rows);
 
   /// Calls Close() on every probe partition, destroys the partitions and cleans up any
-  /// references to the partitions. Also closes and destroys 'null_probe_rows_'.
-  void CloseAndDeletePartitions();
+  /// references to the partitions. Also closes and destroys 'null_probe_rows_'. If
+  /// 'row_batch' is not NULL, transfers ownership of all row-backing resources to it.
+  void CloseAndDeletePartitions(RowBatch* row_batch);
 
   /// Prepares for probing the next batch.
   void ResetForProbe();

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/select-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/select-node.cc b/be/src/exec/select-node.cc
index 21cfa4b..d5cb7f0 100644
--- a/be/src/exec/select-node.cc
+++ b/be/src/exec/select-node.cc
@@ -111,11 +111,11 @@ Status SelectNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos)
   return Status::OK();
 }
 
-Status SelectNode::Reset(RuntimeState* state) {
-  child_row_batch_->Reset();
+Status SelectNode::Reset(RuntimeState* state, RowBatch* row_batch) {
+  child_row_batch_->TransferResourceOwnership(row_batch);
   child_row_idx_ = 0;
   child_eos_ = false;
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void SelectNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/select-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/select-node.h b/be/src/exec/select-node.h
index ce85cfb..c4d8098 100644
--- a/be/src/exec/select-node.h
+++ b/be/src/exec/select-node.h
@@ -40,7 +40,7 @@ class SelectNode : public ExecNode {
   virtual void Codegen(RuntimeState* state) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
   virtual void Close(RuntimeState* state) override;
 
  private:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/sort-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/sort-node.cc b/be/src/exec/sort-node.cc
index f3e3ffc..3a70c29 100644
--- a/be/src/exec/sort-node.cc
+++ b/be/src/exec/sort-node.cc
@@ -142,10 +142,10 @@ Status SortNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
   return Status::OK();
 }
 
-Status SortNode::Reset(RuntimeState* state) {
+Status SortNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   num_rows_skipped_ = 0;
   if (sorter_.get() != NULL) sorter_->Reset();
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void SortNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/sort-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/sort-node.h b/be/src/exec/sort-node.h
index 8075b8e..ce7b188 100644
--- a/be/src/exec/sort-node.h
+++ b/be/src/exec/sort-node.h
@@ -42,7 +42,7 @@ class SortNode : public ExecNode {
   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);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  protected:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/streaming-aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/streaming-aggregation-node.cc b/be/src/exec/streaming-aggregation-node.cc
index 42f3e9f..d2608ad 100644
--- a/be/src/exec/streaming-aggregation-node.cc
+++ b/be/src/exec/streaming-aggregation-node.cc
@@ -180,7 +180,7 @@ Status StreamingAggregationNode::GetRowsStreaming(
   return Status::OK();
 }
 
-Status StreamingAggregationNode::Reset(RuntimeState* state) {
+Status StreamingAggregationNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   DCHECK(false) << "Cannot reset preaggregation";
   return Status("Cannot reset preaggregation");
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/streaming-aggregation-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/streaming-aggregation-node.h b/be/src/exec/streaming-aggregation-node.h
index 71193b9..4b1ca74 100644
--- a/be/src/exec/streaming-aggregation-node.h
+++ b/be/src/exec/streaming-aggregation-node.h
@@ -50,7 +50,7 @@ class StreamingAggregationNode : public AggregationNodeBase {
 
   virtual Status Open(RuntimeState* state) override;
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-  virtual Status Reset(RuntimeState* state) override;
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch) override;
   virtual void Close(RuntimeState* state) override;
 
   virtual void DebugString(int indentation_level, std::stringstream* out) const override;

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/subplan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/subplan-node.cc b/be/src/exec/subplan-node.cc
index 98ef13a..fdd9660 100644
--- a/be/src/exec/subplan-node.cc
+++ b/be/src/exec/subplan-node.cc
@@ -87,9 +87,9 @@ Status SubplanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos)
   while (true) {
     if (subplan_is_open_) {
       if (subplan_eos_) {
-        // Reset the subplan before opening it again. At this point, all resources from
-        // the subplan are assumed to have been transferred to the output row_batch.
-        RETURN_IF_ERROR(child(1)->Reset(state));
+        // Reset the subplan before opening it again. 'row_batch' is passed in to allow
+        // any remaining resources to be transferred to it.
+        RETURN_IF_ERROR(child(1)->Reset(state, row_batch));
         subplan_is_open_ = false;
       } else {
         // Continue fetching rows from the open subplan into the output row_batch.
@@ -140,16 +140,17 @@ Status SubplanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos)
   return Status::OK();
 }
 
-Status SubplanNode::Reset(RuntimeState* state) {
+Status SubplanNode::Reset(RuntimeState* state, RowBatch* row_batch) {
+  input_batch_->TransferResourceOwnership(row_batch);
   input_eos_ = false;
   input_row_idx_ = 0;
   subplan_eos_ = false;
   num_rows_returned_ = 0;
-  RETURN_IF_ERROR(child(0)->Reset(state));
+  RETURN_IF_ERROR(child(0)->Reset(state, row_batch));
   // If child(1) is not open it means that we have just Reset() it and returned from
   // GetNext() without opening it again. It is not safe to call Reset() on the same
   // exec node twice in a row.
-  if (subplan_is_open_) RETURN_IF_ERROR(child(1)->Reset(state));
+  if (subplan_is_open_) RETURN_IF_ERROR(child(1)->Reset(state, row_batch));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/subplan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/subplan-node.h b/be/src/exec/subplan-node.h
index bf13ca1..ecf7c71 100644
--- a/be/src/exec/subplan-node.h
+++ b/be/src/exec/subplan-node.h
@@ -54,7 +54,7 @@ class SubplanNode : public ExecNode {
   virtual Status Prepare(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  private:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/topn-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.cc b/be/src/exec/topn-node.cc
index c616a89..961416f 100644
--- a/be/src/exec/topn-node.cc
+++ b/be/src/exec/topn-node.cc
@@ -212,12 +212,14 @@ Status TopNNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
   return Status::OK();
 }
 
-Status TopNNode::Reset(RuntimeState* state) {
+Status TopNNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   priority_queue_.clear();
   num_rows_skipped_ = 0;
+  // Transfer ownership of tuple data to output batch.
+  row_batch->tuple_data_pool()->AcquireData(tuple_pool_.get(), false);
   // We deliberately do not free the tuple_pool_ here to allow selective transferring
   // of resources in the future.
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void TopNNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/topn-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.h b/be/src/exec/topn-node.h
index e12cfc3..a80d766 100644
--- a/be/src/exec/topn-node.h
+++ b/be/src/exec/topn-node.h
@@ -47,7 +47,7 @@ class TopNNode : public ExecNode {
   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);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  protected:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/union-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/union-node.cc b/be/src/exec/union-node.cc
index 73e7a29..1caa2a6 100644
--- a/be/src/exec/union-node.cc
+++ b/be/src/exec/union-node.cc
@@ -313,7 +313,7 @@ Status UnionNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
   return Status::OK();
 }
 
-Status UnionNode::Reset(RuntimeState* state) {
+Status UnionNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   child_idx_ = 0;
   child_batch_.reset();
   child_row_idx_ = 0;
@@ -322,7 +322,7 @@ Status UnionNode::Reset(RuntimeState* state) {
   // Since passthrough is disabled in subplans, verify that there is no passthrough child
   // that needs to be closed.
   DCHECK_EQ(to_close_child_idx_, -1);
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void UnionNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/union-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/union-node.h b/be/src/exec/union-node.h
index 9c83276..b459392 100644
--- a/be/src/exec/union-node.h
+++ b/be/src/exec/union-node.h
@@ -52,7 +52,7 @@ class UnionNode : public ExecNode {
   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);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
  private:

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/unnest-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/unnest-node.cc b/be/src/exec/unnest-node.cc
index ef09731..52dd94d 100644
--- a/be/src/exec/unnest-node.cc
+++ b/be/src/exec/unnest-node.cc
@@ -175,9 +175,9 @@ Status UnnestNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos)
   return Status::OK();
 }
 
-Status UnnestNode::Reset(RuntimeState* state) {
+Status UnnestNode::Reset(RuntimeState* state, RowBatch* row_batch) {
   item_idx_ = 0;
-  return ExecNode::Reset(state);
+  return ExecNode::Reset(state, row_batch);
 }
 
 void UnnestNode::Close(RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/be/src/exec/unnest-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/unnest-node.h b/be/src/exec/unnest-node.h
index 247dd76..ff4d661 100644
--- a/be/src/exec/unnest-node.h
+++ b/be/src/exec/unnest-node.h
@@ -59,7 +59,7 @@ class UnnestNode : public ExecNode {
   virtual Status Prepare(RuntimeState* state);
   virtual Status Open(RuntimeState* state);
   virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
-  virtual Status Reset(RuntimeState* state);
+  virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
   virtual void Close(RuntimeState* state);
 
   /// Initializes the expression which produces the collection to be unnested.

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch-limit.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch-limit.test b/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch-limit.test
new file mode 100644
index 0000000..4ee5f59
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch-limit.test
@@ -0,0 +1,30 @@
+====
+---- QUERY
+use tpch_nested_parquet
+====
+---- QUERY
+# IMPALA-3652: test limit on a hash join in a subplan where resources need to be
+# transfered in Reset()
+select count(*)
+from customer c,
+  (select o1.o_orderkey, o2.o_orderdate
+   from c.c_orders o1, c.c_orders o2, c.c_orders o3
+   where o1.o_orderkey = o2.o_orderkey and o1.o_orderkey = o3.o_orderkey
+   limit 1) v
+where c_custkey = 113644;
+---- RESULTS
+1
+====
+---- QUERY
+# IMPALA-3652: test limit on a nested loop join in a subplan where resources need to be
+# transfered in Reset()
+select count(*)
+from customer c,
+  (select o1.o_orderkey, o2.o_orderdate
+   from c.c_orders o1 join /* +broadcast */ c.c_orders o2, c.c_orders o3
+   where o1.o_orderkey < o2.o_orderkey and o1.o_orderkey < o3.o_orderkey
+   limit 1) v
+where c_custkey = 113644;
+---- RESULTS
+1
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/ccabc491/tests/query_test/test_nested_types.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_nested_types.py b/tests/query_test/test_nested_types.py
index 4fccd9d..38a2582 100644
--- a/tests/query_test/test_nested_types.py
+++ b/tests/query_test/test_nested_types.py
@@ -83,6 +83,11 @@ class TestNestedTypes(ImpalaTestSuite):
     """Queries over the larger nested TPCH dataset."""
     self.run_test_case('QueryTest/nested-types-tpch', vector)
 
+  def test_tpch_limit(self, vector):
+    """Queries over the larger nested TPCH dataset with limits in their subplan."""
+    vector.get_value('exec_option')['batch_size'] = 10
+    self.run_test_case('QueryTest/nested-types-tpch-limit', vector)
+
   @SkipIfNotHdfsMinicluster.tuned_for_minicluster
   def test_tpch_mem_limit(self, vector):
     """Queries over the larger nested TPCH dataset with memory limits tuned for


[03/33] impala git commit: IMPALA-5031: memcpy cannot take null arguments

Posted by bo...@apache.org.
IMPALA-5031: memcpy cannot take null arguments

This patch fixes UBSAN "null pointer passed as argument" errors in
data loading. These are undefined behavior according to "7.1.4 Use of
library functions" in the C99 standard (which is included in C++14 in
section [intro.refs]):

    If an argument to a function has an invalid value (such as a value
    outside the domain of the function, or a pointer outside the
    address space of the program, or a null pointer, or a pointer to
    non-modifiable storage when the corresponding parameter is not
    const-qualified) or a type (after promotion) not expected by a
    function with variable number of arguments, the behavior is
    undefined.

The interesting parts of the backtraces for the errors fixed in this
patch are below:

    runtime/string-buffer.h:54:12: runtime error: null pointer passed as argument 1, which is declared to never be null
    /usr/include/string.h:43:45: note: nonnull attribute specified here
    StringBuffer::Append(char const*, long) runtime/string-buffer.h:54:5
    ColumnStatsBase::CopyToBuffer(StringBuffer*, StringValue*) exec/parquet-column-stats.cc:151:51
    ColumnStats<StringValue>::MaterializeStringValuesToInternalBuffers() exec/parquet-column-stats.inline.h:237:70
    HdfsParquetTableWriter::BaseColumnWriter::MaterializeStatsValues() exec/hdfs-parquet-table-writer.cc:149:63
    HdfsParquetTableWriter::AppendRows(RowBatch*, vector<int> const&, bool*) exec/hdfs-parquet-table-writer.cc:1129:53
    HdfsTableSink::WriteRowsToPartition(RuntimeState*, RowBatch*, pair<unique_ptr<OutputPartition, default_delete<OutputPartition> >, vector<int, all> > >*) exec/hdfs-table-sink.cc:256:71
    HdfsTableSink::Send(RuntimeState*, RowBatch*) exec/hdfs-table-sink.cc:591:45

    util/streaming-sampler.h:111:22: runtime error: null pointer passed as argument 2, which is declared to never be null
    /usr/include/string.h:43:45: note: nonnull attribute specified here
    StreamingSampler<long, 64>::SetSamples(int, vector<long> const&) util/streaming-sampler.h:111:5
    RuntimeProfile::Update(vector<TRuntimeProfileNode> const&, int*) util/runtime-profile.cc:313:30
    RuntimeProfile::Update(TRuntimeProfileTree const&) util/runtime-profile.cc:246:3
    Coordinator::BackendState::InstanceStats::Update(TFragmentInstanceExecStatus const&, Coordinator::ExecSummary*, ProgressUpdater*) runtime/coordinator-backend-state.cc:474:13
    Coordinator::BackendState::ApplyExecStatusReport(TReportExecStatusParams const&, Coordinator::ExecSummary*, ProgressUpdater*) runtime/coordinator-backend-state.cc:287:21
    Coordinator::UpdateBackendExecStatus(TReportExecStatusParams const&) runtime/coordinator.cc:679:22
    ClientRequestState::UpdateBackendExecStatus(TReportExecStatusParams const&) service/client-request-state.cc:1254:18
    ImpalaServer::ReportExecStatus(TReportExecStatusResult&, TReportExecStatusParams const&) service/impala-server.cc:1343:18
    ImpalaInternalService::ReportExecStatus(TReportExecStatusResult&, TReportExecStatusParams const&) service/impala-internal-service.cc:87:19

Change-Id: Ib9acc8c32409e67253a987eb3d1fd7d921efcb51
Reviewed-on: http://gerrit.cloudera.org:8080/11812
Reviewed-by: Jim Apple <jb...@apache.org>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: f90931e541d45f24cc305a521756fea618b7c4c5
Parents: c8c06dd
Author: Jim Apple <jb...@apache.org>
Authored: Sat Oct 27 16:57:55 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/runtime/string-buffer.h  | 3 ++-
 be/src/util/streaming-sampler.h | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/f90931e5/be/src/runtime/string-buffer.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/string-buffer.h b/be/src/runtime/string-buffer.h
index 2188e4d..8416405 100644
--- a/be/src/runtime/string-buffer.h
+++ b/be/src/runtime/string-buffer.h
@@ -23,6 +23,7 @@
 #include "runtime/mem-pool.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/string-value.h"
+#include "util/ubsan.h"
 
 namespace impala {
 
@@ -51,7 +52,7 @@ class StringBuffer {
   Status Append(const char* str, int64_t str_len) WARN_UNUSED_RESULT {
     int64_t new_len = len_ + str_len;
     if (UNLIKELY(new_len > buffer_size_)) RETURN_IF_ERROR(GrowBuffer(new_len));
-    memcpy(buffer_ + len_, str, str_len);
+    Ubsan::MemCpy(buffer_ + len_, str, str_len);
     len_ += str_len;
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/f90931e5/be/src/util/streaming-sampler.h
----------------------------------------------------------------------
diff --git a/be/src/util/streaming-sampler.h b/be/src/util/streaming-sampler.h
index dfa3dab..027f330 100644
--- a/be/src/util/streaming-sampler.h
+++ b/be/src/util/streaming-sampler.h
@@ -108,7 +108,7 @@ class StreamingSampler {
     boost::lock_guard<SpinLock> l(lock_);
     period_ = period;
     samples_collected_ = samples.size();
-    memcpy(samples_, samples.data(), sizeof(T) * samples_collected_);
+    Ubsan::MemCpy(samples_, samples.data(), sizeof(T) * samples_collected_);
     current_sample_sum_ = 0;
     current_sample_count_ = 0;
     current_sample_total_time_ = 0;


[31/33] impala git commit: IMPALA-7835: Role and user catalog objects with the same name can cause INVALIDATE METADATA to hang

Posted by bo...@apache.org.
IMPALA-7835: Role and user catalog objects with the same name can cause INVALIDATE METADATA to hang

Before this patch, enabling object ownership and running INVALIDATE
METADATA for the role and user catalog objects with the same principal
name could cause INVALIDATE METADATA to hang because a principal name
without a type is not guaranteed to be unique. This causes the catalog
delta logic to assume principal catalog objects with the same name but
with different types are considered the same causing a catalog version
inconsistency. This patch makes the principal object key to be unique by
appending the principal type information.

Testing:
- Added E2E authorization test
- Ran all FE tests
- Ran all E2E authorization tests

Change-Id: I516cf72e69e142a1349950cfca91f035c1ed445f
Reviewed-on: http://gerrit.cloudera.org:8080/11910
Reviewed-by: Vuk Ercegovac <ve...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 87c2ad0531d66ac7c253f340301b5782f3afe7fd
Parents: 4a10096
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Thu Nov 8 10:47:05 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:36 2018 +0100

----------------------------------------------------------------------
 be/src/catalog/catalog-util.cc                  | 26 +++++++++-
 .../java/org/apache/impala/catalog/Catalog.java | 12 ++++-
 .../org/apache/impala/catalog/Principal.java    |  5 +-
 tests/authorization/test_authorization.py       |  4 +-
 tests/authorization/test_grant_revoke.py        | 52 ++++++++++++++++++--
 5 files changed, 88 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/87c2ad05/be/src/catalog/catalog-util.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-util.cc b/be/src/catalog/catalog-util.cc
index 5084828..8cd2e2c 100644
--- a/be/src/catalog/catalog-util.cc
+++ b/be/src/catalog/catalog-util.cc
@@ -151,6 +151,8 @@ TCatalogObjectType::type TCatalogObjectTypeFromName(const string& name) {
 
 Status TCatalogObjectFromObjectName(const TCatalogObjectType::type& object_type,
     const string& object_name, TCatalogObject* catalog_object) {
+  // See Catalog::toCatalogObjectKey in Catalog.java for more information on the
+  // catalog object key format.
   switch (object_type) {
     case TCatalogObjectType::DATABASE:
       catalog_object->__set_type(object_type);
@@ -202,11 +204,31 @@ Status TCatalogObjectFromObjectName(const TCatalogObjectType::type& object_type,
       catalog_object->__set_cache_pool(THdfsCachePool());
       catalog_object->cache_pool.__set_pool_name(object_name);
       break;
-    case TCatalogObjectType::PRINCIPAL:
+    case TCatalogObjectType::PRINCIPAL: {
+      // The format is <principal name>.<principal type>
+      vector<string> split;
+      boost::split(split, object_name, [](char c) { return c == '.'; });
+      if (split.size() != 2) {
+        stringstream error_msg;
+        error_msg << "Invalid principal name: " << object_name;
+        return Status(error_msg.str());
+      }
+      string principal_name = split[0];
+      string principal_type = split[1];
       catalog_object->__set_type(object_type);
       catalog_object->__set_principal(TPrincipal());
-      catalog_object->principal.__set_principal_name(object_name);
+      catalog_object->principal.__set_principal_name(principal_name);
+      if (principal_type == "ROLE") {
+        catalog_object->principal.__set_principal_type(TPrincipalType::ROLE);
+      } else if (principal_type == "USER") {
+        catalog_object->principal.__set_principal_type(TPrincipalType::USER);
+      } else {
+        stringstream error_msg;
+        error_msg << "Invalid principal type: " << principal_type;
+        return Status(error_msg.str());
+      }
       break;
+    }
     case TCatalogObjectType::PRIVILEGE: {
       // The format is <privilege name>.<principal ID>.<principal type>
       vector<string> split;

http://git-wip-us.apache.org/repos/asf/impala/blob/87c2ad05/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 fe49d5b..4a14428 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -28,6 +28,7 @@ import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TFunction;
 import org.apache.impala.thrift.TPartitionKeyValue;
+import org.apache.impala.thrift.TPrincipalType;
 import org.apache.impala.thrift.TTable;
 import org.apache.impala.thrift.TTableName;
 import org.apache.impala.thrift.TUniqueId;
@@ -545,6 +546,7 @@ public abstract class Catalog {
    * Returns a unique string key of a catalog object.
    */
   public static String toCatalogObjectKey(TCatalogObject catalogObject) {
+    // TODO (IMPALA-7839): Refactor this method to reduce code repetition.
     Preconditions.checkNotNull(catalogObject);
     switch (catalogObject.getType()) {
       case DATABASE:
@@ -558,8 +560,14 @@ public abstract class Catalog {
         return "FUNCTION:" + catalogObject.getFn().getName() + "(" +
             catalogObject.getFn().getSignature() + ")";
       case PRINCIPAL:
-        return "PRINCIPAL:" + catalogObject.getPrincipal().getPrincipal_name()
-            .toLowerCase();
+        // It is important to make the principal object key unique since it is possible
+        // to have the same name for both role and user.
+        String principalName = catalogObject.getPrincipal().getPrincipal_name();
+        if (catalogObject.getPrincipal().getPrincipal_type() == TPrincipalType.ROLE) {
+          principalName = principalName.toLowerCase();
+        }
+        return "PRINCIPAL:" + principalName + "." +
+            catalogObject.getPrincipal().getPrincipal_type().name();
       case PRIVILEGE:
         // The combination of privilege name + principal ID + principal type is
         // guaranteed to be unique.

http://git-wip-us.apache.org/repos/asf/impala/blob/87c2ad05/fe/src/main/java/org/apache/impala/catalog/Principal.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Principal.java b/fe/src/main/java/org/apache/impala/catalog/Principal.java
index d048d10..8f65e95 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Principal.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Principal.java
@@ -159,8 +159,9 @@ public abstract class Principal extends CatalogObjectImpl {
 
   @Override
   public String getUniqueName() {
-    return this.getPrincipalType() == TPrincipalType.ROLE ? "ROLE:" : "USER:"
-        + getName().toLowerCase();
+    String principalName = getPrincipalType() == TPrincipalType.ROLE ?
+        getName().toLowerCase() : getName();
+    return "PRINCIPAL:" + principalName + "." + getPrincipalType().name();
   }
 
   public TCatalogObject toTCatalogObject() {

http://git-wip-us.apache.org/repos/asf/impala/blob/87c2ad05/tests/authorization/test_authorization.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_authorization.py b/tests/authorization/test_authorization.py
index a036680..e315f6d 100644
--- a/tests/authorization/test_authorization.py
+++ b/tests/authorization/test_authorization.py
@@ -456,7 +456,7 @@ class TestAuthorization(CustomClusterTestSuite):
       self.client.execute("grant select on database functional to role %s" % unique_role)
       for service in [self.cluster.catalogd.service,
                       self.cluster.get_first_impalad().service]:
-        obj_dump = service.get_catalog_object_dump("PRINCIPAL", unique_role)
+        obj_dump = service.get_catalog_object_dump("PRINCIPAL", "%s.ROLE" % unique_role)
         assert "catalog_version" in obj_dump
 
         # Get the privilege associated with that principal ID.
@@ -468,7 +468,7 @@ class TestAuthorization(CustomClusterTestSuite):
         assert "catalog_version" in obj_dump
 
         # Get the principal that does not exist.
-        obj_dump = service.get_catalog_object_dump("PRINCIPAL", "doesnotexist")
+        obj_dump = service.get_catalog_object_dump("PRINCIPAL", "doesnotexist.ROLE")
         assert "CatalogException" in obj_dump
 
         # Get the privilege that does not exist.

http://git-wip-us.apache.org/repos/asf/impala/blob/87c2ad05/tests/authorization/test_grant_revoke.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_grant_revoke.py b/tests/authorization/test_grant_revoke.py
index f7299bf..d568859 100644
--- a/tests/authorization/test_grant_revoke.py
+++ b/tests/authorization/test_grant_revoke.py
@@ -28,8 +28,9 @@ from tests.common.sentry_cache_test_suite import SentryCacheTestSuite, TestObjec
 from tests.util.calculation_util import get_random_id
 from tests.verifiers.metric_verifier import MetricVerifier
 
-SENTRY_CONFIG_FILE = getenv('IMPALA_HOME') + \
-    '/fe/src/test/resources/sentry-site.xml'
+SENTRY_CONFIG_DIR = "%s/%s" % (getenv("IMPALA_HOME"), "fe/src/test/resources")
+SENTRY_CONFIG_FILE = "%s/sentry-site.xml" % SENTRY_CONFIG_DIR
+SENTRY_CONFIG_FILE_OO = "%s/sentry-site_oo.xml" % SENTRY_CONFIG_DIR
 
 # Sentry long polling frequency to make Sentry refresh not run.
 SENTRY_LONG_POLLING_FREQUENCY_S = 3600
@@ -306,7 +307,7 @@ class TestGrantRevoke(SentryCacheTestSuite):
       impalad_args="--server_name=server1",
       catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE),
       sentry_config=SENTRY_CONFIG_FILE)
-  def test_invalidate_metadata(self, unique_role):
+  def test_role_name_case_insensitive_invalidate_metadata(self, unique_role):
     """IMPALA-7729: Tests running invalidate metadata with role names that have different
     case sensitivity."""
     for role_name in [unique_role.lower(), unique_role.upper(), unique_role.capitalize()]:
@@ -322,3 +323,48 @@ class TestGrantRevoke(SentryCacheTestSuite):
         assert self.client.wait_for_finished_timeout(handle, timeout=60)
       finally:
         self.client.execute("drop role {0}".format(role_name))
+
+  @pytest.mark.execute_serially
+  @SentryCacheTestSuite.with_args(
+      impalad_args="--server_name=server1 --sentry_config={0} "
+                   "--authorization_policy_provider_class="
+                   "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
+                   .format(SENTRY_CONFIG_FILE_OO),
+      catalogd_args="--sentry_config={0} "
+                    "--authorization_policy_provider_class="
+                    "org.apache.impala.service.CustomClusterResourceAuthorizationProvider"
+                    .format(SENTRY_CONFIG_FILE_OO),
+      sentry_config=SENTRY_CONFIG_FILE_OO)
+  def test_same_name_for_role_and_user_invalidate_metadata(self, testid_checksum):
+    """IMPALA-7729: Tests running invalidate metadata with for the same name for both
+    user and role should not cause Impala query to hang."""
+    db_prefix = testid_checksum
+    role_name = "foobar"
+    # Use two different clients so that the sessions will use two different user names.
+    foobar_impalad_client = self.create_impala_client()
+    FOOBAR_impalad_client = self.create_impala_client()
+    try:
+      # This will create "foobar" role catalog object.
+      self.client.execute("create role {0}".format(role_name))
+      self.client.execute("grant all on server to {0}".format(role_name))
+      self.client.execute("grant role {0} to group `{1}`".format(
+          role_name, grp.getgrnam(getuser()).gr_name))
+
+      # User names are case sensitive, so "foobar" and "FOOBAR" users should be treated
+      # as two different catalog objects.
+
+      # This will create "foobar" user catalog object.
+      self.user_query(foobar_impalad_client, "create database {0}_db1"
+                      .format(db_prefix, user="foobar"))
+      # This will create "FOOBAR" user catalog object.
+      self.user_query(FOOBAR_impalad_client, "create database {0}_db2"
+                      .format(db_prefix, user="FOOBAR"))
+
+      # Verify that running invalidate metadata won't hang due to having the same name
+      # in both user and role.
+      handle = self.client.execute_async("invalidate metadata")
+      assert self.client.wait_for_finished_timeout(handle, timeout=60)
+    finally:
+      self.client.execute("drop database {0}_db1".format(db_prefix))
+      self.client.execute("drop database {0}_db2".format(db_prefix))
+      self.client.execute("drop role {0}".format(role_name))


[10/33] impala git commit: IMPALA-402: test for random partitioning in insert

Posted by bo...@apache.org.
IMPALA-402: test for random partitioning in insert

This adds a basic regression test for the bug reported in IMPALA-402.

Testing:
Exhaustive build.

Looped the modified test overnight.

Change-Id: I4bbca5c64977cadf79dabd72f0c8876a40fdf410
Reviewed-on: http://gerrit.cloudera.org:8080/11799
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 8e9298ccf254311f49e4adb2206813094745c479
Parents: e0642c9
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Oct 25 11:40:17 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 .../QueryTest/insert-random-partition.test      | 33 ++++++++++++++++++++
 tests/query_test/test_insert.py                 |  8 +++++
 2 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/8e9298cc/testdata/workloads/functional-query/queries/QueryTest/insert-random-partition.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/insert-random-partition.test b/testdata/workloads/functional-query/queries/QueryTest/insert-random-partition.test
new file mode 100644
index 0000000..aa89c46
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert-random-partition.test
@@ -0,0 +1,33 @@
+====
+---- QUERY
+create table t (i int) partitioned by (p string)
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+# IMPALA-402: dynamic partitioning with random functions should behave sanely. Use uuid()
+# to guarantee that the partition keys will be unique.
+insert into t partition(p)
+select id, uuid() from functional.alltypestiny
+====
+---- QUERY
+select count(distinct p)
+from t
+---- TYPES
+BIGINT
+---- RESULTS
+8
+====
+---- QUERY
+# IMPALA-402: also test without clustering
+insert into t partition(p) /*+noclustered*/
+select id, uuid() from functional.alltypestiny
+====
+---- QUERY
+select count(distinct p)
+from t
+---- TYPES
+BIGINT
+---- RESULTS
+16
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/8e9298cc/tests/query_test/test_insert.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_insert.py b/tests/query_test/test_insert.py
index 8ded894..f34ba51 100644
--- a/tests/query_test/test_insert.py
+++ b/tests/query_test/test_insert.py
@@ -22,6 +22,7 @@ import pytest
 from testdata.common import widetable
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.parametrize import UniqueDatabase
 from tests.common.skip import SkipIfABFS, SkipIfEC, SkipIfLocal, SkipIfNotHdfsMinicluster
 from tests.common.test_dimensions import (
     create_exec_option_dimension,
@@ -149,6 +150,13 @@ class TestInsertQueries(ImpalaTestSuite):
       self.run_test_case('QueryTest/insert_bad_expr', vector,
           multiple_impalad=vector.get_value('exec_option')['sync_ddl'] == 1)
 
+  @UniqueDatabase.parametrize(sync_ddl=True)
+  def test_insert_random_partition(self, vector, unique_database):
+    """Regression test for IMPALA-402: partitioning by rand() leads to strange behaviour
+    or crashes."""
+    self.run_test_case('QueryTest/insert-random-partition', vector, unique_database,
+        multiple_impalad=vector.get_value('exec_option')['sync_ddl'] == 1)
+
 class TestInsertWideTable(ImpalaTestSuite):
   @classmethod
   def get_workload(self):


[21/33] impala git commit: IMPALA-7809: support Kudu 1.9 in test_concurrent_schema_change

Posted by bo...@apache.org.
IMPALA-7809: support Kudu 1.9 in test_concurrent_schema_change

This patch extends the match of allowable errors in
test_concurrent_schema_change to work with Kudu 1.9.

Testing: Local with a Kudu 1.9 minicluster environment.

Change-Id: I7fc24bb6a18aecc0cb726b8d66f0aeccf56bbb9b
Reviewed-on: http://gerrit.cloudera.org:8080/11882
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: d8b1e43a86f172bc2a76b53fae2d82420e5aca53
Parents: 0bc5446
Author: Michael Brown <mi...@cloudera.com>
Authored: Mon Nov 5 14:36:03 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 tests/query_test/test_kudu.py | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d8b1e43a/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index 636c6c0..005fa63 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -443,7 +443,8 @@ class TestKuduOperations(KuduTestSuite):
         or "(type: TINYINT) is not compatible with column 'col1' (type: STRING)" in msg \
         or "has fewer columns than expected." in msg \
         or "Column col1 has unexpected type." in msg \
-        or "Client provided column col1[int64 NULLABLE] not present in tablet" in msg
+        or "Client provided column col1[int64 NULLABLE] not present in tablet" in msg \
+        or "Client provided column col1 INT64 NULLABLE not present in tablet"
 
   def _retry_query(self, cursor, query, expected):
     retries = 0


[15/33] impala git commit: IMPALA-7808: Refactor Analyzer for easier debugging

Posted by bo...@apache.org.
IMPALA-7808: Refactor Analyzer for easier debugging

Changes two blocks of code to make debugging easier. No functional
changes occur; changes are pure refactoring. A trivial change in
AnalyzerContext removes a nested conditional clause.

A larger change in SelectStmt takes the large analysis function and
breaks it into a series of smaller functions. The functions were large
because they shared state: variables created near the top are used much
later near the bottom.

To solve this, moved the code into an "algorithm" class whose only job
is to hold onto the temporary state so that the big function can be
broken into smaller pieces, with the temporary class fields used in
place of the former local variables.

For the most part, the existign code was simply split into functions
and indented.  One block of code had to be moved below the inner class
since it is not part of the analysis process.

Testing: No functional change, changes are purely structure.
Reran all tests, which passed.

Change-Id: I576c80c4c7a974df226fc91d8903db275069ed52
Reviewed-on: http://gerrit.cloudera.org:8080/11883
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 14fc20a971abf6b0ab2f92c5a02d7e57b6553615
Parents: 3a7e382
Author: Paul Rogers <pr...@cloudera.com>
Authored: Mon Nov 5 15:55:25 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 .../apache/impala/analysis/AnalysisContext.java |   86 +-
 .../org/apache/impala/analysis/SelectStmt.java  | 1289 ++++++++++--------
 2 files changed, 727 insertions(+), 648 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/14fc20a9/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index a10f5f0..729a84a 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -380,7 +380,7 @@ public class AnalysisContext {
           && !(stmt_ instanceof AlterViewStmt) && !(stmt_ instanceof ShowCreateTableStmt);
     }
     public boolean requiresExprRewrite() {
-      return isQueryStmt() ||isInsertStmt() || isCreateTableAsSelectStmt()
+      return isQueryStmt() || isInsertStmt() || isCreateTableAsSelectStmt()
           || isUpdateStmt() || isDeleteStmt();
     }
     public TLineageGraph getThriftLineageGraph() {
@@ -458,51 +458,51 @@ public class AnalysisContext {
       new StmtRewriter.SubqueryRewriter().rewrite(analysisResult_);
       reAnalyze = true;
     }
-    if (reAnalyze) {
-      // The rewrites should have no user-visible effect. Remember the original result
-      // types and column labels to restore them after the rewritten stmt has been
-      // reset() and re-analyzed. For a CTAS statement, the types represent column types
-      // of the table that will be created, including the partition columns, if any.
-      List<Type> origResultTypes = Lists.newArrayList();
-      for (Expr e : analysisResult_.stmt_.getResultExprs()) {
-        origResultTypes.add(e.getType());
-      }
-      List<String> origColLabels =
-          Lists.newArrayList(analysisResult_.stmt_.getColLabels());
-
-      // Some expressions, such as function calls with constant arguments, can get
-      // folded into literals. Since literals do not require privilege requests, we
-      // must save the original privileges in order to not lose them during
-      // re-analysis.
-      ImmutableList<PrivilegeRequest> origPrivReqs =
-          analysisResult_.analyzer_.getPrivilegeReqs();
-
-      // Re-analyze the stmt with a new analyzer.
-      analysisResult_.analyzer_ = createAnalyzer(stmtTableCache);
-      analysisResult_.stmt_.reset();
-      try {
-        analysisResult_.stmt_.analyze(analysisResult_.analyzer_);
-      } catch (AnalysisException e) {
-        LOG.error(String.format("Error analyzing the rewritten query.\n" +
-            "Original SQL: %s\nRewritten SQL: %s", analysisResult_.stmt_.toSql(),
-            analysisResult_.stmt_.toSql(true)));
-        throw e;
-      }
+    if (!reAnalyze) return;
 
-      // Restore the original result types and column labels.
-      analysisResult_.stmt_.castResultExprs(origResultTypes);
-      analysisResult_.stmt_.setColLabels(origColLabels);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Rewritten SQL: " + analysisResult_.stmt_.toSql(true));
-      }
+    // The rewrites should have no user-visible effect. Remember the original result
+    // types and column labels to restore them after the rewritten stmt has been
+    // reset() and re-analyzed. For a CTAS statement, the types represent column types
+    // of the table that will be created, including the partition columns, if any.
+    List<Type> origResultTypes = Lists.newArrayList();
+    for (Expr e : analysisResult_.stmt_.getResultExprs()) {
+      origResultTypes.add(e.getType());
+    }
+    List<String> origColLabels =
+        Lists.newArrayList(analysisResult_.stmt_.getColLabels());
 
-      // Restore privilege requests found during the first pass
-      for (PrivilegeRequest req : origPrivReqs) {
-        analysisResult_.analyzer_.registerPrivReq(req);
-      }
-      if (isExplain) analysisResult_.stmt_.setIsExplain();
-      Preconditions.checkState(!analysisResult_.requiresSubqueryRewrite());
+    // Some expressions, such as function calls with constant arguments, can get
+    // folded into literals. Since literals do not require privilege requests, we
+    // must save the original privileges in order to not lose them during
+    // re-analysis.
+    ImmutableList<PrivilegeRequest> origPrivReqs =
+        analysisResult_.analyzer_.getPrivilegeReqs();
+
+    // Re-analyze the stmt with a new analyzer.
+    analysisResult_.analyzer_ = createAnalyzer(stmtTableCache);
+    analysisResult_.stmt_.reset();
+    try {
+      analysisResult_.stmt_.analyze(analysisResult_.analyzer_);
+    } catch (AnalysisException e) {
+      LOG.error(String.format("Error analyzing the rewritten query.\n" +
+          "Original SQL: %s\nRewritten SQL: %s", analysisResult_.stmt_.toSql(),
+          analysisResult_.stmt_.toSql(true)));
+      throw e;
+    }
+
+    // Restore the original result types and column labels.
+    analysisResult_.stmt_.castResultExprs(origResultTypes);
+    analysisResult_.stmt_.setColLabels(origColLabels);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Rewritten SQL: " + analysisResult_.stmt_.toSql(true));
+    }
+
+    // Restore privilege requests found during the first pass
+    for (PrivilegeRequest req : origPrivReqs) {
+      analysisResult_.analyzer_.registerPrivReq(req);
     }
+    if (isExplain) analysisResult_.stmt_.setIsExplain();
+    Preconditions.checkState(!analysisResult_.requiresSubqueryRewrite());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/14fc20a9/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java b/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
index 54cb1c6..ce90cd3 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SelectStmt.java
@@ -164,710 +164,789 @@ public class SelectStmt extends QueryStmt {
   public void analyze(Analyzer analyzer) throws AnalysisException {
     if (isAnalyzed()) return;
     super.analyze(analyzer);
+    new SelectAnalyzer(analyzer).analyze();
+  }
 
-    // Start out with table refs to establish aliases.
-    fromClause_.analyze(analyzer);
-
-    // Generate !empty() predicates to filter out empty collections.
-    // Skip this step when analyzing a WITH-clause because CollectionTableRefs
-    // do not register collection slots in their parent in that context
-    // (see CollectionTableRef.analyze()).
-    if (!analyzer.isWithClause()) registerIsNotEmptyPredicates(analyzer);
+  /**
+   * Algorithm class for the SELECT statement analyzer. Holds
+   * the analyzer and intermediate state.
+   */
+  private class SelectAnalyzer {
 
-    // analyze plan hints from select list
-    selectList_.analyzePlanHints(analyzer);
+    private final Analyzer analyzer_;
+    private ArrayList<Expr> groupingExprsCopy_;
+    private List<FunctionCallExpr> aggExprs_;
+    private ExprSubstitutionMap ndvSmap_;
+    private ExprSubstitutionMap countAllMap_;
 
-    // populate resultExprs_, aliasSmap_, and colLabels_
-    for (int i = 0; i < selectList_.getItems().size(); ++i) {
-      SelectListItem item = selectList_.getItems().get(i);
-      if (item.isStar()) {
-        if (item.getRawPath() != null) {
-          Path resolvedPath = analyzeStarPath(item.getRawPath(), analyzer);
-          expandStar(resolvedPath, analyzer);
-        } else {
-          expandStar(analyzer);
-        }
-      } else {
-        // Analyze the resultExpr before generating a label to ensure enforcement
-        // of expr child and depth limits (toColumn() label may call toSql()).
-        item.getExpr().analyze(analyzer);
-        if (item.getExpr().contains(Predicates.instanceOf(Subquery.class))) {
-          throw new AnalysisException(
-              "Subqueries are not supported in the select list.");
-        }
-        resultExprs_.add(item.getExpr());
-        String label = item.toColumnLabel(i, analyzer.useHiveColLabels());
-        SlotRef aliasRef = new SlotRef(label);
-        Expr existingAliasExpr = aliasSmap_.get(aliasRef);
-        if (existingAliasExpr != null && !existingAliasExpr.equals(item.getExpr())) {
-          // If we have already seen this alias, it refers to more than one column and
-          // therefore is ambiguous.
-          ambiguousAliasList_.add(aliasRef);
-        }
-        aliasSmap_.put(aliasRef, item.getExpr().clone());
-        colLabels_.add(label);
-      }
+    private SelectAnalyzer(Analyzer analyzer) {
+      this.analyzer_ = analyzer;
     }
 
-    // Star exprs only expand to the scalar-typed columns/fields, so
-    // the resultExprs_ could be empty.
-    if (resultExprs_.isEmpty()) {
-      throw new AnalysisException("The star exprs expanded to an empty select list " +
-          "because the referenced tables only have complex-typed columns.\n" +
-          "Star exprs only expand to scalar-typed columns because complex-typed exprs " +
-          "are currently not supported in the select list.\n" +
-          "Affected select statement:\n" + toSql());
-    }
+    private void analyze() throws AnalysisException {
+      // Start out with table refs to establish aliases.
+      fromClause_.analyze(analyzer_);
 
-    for (Expr expr: resultExprs_) {
-      // Complex types are currently not supported in the select list because we'd need
-      // to serialize them in a meaningful way.
-      if (expr.getType().isComplexType()) {
-        throw new AnalysisException(String.format(
-            "Expr '%s' in select list returns a complex type '%s'.\n" +
-            "Only scalar types are allowed in the select list.",
-            expr.toSql(), expr.getType().toSql()));
-      }
-      if (!expr.getType().isSupported()) {
-        throw new AnalysisException("Unsupported type '"
-            + expr.getType().toSql() + "' in '" + expr.toSql() + "'.");
-      }
-    }
+      analyzeSelectClause();
+      verifyResultExprs();
+      analyzeWhereClause();
 
-    if (TreeNode.contains(resultExprs_, AnalyticExpr.class)) {
-      if (fromClause_.isEmpty()) {
-        throw new AnalysisException("Analytic expressions require FROM clause.");
-      }
+      createSortInfo(analyzer_);
+      analyzeAggregation();
+      createAnalyticInfo();
+      if (evaluateOrderBy_) createSortTupleInfo(analyzer_);
 
-      // do this here, not after analyzeAggregation(), otherwise the AnalyticExprs
-      // will get substituted away
-      if (selectList_.isDistinct()) {
-        throw new AnalysisException(
-            "cannot combine SELECT DISTINCT with analytic functions");
+      // Remember the SQL string before inline-view expression substitution.
+      sqlString_ = toSql();
+      if (origSqlString_ == null) origSqlString_ = sqlString_;
+      resolveInlineViewRefs();
+
+      // If this block's select-project-join portion returns an empty result set and the
+      // block has no aggregation, then mark this block as returning an empty result set.
+      if (analyzer_.hasEmptySpjResultSet() && multiAggInfo_ == null) {
+        analyzer_.setHasEmptyResultSet();
       }
+
+      buildColumnLineageGraph();
     }
 
-    if (whereClause_ != null) {
-      whereClause_.analyze(analyzer);
-      if (whereClause_.contains(Expr.isAggregatePredicate())) {
-        throw new AnalysisException(
-            "aggregate function not allowed in WHERE clause");
+    private void buildColumnLineageGraph() {
+      ColumnLineageGraph graph = analyzer_.getColumnLineageGraph();
+      if (multiAggInfo_ != null && multiAggInfo_.hasAggregateExprs()) {
+        graph.addDependencyPredicates(multiAggInfo_.getGroupingExprs());
       }
-      whereClause_.checkReturnsBool("WHERE clause", false);
-      Expr e = whereClause_.findFirstOf(AnalyticExpr.class);
-      if (e != null) {
-        throw new AnalysisException(
-            "WHERE clause must not contain analytic expressions: " + e.toSql());
+      if (sortInfo_ != null && hasLimit()) {
+        // When there is a LIMIT clause in conjunction with an ORDER BY, the
+        // ordering exprs must be added in the column lineage graph.
+        graph.addDependencyPredicates(sortInfo_.getSortExprs());
       }
-      analyzer.registerConjuncts(whereClause_, false);
     }
 
-    createSortInfo(analyzer);
-    analyzeAggregation(analyzer);
-    createAnalyticInfo(analyzer);
-    if (evaluateOrderBy_) createSortTupleInfo(analyzer);
+    private void analyzeSelectClause() throws AnalysisException {
+      // Generate !empty() predicates to filter out empty collections.
+      // Skip this step when analyzing a WITH-clause because CollectionTableRefs
+      // do not register collection slots in their parent in that context
+      // (see CollectionTableRef.analyze()).
+      if (!analyzer_.isWithClause()) registerIsNotEmptyPredicates();
 
-    // Remember the SQL string before inline-view expression substitution.
-    sqlString_ = toSql();
-    if (origSqlString_ == null) origSqlString_ = sqlString_;
-    resolveInlineViewRefs(analyzer);
-
-    // If this block's select-project-join portion returns an empty result set and the
-    // block has no aggregation, then mark this block as returning an empty result set.
-    if (analyzer.hasEmptySpjResultSet() && multiAggInfo_ == null) {
-      analyzer.setHasEmptyResultSet();
-    }
+      // analyze plan hints from select list
+      selectList_.analyzePlanHints(analyzer_);
 
-    ColumnLineageGraph graph = analyzer.getColumnLineageGraph();
-    if (multiAggInfo_ != null && multiAggInfo_.hasAggregateExprs()) {
-      graph.addDependencyPredicates(multiAggInfo_.getGroupingExprs());
-    }
-    if (sortInfo_ != null && hasLimit()) {
-      // When there is a LIMIT clause in conjunction with an ORDER BY, the ordering exprs
-      // must be added in the column lineage graph.
-      graph.addDependencyPredicates(sortInfo_.getSortExprs());
+      // populate resultExprs_, aliasSmap_, and colLabels_
+      for (int i = 0; i < selectList_.getItems().size(); ++i) {
+        SelectListItem item = selectList_.getItems().get(i);
+        if (item.isStar()) {
+          if (item.getRawPath() != null) {
+            Path resolvedPath = analyzeStarPath(item.getRawPath(), analyzer_);
+            expandStar(resolvedPath);
+          } else {
+            expandStar();
+          }
+        } else {
+          // Analyze the resultExpr before generating a label to ensure enforcement
+          // of expr child and depth limits (toColumn() label may call toSql()).
+          item.getExpr().analyze(analyzer_);
+          if (item.getExpr().contains(Predicates.instanceOf(Subquery.class))) {
+            throw new AnalysisException(
+                "Subqueries are not supported in the select list.");
+          }
+          resultExprs_.add(item.getExpr());
+          String label = item.toColumnLabel(i, analyzer_.useHiveColLabels());
+          SlotRef aliasRef = new SlotRef(label);
+          Expr existingAliasExpr = aliasSmap_.get(aliasRef);
+          if (existingAliasExpr != null && !existingAliasExpr.equals(item.getExpr())) {
+            // If we have already seen this alias, it refers to more than one column and
+            // therefore is ambiguous.
+            ambiguousAliasList_.add(aliasRef);
+          }
+          aliasSmap_.put(aliasRef, item.getExpr().clone());
+          colLabels_.add(label);
+        }
+      }
     }
-  }
 
-  /**
-   * Generates and registers !empty() predicates to filter out empty collections directly
-   * in the parent scan of collection table refs. This is a performance optimization to
-   * avoid the expensive processing of empty collections inside a subplan that would
-   * yield an empty result set.
-   *
-   * For correctness purposes, the predicates are generated in cases where we can ensure
-   * that they will be assigned only to the parent scan, and no other plan node.
-   *
-   * The conditions are as follows:
-   * - collection table ref is relative and non-correlated
-   * - collection table ref represents the rhs of an inner/cross/semi join
-   * - collection table ref's parent tuple is not outer joined
-   *
-   * Example: table T has field A which is of type array<array<int>>.
-   * 1) ... T join T.A a join a.item a_nest ... : all nodes on the path T -> a -> a_nest
-   *                                              are required so are checked for !empty.
-   * 2) ... T left outer join T.A a join a.item a_nest ... : no !empty.
-   * 3) ... T join T.A a left outer join a.item a_nest ... : a checked for !empty.
-   * 4) ... T left outer join T.A a left outer join a.item a_nest ... : no !empty.
-   *
-   *
-   * TODO: In some cases, it is possible to generate !empty() predicates for a correlated
-   * table ref, but in general, that is not correct for non-trivial query blocks.
-   * For example, if the block with the correlated ref has an aggregation then adding a
-   * !empty() predicate would incorrectly discard rows from the final result set.
-   * TODO: Evaluating !empty() predicates at non-scan nodes interacts poorly with our BE
-   * projection of collection slots. For example, rows could incorrectly be filtered if
-   * a !empty() predicate is assigned to a plan node that comes after the unnest of the
-   * collection that also performs the projection.
-   */
-  private void registerIsNotEmptyPredicates(Analyzer analyzer) throws AnalysisException {
-    for (TableRef tblRef: fromClause_.getTableRefs()) {
-      Preconditions.checkState(tblRef.isResolved());
-      if (!(tblRef instanceof CollectionTableRef)) continue;
-      CollectionTableRef ref = (CollectionTableRef) tblRef;
-      // Skip non-relative and correlated refs.
-      if (!ref.isRelative() || ref.isCorrelated()) continue;
-      // Skip outer and anti joins.
-      if (ref.getJoinOp().isOuterJoin() || ref.getJoinOp().isAntiJoin()) continue;
-      // Do not generate a predicate if the parent tuple is outer joined.
-      if (analyzer.isOuterJoined(ref.getResolvedPath().getRootDesc().getId())) continue;
-      IsNotEmptyPredicate isNotEmptyPred =
-          new IsNotEmptyPredicate(ref.getCollectionExpr().clone());
-      isNotEmptyPred.analyze(analyzer);
-      // Register the predicate as an On-clause conjunct because it should only
-      // affect the result of this join and not the whole FROM clause.
-      analyzer.registerOnClauseConjuncts(
-          Lists.<Expr>newArrayList(isNotEmptyPred), ref);
-    }
-  }
+    private void verifyResultExprs() throws AnalysisException {
+      // Star exprs only expand to the scalar-typed columns/fields, so
+      // the resultExprs_ could be empty.
+      if (resultExprs_.isEmpty()) {
+        throw new AnalysisException("The star exprs expanded to an empty select list " +
+            "because the referenced tables only have complex-typed columns.\n" +
+            "Star exprs only expand to scalar-typed columns because " +
+            "complex-typed exprs " +
+            "are currently not supported in the select list.\n" +
+            "Affected select statement:\n" + toSql());
+      }
 
-  /**
-   * Marks all unassigned join predicates as well as exprs in aggInfo and sortInfo.
-   */
-  @Override
-  public void materializeRequiredSlots(Analyzer analyzer) {
-    // Mark unassigned join predicates. Some predicates that must be evaluated by a join
-    // can also be safely evaluated below the join (picked up by getBoundPredicates()).
-    // Such predicates will be marked twice and that is ok.
-    List<Expr> unassigned =
-        analyzer.getUnassignedConjuncts(getTableRefIds(), true);
-    List<Expr> unassignedJoinConjuncts = Lists.newArrayList();
-    for (Expr e: unassigned) {
-      if (analyzer.evalAfterJoin(e)) unassignedJoinConjuncts.add(e);
-    }
-    List<Expr> baseTblJoinConjuncts =
-        Expr.substituteList(unassignedJoinConjuncts, baseTblSmap_, analyzer, false);
-    materializeSlots(analyzer, baseTblJoinConjuncts);
+      for (Expr expr: resultExprs_) {
+        // Complex types are currently not supported in the select list because
+        // we'd need to serialize them in a meaningful way.
+        if (expr.getType().isComplexType()) {
+          throw new AnalysisException(String.format(
+              "Expr '%s' in select list returns a complex type '%s'.\n" +
+              "Only scalar types are allowed in the select list.",
+              expr.toSql(), expr.getType().toSql()));
+        }
+        if (!expr.getType().isSupported()) {
+          throw new AnalysisException("Unsupported type '"
+              + expr.getType().toSql() + "' in '" + expr.toSql() + "'.");
+        }
+      }
 
-    if (evaluateOrderBy_) {
-      // mark ordering exprs before marking agg/analytic exprs because they could contain
-      // agg/analytic exprs that are not referenced anywhere but the ORDER BY clause
-      sortInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
-    }
+      if (TreeNode.contains(resultExprs_, AnalyticExpr.class)) {
+        if (fromClause_.isEmpty()) {
+          throw new AnalysisException("Analytic expressions require FROM clause.");
+        }
 
-    if (hasAnalyticInfo()) {
-      // Mark analytic exprs before marking agg exprs because they could contain agg
-      // exprs that are not referenced anywhere but the analytic expr.
-      // Gather unassigned predicates and mark their slots. It is not desirable
-      // to account for propagated predicates because if an analytic expr is only
-      // referenced by a propagated predicate, then it's better to not materialize the
-      // analytic expr at all.
-      ArrayList<TupleId> tids = Lists.newArrayList();
-      getMaterializedTupleIds(tids); // includes the analytic tuple
-      List<Expr> conjuncts = analyzer.getUnassignedConjuncts(tids, false);
-      materializeSlots(analyzer, conjuncts);
-      analyticInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
+        // do this here, not after analyzeAggregation(), otherwise the AnalyticExprs
+        // will get substituted away
+        if (selectList_.isDistinct()) {
+          throw new AnalysisException(
+              "cannot combine SELECT DISTINCT with analytic functions");
+        }
+      }
     }
 
-    if (multiAggInfo_ != null) {
-      // Mark all agg slots required for conjunct evaluation as materialized before
-      // calling MultiAggregateInfo.materializeRequiredSlots().
-      List<Expr> conjuncts = multiAggInfo_.collectConjuncts(analyzer, false);
-      materializeSlots(analyzer, conjuncts);
-      multiAggInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
+    private void analyzeWhereClause() throws AnalysisException {
+      if (whereClause_ != null) {
+        whereClause_.analyze(analyzer_);
+        if (whereClause_.contains(Expr.isAggregatePredicate())) {
+          throw new AnalysisException(
+              "aggregate function not allowed in WHERE clause");
+        }
+        whereClause_.checkReturnsBool("WHERE clause", false);
+        Expr e = whereClause_.findFirstOf(AnalyticExpr.class);
+        if (e != null) {
+          throw new AnalysisException(
+              "WHERE clause must not contain analytic expressions: " + e.toSql());
+        }
+        analyzer_.registerConjuncts(whereClause_, false);
+      }
     }
-  }
 
-  /**
-    * Populates baseTblSmap_ with our combined inline view smap and creates
-    * baseTblResultExprs.
-    */
-  protected void resolveInlineViewRefs(Analyzer analyzer)
-      throws AnalysisException {
-    // Gather the inline view substitution maps from the enclosed inline views
-    for (TableRef tblRef: fromClause_) {
-      if (tblRef instanceof InlineViewRef) {
-        InlineViewRef inlineViewRef = (InlineViewRef) tblRef;
-        baseTblSmap_ =
-            ExprSubstitutionMap.combine(baseTblSmap_, inlineViewRef.getBaseTblSmap());
+    /**
+     * Generates and registers !empty() predicates to filter out empty
+     * collections directly in the parent scan of collection table refs. This is
+     * a performance optimization to avoid the expensive processing of empty
+     * collections inside a subplan that would yield an empty result set.
+     *
+     * For correctness purposes, the predicates are generated in cases where we
+     * can ensure that they will be assigned only to the parent scan, and no other
+     * plan node.
+     *
+     * The conditions are as follows:
+     * - collection table ref is relative and non-correlated
+     * - collection table ref represents the rhs of an inner/cross/semi join
+     * - collection table ref's parent tuple is not outer joined
+     *
+     * Example: table T has field A which is of type array<array<int>>.
+     * 1) ... T join T.A a join a.item a_nest ... :
+     *                       all nodes on the path T -> a -> a_nest
+     *                       are required so are checked for !empty.
+     * 2) ... T left outer join T.A a join a.item a_nest ... : no !empty.
+     * 3) ... T join T.A a left outer join a.item a_nest ... :
+     *                       a checked for !empty.
+     * 4) ... T left outer join T.A a left outer join a.item a_nest ... :
+     *                       no !empty.
+     *
+     *
+     * TODO: In some cases, it is possible to generate !empty() predicates for
+     * a correlated table ref, but in general, that is not correct for non-trivial
+     * query blocks. For example, if the block with the correlated ref has an
+     * aggregation then adding a !empty() predicate would incorrectly discard rows
+     * from the final result set.
+     *
+     * TODO: Evaluating !empty() predicates at non-scan nodes interacts poorly with
+     * our BE projection of collection slots. For example, rows could incorrectly
+     * be filtered if a !empty() predicate is assigned to a plan node that comes
+     * after the unnest of the collection that also performs the projection.
+     */
+    private void registerIsNotEmptyPredicates() throws AnalysisException {
+      for (TableRef tblRef: fromClause_.getTableRefs()) {
+        Preconditions.checkState(tblRef.isResolved());
+        if (!(tblRef instanceof CollectionTableRef)) continue;
+        CollectionTableRef ref = (CollectionTableRef) tblRef;
+        // Skip non-relative and correlated refs.
+        if (!ref.isRelative() || ref.isCorrelated()) continue;
+        // Skip outer and anti joins.
+        if (ref.getJoinOp().isOuterJoin() || ref.getJoinOp().isAntiJoin()) continue;
+        // Do not generate a predicate if the parent tuple is outer joined.
+        if (analyzer_.isOuterJoined(ref.getResolvedPath().getRootDesc().getId()))
+          continue;
+        IsNotEmptyPredicate isNotEmptyPred =
+            new IsNotEmptyPredicate(ref.getCollectionExpr().clone());
+        isNotEmptyPred.analyze(analyzer_);
+        // Register the predicate as an On-clause conjunct because it should only
+        // affect the result of this join and not the whole FROM clause.
+        analyzer_.registerOnClauseConjuncts(
+            Lists.<Expr>newArrayList(isNotEmptyPred), ref);
       }
     }
-    baseTblResultExprs_ =
-        Expr.trySubstituteList(resultExprs_, baseTblSmap_, analyzer, false);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("baseTblSmap_: " + baseTblSmap_.debugString());
-      LOG.trace("resultExprs: " + Expr.debugString(resultExprs_));
-      LOG.trace("baseTblResultExprs: " + Expr.debugString(baseTblResultExprs_));
-    }
-  }
 
-  public List<TupleId> getTableRefIds() {
-    List<TupleId> result = Lists.newArrayList();
-    for (TableRef ref: fromClause_) {
-      result.add(ref.getId());
+    /**
+      * Populates baseTblSmap_ with our combined inline view smap and creates
+      * baseTblResultExprs.
+      */
+    private void resolveInlineViewRefs()
+        throws AnalysisException {
+      // Gather the inline view substitution maps from the enclosed inline views
+      for (TableRef tblRef: fromClause_) {
+        if (tblRef instanceof InlineViewRef) {
+          InlineViewRef inlineViewRef = (InlineViewRef) tblRef;
+          baseTblSmap_ =
+              ExprSubstitutionMap.combine(baseTblSmap_, inlineViewRef.getBaseTblSmap());
+        }
+      }
+      baseTblResultExprs_ =
+          Expr.trySubstituteList(resultExprs_, baseTblSmap_, analyzer_, false);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("baseTblSmap_: " + baseTblSmap_.debugString());
+        LOG.trace("resultExprs: " + Expr.debugString(resultExprs_));
+        LOG.trace("baseTblResultExprs: " + Expr.debugString(baseTblResultExprs_));
+      }
     }
-    return result;
-  }
 
-  /**
-   * Resolves the given raw path as a STAR path and checks its legality.
-   * Returns the resolved legal path, or throws if the raw path could not
-   * be resolved or is an illegal star path.
-   */
-  private Path analyzeStarPath(List<String> rawPath, Analyzer analyzer)
-      throws AnalysisException {
-    Path resolvedPath = null;
-    try {
-      resolvedPath = analyzer.resolvePath(rawPath, PathType.STAR);
-    } catch (TableLoadingException e) {
-      // Should never happen because we only check registered table aliases.
-      Preconditions.checkState(false);
+    /**
+     * Resolves the given raw path as a STAR path and checks its legality.
+     * Returns the resolved legal path, or throws if the raw path could not
+     * be resolved or is an illegal star path.
+     */
+    private Path analyzeStarPath(List<String> rawPath, Analyzer analyzer)
+        throws AnalysisException {
+      Path resolvedPath = null;
+      try {
+        resolvedPath = analyzer.resolvePath(rawPath, PathType.STAR);
+      } catch (TableLoadingException e) {
+        // Should never happen because we only check registered table aliases.
+        Preconditions.checkState(false);
+      }
+      Preconditions.checkNotNull(resolvedPath);
+      return resolvedPath;
     }
-    Preconditions.checkNotNull(resolvedPath);
-    return resolvedPath;
-  }
 
-  /**
-   * Expand "*" select list item, ignoring semi-joined tables as well as
-   * complex-typed fields because those are currently illegal in any select
-   * list (even for inline views, etc.)
-   */
-  private void expandStar(Analyzer analyzer) throws AnalysisException {
-    if (fromClause_.isEmpty()) {
-      throw new AnalysisException("'*' expression in select list requires FROM clause.");
-    }
-    // expand in From clause order
-    for (TableRef tableRef: fromClause_) {
-      if (analyzer.isSemiJoined(tableRef.getId())) continue;
-      Path resolvedPath = new Path(tableRef.getDesc(), Collections.<String>emptyList());
-      Preconditions.checkState(resolvedPath.resolve());
-      expandStar(resolvedPath, analyzer);
+    /**
+     * Expand "*" select list item, ignoring semi-joined tables as well as
+     * complex-typed fields because those are currently illegal in any select
+     * list (even for inline views, etc.)
+     */
+    private void expandStar() throws AnalysisException {
+      if (fromClause_.isEmpty()) {
+        throw new AnalysisException(
+            "'*' expression in select list requires FROM clause.");
+      }
+      // expand in From clause order
+      for (TableRef tableRef: fromClause_) {
+        if (analyzer_.isSemiJoined(tableRef.getId())) continue;
+        Path resolvedPath = new Path(tableRef.getDesc(),
+            Collections.<String>emptyList());
+        Preconditions.checkState(resolvedPath.resolve());
+        expandStar(resolvedPath);
+      }
     }
-  }
 
-  /**
-   * Expand "path.*" from a resolved path, ignoring complex-typed fields because those
-   * are currently illegal in any select list (even for inline views, etc.)
-   */
-  private void expandStar(Path resolvedPath, Analyzer analyzer)
-      throws AnalysisException {
-    Preconditions.checkState(resolvedPath.isResolved());
-    if (resolvedPath.destTupleDesc() != null &&
-        resolvedPath.destTupleDesc().getTable() != null &&
-        resolvedPath.destTupleDesc().getPath().getMatchedTypes().isEmpty()) {
-      // The resolved path targets a registered tuple descriptor of a catalog
-      // table. Expand the '*' based on the Hive-column order.
-      TupleDescriptor tupleDesc = resolvedPath.destTupleDesc();
-      FeTable table = tupleDesc.getTable();
-      for (Column c: table.getColumnsInHiveOrder()) {
-        addStarResultExpr(resolvedPath, analyzer, c.getName());
-      }
-    } else {
-      // The resolved path does not target the descriptor of a catalog table.
-      // Expand '*' based on the destination type of the resolved path.
-      Preconditions.checkState(resolvedPath.destType().isStructType());
-      StructType structType = (StructType) resolvedPath.destType();
-      Preconditions.checkNotNull(structType);
-
-      // Star expansion for references to nested collections.
-      // Collection Type                    Star Expansion
-      // array<int>                     --> item
-      // array<struct<f1,f2,...,fn>>    --> f1, f2, ..., fn
-      // map<int,int>                   --> key, value
-      // map<int,struct<f1,f2,...,fn>>  --> key, f1, f2, ..., fn
-      if (structType instanceof CollectionStructType) {
-        CollectionStructType cst = (CollectionStructType) structType;
-        if (cst.isMapStruct()) {
-          addStarResultExpr(resolvedPath, analyzer, Path.MAP_KEY_FIELD_NAME);
+    /**
+     * Expand "path.*" from a resolved path, ignoring complex-typed fields
+     * because those are currently illegal in any select list (even for
+     * inline views, etc.)
+     */
+    private void expandStar(Path resolvedPath)
+        throws AnalysisException {
+      Preconditions.checkState(resolvedPath.isResolved());
+      if (resolvedPath.destTupleDesc() != null &&
+          resolvedPath.destTupleDesc().getTable() != null &&
+          resolvedPath.destTupleDesc().getPath().getMatchedTypes().isEmpty()) {
+        // The resolved path targets a registered tuple descriptor of a catalog
+        // table. Expand the '*' based on the Hive-column order.
+        TupleDescriptor tupleDesc = resolvedPath.destTupleDesc();
+        FeTable table = tupleDesc.getTable();
+        for (Column c: table.getColumnsInHiveOrder()) {
+          addStarResultExpr(resolvedPath, c.getName());
         }
-        if (cst.getOptionalField().getType().isStructType()) {
-          structType = (StructType) cst.getOptionalField().getType();
-          for (StructField f: structType.getFields()) {
-            addStarResultExpr(
-                resolvedPath, analyzer, cst.getOptionalField().getName(), f.getName());
+      } else {
+        // The resolved path does not target the descriptor of a catalog table.
+        // Expand '*' based on the destination type of the resolved path.
+        Preconditions.checkState(resolvedPath.destType().isStructType());
+        StructType structType = (StructType) resolvedPath.destType();
+        Preconditions.checkNotNull(structType);
+
+        // Star expansion for references to nested collections.
+        // Collection Type                    Star Expansion
+        // array<int>                     --> item
+        // array<struct<f1,f2,...,fn>>    --> f1, f2, ..., fn
+        // map<int,int>                   --> key, value
+        // map<int,struct<f1,f2,...,fn>>  --> key, f1, f2, ..., fn
+        if (structType instanceof CollectionStructType) {
+          CollectionStructType cst = (CollectionStructType) structType;
+          if (cst.isMapStruct()) {
+            addStarResultExpr(resolvedPath, Path.MAP_KEY_FIELD_NAME);
+          }
+          if (cst.getOptionalField().getType().isStructType()) {
+            structType = (StructType) cst.getOptionalField().getType();
+            for (StructField f: structType.getFields()) {
+              addStarResultExpr(
+                  resolvedPath, cst.getOptionalField().getName(), f.getName());
+            }
+          } else if (cst.isMapStruct()) {
+            addStarResultExpr(resolvedPath, Path.MAP_VALUE_FIELD_NAME);
+          } else {
+            addStarResultExpr(resolvedPath, Path.ARRAY_ITEM_FIELD_NAME);
           }
-        } else if (cst.isMapStruct()) {
-          addStarResultExpr(resolvedPath, analyzer, Path.MAP_VALUE_FIELD_NAME);
         } else {
-          addStarResultExpr(resolvedPath, analyzer, Path.ARRAY_ITEM_FIELD_NAME);
-        }
-      } else {
-        // Default star expansion.
-        for (StructField f: structType.getFields()) {
-          addStarResultExpr(resolvedPath, analyzer, f.getName());
+          // Default star expansion.
+          for (StructField f: structType.getFields()) {
+            addStarResultExpr(resolvedPath, f.getName());
+          }
         }
       }
     }
-  }
 
-  /**
-   * Helper function used during star expansion to add a single result expr
-   * based on a given raw path to be resolved relative to an existing path.
-   * Ignores paths with a complex-typed destination because they are currently
-   * illegal in any select list (even for inline views, etc.)
-   */
-  private void addStarResultExpr(Path resolvedPath, Analyzer analyzer,
-      String... relRawPath) throws AnalysisException {
-    Path p = Path.createRelPath(resolvedPath, relRawPath);
-    Preconditions.checkState(p.resolve());
-    if (p.destType().isComplexType()) return;
-    SlotDescriptor slotDesc = analyzer.registerSlotRef(p);
-    SlotRef slotRef = new SlotRef(slotDesc);
-    slotRef.analyze(analyzer);
-    resultExprs_.add(slotRef);
-    colLabels_.add(relRawPath[relRawPath.length - 1]);
-  }
-
-  /**
-   * Analyze aggregation-relevant components of the select block (Group By clause,
-   * select list, Order By clause), substitute AVG with SUM/COUNT, create the
-   * AggregationInfo, including the agg output tuple, and transform all post-agg exprs
-   * given AggregationInfo's smap.
-   */
-  private void analyzeAggregation(Analyzer analyzer) throws AnalysisException {
-    // Analyze the HAVING clause first so we can check if it contains aggregates.
-    // We need to analyze/register it even if we are not computing aggregates.
-    if (havingClause_ != null) {
-      // can't contain subqueries
-      if (havingClause_.contains(Predicates.instanceOf(Subquery.class))) {
-        throw new AnalysisException(
-            "Subqueries are not supported in the HAVING clause.");
+    /**
+     * Helper function used during star expansion to add a single result expr
+     * based on a given raw path to be resolved relative to an existing path.
+     * Ignores paths with a complex-typed destination because they are currently
+     * illegal in any select list (even for inline views, etc.)
+     */
+    private void addStarResultExpr(Path resolvedPath,
+        String... relRawPath) throws AnalysisException {
+      Path p = Path.createRelPath(resolvedPath, relRawPath);
+      Preconditions.checkState(p.resolve());
+      if (p.destType().isComplexType()) return;
+      SlotDescriptor slotDesc = analyzer_.registerSlotRef(p);
+      SlotRef slotRef = new SlotRef(slotDesc);
+      slotRef.analyze(analyzer_);
+      resultExprs_.add(slotRef);
+      colLabels_.add(relRawPath[relRawPath.length - 1]);
+    }
+
+    /**
+     * Analyze aggregation-relevant components of the select block (Group By clause,
+     * select list, Order By clause), substitute AVG with SUM/COUNT, create the
+     * AggregationInfo, including the agg output tuple, and transform all post-agg exprs
+     * given AggregationInfo's smap.
+     */
+    private void analyzeAggregation() throws AnalysisException {
+      analyzeHavingClause();
+      if (!checkForAggregates()) {
+        return;
       }
-      havingPred_ = substituteOrdinalOrAlias(havingClause_, "HAVING", analyzer);
-      // can't contain analytic exprs
-      Expr analyticExpr = havingPred_.findFirstOf(AnalyticExpr.class);
-      if (analyticExpr != null) {
-        throw new AnalysisException(
-            "HAVING clause must not contain analytic expressions: "
-               + analyticExpr.toSql());
+      verifyAggSemantics();
+      analyzeGroupingExprs();
+      collectAggExprs();
+      rewriteCountDistinct();
+      buildAggregateExprs();
+      buildResultExprs();
+      verifyAggregation();
+    }
+
+    private void analyzeHavingClause() throws AnalysisException {
+      // Analyze the HAVING clause first so we can check if it contains aggregates.
+      // We need to analyze/register it even if we are not computing aggregates.
+      if (havingClause_ != null) {
+        // can't contain subqueries
+        if (havingClause_.contains(Predicates.instanceOf(Subquery.class))) {
+          throw new AnalysisException(
+              "Subqueries are not supported in the HAVING clause.");
+        }
+        havingPred_ = substituteOrdinalOrAlias(havingClause_, "HAVING", analyzer_);
+        // can't contain analytic exprs
+        Expr analyticExpr = havingPred_.findFirstOf(AnalyticExpr.class);
+        if (analyticExpr != null) {
+          throw new AnalysisException(
+              "HAVING clause must not contain analytic expressions: "
+                 + analyticExpr.toSql());
+        }
+        havingPred_.checkReturnsBool("HAVING clause", true);
       }
-      havingPred_.checkReturnsBool("HAVING clause", true);
     }
 
-    if (groupingExprs_ == null && !selectList_.isDistinct()
-        && !TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())
-        && (havingPred_ == null
-            || !havingPred_.contains(Expr.isAggregatePredicate()))
-        && (sortInfo_ == null
-            || !TreeNode.contains(sortInfo_.getSortExprs(),
-                                  Expr.isAggregatePredicate()))) {
-      // We're not computing aggregates but we still need to register the HAVING
-      // clause which could, e.g., contain a constant expression evaluating to false.
-      if (havingPred_ != null) analyzer.registerConjuncts(havingPred_, true);
-      return;
+    private boolean checkForAggregates() throws AnalysisException {
+      if (groupingExprs_ == null && !selectList_.isDistinct()
+          && !TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())
+          && (havingPred_ == null
+              || !havingPred_.contains(Expr.isAggregatePredicate()))
+          && (sortInfo_ == null
+              || !TreeNode.contains(sortInfo_.getSortExprs(),
+                                    Expr.isAggregatePredicate()))) {
+        // We're not computing aggregates but we still need to register the HAVING
+        // clause which could, e.g., contain a constant expression evaluating to false.
+        if (havingPred_ != null) analyzer_.registerConjuncts(havingPred_, true);
+        return false;
+      }
+      return true;
     }
 
-    // If we're computing an aggregate, we must have a FROM clause.
-    if (fromClause_.isEmpty()) {
-      throw new AnalysisException(
-          "aggregation without a FROM clause is not allowed");
-    }
+    private void verifyAggSemantics() throws AnalysisException {
+      // If we're computing an aggregate, we must have a FROM clause.
+      if (fromClause_.isEmpty()) {
+        throw new AnalysisException(
+            "aggregation without a FROM clause is not allowed");
+      }
 
-    if (selectList_.isDistinct()
-        && (groupingExprs_ != null
-            || TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())
-            || (havingPred_ != null
-                && havingPred_.contains(Expr.isAggregatePredicate())))) {
-      throw new AnalysisException(
-        "cannot combine SELECT DISTINCT with aggregate functions or GROUP BY");
-    }
+      if (selectList_.isDistinct()
+          && (groupingExprs_ != null
+              || TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())
+              || (havingPred_ != null
+                  && havingPred_.contains(Expr.isAggregatePredicate())))) {
+        throw new AnalysisException(
+          "cannot combine SELECT DISTINCT with aggregate functions or GROUP BY");
+      }
 
-    // Disallow '*' with explicit GROUP BY or aggregation function (we can't group by
-    // '*', and if you need to name all star-expanded cols in the group by clause you
-    // might as well do it in the select list).
-    if (groupingExprs_ != null ||
-        TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())) {
-      for (SelectListItem item : selectList_.getItems()) {
-        if (item.isStar()) {
-          throw new AnalysisException(
-              "cannot combine '*' in select list with grouping or aggregation");
+      // Disallow '*' with explicit GROUP BY or aggregation function (we can't group by
+      // '*', and if you need to name all star-expanded cols in the group by clause you
+      // might as well do it in the select list).
+      if (groupingExprs_ != null ||
+          TreeNode.contains(resultExprs_, Expr.isAggregatePredicate())) {
+        for (SelectListItem item : selectList_.getItems()) {
+          if (item.isStar()) {
+            throw new AnalysisException(
+                "cannot combine '*' in select list with grouping or aggregation");
+          }
         }
       }
-    }
 
-    // disallow subqueries in the GROUP BY clause
-    if (groupingExprs_ != null) {
-      for (Expr expr: groupingExprs_) {
-        if (expr.contains(Predicates.instanceOf(Subquery.class))) {
-          throw new AnalysisException(
-              "Subqueries are not supported in the GROUP BY clause.");
+      // disallow subqueries in the GROUP BY clause
+      if (groupingExprs_ != null) {
+        for (Expr expr: groupingExprs_) {
+          if (expr.contains(Predicates.instanceOf(Subquery.class))) {
+            throw new AnalysisException(
+                "Subqueries are not supported in the GROUP BY clause.");
+          }
         }
       }
     }
 
-    // analyze grouping exprs
-    ArrayList<Expr> groupingExprsCopy = Lists.newArrayList();
-    if (groupingExprs_ != null) {
-      // make a deep copy here, we don't want to modify the original
-      // exprs during analysis (in case we need to print them later)
-      groupingExprsCopy = Expr.cloneList(groupingExprs_);
-      substituteOrdinalsAndAliases(groupingExprsCopy, "GROUP BY", analyzer);
-
-      for (int i = 0; i < groupingExprsCopy.size(); ++i) {
-        groupingExprsCopy.get(i).analyze(analyzer);
-        if (groupingExprsCopy.get(i).contains(Expr.isAggregatePredicate())) {
-          // reference the original expr in the error msg
-          throw new AnalysisException(
-              "GROUP BY expression must not contain aggregate functions: "
-                  + groupingExprs_.get(i).toSql());
-        }
-        if (groupingExprsCopy.get(i).contains(AnalyticExpr.class)) {
-          // reference the original expr in the error msg
-          throw new AnalysisException(
-              "GROUP BY expression must not contain analytic expressions: "
-                  + groupingExprsCopy.get(i).toSql());
+    private void analyzeGroupingExprs() throws AnalysisException {
+      // analyze grouping exprs
+      groupingExprsCopy_ = Lists.newArrayList();
+      if (groupingExprs_ != null) {
+        // make a deep copy here, we don't want to modify the original
+        // exprs during analysis (in case we need to print them later)
+        groupingExprsCopy_ = Expr.cloneList(groupingExprs_);
+        substituteOrdinalsAndAliases(groupingExprsCopy_, "GROUP BY", analyzer_);
+
+        for (int i = 0; i < groupingExprsCopy_.size(); ++i) {
+          groupingExprsCopy_.get(i).analyze(analyzer_);
+          if (groupingExprsCopy_.get(i).contains(Expr.isAggregatePredicate())) {
+            // reference the original expr in the error msg
+            throw new AnalysisException(
+                "GROUP BY expression must not contain aggregate functions: "
+                    + groupingExprs_.get(i).toSql());
+          }
+          if (groupingExprsCopy_.get(i).contains(AnalyticExpr.class)) {
+            // reference the original expr in the error msg
+            throw new AnalysisException(
+                "GROUP BY expression must not contain analytic expressions: "
+                    + groupingExprsCopy_.get(i).toSql());
+          }
         }
       }
     }
 
-    // Collect the aggregate expressions from the SELECT, HAVING and ORDER BY clauses
-    // of this statement.
-    List<FunctionCallExpr> aggExprs = Lists.newArrayList();
-    TreeNode.collect(resultExprs_, Expr.isAggregatePredicate(), aggExprs);
-    if (havingPred_ != null) {
-      havingPred_.collect(Expr.isAggregatePredicate(), aggExprs);
-    }
-    if (sortInfo_ != null) {
-      // TODO: Avoid evaluating aggs in ignored order-bys
-      TreeNode.collect(sortInfo_.getSortExprs(), Expr.isAggregatePredicate(),
-          aggExprs);
+    private void collectAggExprs() {
+      // Collect the aggregate expressions from the SELECT, HAVING and ORDER BY clauses
+      // of this statement.
+      aggExprs_ = Lists.newArrayList();
+      TreeNode.collect(resultExprs_, Expr.isAggregatePredicate(), aggExprs_);
+      if (havingPred_ != null) {
+        havingPred_.collect(Expr.isAggregatePredicate(), aggExprs_);
+      }
+      if (sortInfo_ != null) {
+        // TODO: Avoid evaluating aggs in ignored order-bys
+        TreeNode.collect(sortInfo_.getSortExprs(), Expr.isAggregatePredicate(),
+            aggExprs_);
+      }
     }
 
-    // Optionally rewrite all count(distinct <expr>) into equivalent NDV() calls.
-    ExprSubstitutionMap ndvSmap = null;
-    if (analyzer.getQueryCtx().client_request.query_options.appx_count_distinct) {
-      ndvSmap = new ExprSubstitutionMap();
-      for (FunctionCallExpr aggExpr: aggExprs) {
-        if (!aggExpr.isDistinct()
-            || !aggExpr.getFnName().getFunction().equals("count")
-            || aggExpr.getParams().size() != 1) {
-          continue;
+    private void rewriteCountDistinct() {
+      // Optionally rewrite all count(distinct <expr>) into equivalent NDV() calls.
+      if (analyzer_.getQueryCtx().client_request.query_options.appx_count_distinct) {
+        ndvSmap_ = new ExprSubstitutionMap();
+        for (FunctionCallExpr aggExpr: aggExprs_) {
+          if (!aggExpr.isDistinct()
+              || !aggExpr.getFnName().getFunction().equals("count")
+              || aggExpr.getParams().size() != 1) {
+            continue;
+          }
+          FunctionCallExpr ndvFnCall =
+              new FunctionCallExpr("ndv", aggExpr.getParams().exprs());
+          ndvFnCall.analyzeNoThrow(analyzer_);
+          Preconditions.checkState(ndvFnCall.getType().equals(aggExpr.getType()));
+          ndvSmap_.put(aggExpr, ndvFnCall);
+        }
+        // Replace all count(distinct <expr>) with NDV(<expr>).
+        List<Expr> substAggExprs = Expr.substituteList(aggExprs_,
+            ndvSmap_, analyzer_, false);
+        aggExprs_.clear();
+        for (Expr aggExpr: substAggExprs) {
+          Preconditions.checkState(aggExpr instanceof FunctionCallExpr);
+          aggExprs_.add((FunctionCallExpr) aggExpr);
         }
-        FunctionCallExpr ndvFnCall =
-            new FunctionCallExpr("ndv", aggExpr.getParams().exprs());
-        ndvFnCall.analyzeNoThrow(analyzer);
-        Preconditions.checkState(ndvFnCall.getType().equals(aggExpr.getType()));
-        ndvSmap.put(aggExpr, ndvFnCall);
-      }
-      // Replace all count(distinct <expr>) with NDV(<expr>).
-      List<Expr> substAggExprs = Expr.substituteList(aggExprs, ndvSmap, analyzer, false);
-      aggExprs.clear();
-      for (Expr aggExpr: substAggExprs) {
-        Preconditions.checkState(aggExpr instanceof FunctionCallExpr);
-        aggExprs.add((FunctionCallExpr) aggExpr);
       }
     }
 
-    // When DISTINCT aggregates are present, non-distinct (i.e. ALL) aggregates are
-    // evaluated in two phases (see AggregateInfo for more details). In particular,
-    // COUNT(c) in "SELECT COUNT(c), AGG(DISTINCT d) from R" is transformed to
-    // "SELECT SUM(cnt) FROM (SELECT COUNT(c) as cnt from R group by d ) S".
-    // Since a group-by expression is added to the inner query it returns no rows if
-    // R is empty, in which case the SUM of COUNTs will return NULL.
-    // However the original COUNT(c) should have returned 0 instead of NULL in this case.
-    // Therefore, COUNT([ALL]) is transformed into zeroifnull(COUNT([ALL]) if
-    // i) There is no GROUP-BY clause, and
-    // ii) Other DISTINCT aggregates are present.
-    ExprSubstitutionMap countAllMap = createCountAllMap(aggExprs, analyzer);
-    countAllMap = ExprSubstitutionMap.compose(ndvSmap, countAllMap, analyzer);
-    List<Expr> substitutedAggs =
-        Expr.substituteList(aggExprs, countAllMap, analyzer, false);
-    aggExprs.clear();
-    TreeNode.collect(substitutedAggs, Expr.isAggregatePredicate(), aggExprs);
-
-    List<Expr> groupingExprs = groupingExprsCopy;
-    if (selectList_.isDistinct()) {
-      // Create multiAggInfo for SELECT DISTINCT:
-      // - all select list items turn into grouping exprs
-      // - there are no aggregate exprs
-      Preconditions.checkState(groupingExprsCopy.isEmpty());
-      Preconditions.checkState(aggExprs.isEmpty());
-      groupingExprs = Expr.cloneList(resultExprs_);
-    }
-    Expr.removeDuplicates(aggExprs);
-    Expr.removeDuplicates(groupingExprs);
-    multiAggInfo_ = new MultiAggregateInfo(groupingExprs, aggExprs);
-    multiAggInfo_.analyze(analyzer);
-
-    ExprSubstitutionMap finalOutputSmap = multiAggInfo_.getOutputSmap();
-    ExprSubstitutionMap combinedSmap =
-        ExprSubstitutionMap.compose(countAllMap, finalOutputSmap, analyzer);
-
-    // change select list, having and ordering exprs to point to agg output. We need
-    // to reanalyze the exprs at this point.
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("combined smap: " + combinedSmap.debugString());
-      LOG.trace("desctbl: " + analyzer.getDescTbl().debugString());
-      LOG.trace("resultexprs: " + Expr.debugString(resultExprs_));
-    }
-    resultExprs_ = Expr.substituteList(resultExprs_, combinedSmap, analyzer, false);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("post-agg selectListExprs: " + Expr.debugString(resultExprs_));
+    private void buildAggregateExprs() throws AnalysisException {
+      // When DISTINCT aggregates are present, non-distinct (i.e. ALL) aggregates are
+      // evaluated in two phases (see AggregateInfo for more details). In particular,
+      // COUNT(c) in "SELECT COUNT(c), AGG(DISTINCT d) from R" is transformed to
+      // "SELECT SUM(cnt) FROM (SELECT COUNT(c) as cnt from R group by d ) S".
+      // Since a group-by expression is added to the inner query it returns no rows if
+      // R is empty, in which case the SUM of COUNTs will return NULL.
+      // However the original COUNT(c) should have returned 0 instead of NULL in this
+      // case.
+      // Therefore, COUNT([ALL]) is transformed into zeroifnull(COUNT([ALL]) if
+      // i) There is no GROUP-BY clause, and
+      // ii) Other DISTINCT aggregates are present.
+      countAllMap_ = createCountAllMap();
+      countAllMap_ = ExprSubstitutionMap.compose(ndvSmap_, countAllMap_, analyzer_);
+      List<Expr> substitutedAggs =
+          Expr.substituteList(aggExprs_, countAllMap_, analyzer_, false);
+      aggExprs_.clear();
+      TreeNode.collect(substitutedAggs, Expr.isAggregatePredicate(), aggExprs_);
+
+      List<Expr> groupingExprs = groupingExprsCopy_;
+      if (selectList_.isDistinct()) {
+        // Create multiAggInfo for SELECT DISTINCT:
+        // - all select list items turn into grouping exprs
+        // - there are no aggregate exprs
+        Preconditions.checkState(groupingExprsCopy_.isEmpty());
+        Preconditions.checkState(aggExprs_.isEmpty());
+        groupingExprs = Expr.cloneList(resultExprs_);
+      }
+      Expr.removeDuplicates(aggExprs_);
+      Expr.removeDuplicates(groupingExprs);
+      multiAggInfo_ = new MultiAggregateInfo(groupingExprs, aggExprs_);
+      multiAggInfo_.analyze(analyzer_);
     }
-    if (havingPred_ != null) {
-      // Make sure the predicate in the HAVING clause does not contain a
-      // subquery.
-      Preconditions.checkState(!havingPred_.contains(
-          Predicates.instanceOf(Subquery.class)));
-      havingPred_ = havingPred_.substitute(combinedSmap, analyzer, false);
-      analyzer.registerConjuncts(havingPred_, true);
+
+    private void buildResultExprs() throws AnalysisException {
+      ExprSubstitutionMap finalOutputSmap = multiAggInfo_.getOutputSmap();
+      ExprSubstitutionMap combinedSmap =
+          ExprSubstitutionMap.compose(countAllMap_, finalOutputSmap, analyzer_);
+
+      // change select list, having and ordering exprs to point to agg output. We need
+      // to reanalyze the exprs at this point.
       if (LOG.isTraceEnabled()) {
-        LOG.trace("post-agg havingPred: " + havingPred_.debugString());
+        LOG.trace("combined smap: " + combinedSmap.debugString());
+        LOG.trace("desctbl: " + analyzer_.getDescTbl().debugString());
+        LOG.trace("resultexprs: " + Expr.debugString(resultExprs_));
       }
-    }
-    if (sortInfo_ != null) {
-      sortInfo_.substituteSortExprs(combinedSmap, analyzer);
+      resultExprs_ = Expr.substituteList(resultExprs_, combinedSmap, analyzer_, false);
       if (LOG.isTraceEnabled()) {
-        LOG.trace("post-agg orderingExprs: " +
-            Expr.debugString(sortInfo_.getSortExprs()));
+        LOG.trace("post-agg selectListExprs: " + Expr.debugString(resultExprs_));
+      }
+      if (havingPred_ != null) {
+        // Make sure the predicate in the HAVING clause does not contain a
+        // subquery.
+        Preconditions.checkState(!havingPred_.contains(
+            Predicates.instanceOf(Subquery.class)));
+        havingPred_ = havingPred_.substitute(combinedSmap, analyzer_, false);
+        analyzer_.registerConjuncts(havingPred_, true);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("post-agg havingPred: " + havingPred_.debugString());
+        }
+      }
+      if (sortInfo_ != null) {
+        sortInfo_.substituteSortExprs(combinedSmap, analyzer_);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("post-agg orderingExprs: " +
+              Expr.debugString(sortInfo_.getSortExprs()));
+        }
       }
     }
 
-    // check that all post-agg exprs point to agg output
-    for (int i = 0; i < selectList_.getItems().size(); ++i) {
-      if (!resultExprs_.get(i).isBound(multiAggInfo_.getResultTupleId())) {
-        SelectListItem selectListItem = selectList_.getItems().get(i);
-        throw new AnalysisException(
-            "select list expression not produced by aggregation output "
-            + "(missing from GROUP BY clause?): "
-            + selectListItem.toSql());
+    private void verifyAggregation() throws AnalysisException {
+      // check that all post-agg exprs point to agg output
+      for (int i = 0; i < selectList_.getItems().size(); ++i) {
+        if (!resultExprs_.get(i).isBound(multiAggInfo_.getResultTupleId())) {
+          SelectListItem selectListItem = selectList_.getItems().get(i);
+          throw new AnalysisException(
+              "select list expression not produced by aggregation output "
+              + "(missing from GROUP BY clause?): "
+              + selectListItem.toSql());
+        }
       }
-    }
-    if (orderByElements_ != null) {
-      for (int i = 0; i < orderByElements_.size(); ++i) {
-        if (!sortInfo_.getSortExprs().get(i).isBound(multiAggInfo_.getResultTupleId())) {
+      if (orderByElements_ != null) {
+        for (int i = 0; i < orderByElements_.size(); ++i) {
+          if (!sortInfo_.getSortExprs().get(i).isBound(
+              multiAggInfo_.getResultTupleId())) {
+            throw new AnalysisException(
+                "ORDER BY expression not produced by aggregation output "
+                + "(missing from GROUP BY clause?): "
+                + orderByElements_.get(i).getExpr().toSql());
+          }
+        }
+      }
+      if (havingPred_ != null) {
+        if (!havingPred_.isBound(multiAggInfo_.getResultTupleId())) {
           throw new AnalysisException(
-              "ORDER BY expression not produced by aggregation output "
+              "HAVING clause not produced by aggregation output "
               + "(missing from GROUP BY clause?): "
-              + orderByElements_.get(i).getExpr().toSql());
+              + havingClause_.toSql());
         }
       }
     }
-    if (havingPred_ != null) {
-      if (!havingPred_.isBound(multiAggInfo_.getResultTupleId())) {
-        throw new AnalysisException(
-            "HAVING clause not produced by aggregation output "
-            + "(missing from GROUP BY clause?): "
-            + havingClause_.toSql());
+
+    /**
+     * Create a map from COUNT([ALL]) -> zeroifnull(COUNT([ALL])) if
+     * i) There is no GROUP-BY, and
+     * ii) There are other distinct aggregates to be evaluated.
+     * This transformation is necessary for COUNT to correctly return 0
+     * for empty input relations.
+     */
+    private ExprSubstitutionMap createCountAllMap()
+        throws AnalysisException {
+      ExprSubstitutionMap scalarCountAllMap = new ExprSubstitutionMap();
+
+      if (groupingExprs_ != null && !groupingExprs_.isEmpty()) {
+        // There are grouping expressions, so no substitution needs to be done.
+        return scalarCountAllMap;
       }
-    }
-  }
 
-  /**
-   * Create a map from COUNT([ALL]) -> zeroifnull(COUNT([ALL])) if
-   * i) There is no GROUP-BY, and
-   * ii) There are other distinct aggregates to be evaluated.
-   * This transformation is necessary for COUNT to correctly return 0 for empty
-   * input relations.
-   */
-  private ExprSubstitutionMap createCountAllMap(
-      List<FunctionCallExpr> aggExprs, Analyzer analyzer)
-      throws AnalysisException {
-    ExprSubstitutionMap scalarCountAllMap = new ExprSubstitutionMap();
+      com.google.common.base.Predicate<FunctionCallExpr> isNotDistinctPred =
+          new com.google.common.base.Predicate<FunctionCallExpr>() {
+            public boolean apply(FunctionCallExpr expr) {
+              return !expr.isDistinct();
+            }
+          };
+      if (Iterables.all(aggExprs_, isNotDistinctPred)) {
+        // Only [ALL] aggs, so no substitution needs to be done.
+        return scalarCountAllMap;
+      }
 
-    if (groupingExprs_ != null && !groupingExprs_.isEmpty()) {
-      // There are grouping expressions, so no substitution needs to be done.
-      return scalarCountAllMap;
-    }
+      com.google.common.base.Predicate<FunctionCallExpr> isCountPred =
+          new com.google.common.base.Predicate<FunctionCallExpr>() {
+            public boolean apply(FunctionCallExpr expr) {
+              return expr.getFnName().getFunction().equals("count");
+            }
+          };
+
+      Iterable<FunctionCallExpr> countAllAggs =
+          Iterables.filter(aggExprs_, Predicates.and(isCountPred, isNotDistinctPred));
+      for (FunctionCallExpr countAllAgg: countAllAggs) {
+        // Replace COUNT(ALL) with zeroifnull(COUNT(ALL))
+        ArrayList<Expr> zeroIfNullParam = Lists.newArrayList(countAllAgg.clone());
+        FunctionCallExpr zeroIfNull =
+            new FunctionCallExpr("zeroifnull", zeroIfNullParam);
+        zeroIfNull.analyze(analyzer_);
+        scalarCountAllMap.put(countAllAgg, zeroIfNull);
+      }
 
-    com.google.common.base.Predicate<FunctionCallExpr> isNotDistinctPred =
-        new com.google.common.base.Predicate<FunctionCallExpr>() {
-          public boolean apply(FunctionCallExpr expr) {
-            return !expr.isDistinct();
-          }
-        };
-    if (Iterables.all(aggExprs, isNotDistinctPred)) {
-      // Only [ALL] aggs, so no substitution needs to be done.
       return scalarCountAllMap;
     }
 
-    com.google.common.base.Predicate<FunctionCallExpr> isCountPred =
-        new com.google.common.base.Predicate<FunctionCallExpr>() {
-          public boolean apply(FunctionCallExpr expr) {
-            return expr.getFnName().getFunction().equals("count");
+    /**
+     * If the select list contains AnalyticExprs, create AnalyticInfo and substitute
+     * AnalyticExprs using the AnalyticInfo's smap.
+     */
+    private void createAnalyticInfo()
+        throws AnalysisException {
+      // collect AnalyticExprs from the SELECT and ORDER BY clauses
+      ArrayList<Expr> analyticExprs = Lists.newArrayList();
+      TreeNode.collect(resultExprs_, AnalyticExpr.class, analyticExprs);
+      if (sortInfo_ != null) {
+        TreeNode.collect(sortInfo_.getSortExprs(), AnalyticExpr.class,
+            analyticExprs);
+      }
+      if (analyticExprs.isEmpty()) return;
+      ExprSubstitutionMap rewriteSmap = new ExprSubstitutionMap();
+      for (Expr expr: analyticExprs) {
+        AnalyticExpr toRewrite = (AnalyticExpr)expr;
+        Expr newExpr = AnalyticExpr.rewrite(toRewrite);
+        if (newExpr != null) {
+          newExpr.analyze(analyzer_);
+          if (!rewriteSmap.containsMappingFor(toRewrite)) {
+            rewriteSmap.put(toRewrite, newExpr);
           }
-        };
-
-    Iterable<FunctionCallExpr> countAllAggs =
-        Iterables.filter(aggExprs, Predicates.and(isCountPred, isNotDistinctPred));
-    for (FunctionCallExpr countAllAgg: countAllAggs) {
-      // Replace COUNT(ALL) with zeroifnull(COUNT(ALL))
-      ArrayList<Expr> zeroIfNullParam = Lists.newArrayList(countAllAgg.clone());
-      FunctionCallExpr zeroIfNull =
-          new FunctionCallExpr("zeroifnull", zeroIfNullParam);
-      zeroIfNull.analyze(analyzer);
-      scalarCountAllMap.put(countAllAgg, zeroIfNull);
-    }
+        }
+      }
+      if (rewriteSmap.size() > 0) {
+        // Substitute the exprs with their rewritten versions.
+        ArrayList<Expr> updatedAnalyticExprs =
+            Expr.substituteList(analyticExprs, rewriteSmap, analyzer_, false);
+        // This is to get rid the original exprs which have been rewritten.
+        analyticExprs.clear();
+        // Collect the new exprs introduced through the rewrite and the
+        // non-rewrite exprs.
+        TreeNode.collect(updatedAnalyticExprs, AnalyticExpr.class, analyticExprs);
+      }
 
-    return scalarCountAllMap;
-  }
+      analyticInfo_ = AnalyticInfo.create(analyticExprs, analyzer_);
 
-  /**
-   * If the select list contains AnalyticExprs, create AnalyticInfo and substitute
-   * AnalyticExprs using the AnalyticInfo's smap.
-   */
-  private void createAnalyticInfo(Analyzer analyzer)
-      throws AnalysisException {
-    // collect AnalyticExprs from the SELECT and ORDER BY clauses
-    ArrayList<Expr> analyticExprs = Lists.newArrayList();
-    TreeNode.collect(resultExprs_, AnalyticExpr.class, analyticExprs);
-    if (sortInfo_ != null) {
-      TreeNode.collect(sortInfo_.getSortExprs(), AnalyticExpr.class,
-          analyticExprs);
-    }
-    if (analyticExprs.isEmpty()) return;
-    ExprSubstitutionMap rewriteSmap = new ExprSubstitutionMap();
-    for (Expr expr: analyticExprs) {
-      AnalyticExpr toRewrite = (AnalyticExpr)expr;
-      Expr newExpr = AnalyticExpr.rewrite(toRewrite);
-      if (newExpr != null) {
-        newExpr.analyze(analyzer);
-        if (!rewriteSmap.containsMappingFor(toRewrite)) {
-          rewriteSmap.put(toRewrite, newExpr);
+      ExprSubstitutionMap smap = analyticInfo_.getSmap();
+      // If 'exprRewritten' is true, we have to compose the new smap with
+      // the existing one.
+      if (rewriteSmap.size() > 0) {
+        smap = ExprSubstitutionMap.compose(
+            rewriteSmap, analyticInfo_.getSmap(), analyzer_);
+      }
+      // change select list and ordering exprs to point to analytic output. We need
+      // to reanalyze the exprs at this point.
+      resultExprs_ = Expr.substituteList(resultExprs_, smap, analyzer_, false);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("post-analytic selectListExprs: " + Expr.debugString(resultExprs_));
+      }
+      if (sortInfo_ != null) {
+        sortInfo_.substituteSortExprs(smap, analyzer_);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("post-analytic orderingExprs: " +
+              Expr.debugString(sortInfo_.getSortExprs()));
         }
       }
     }
-    if (rewriteSmap.size() > 0) {
-      // Substitute the exprs with their rewritten versions.
-      ArrayList<Expr> updatedAnalyticExprs =
-          Expr.substituteList(analyticExprs, rewriteSmap, analyzer, false);
-      // This is to get rid the original exprs which have been rewritten.
-      analyticExprs.clear();
-      // Collect the new exprs introduced through the rewrite and the non-rewrite exprs.
-      TreeNode.collect(updatedAnalyticExprs, AnalyticExpr.class, analyticExprs);
+  }
+
+  /**
+   * Marks all unassigned join predicates as well as exprs in aggInfo and sortInfo.
+   */
+  @Override
+  public void materializeRequiredSlots(Analyzer analyzer) {
+    // Mark unassigned join predicates. Some predicates that must be evaluated by a join
+    // can also be safely evaluated below the join (picked up by getBoundPredicates()).
+    // Such predicates will be marked twice and that is ok.
+    List<Expr> unassigned =
+        analyzer.getUnassignedConjuncts(getTableRefIds(), true);
+    List<Expr> unassignedJoinConjuncts = Lists.newArrayList();
+    for (Expr e: unassigned) {
+      if (analyzer.evalAfterJoin(e)) unassignedJoinConjuncts.add(e);
     }
+    List<Expr> baseTblJoinConjuncts =
+        Expr.substituteList(unassignedJoinConjuncts, baseTblSmap_, analyzer, false);
+    materializeSlots(analyzer, baseTblJoinConjuncts);
 
-    analyticInfo_ = AnalyticInfo.create(analyticExprs, analyzer);
+    if (evaluateOrderBy_) {
+      // mark ordering exprs before marking agg/analytic exprs because they could contain
+      // agg/analytic exprs that are not referenced anywhere but the ORDER BY clause
+      sortInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
+    }
 
-    ExprSubstitutionMap smap = analyticInfo_.getSmap();
-    // If 'exprRewritten' is true, we have to compose the new smap with the existing one.
-    if (rewriteSmap.size() > 0) {
-      smap = ExprSubstitutionMap.compose(
-          rewriteSmap, analyticInfo_.getSmap(), analyzer);
+    if (hasAnalyticInfo()) {
+      // Mark analytic exprs before marking agg exprs because they could contain agg
+      // exprs that are not referenced anywhere but the analytic expr.
+      // Gather unassigned predicates and mark their slots. It is not desirable
+      // to account for propagated predicates because if an analytic expr is only
+      // referenced by a propagated predicate, then it's better to not materialize the
+      // analytic expr at all.
+      ArrayList<TupleId> tids = Lists.newArrayList();
+      getMaterializedTupleIds(tids); // includes the analytic tuple
+      List<Expr> conjuncts = analyzer.getUnassignedConjuncts(tids, false);
+      materializeSlots(analyzer, conjuncts);
+      analyticInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
     }
-    // change select list and ordering exprs to point to analytic output. We need
-    // to reanalyze the exprs at this point.
-    resultExprs_ = Expr.substituteList(resultExprs_, smap, analyzer, false);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("post-analytic selectListExprs: " + Expr.debugString(resultExprs_));
+
+    if (multiAggInfo_ != null) {
+      // Mark all agg slots required for conjunct evaluation as materialized before
+      // calling MultiAggregateInfo.materializeRequiredSlots().
+      List<Expr> conjuncts = multiAggInfo_.collectConjuncts(analyzer, false);
+      materializeSlots(analyzer, conjuncts);
+      multiAggInfo_.materializeRequiredSlots(analyzer, baseTblSmap_);
     }
-    if (sortInfo_ != null) {
-      sortInfo_.substituteSortExprs(smap, analyzer);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("post-analytic orderingExprs: " +
-            Expr.debugString(sortInfo_.getSortExprs()));
-      }
+  }
+
+  public List<TupleId> getTableRefIds() {
+    List<TupleId> result = Lists.newArrayList();
+    for (TableRef ref: fromClause_) {
+      result.add(ref.getId());
     }
+    return result;
   }
 
   /**


[07/33] impala git commit: IMPALA-7777: Fix crash due to arithmetic overflows in Exchange Node

Posted by bo...@apache.org.
IMPALA-7777: Fix crash due to arithmetic overflows in Exchange Node

Fixes an arithmetic overflow in ExchangeNode::GetNextMerging. Prior to
this patch, the code read:

int rows_to_keep = num_rows_skipped_ - offset_;

Where num_rows_skipped_ and offset_ were of type int64_t. The result was
cast to an int which can lead to an overflow if the result exceeds the
value of 2^31. The value of rows_to_keep would be passed into
row-batch.h::CopyRows which would crash due to a DCHECK_LE error.

This crash arises when the value of the OFFSET is a large number, for
example, the query:

select int_col from functional.alltypes order by 1 limit
1 offset 9223372036854775800;

Would crash the Impalad executor for this query.

The fix is to change rows_to_keep to an int64_t to avoid the overflow,
which prevents the DCHECK_LE from failing.

Change-Id: I8bb8064aae6ad25c8a19f6a8869086be7e70400a
Reviewed-on: http://gerrit.cloudera.org:8080/11844
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: c73530d1b1ad566f34f68b677c8423fb0abafb27
Parents: e5d0757
Author: stakiar <ta...@gmail.com>
Authored: Wed Oct 31 12:06:08 2018 -0500
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/exec/exchange-node.cc                                 | 2 +-
 be/src/runtime/row-batch.h                                   | 2 +-
 .../workloads/functional-query/queries/QueryTest/top-n.test  | 8 ++++++++
 3 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/c73530d1/be/src/exec/exchange-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exchange-node.cc b/be/src/exec/exchange-node.cc
index 26579c1..ca8b973 100644
--- a/be/src/exec/exchange-node.cc
+++ b/be/src/exec/exchange-node.cc
@@ -221,7 +221,7 @@ Status ExchangeNode::GetNextMerging(RuntimeState* state, RowBatch* output_batch,
   while (num_rows_skipped_ < offset_) {
     num_rows_skipped_ += output_batch->num_rows();
     // Throw away rows in the output batch until the offset is skipped.
-    int rows_to_keep = num_rows_skipped_ - offset_;
+    int64_t rows_to_keep = num_rows_skipped_ - offset_;
     if (rows_to_keep > 0) {
       output_batch->CopyRows(0, output_batch->num_rows() - rows_to_keep, rows_to_keep);
       output_batch->set_num_rows(rows_to_keep);

http://git-wip-us.apache.org/repos/asf/impala/blob/c73530d1/be/src/runtime/row-batch.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/row-batch.h b/be/src/runtime/row-batch.h
index 1334858..dd737d7 100644
--- a/be/src/runtime/row-batch.h
+++ b/be/src/runtime/row-batch.h
@@ -330,7 +330,7 @@ class RowBatch {
 
   /// Copy 'num_rows' rows from 'src' to 'dest' within the batch. Useful for exec
   /// nodes that skip an offset and copied more than necessary.
-  void CopyRows(int dest, int src, int num_rows) {
+  void CopyRows(int64_t dest, int64_t src, int64_t num_rows) {
     DCHECK_LE(dest, src);
     DCHECK_LE(src + num_rows, capacity_);
     memmove(tuple_ptrs_ + num_tuples_per_row_ * dest,

http://git-wip-us.apache.org/repos/asf/impala/blob/c73530d1/testdata/workloads/functional-query/queries/QueryTest/top-n.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/top-n.test b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
index 32b1504..37c986c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/top-n.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
@@ -1384,3 +1384,11 @@ select cast(string_col as char(20)) from alltypes order by 1 limit 5
 ---- TYPES
 CHAR
 ====
+---- QUERY
+# Test queries with the maximum value for the limit and offset
+select string_col from alltypes order by 1 limit 9223372036854775807
+offset 9223372036854775807
+---- RESULTS
+---- TYPES
+STRING
+====
\ No newline at end of file


[26/33] impala git commit: IMPALA-7822: handle overflows in repeat() builtin

Posted by bo...@apache.org.
IMPALA-7822: handle overflows in repeat() builtin

We need to carefully check that the intermediate value fits in an
int64_t and the final size fits in an int. If they don't we
raise an error and fail the query.

Testing:
Added a couple of backend tests to exercise the
overflow check code paths.

Change-Id: I872ce77bc2cb29116881c27ca2a5216f722cdb2a
Reviewed-on: http://gerrit.cloudera.org:8080/11889
Reviewed-by: Thomas Marshall <th...@cmu.edu>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 1760339a5fb4375a22c3579e5eeb505335bcf8f6
Parents: ac2b7c9
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Nov 6 13:31:38 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:40 2018 +0100

----------------------------------------------------------------------
 be/src/exprs/expr-test.cc                           |  8 ++++++--
 be/src/exprs/string-functions-ir.cc                 | 16 +++++++++++++++-
 .../queries/QueryTest/large_strings.test            |  9 +++++++--
 3 files changed, 28 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/1760339a/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index dd6f1b3..bab16ca 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -611,8 +611,8 @@ class ExprTest : public testing::Test {
     Status status = executor_->Exec(stmt, &result_types);
     status = executor_->FetchResult(&result_row);
     ASSERT_FALSE(status.ok());
-    ASSERT_TRUE(EndsWith(status.msg().msg(), error_string)) << "Actual: "
-        << status.msg().msg() << endl << "Expected: " << error_string;
+    ASSERT_TRUE(EndsWith(status.msg().msg(), error_string)) << "Actual: '"
+        << status.msg().msg() << "'" << endl << "Expected: '" << error_string << "'";
   }
 
   template <typename T> void TestFixedPointComparisons(bool test_boundaries) {
@@ -4082,6 +4082,10 @@ TEST_F(ExprTest, StringFunctions) {
   TestIsNull("repeat(NULL, 6)", TYPE_STRING);
   TestIsNull("repeat('ab', NULL)", TYPE_STRING);
   TestIsNull("repeat(NULL, NULL)", TYPE_STRING);
+  TestErrorString("repeat('x', 1024 * 1024 * 1024 * 10)", "Number of repeats in "
+      "repeat() call is larger than allowed limit of 1 GB character data.\n");
+  TestErrorString("repeat('xx', 1024 * 1024 * 1024)", "repeat() result is larger than "
+      "allowed limit of 1 GB character data.\n");
 
   TestValue("ascii('')", TYPE_INT, 0);
   TestValue("ascii('abcde')", TYPE_INT, 'a');

http://git-wip-us.apache.org/repos/asf/impala/blob/1760339a/be/src/exprs/string-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/string-functions-ir.cc b/be/src/exprs/string-functions-ir.cc
index f5e8213..258f2cb 100644
--- a/be/src/exprs/string-functions-ir.cc
+++ b/be/src/exprs/string-functions-ir.cc
@@ -93,7 +93,21 @@ StringVal StringFunctions::Repeat(
     FunctionContext* context, const StringVal& str, const BigIntVal& n) {
   if (str.is_null || n.is_null) return StringVal::null();
   if (str.len == 0 || n.val <= 0) return StringVal();
-  StringVal result(context, str.len * n.val);
+  if (n.val > StringVal::MAX_LENGTH) {
+    context->SetError("Number of repeats in repeat() call is larger than allowed limit "
+        "of 1 GB character data.");
+    return StringVal::null();
+  }
+  static_assert(numeric_limits<int64_t>::max() / numeric_limits<int>::max()
+      >= StringVal::MAX_LENGTH,
+      "multiplying StringVal::len with positive int fits in int64_t");
+  int64_t out_len = str.len * n.val;
+  if (out_len > StringVal::MAX_LENGTH) {
+    context->SetError(
+        "repeat() result is larger than allowed limit of 1 GB character data.");
+    return StringVal::null();
+  }
+  StringVal result(context, static_cast<int>(out_len));
   if (UNLIKELY(result.is_null)) return StringVal::null();
   uint8_t* ptr = result.ptr;
   for (int64_t i = 0; i < n.val; ++i) {

http://git-wip-us.apache.org/repos/asf/impala/blob/1760339a/testdata/workloads/functional-query/queries/QueryTest/large_strings.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/large_strings.test b/testdata/workloads/functional-query/queries/QueryTest/large_strings.test
index 1d930db..0cca71b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/large_strings.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/large_strings.test
@@ -129,7 +129,7 @@ select length(repeat(s, 10)) from (
     select l_comment from tpch.lineitem) t1
   ) t2
 ---- CATCH
-String length larger than allowed limit of 1 GB character data
+repeat() result is larger than allowed limit of 1 GB character data
 =====
 ---- QUERY
 select length(lpad(s, 1073741830, '!')) from (
@@ -225,4 +225,9 @@ select cast(fnv_hash(concat(l_comment, 'e')) as string) as h from tpch_parquet.l
 INT,INT
 ---- RESULTS
 611468161,611468161
-=====
\ No newline at end of file
+=====
+---- QUERY
+select repeat('the quick brown fox', 1024 * 1024 * 100)
+---- CATCH
+repeat() result is larger than allowed limit of 1 GB character data
+=====


[14/33] impala git commit: IMPALA-5004: Switch to sorting node for large TopN queries

Posted by bo...@apache.org.
IMPALA-5004: Switch to sorting node for large TopN queries

Adds a new query option 'topn_bytes_limit' that places a limit on the
number of estimated bytes that a TopN operator can process. If the
Impala planner estimates that a TopN operator will process more bytes
than this limit, it will replace the TopN operator with a sort operator.

Since the TopN operator cannot spill to disk, it has to buffer everything
in memory. This can cause frequent OOM issues when running with a large
limit + offset. Switching to a sort operator allows Impala to spill to
disk. We prefer to use the TopN operator when possible as it has better
performance than the sort operator for 'order by limit [offset]' queries.

The default limit is set to 512MB and is based on micro-benchmarking the
topn vs. sort operator for various limits (see the JIRA for full details).
The default is set to an intentionally high value in order to avoid
performance regressions.

Testing:

* Added a new planner test to fuctional-planner/ to validate that
'topn_bytes_limit' properly switches between topn and sort operators.

Change-Id: I34c9db33c9302b55e9978f53f9c7061f2806c8a9
Reviewed-on: http://gerrit.cloudera.org:8080/11698
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 3a7e382805d1d19a3a9c7c69c7ff1adce9677675
Parents: 030f0ac
Author: stakiar <ta...@gmail.com>
Authored: Thu Oct 4 16:50:53 2018 -0500
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 be/src/service/query-options-test.cc            |  1 +
 be/src/service/query-options.cc                 |  6 ++
 be/src/service/query-options.h                  |  3 +-
 common/thrift/ImpalaInternalService.thrift      |  4 ++
 common/thrift/ImpalaService.thrift              |  6 ++
 .../org/apache/impala/analysis/SortInfo.java    | 14 ++++
 .../apache/impala/planner/AggregationNode.java  |  2 +-
 .../apache/impala/planner/AnalyticEvalNode.java |  2 +-
 .../impala/planner/DataSourceScanNode.java      |  2 +-
 .../impala/planner/DistributedPlanner.java      |  2 +-
 .../apache/impala/planner/HBaseScanNode.java    |  2 +-
 .../org/apache/impala/planner/HdfsScanNode.java |  2 +-
 .../org/apache/impala/planner/JoinNode.java     |  2 +-
 .../org/apache/impala/planner/KuduScanNode.java |  2 +-
 .../org/apache/impala/planner/PlanNode.java     | 12 ++--
 .../org/apache/impala/planner/SelectNode.java   |  2 +-
 .../impala/planner/SingleNodePlanner.java       | 54 +++++++++++----
 .../org/apache/impala/planner/SortNode.java     |  7 +-
 .../org/apache/impala/planner/SubplanNode.java  |  2 +-
 .../org/apache/impala/planner/UnionNode.java    |  2 +-
 .../org/apache/impala/planner/UnnestNode.java   |  2 +-
 .../org/apache/impala/planner/PlannerTest.java  | 16 ++++-
 .../PlannerTest/topn-bytes-limit-small.test     | 72 ++++++++++++++++++++
 .../queries/PlannerTest/topn-bytes-limit.test   | 23 +++++++
 .../QueryTest/spilling-no-debug-action.test     |  1 +
 25 files changed, 205 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/be/src/service/query-options-test.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 114d830..059b355 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -144,6 +144,7 @@ TEST(QueryOptions, SetByteOptions) {
       {MAKE_OPTIONDEF(compute_stats_min_sample_size), {-1, I64_MAX}},
       {MAKE_OPTIONDEF(max_mem_estimate_for_admission), {-1, I64_MAX}},
       {MAKE_OPTIONDEF(scan_bytes_limit), {-1, I64_MAX}},
+      {MAKE_OPTIONDEF(topn_bytes_limit), {-1, I64_MAX}},
   };
   vector<pair<OptionDef<int32_t>, Range<int32_t>>> case_set_i32{
       {MAKE_OPTIONDEF(runtime_filter_min_size),

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/be/src/service/query-options.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 1424896..bb49762 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -714,6 +714,12 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_cpu_limit_s(cpu_limit_s);
         break;
       }
+      case TImpalaQueryOptions::TOPN_BYTES_LIMIT: {
+        int64_t topn_bytes_limit;
+        RETURN_IF_ERROR(ParseMemValue(value, "topn bytes limit", &topn_bytes_limit));
+        query_options->__set_topn_bytes_limit(topn_bytes_limit);
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/be/src/service/query-options.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 46cdf05..95263f7 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -41,7 +41,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // the DCHECK.
 #define QUERY_OPTS_TABLE\
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),\
-      TImpalaQueryOptions::CPU_LIMIT_S + 1);\
+      TImpalaQueryOptions::TOPN_BYTES_LIMIT + 1);\
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED)\
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)\
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)\
@@ -143,6 +143,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
   QUERY_OPT_FN(scan_bytes_limit, SCAN_BYTES_LIMIT,\
       TQueryOptionLevel::ADVANCED)\
   QUERY_OPT_FN(cpu_limit_s, CPU_LIMIT_S, TQueryOptionLevel::DEVELOPMENT)\
+  QUERY_OPT_FN(topn_bytes_limit, TOPN_BYTES_LIMIT, TQueryOptionLevel::ADVANCED)\
   ;
 
 /// Enforce practical limits on some query options to avoid undesired query state.

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/common/thrift/ImpalaInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index 7222dae..47f43e2 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -304,6 +304,10 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift.
   72: optional i64 cpu_limit_s = 0;
+
+  // See comment in ImpalaService.thrift
+  // The default value is set to 512MB based on empirical data
+  73: optional i64 topn_bytes_limit = 536870912;
 }
 
 // Impala currently has two types of sessions: Beeswax and HiveServer2

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/common/thrift/ImpalaService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index 05a1431..758617d 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -339,6 +339,12 @@ enum TImpalaQueryOptions {
   // Note that until IMPALA-7318 is fixed, CPU usage can be very stale and this may not
   // terminate queries soon enough.
   CPU_LIMIT_S,
+
+  // The max number of estimated bytes a TopN operator is allowed to materialize, if the
+  // planner thinks a TopN operator will exceed this limit, it falls back to a TotalSort
+  // operator which is capable of spilling to disk (unlike the TopN operator which keeps
+  // everything in memory). 0 or -1 means this has no effect.
+  TOPN_BYTES_LIMIT,
 }
 
 // The summary of a DML statement.

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
index fba7286..8472725 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
@@ -16,11 +16,13 @@
 // under the License.
 
 package org.apache.impala.analysis;
+
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.impala.common.TreeNode;
+import org.apache.impala.planner.PlanNode;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
@@ -233,6 +235,18 @@ public class SortInfo {
   }
 
   /**
+   * Estimates the size of the data materialized in memory by the TopN operator. The
+   * method uses the formula <code>estimatedSize = estimated # of rows in memory *
+   * average tuple serialized size</code>. 'cardinality' is the cardinality of the TopN
+   * operator and 'offset' is the value in the 'OFFSET [x]' clause.
+   */
+  public long estimateTopNMaterializedSize(long cardinality, long offset) {
+    getSortTupleDescriptor().computeMemLayout();
+    return (long) Math.ceil(getSortTupleDescriptor().getAvgSerializedSize()
+        * (PlanNode.checkedAdd(cardinality, offset)));
+  }
+
+  /**
    * Returns the subset of 'sortExprs_' that should be materialized. A sort expr is
    * is materialized if it:
    * - contains a non-deterministic expr

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
index ab234e4..72694ca 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -249,7 +249,7 @@ public class AggregationNode extends PlanNode {
         cardinality_ = Math.min(getChild(0).getCardinality(), cardinality_);
       }
     }
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
index 5ca666c..86e5166 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
@@ -138,7 +138,7 @@ public class AnalyticEvalNode extends PlanNode {
   protected void computeStats(Analyzer analyzer) {
     super.computeStats(analyzer);
     cardinality_ = getChild(0).cardinality_;
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
index 1ddb394..d4740e8 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -268,7 +268,7 @@ public class DataSourceScanNode extends ScanNode {
     cardinality_ = numRowsEstimate_;
     cardinality_ *= computeSelectivity();
     cardinality_ = Math.max(1, cardinality_);
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("computeStats DataSourceScan: cardinality=" + Long.toString(cardinality_));

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
index ac3115b..05563b0 100644
--- a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
@@ -1054,7 +1054,7 @@ public class DistributedPlanner {
     Preconditions.checkState(node == childSortNode);
     if (hasLimit) {
       childSortNode.unsetLimit();
-      childSortNode.setLimit(limit + offset);
+      childSortNode.setLimit(PlanNode.checkedAdd(limit, offset));
     }
     childSortNode.setOffset(0);
     childSortNode.computeStats(ctx_.getRootAnalyzer());

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
index 13ecb6a..1b60ad4 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -226,7 +226,7 @@ public class HBaseScanNode extends ScanNode {
 
     cardinality_ *= computeSelectivity();
     cardinality_ = Math.max(1, cardinality_);
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("computeStats HbaseScan: cardinality=" + Long.toString(cardinality_));
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/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 0246d8c..c1ff092 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -1054,7 +1054,7 @@ public class HdfsScanNode extends ScanNode {
       // IMPALA-2165: Avoid setting the cardinality to 0 after rounding.
       cardinality_ = Math.max(cardinality_, 1);
     }
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("HdfsScan: cardinality_=" + Long.toString(cardinality_));
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/JoinNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index e8adcfa..cc50318 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -617,7 +617,7 @@ public abstract class JoinNode extends PlanNode {
         break;
       }
     }
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     Preconditions.checkState(hasValidStats());
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Join: cardinality=" + Long.toString(cardinality_));

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/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 adeaa72..fc1f371 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -265,7 +265,7 @@ public class KuduScanNode extends ScanNode {
     inputCardinality_ = cardinality_ = kuduTable_.getNumRows();
     cardinality_ *= computeSelectivity();
     cardinality_ = Math.min(Math.max(1, cardinality_), kuduTable_.getNumRows());
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("computeStats KuduScan: cardinality=" + Long.toString(cardinality_));
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/PlanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index 744aa09..7751eeb 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -521,17 +521,17 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     if (!children_.isEmpty()) numNodes_ = getChild(0).numNodes_;
   }
 
-  protected long capAtLimit(long cardinality) {
+  protected long capCardinalityAtLimit(long cardinality) {
     if (hasLimit()) {
-      if (cardinality == -1) {
-        return limit_;
-      } else {
-        return Math.min(cardinality, limit_);
-      }
+      return capCardinalityAtLimit(cardinality, limit_);
     }
     return cardinality;
   }
 
+  static long capCardinalityAtLimit(long cardinality, long limit) {
+    return cardinality == -1 ? limit : Math.min(cardinality, limit);
+  }
+
   /**
    * Call computeMemLayout() for all materialized tuples.
    */

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/SelectNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SelectNode.java b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
index 3ffc975..7b637e6 100644
--- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
@@ -75,7 +75,7 @@ public class SelectNode extends PlanNode {
           Math.round(((double) getChild(0).cardinality_) * computeSelectivity());
       Preconditions.checkState(cardinality_ >= 0);
     }
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Select: cardinality=" + Long.toString(cardinality_));
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
index 313597e..740cbfa 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
@@ -48,6 +48,7 @@ import org.apache.impala.analysis.SingularRowSrcTableRef;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotId;
 import org.apache.impala.analysis.SlotRef;
+import org.apache.impala.analysis.SortInfo;
 import org.apache.impala.analysis.TableRef;
 import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.analysis.TupleId;
@@ -294,20 +295,8 @@ public class SingleNodePlanner {
     }
 
     if (stmt.evaluateOrderBy() && sortHasMaterializedSlots) {
-      long limit = stmt.getLimit();
-      // TODO: External sort could be used for very large limits
-      // not just unlimited order-by
-      boolean useTopN = stmt.hasLimit() && !disableTopN;
-      if (useTopN) {
-        root = SortNode.createTopNSortNode(
-            ctx_.getNextNodeId(), root, stmt.getSortInfo(), stmt.getOffset());
-      } else {
-        root = SortNode.createTotalSortNode(
-            ctx_.getNextNodeId(), root, stmt.getSortInfo(), stmt.getOffset());
-      }
-      Preconditions.checkState(root.hasValidStats());
-      root.setLimit(limit);
-      root.init(analyzer);
+      root = createSortNode(analyzer, root, stmt.getSortInfo(), stmt.getLimit(),
+          stmt.getOffset(), stmt.hasLimit(), disableTopN);
     } else {
       root.setLimit(stmt.getLimit());
       root.computeStats(analyzer);
@@ -317,6 +306,43 @@ public class SingleNodePlanner {
   }
 
   /**
+   * Creates and initializes either a SortNode or a TopNNode depending on various
+   * heuristics and configuration parameters.
+   */
+  private SortNode createSortNode(Analyzer analyzer, PlanNode root, SortInfo sortInfo,
+      long limit, long offset, boolean hasLimit, boolean disableTopN)
+      throws ImpalaException {
+    SortNode sortNode;
+    long topNBytesLimit = ctx_.getQueryOptions().topn_bytes_limit;
+
+    if (hasLimit && !disableTopN) {
+      if (topNBytesLimit <= 0) {
+        sortNode =
+            SortNode.createTopNSortNode(ctx_.getNextNodeId(), root, sortInfo, offset);
+      } else {
+        long topNCardinality = PlanNode.capCardinalityAtLimit(root.cardinality_, limit);
+        long estimatedTopNMaterializedSize =
+            sortInfo.estimateTopNMaterializedSize(topNCardinality, offset);
+
+        if (estimatedTopNMaterializedSize < topNBytesLimit) {
+          sortNode =
+              SortNode.createTopNSortNode(ctx_.getNextNodeId(), root, sortInfo, offset);
+        } else {
+          sortNode =
+              SortNode.createTotalSortNode(ctx_.getNextNodeId(), root, sortInfo, offset);
+        }
+      }
+    } else {
+      sortNode =
+          SortNode.createTotalSortNode(ctx_.getNextNodeId(), root, sortInfo, offset);
+    }
+    Preconditions.checkState(sortNode.hasValidStats());
+    sortNode.setLimit(limit);
+    sortNode.init(analyzer);
+    return sortNode;
+  }
+
+  /**
    * If there are unassigned conjuncts that are bound by tupleIds or if there are slot
    * equivalences for tupleIds that have not yet been enforced, returns a SelectNode on
    * top of root that evaluates those conjuncts; otherwise returns root unchanged.

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/SortNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SortNode.java b/fe/src/main/java/org/apache/impala/planner/SortNode.java
index ccef721..e23b933 100644
--- a/fe/src/main/java/org/apache/impala/planner/SortNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SortNode.java
@@ -164,7 +164,7 @@ public class SortNode extends PlanNode {
   @Override
   protected void computeStats(Analyzer analyzer) {
     super.computeStats(analyzer);
-    cardinality_ = capAtLimit(getChild(0).cardinality_);
+    cardinality_ = capCardinalityAtLimit(getChild(0).cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Sort: cardinality=" + Long.toString(cardinality_));
     }
@@ -257,9 +257,8 @@ public class SortNode extends PlanNode {
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     Preconditions.checkState(hasValidStats());
     if (type_ == TSortType.TOPN) {
-      long perInstanceMemEstimate =
-              (long) Math.ceil((cardinality_ + offset_) * avgRowSize_);
-      nodeResourceProfile_ = ResourceProfile.noReservation(perInstanceMemEstimate);
+      nodeResourceProfile_ = ResourceProfile.noReservation(
+          getSortInfo().estimateTopNMaterializedSize(cardinality_, offset_));
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
index a22c397..5b1e323 100644
--- a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
@@ -91,7 +91,7 @@ public class SubplanNode extends PlanNode {
     } else {
       cardinality_ = -1;
     }
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/UnionNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/UnionNode.java b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
index a8cda0d..00a6d20 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -124,7 +124,7 @@ public class UnionNode extends PlanNode {
     // are inline views (e.g. select 1 FROM (VALUES(1 x, 1 y)) a FULL OUTER JOIN
     // (VALUES(1 x, 1 y)) b ON (a.x = b.y)). We need to set the correct value.
     if (numNodes_ == -1) numNodes_ = 1;
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Union: cardinality=" + Long.toString(cardinality_));
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
index 7e0a87e..857a949 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
@@ -70,7 +70,7 @@ public class UnnestNode extends PlanNode {
     // The containing SubplanNode has not yet been initialized, so get the number
     // of nodes from the SubplanNode's input.
     numNodes_ = containingSubplanNode_.getChild(0).getNumNodes();
-    cardinality_ = capAtLimit(cardinality_);
+    cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index e8e4fb8..2ec4b15 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -193,7 +193,21 @@ public class PlannerTest extends PlannerTestBase {
 
   @Test
   public void testTopN() {
-    runPlannerTestFile("topn");
+    TQueryOptions options = new TQueryOptions();
+    options.setTopn_bytes_limit(0);
+    runPlannerTestFile("topn", options);
+  }
+
+  @Test
+  public void testTopNBytesLimit() {
+    runPlannerTestFile("topn-bytes-limit");
+  }
+
+  @Test
+  public void testTopNBytesLimitSmall() {
+    TQueryOptions options = new TQueryOptions();
+    options.setTopn_bytes_limit(6);
+    runPlannerTestFile("topn-bytes-limit-small", options);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test b/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
new file mode 100644
index 0000000..db3e9a5
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
@@ -0,0 +1,72 @@
+# topn_bytes_limit is set to 6 so a limit of 1 will return a single int
+# a single int is 4 bytes, which is under the limit of 6 so a TOP-N should be triggered
+select int_col from functional.alltypes order by 1 limit 1
+---- PLAN
+PLAN-ROOT SINK
+|
+01:TOP-N [LIMIT=1]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC
+|  limit: 1
+|
+01:TOP-N [LIMIT=1]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+# returns 2 ints, with a total size of 8 bytes, which exceeds the limit of 6 and thus triggers a SORT
+select int_col from functional.alltypes order by 1 limit 2
+---- PLAN
+PLAN-ROOT SINK
+|
+01:SORT [LIMIT=2]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC
+|  limit: 2
+|
+01:SORT [LIMIT=2]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+# test that offset is taken into account; the query only returns a single row but needs to sort two rows
+# sorting two ints requires 8 bytes of memory, which exceeds the threshold of 6
+select int_col from functional.alltypes order by 1 limit 1 offset 1
+---- PLAN
+PLAN-ROOT SINK
+|
+01:SORT [LIMIT=1 OFFSET=1]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  offset: 1
+|  order by: int_col ASC
+|  limit: 1
+|
+01:SORT [LIMIT=2]
+|  order by: int_col ASC
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test b/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
new file mode 100644
index 0000000..9ad000e
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
@@ -0,0 +1,23 @@
+# check that topn is triggered for low limits with the default value of topn_bytes_limit
+select id from functional.alltypestiny order by id limit 7
+---- PLAN
+PLAN-ROOT SINK
+|
+01:TOP-N [LIMIT=7]
+|  order by: id ASC
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partitions=4/4 files=4 size=460B
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: id ASC
+|  limit: 7
+|
+01:TOP-N [LIMIT=7]
+|  order by: id ASC
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partitions=4/4 files=4 size=460B
+====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/3a7e3828/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
index 7fe1c96..c82ea17 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
@@ -79,6 +79,7 @@ Memory limit exceeded
 ---- QUERY
 # Top-N query with large limit that will OOM because spilling is not implemented:
 # IMPALA-3471. It does not need any help from DEBUG_ACTION.
+set topn_bytes_limit=-1;
 set mem_limit=100m;
 select *
 from lineitem


[12/33] impala git commit: IMPALA-7244: [DOCS] Remove unsupported format writer support

Posted by bo...@apache.org.
IMPALA-7244: [DOCS] Remove unsupported format writer support

- Added a "removed" note for the ALLOW_UNSUPPORTED_FORMATS and
SEQ_COMPRESSION_MODE query options.
- Will remove the above options from the docs at the next
compatibility breaking release.

Change-Id: I363accf5f284d2a1535cea0652b2b579379b9588
Reviewed-on: http://gerrit.cloudera.org:8080/11842
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Bikramjeet Vig <bi...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: cb2574b8a131a1b15136ceb4a5f8d3896afa2731
Parents: c124d26
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Wed Oct 31 18:44:18 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 .../topics/impala_allow_unsupported_formats.xml |  7 +-
 docs/topics/impala_avro.xml                     | 19 +---
 docs/topics/impala_file_formats.xml             | 32 +++----
 docs/topics/impala_seq_compression_mode.xml     | 11 +--
 docs/topics/impala_seqfile.xml                  | 26 ++----
 docs/topics/impala_txtfile.xml                  | 96 ++++----------------
 6 files changed, 43 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_allow_unsupported_formats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_allow_unsupported_formats.xml b/docs/topics/impala_allow_unsupported_formats.xml
index 55ec545..d140c1c 100644
--- a/docs/topics/impala_allow_unsupported_formats.xml
+++ b/docs/topics/impala_allow_unsupported_formats.xml
@@ -31,11 +31,8 @@ under the License.
   </prolog>
 
   <conbody>
-
-    <p>
-      An obsolete query option from early work on support for file formats. Do not use. Might be removed in the
-      future.
-    </p>
+    <note>This query option was removed in <keyword keyref="impala31"/> and no
+      longer has any effect. Do not use.</note>
 
     <p conref="../shared/impala_common.xml#common/type_boolean"/>
     <p conref="../shared/impala_common.xml#common/default_false_0"/>

http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_avro.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_avro.xml b/docs/topics/impala_avro.xml
index 8e07d47..fd85f1a 100644
--- a/docs/topics/impala_avro.xml
+++ b/docs/topics/impala_avro.xml
@@ -34,12 +34,10 @@ under the License.
 
   <conbody>
 
-    <p rev="1.4.0">
-      <indexterm audience="hidden">Avro support in Impala</indexterm>
-      Impala supports using tables whose data files use the Avro file format. Impala can query Avro
-      tables, and in Impala 1.4.0 and higher can create them, but currently cannot insert data into them. For
-      insert operations, use Hive, then switch back to Impala to run queries.
-    </p>
+    <p rev="1.4.0"> Impala supports using tables whose data files use the Avro
+      file format. Impala can query Avro tables. In Impala 1.4.0 and higher,
+      Impala can create Avro tables, but cannot insert data into them. For
+      insert operations, use Hive, then switch back to Impala to run queries. </p>
 
     <table>
       <title>Avro Format Support in Impala</title>
@@ -192,15 +190,6 @@ hive> CREATE TABLE hive_avro_table
         name, is ignored.
       </p>
 
-<!-- Have not got a working example of this syntax yet from Lenni.
-<p>
-The schema can be specified either through the <codeph>TBLPROPERTIES</codeph> clause or the
-<codeph>WITH SERDEPROPERTIES</codeph> clause.
-For best compatibility with future versions of Hive, use the <codeph>WITH SERDEPROPERTIES</codeph> clause
-for this information.
-</p>
--->
-
       <note>
         For nullable Avro columns, make sure to put the <codeph>"null"</codeph> entry before the actual type name.
         In Impala, all columns are nullable; Impala currently does not have a <codeph>NOT NULL</codeph> clause. Any

http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_file_formats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_file_formats.xml b/docs/topics/impala_file_formats.xml
index 7516eae..7f59d2c 100644
--- a/docs/topics/impala_file_formats.xml
+++ b/docs/topics/impala_file_formats.xml
@@ -38,14 +38,12 @@ under the License.
 
   <conbody>
 
-    <p>
-      <indexterm audience="hidden">file formats</indexterm>
-      <indexterm audience="hidden">compression</indexterm>
-      Impala supports several familiar file formats used in Apache Hadoop. Impala can load and query data files
-      produced by other Hadoop components such as Pig or MapReduce, and data files produced by Impala can be used
-      by other components also. The following sections discuss the procedures, limitations, and performance
-      considerations for using each file format with Impala.
-    </p>
+    <p>Impala supports several familiar file formats used in Apache Hadoop.
+      Impala can load and query data files produced by other Hadoop components
+      such as Pig or MapReduce, and data files produced by Impala can be used by
+      other components also. The following sections discuss the procedures,
+      limitations, and performance considerations for using each file format
+      with Impala. </p>
 
     <p>
       The file format used for an Impala table has significant performance consequences. Some file formats include
@@ -143,14 +141,11 @@ under the License.
               format is uncompressed text, with values separated by ASCII <codeph>0x01</codeph> characters
               (typically represented as Ctrl-A).
             </entry>
-            <entry>
-              Yes: <codeph>CREATE TABLE</codeph>, <codeph>INSERT</codeph>, <codeph>LOAD DATA</codeph>, and query.
-              If LZO compression is used, you must create the table and load data in Hive. If other kinds of
-              compression are used, you must load data through <codeph>LOAD DATA</codeph>, Hive, or manually in
-              HDFS.
-
-<!--            <ph rev="2.0.0">Impala 2.0 and higher can write LZO-compressed text data; for earlier Impala releases,  you must create the table and load data in Hive.</ph> -->
-            </entry>
+            <entry> Yes: <codeph>CREATE TABLE</codeph>, <codeph>INSERT</codeph>,
+                <codeph>LOAD DATA</codeph>, and query. If LZO compression is
+              used, you must create the table and load data in Hive. If other
+              kinds of compression are used, you must load data through
+                <codeph>LOAD DATA</codeph>, Hive, or manually in HDFS.</entry>
           </row>
           <row id="avro_support">
             <entry>
@@ -162,9 +157,8 @@ under the License.
             <entry>
               Snappy, gzip, deflate, bzip2
             </entry>
-            <entry rev="1.4.0">
-              Yes, in Impala 1.4.0 and higher. Before that, create the table using Hive.
-            </entry>
+            <entry rev="1.4.0"> Yes, in Impala 1.4.0 and higher. In lower
+              versions, create the table using Hive. </entry>
             <entry>
               No. Import data by using <codeph>LOAD DATA</codeph> on data files already in the right format, or use
               <codeph>INSERT</codeph> in Hive followed by <codeph>REFRESH <varname>table_name</varname></codeph> in Impala.

http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_seq_compression_mode.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_seq_compression_mode.xml b/docs/topics/impala_seq_compression_mode.xml
index 09b6fd5..d1d6e93 100644
--- a/docs/topics/impala_seq_compression_mode.xml
+++ b/docs/topics/impala_seq_compression_mode.xml
@@ -33,15 +33,8 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="hidden">RM_INITIAL_MEM query option</indexterm>
-    </p>
-
-    <p>
-      <b>Type:</b>
-    </p>
-
-    <p>
-      <b>Default:</b>
+      <note>This query option was removed in <keyword keyref="impala31"/> and no
+        longer has any effect. Do not use.</note>
     </p>
   </conbody>
 </concept>

http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_seqfile.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_seqfile.xml b/docs/topics/impala_seqfile.xml
index 7143530..db5a231 100644
--- a/docs/topics/impala_seqfile.xml
+++ b/docs/topics/impala_seqfile.xml
@@ -34,10 +34,7 @@ under the License.
 
   <conbody>
 
-    <p>
-      <indexterm audience="hidden">SequenceFile support in Impala</indexterm>
-      Impala supports using SequenceFile data files.
-    </p>
+    <p> Impala supports using SequenceFile data files. </p>
 
     <table>
       <title>SequenceFile Format Support in Impala</title>
@@ -160,12 +157,11 @@ Returned 3 row(s) in 0.23s</codeblock>
 
     <conbody>
 
-      <p>
-        <indexterm audience="hidden">compression</indexterm>
-        You may want to enable compression on existing tables. Enabling compression provides performance gains in
-        most cases and is supported for SequenceFile tables. For example, to enable Snappy compression, you would
-        specify the following additional settings when loading data through the Hive shell:
-      </p>
+      <p> You may want to enable compression on existing tables. Enabling
+        compression provides performance gains in most cases and is supported
+        for SequenceFile tables. For example, to enable Snappy compression, you
+        would specify the following additional settings when loading data
+        through the Hive shell: </p>
 
 <codeblock>hive&gt; SET hive.exec.compress.output=true;
 hive&gt; SET mapred.max.split.size=256000000;
@@ -225,16 +221,6 @@ hive&gt; INSERT OVERWRITE TABLE tbl_seq PARTITION(year) SELECT * FROM tbl;</code
     </conbody>
   </concept>
 
-  <concept audience="hidden" id="seqfile_data_types">
-
-    <title>Data Type Considerations for SequenceFile Tables</title>
-
-    <conbody>
-
-      <p></p>
-    </conbody>
-  </concept>
-
   <concept id="seqfile_performance">
 
     <title>Query Performance for Impala SequenceFile Tables</title>

http://git-wip-us.apache.org/repos/asf/impala/blob/cb2574b8/docs/topics/impala_txtfile.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_txtfile.xml b/docs/topics/impala_txtfile.xml
index 3c09b80..2b8fb58 100644
--- a/docs/topics/impala_txtfile.xml
+++ b/docs/topics/impala_txtfile.xml
@@ -34,12 +34,10 @@ under the License.
 
   <conbody>
 
-    <p>
-      <indexterm audience="hidden">Text support in Impala</indexterm>
-      Impala supports using text files as the storage format for input and output. Text files are a
-      convenient format to use for interchange with other applications or scripts that produce or read delimited
-      text files, such as CSV or TSV with commas or tabs for delimiters.
-    </p>
+    <p> Impala supports using text files as the storage format for input and
+      output. Text files are a convenient format to use for interchange with
+      other applications or scripts that produce or read delimited text files,
+      such as CSV or TSV with commas or tabs for delimiters. </p>
 
     <p>
       Text files are also very flexible in their column definitions. For example, a text file could have more
@@ -223,20 +221,6 @@ create table pipe_separated(id int, s string, n int, t timestamp, b boolean)
         </p>
       </note>
 
-<!--
-      <p>
-        In the <cmdname>impala-shell</cmdname> interpreter, issue a command similar to:
-      </p>
-
-<codeblock>create table textfile_table (<varname>column_specs</varname>) stored as textfile;
-/* If the STORED AS clause is omitted, the default is a TEXTFILE with hex 01 characters as the delimiter. */
-create table default_table (<varname>column_specs</varname>);
-/* Some optional clauses in the CREATE TABLE statement apply only to Text tables. */
-create table csv_table (<varname>column_specs</varname>) row format delimited fields terminated by ',';
-create table tsv_table (<varname>column_specs</varname>) row format delimited fields terminated by '\t';
-create table dos_table (<varname>column_specs</varname>) lines terminated by '\r';</codeblock>
--->
-
       <p>
         Issue a <codeph>DESCRIBE FORMATTED <varname>table_name</varname></codeph> statement to see the details of
         how each table is represented internally in Impala.
@@ -271,7 +255,6 @@ create table dos_table (<varname>column_specs</varname>) lines terminated by '\r
         </li>
 
         <li>
-<!-- Copied and slightly adapted text from later on in this same file. Turn into a conref. -->
           <p>
             Impala uses suffixes to recognize when text data files are compressed text. For Impala to recognize the
             compressed text files, they must have the appropriate file extension corresponding to the compression
@@ -438,14 +421,11 @@ INSERT INTO csv SELECT * FROM other_file_format_table;</codeblock>
 
     <conbody>
 
-      <p>
-        <indexterm audience="hidden">LZO support in Impala</indexterm>
-
-        <indexterm audience="hidden">compression</indexterm>
-        Impala supports using text data files that employ LZO compression. Where practical, apply compression to
-        text data files. Impala queries are usually I/O-bound; reducing the amount of data read from
-        disk typically speeds up a query, despite the extra CPU work to uncompress the data in memory.
-      </p>
+      <p> Impala supports using text data files that employ LZO compression.
+        Where practical, apply compression to text data files. Impala queries
+        are usually I/O-bound; reducing the amount of data read from disk
+        typically speeds up a query, despite the extra CPU work to uncompress
+        the data in memory. </p>
 
       <p>
         Impala can work with LZO-compressed text files are preferable to files compressed by other codecs, because
@@ -581,15 +561,6 @@ drwxr-xr-x. 2 root root 4096 Oct 28 15:46 conf.pseudo</codeblock>
     INPUTFORMAT 'com.hadoop.mapred.DeprecatedLzoTextInputFormat'
     OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'</codeblock>
 
-<!--
-      <p>
-        In Hive, when writing LZO compressed text tables, you must include the following specification:
-      </p>
-
-<codeblock>hive&gt; SET hive.exec.compress.output=true;
-hive&gt; SET mapred.output.compression.codec=com.hadoop.compression.lzo.LzopCodec;</codeblock>
--->
-
         <p>
           Also, certain Hive settings need to be in effect. For example:
         </p>
@@ -639,13 +610,6 @@ hive&gt; INSERT INTO TABLE lzo_t SELECT col1, col2 FROM uncompressed_text_table;
           DataNodes, which is very inefficient.
         </p>
 
-        <!-- To do:
-          Here is the place to put some end-to-end examples once I have it
-          all working. Or at least the final step with Impala queries.
-          Have never actually gotten this part working yet due to mismatches
-          between the levels of Impala and LZO packages.
-        -->
-
         <p>
           Once the LZO-compressed tables are created, and data is loaded and indexed, you can query them through
           Impala. As always, the first time you start <cmdname>impala-shell</cmdname> after creating a table in
@@ -673,20 +637,13 @@ hive&gt; INSERT INTO TABLE lzo_t SELECT col1, col2 FROM uncompressed_text_table;
 
     <conbody>
 
-      <p>
-        <indexterm audience="hidden">gzip support in Impala</indexterm>
-
-        <indexterm audience="hidden">bzip2 support in Impala</indexterm>
-
-        <indexterm audience="hidden">Snappy support in Impala</indexterm>
-
-        <indexterm audience="hidden">compression</indexterm>
-        In Impala 2.0 and later, Impala supports using text data files that employ gzip, bzip2, or Snappy
-        compression. These compression types are primarily for convenience within an existing ETL pipeline rather
-        than maximum performance. Although it requires less I/O to read compressed text than the equivalent
-        uncompressed text, files compressed by these codecs are not <q>splittable</q> and therefore cannot take
-        full advantage of the Impala parallel query capability.
-      </p>
+      <p> In Impala 2.0 and later, Impala supports using text data files that
+        employ gzip, bzip2, or Snappy compression. These compression types are
+        primarily for convenience within an existing ETL pipeline rather than
+        maximum performance. Although it requires less I/O to read compressed
+        text than the equivalent uncompressed text, files compressed by these
+        codecs are not <q>splittable</q> and therefore cannot take full
+        advantage of the Impala parallel query capability. </p>
 
       <p>
         As each bzip2- or Snappy-compressed text file is processed, the node doing the work reads the entire file
@@ -697,15 +654,6 @@ hive&gt; INSERT INTO TABLE lzo_t SELECT col1, col2 FROM uncompressed_text_table;
         gzip-compressed text files. The gzipped data is decompressed as it is read, rather than all at once.</ph>
       </p>
 
-<!--
-    <p>
-    Impala can work with LZO-compressed text files but not GZip-compressed text.
-    LZO-compressed files are <q>splittable</q>, meaning that different portions of a file
-    can be uncompressed and processed independently by different nodes. GZip-compressed
-    files are not splittable, making them unsuitable for Impala-style distributed queries.
-    </p>
--->
-
       <p>
         To create a table to hold gzip, bzip2, or Snappy-compressed text, create a text table with no special
         compression options. Specify the delimiter and escape character if required, using the <codeph>ROW
@@ -764,16 +712,4 @@ $ hdfs dfs -ls 'hdfs://127.0.0.1:8020/user/hive/warehouse/file_formats.db/csv_co
 
   </concept>
 
-  <concept audience="hidden" id="txtfile_data_types">
-
-    <title>Data Type Considerations for Text Tables</title>
-
-    <conbody>
-
-      <p></p>
-
-    </conbody>
-
-  </concept>
-
 </concept>


[30/33] impala git commit: IMPALA-7824: INVALIDATE METADATA should not hang when Sentry is unavailable

Posted by bo...@apache.org.
IMPALA-7824: INVALIDATE METADATA should not hang when Sentry is unavailable

Before this patch, running INVALIDATE METADATA when Sentry is
unavailable could cause Impala query to hang. PolicyReader thread in
SentryProxy is used by two use cases, one as a background thread
that periodically refreshes Sentry policy and another one as a
synchronous operation for INVALIDATE METADATA. For the background
thread, we need to swallow any exception thrown while refreshing the
Sentry policy in order to not kill the background thread. For a
synchronous reset operation, such as INVALIDATE METADATA, swallowing
an exception causes the Impala catalog to wait indefinitely for
authorization catalog objects that never get processed due to Sentry
being unavailable. The patch updates the code by not swallowing any
exception in INVALIDATE METADATA and return the exception to the
caller.

Testing:
- Ran all FE tests
- Added a new E2E test
- Ran all E2E authorization tests

Change-Id: Icff987a6184f62a338faadfdc1a0d349d912fc37
Reviewed-on: http://gerrit.cloudera.org:8080/11897
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 59f3f6b901581d2f6b5084fa739db2b5870f2af0
Parents: d07bc81
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Tue Nov 6 19:59:09 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:36 2018 +0100

----------------------------------------------------------------------
 .../common/SentryPolicyReaderException.java     | 35 ++++++++++++++++
 .../common/SentryUnavailableException.java      | 35 ++++++++++++++++
 .../org/apache/impala/util/SentryProxy.java     | 44 +++++++++++++++-----
 .../impala/testutil/ImpaladTestCatalog.java     |  5 ++-
 tests/authorization/test_authorization.py       | 32 ++++++++++++++
 tests/common/custom_cluster_test_suite.py       | 10 ++++-
 6 files changed, 147 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/fe/src/main/java/org/apache/impala/common/SentryPolicyReaderException.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/common/SentryPolicyReaderException.java b/fe/src/main/java/org/apache/impala/common/SentryPolicyReaderException.java
new file mode 100644
index 0000000..b3c27f8
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/common/SentryPolicyReaderException.java
@@ -0,0 +1,35 @@
+// 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.common;
+
+/**
+ * Thrown as a generic exception when processing Sentry policy.
+ */
+public class SentryPolicyReaderException extends RuntimeException {
+  public SentryPolicyReaderException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+  public SentryPolicyReaderException(String msg) {
+    super(msg);
+  }
+
+  public SentryPolicyReaderException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/fe/src/main/java/org/apache/impala/common/SentryUnavailableException.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/common/SentryUnavailableException.java b/fe/src/main/java/org/apache/impala/common/SentryUnavailableException.java
new file mode 100644
index 0000000..2c32a44
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/common/SentryUnavailableException.java
@@ -0,0 +1,35 @@
+// 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.common;
+
+/**
+ * Thrown when Sentry is not available.
+ */
+public class SentryUnavailableException extends RuntimeException {
+  public SentryUnavailableException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+  public SentryUnavailableException(String msg) {
+    super(msg);
+  }
+
+  public SentryUnavailableException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/fe/src/main/java/org/apache/impala/util/SentryProxy.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/SentryProxy.java b/fe/src/main/java/org/apache/impala/util/SentryProxy.java
index be2f7f3..2470bc3 100644
--- a/fe/src/main/java/org/apache/impala/util/SentryProxy.java
+++ b/fe/src/main/java/org/apache/impala/util/SentryProxy.java
@@ -32,6 +32,8 @@ import org.apache.impala.catalog.Principal;
 import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.impala.catalog.Role;
 import org.apache.impala.common.Reference;
+import org.apache.impala.common.SentryPolicyReaderException;
+import org.apache.impala.common.SentryUnavailableException;
 import org.apache.impala.thrift.TPrincipalType;
 import org.apache.log4j.Logger;
 import org.apache.sentry.api.service.thrift.TSentryGroup;
@@ -48,8 +50,10 @@ 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 org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.service.common.SentryOwnerPrivilegeType;
 import org.apache.sentry.service.common.ServiceConstants;
+import org.apache.thrift.transport.TTransportException;
 
 /**
  * Thread safe class that acts as a link between the Sentry Service and the Catalog
@@ -105,9 +109,11 @@ public class SentryProxy {
     } else {
       objectOwnershipConfigValue_ = SentryOwnerPrivilegeType.NONE.toString();
     }
-    policyReader_.scheduleAtFixedRate(new PolicyReader(false), 0,
-        BackendConfig.INSTANCE.getSentryCatalogPollingFrequency(),
-        TimeUnit.SECONDS);
+    // We configure the PolicyReader to swallow any exception because we do not want to
+    // kill the PolicyReader background thread on exception.
+    policyReader_.scheduleAtFixedRate(new PolicyReader(/*reset versions*/ false,
+        /*swallow exception*/ true), 0,
+        BackendConfig.INSTANCE.getSentryCatalogPollingFrequency(), TimeUnit.SECONDS);
   }
 
   /**
@@ -125,10 +131,13 @@ public class SentryProxy {
    * atomically.
    */
   private class PolicyReader implements Runnable {
-    private boolean resetVersions_;
+    private final boolean resetVersions_;
+    // A flag to indicate whether or not to swallow any exception thrown.
+    private final boolean swallowException_;
 
-    public PolicyReader(boolean resetVersions) {
+    public PolicyReader(boolean resetVersions, boolean swallowException) {
       resetVersions_ = resetVersions;
+      swallowException_ = swallowException;
     }
 
     public void run() {
@@ -141,7 +150,17 @@ public class SentryProxy {
           usersToRemove = refreshUserPrivileges();
         } catch (Exception e) {
           LOG.error("Error refreshing Sentry policy: ", e);
-          return;
+          if (swallowException_) return;
+          // We need to differentiate between Sentry not available exception and
+          // any other exceptions.
+          if (e.getCause() != null && e.getCause() instanceof SentryUserException) {
+            Throwable sentryException = e.getCause();
+            if (sentryException.getCause() != null &&
+                sentryException.getCause() instanceof TTransportException) {
+              throw new SentryUnavailableException(e);
+            }
+          }
+          throw new SentryPolicyReaderException(e);
         } finally {
           LOG.debug("Refreshing Sentry policy took " +
               (System.currentTimeMillis() - startTime) + "ms");
@@ -527,16 +546,21 @@ public class SentryProxy {
   }
 
   /**
-   * Perfoms a synchronous refresh of all authorization policy metadata and updates
+   * Performs a synchronous refresh of all authorization policy metadata and updates
    * the Catalog with any changes. Throws an ImpalaRuntimeException if there are any
    * errors executing the refresh job.
    */
   public void refresh(boolean resetVersions) throws ImpalaRuntimeException {
     try {
-      policyReader_.submit(new PolicyReader(resetVersions)).get();
+      // Since this is a synchronous refresh, any exception thrown while running the
+      // refresh should be thrown here instead of silently swallowing it.
+      policyReader_.submit(new PolicyReader(resetVersions, /*swallow exception*/ false))
+          .get();
     } catch (Exception e) {
-      // We shouldn't make it here. It means an exception leaked from the
-      // AuthorizationPolicyReader.
+      if (e.getCause() != null && e.getCause() instanceof SentryUnavailableException) {
+        throw new ImpalaRuntimeException("Error refreshing authorization policy. " +
+            "Sentry is unavailable. Ensure Sentry is up: ", e);
+      }
       throw new ImpalaRuntimeException("Error refreshing authorization policy, " +
           "current policy state may be inconsistent. Running 'invalidate metadata' " +
           "may resolve this problem: ", e);

http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/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 3186113..0734ee0 100644
--- a/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
+++ b/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
@@ -55,8 +55,9 @@ public class ImpaladTestCatalog extends ImpaladCatalog {
    */
   public ImpaladTestCatalog(AuthorizationConfig authzConfig) {
     super("127.0.0.1");
-    CatalogServiceCatalog catalogServerCatalog =
-        CatalogServiceTestCatalog.createWithAuth(authzConfig.getSentryConfig());
+    CatalogServiceCatalog catalogServerCatalog = authzConfig.isEnabled() ?
+        CatalogServiceTestCatalog.createWithAuth(authzConfig.getSentryConfig()) :
+        CatalogServiceTestCatalog.create();
     // Bootstrap the catalog by adding all dbs, tables, and functions.
     for (FeDb db: catalogServerCatalog.getDbs(PatternMatcher.MATCHER_MATCH_ALL)) {
       // Adding DB should include all tables/fns in that database.

http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/tests/authorization/test_authorization.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_authorization.py b/tests/authorization/test_authorization.py
index 9da43d3..a036680 100644
--- a/tests/authorization/test_authorization.py
+++ b/tests/authorization/test_authorization.py
@@ -478,3 +478,35 @@ class TestAuthorization(CustomClusterTestSuite):
         assert "CatalogException" in obj_dump
     finally:
       self.role_cleanup(unique_role)
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server_name=server1 --sentry_config=%s" % SENTRY_CONFIG_FILE,
+    catalogd_args="--sentry_config=%s --sentry_catalog_polling_frequency_s=3600" %
+                  SENTRY_CONFIG_FILE,
+    impala_log_dir=tempfile.mkdtemp(prefix="test_invalidate_metadata_sentry_unavailable_",
+                                    dir=os.getenv("LOG_DIR")))
+  def test_invalidate_metadata_sentry_unavailable(self, unique_role):
+    """IMPALA-7824: Tests that running INVALIDATE METADATA when Sentry is unavailable
+    should not cause Impala to hang."""
+    self.role_cleanup(unique_role)
+    try:
+      group_name = grp.getgrnam(getuser()).gr_name
+      self.client.execute("create role %s" % unique_role)
+      self.client.execute("grant all on server to role %s" % unique_role)
+      self.client.execute("grant role %s to group `%s`" % (unique_role, group_name))
+
+      self._stop_sentry_service()
+      # Calling INVALIDATE METADATA when Sentry is unavailable should return an error.
+      result = self.execute_query_expect_failure(self.client, "invalidate metadata")
+      result_str = str(result)
+      assert "MESSAGE: CatalogException: Error updating authorization policy:" \
+             in result_str
+      assert "CAUSED BY: ImpalaRuntimeException: Error refreshing authorization policy." \
+             " Sentry is unavailable. Ensure Sentry is up:" in result_str
+
+      self._start_sentry_service(SENTRY_CONFIG_FILE)
+      # Calling INVALIDATE METADATA after Sentry is up should not return an error.
+      self.execute_query_expect_success(self.client, "invalidate metadata")
+    finally:
+      self.role_cleanup(unique_role)

http://git-wip-us.apache.org/repos/asf/impala/blob/59f3f6b9/tests/common/custom_cluster_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/custom_cluster_test_suite.py b/tests/common/custom_cluster_test_suite.py
index 0140303..90cd4e9 100644
--- a/tests/common/custom_cluster_test_suite.py
+++ b/tests/common/custom_cluster_test_suite.py
@@ -164,11 +164,17 @@ class CustomClusterTestSuite(ImpalaTestSuite):
     sentry_env['SENTRY_SERVICE_CONFIG'] = sentry_service_config
     call = subprocess.Popen(
         ['/bin/bash', '-c', os.path.join(IMPALA_HOME,
-        'testdata/bin/run-sentry-service.sh')],
+                                         'testdata/bin/run-sentry-service.sh')],
         env=sentry_env)
     call.wait()
     if call.returncode != 0:
-      raise RuntimeError("unable to start sentry")
+      raise RuntimeError("Unable to start Sentry")
+
+  @classmethod
+  def _stop_sentry_service(cls):
+    subprocess.check_call([os.path.join(os.environ["IMPALA_HOME"],
+                                        "testdata/bin/kill-sentry-service.sh")],
+                          close_fds=True)
 
   @classmethod
   def _start_impala_cluster(cls, options, impala_log_dir=os.getenv('LOG_DIR', "/tmp/"),


[19/33] impala git commit: IMPALA-7788: [DOCS] Impala supports ADLS Gen 2 (ABFS)

Posted by bo...@apache.org.
IMPALA-7788: [DOCS] Impala supports ADLS Gen 2 (ABFS)

Change-Id: Ic06d9ac92ed78b9092369e211de8a81db1d7ce90
Reviewed-on: http://gerrit.cloudera.org:8080/11853
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Joe McDonnell <jo...@cloudera.com>
Reviewed-by: Jim Apple <jb...@apache.org>


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

Branch: refs/heads/branch-3.1.0
Commit: 030f0ac303f044ad1661cc3601ca0cedc675aba0
Parents: 0f63b2c
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Thu Nov 1 16:55:27 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 docs/shared/impala_common.xml    |  29 +++---
 docs/topics/impala_adls.xml      | 179 ++++++++++++++++++++--------------
 docs/topics/impala_insert.xml    |   3 +-
 docs/topics/impala_load_data.xml |  13 ++-
 4 files changed, 127 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/030f0ac3/docs/shared/impala_common.xml
----------------------------------------------------------------------
diff --git a/docs/shared/impala_common.xml b/docs/shared/impala_common.xml
index 8b79596..f4aaedd 100644
--- a/docs/shared/impala_common.xml
+++ b/docs/shared/impala_common.xml
@@ -1297,17 +1297,21 @@ drop database temp;
         See <xref href="../topics/impala_s3.xml#s3"/> for details about reading and writing S3 data with Impala.
       </p>
 
-      <p rev="2.9.0 IMPALA-5333" id="adls_dml">
-        In <keyword keyref="impala29_full"/> and higher, the Impala DML statements (<codeph>INSERT</codeph>, <codeph>LOAD DATA</codeph>,
-        and <codeph>CREATE TABLE AS SELECT</codeph>) can write data into a table or partition that resides in the
-        Azure Data Lake Store (ADLS).
-        The syntax of the DML statements is the same as for any other tables, because the ADLS location for tables and
-        partitions is specified by an <codeph>adl://</codeph> prefix in the
-        <codeph>LOCATION</codeph> attribute of
-        <codeph>CREATE TABLE</codeph> or <codeph>ALTER TABLE</codeph> statements.
-        If you bring data into ADLS using the normal ADLS transfer mechanisms instead of Impala DML statements,
-        issue a <codeph>REFRESH</codeph> statement for the table before using Impala to query the ADLS data.
-      </p>
+      <p rev="2.9.0 IMPALA-5333" id="adls_dml"> In <keyword
+          keyref="impala29_full"/> and higher, the Impala DML statements
+          (<codeph>INSERT</codeph>, <codeph>LOAD DATA</codeph>, and
+          <codeph>CREATE TABLE AS SELECT</codeph>) can write data into a table
+        or partition that resides in the Azure Data Lake Store (ADLS). ADLS Gen2
+        is supported in <keyword keyref="impala31"/> and higher.</p>
+      <p rev="2.9.0 IMPALA-5333">In the<codeph>CREATE TABLE</codeph> or
+          <codeph>ALTER TABLE</codeph> statements, specify the ADLS location for
+        tables and partitions with the <codeph>adl://</codeph> prefix for ADLS
+        Gen1 and <codeph>abfs://</codeph> or <codeph>abfss://</codeph> for ADLS
+        Gen2 in the <codeph>LOCATION</codeph> attribute.</p>
+      <p rev="2.9.0 IMPALA-5333" id="adls_dml_end">If you bring data into ADLS
+        using the normal ADLS transfer mechanisms instead of Impala DML
+        statements, issue a <codeph>REFRESH</codeph> statement for the table
+        before using Impala to query the ADLS data. </p>
 
       <p rev="2.6.0 IMPALA-1878" id="s3_dml">
         In <keyword keyref="impala26_full"/> and higher, the Impala DML statements (<codeph>INSERT</codeph>, <codeph>LOAD DATA</codeph>,
@@ -1321,9 +1325,6 @@ drop database temp;
         issue a <codeph>REFRESH</codeph> statement for the table before using Impala to query the S3 data.
       </p>
 
-        <!-- Formerly part of s3_dml element. Moved out to avoid a circular link in the S3 topic itelf. -->
-        <!-- See <xref href="../topics/impala_s3.xml#s3"/> for details about reading and writing S3 data with Impala. -->
-
       <p rev="2.2.0" id="s3_metadata"> Impala caches metadata for tables where
         the data resides in the Amazon Simple Storage Service (S3), and the
           <codeph>REFRESH</codeph> and <codeph>INVALIDATE METADATA</codeph>

http://git-wip-us.apache.org/repos/asf/impala/blob/030f0ac3/docs/topics/impala_adls.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_adls.xml b/docs/topics/impala_adls.xml
index 5d790c5..f5103f4 100644
--- a/docs/topics/impala_adls.xml
+++ b/docs/topics/impala_adls.xml
@@ -35,14 +35,12 @@ under the License.
 
   <conbody>
 
-    <p>
-      <indexterm audience="hidden">ADLS with Impala</indexterm>
-      You can use Impala to query data residing on the Azure Data Lake Store (ADLS) filesystem.
-      This capability allows convenient access to a storage system that is remotely managed,
-      accessible from anywhere, and integrated with various cloud-based services. Impala can
-      query files in any supported file format from ADLS. The ADLS storage location
-      can be for an entire table, or individual partitions in a partitioned table.
-    </p>
+    <p> You can use Impala to query data residing on the Azure Data Lake Store
+      (ADLS) filesystem. This capability allows convenient access to a storage
+      system that is remotely managed, accessible from anywhere, and integrated
+      with various cloud-based services. Impala can query files in any supported
+      file format from ADLS. The ADLS storage location can be for an entire
+      table, or individual partitions in a partitioned table. </p>
 
     <p>
       The default Impala tables use data files stored on HDFS, which are ideal for bulk loads and queries using
@@ -51,6 +49,8 @@ under the License.
       HDFS. In a partitioned table, you can set the <codeph>LOCATION</codeph> attribute for individual partitions
       to put some partitions on HDFS and others on ADLS, typically depending on the age of the data.
     </p>
+    <p>Starting in <keyword keyref="impala31"/>, Impala supports ADLS Gen2
+      filesystem, Azure Blob File System (ABFS).</p>
 
     <p outputclass="toc inpage"/>
 
@@ -70,6 +70,9 @@ under the License.
             <xref href="https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-get-started-portal" scope="external" format="html">Get started with Azure Data Lake Store using the Azure Portal</xref>
           </p>
         </li>
+        <li><xref
+            href="https://docs.microsoft.com/en-us/azure/storage/data-lake-storage/quickstart-create-account"
+            format="html" scope="external">Azure Data Lake Storage Gen2</xref></li>
         <li>
           <p>
             <xref href="https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html" scope="external" format="html">Hadoop Azure Data Lake Support</xref>
@@ -82,27 +85,22 @@ under the License.
   <concept id="sql">
     <title>How Impala SQL Statements Work with ADLS</title>
     <conbody>
-      <p>
-        Impala SQL statements work with data on ADLS as follows:
-      </p>
+      <p> Impala SQL statements work with data on ADLS as follows. </p>
       <ul>
-        <li>
-          <p>
-            The <xref href="impala_create_table.xml#create_table"/>
-            or <xref href="impala_alter_table.xml#alter_table"/> statements
-            can specify that a table resides on the ADLS filesystem by
-            encoding an <codeph>adl://</codeph> prefix for the <codeph>LOCATION</codeph>
-            property. <codeph>ALTER TABLE</codeph> can also set the <codeph>LOCATION</codeph>
-            property for an individual partition, so that some data in a table resides on
-            ADLS and other data in the same table resides on HDFS.
-          </p>
-          <p>
-            The full format of the location URI is typically:
-<codeblock>
-adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_directory_path</varname>
-</codeblock>
-          </p>
-        </li>
+        <li><p> The <xref href="impala_create_table.xml#create_table"/> or <xref
+              href="impala_alter_table.xml#alter_table"/> statements can specify
+            that a table resides on the ADLS filesystem by specifying an ADLS
+            prefix for the <codeph>LOCATION</codeph> property.<ul>
+              <li><codeph>adl://</codeph> for ADLS Gen1</li>
+              <li><codeph>abfs://</codeph> for ADLS Gen2</li>
+              <li><codeph>abfss://</codeph> for ADLS Gen2 with a secure socket
+                layer connection</li>
+            </ul>
+            <codeph>ALTER TABLE</codeph> can also set the
+              <codeph>LOCATION</codeph> property for an individual partition, so
+            that some data in a table resides on ADLS and other data in the same
+            table resides on HDFS. </p> See <xref href="impala_adls.xml#ddl"/>
+          for usage information.</li>
         <li>
           <p>
             Once a table or partition is designated as residing on ADLS, the <xref href="impala_select.xml#select"/>
@@ -135,10 +133,8 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
           </p>
         </li>
       </ul>
-      <p>
-        For usage information about Impala SQL statements with ADLS tables, see <xref href="impala_adls.xml#ddl"/>
-        and <xref href="impala_adls.xml#dml"/>.
-      </p>
+      <p> For usage information about Impala SQL statements with ADLS tables,
+        see <xref href="impala_adls.xml#dml"/>. </p>
     </conbody>
   </concept>
 
@@ -148,30 +144,54 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
 
     <conbody>
 
-      <p>
-        To allow Impala to access data in ADLS, specify values for the following configuration settings in your
-        <filepath>core-site.xml</filepath> file:
-      </p>
+      <p> To allow Impala to access data in ADLS, specify values for the
+        following configuration settings in your
+          <filepath>core-site.xml</filepath> file.</p>
+      <p>For ADLS Gen1:</p>
+
+<codeblock>&lt;property>
+   &lt;name>dfs.adls.oauth2.access.token.provider.type&lt;/name>
+   &lt;value>ClientCredential&lt;/value>
+&lt;/property>
+&lt;property>
+   &lt;name>dfs.adls.oauth2.client.id&lt;/name>
+   &lt;value><varname>your_client_id</varname>&lt;/value>
+&lt;/property>
+&lt;property>
+   &lt;name>dfs.adls.oauth2.credential&lt;/name>
+   &lt;value><varname>your_client_secret</varname>&lt;/value>
+&lt;/property>
+&lt;property>
+   &lt;name>dfs.adls.oauth2.refresh.url&lt;/name>
+   &lt;value>https://login.windows.net/<varname>your_azure_tenant_id</varname>/oauth2/token&lt;/value>
+&lt;/property>
 
-<codeblock><![CDATA[
-<property>
-   <name>dfs.adls.oauth2.access.token.provider.type</name>
-   <value>ClientCredential</value>
-</property>
-<property>
-   <name>dfs.adls.oauth2.client.id</name>
-   <value><varname>your_client_id</varname></value>
-</property>
-<property>
-   <name>dfs.adls.oauth2.credential</name>
-   <value><varname>your_client_secret</varname></value>
-</property>
-<property>
-   <name>dfs.adls.oauth2.refresh.url</name>
-   <value><varname>refresh_URL</varname></value>
-</property>
-]]>
 </codeblock>
+      <p>For ADLS Gen2:</p>
+      <codeblock> &lt;property>
+    &lt;name>fs.azure.account.auth.type&lt;/name>
+    &lt;value>OAuth&lt;/value>
+  &lt;/property>
+
+  &lt;property>
+    &lt;name>fs.azure.account.oauth.provider.type&lt;/name>
+    &lt;value>org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider&lt;/value>
+  &lt;/property>
+
+  &lt;property>
+    &lt;name>fs.azure.account.oauth2.client.id&lt;/name>
+    &lt;value><varname>your_client_id</varname>&lt;/value>
+  &lt;/property>
+
+  &lt;property>
+    &lt;name>fs.azure.account.oauth2.client.secret&lt;/name>
+    &lt;value><varname>your_client_secret</varname>&lt;/value>
+  &lt;/property>
+
+  &lt;property>
+    &lt;name>fs.azure.account.oauth2.client.endpoint&lt;/name>
+    &lt;value>https://login.microsoftonline.com/<varname>your_azure_tenant_id</varname>/oauth2/token&lt;/value>
+  &lt;/property></codeblock>
 
       <note>
         <p>
@@ -180,11 +200,10 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
         </p>
       </note>
 
-      <p>
-        After specifying the credentials, restart both the Impala and
-        Hive services. (Restarting Hive is required because Impala queries, CREATE TABLE statements, and so on go
-        through the Hive metastore.)
-      </p>
+      <p> After specifying the credentials, restart both the Impala and Hive
+        services. Restarting Hive is required because certain Impala queries,
+        such as <codeph>CREATE TABLE</codeph> statements, go through the Hive
+        metastore.</p>
 
     </conbody>
 
@@ -213,7 +232,8 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
     <concept id="dml">
       <title>Using Impala DML Statements for ADLS Data</title>
       <conbody>
-        <p conref="../shared/impala_common.xml#common/adls_dml"/>
+        <p conref="../shared/impala_common.xml#common/adls_dml"
+          conrefend="../shared/impala_common.xml#common/adls_dml_end"/>
       </conbody>
     </concept>
 
@@ -249,12 +269,24 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
 
     <conbody>
 
-      <p>
-        Impala reads data for a table or partition from ADLS based on the <codeph>LOCATION</codeph> attribute for the
-        table or partition. Specify the ADLS details in the <codeph>LOCATION</codeph> clause of a <codeph>CREATE
-        TABLE</codeph> or <codeph>ALTER TABLE</codeph> statement. The notation for the <codeph>LOCATION</codeph>
-        clause is <codeph>adl://<varname>store</varname>/<varname>path/to/file</varname></codeph>.
-      </p>
+      <p> Impala reads data for a table or partition from ADLS based on the
+          <codeph>LOCATION</codeph> attribute for the table or partition.
+        Specify the ADLS details in the <codeph>LOCATION</codeph> clause of a
+          <codeph>CREATE TABLE</codeph> or <codeph>ALTER TABLE</codeph>
+        statement. The syntax for the <codeph>LOCATION</codeph> clause is:<ul>
+          <li>For ADLS Gen1,
+                <codeph>adl://<varname>account</varname>.azuredatalakestore.net/<varname>path/file</varname></codeph>
+          </li>
+          <li>For ADLS Gen2,
+                <codeph>abfs://<varname>container</varname>@<varname>account</varname>.dfs.core.windows.net/<varname>path</varname>/<varname>file</varname></codeph></li>
+          <li>For ADLS Gen2 with a secure socket layer connection,
+                <codeph>abfss://<varname>container</varname>@<varname>account</varname>.dfs.core.windows.net/<varname>path</varname>/<varname>file</varname></codeph></li>
+        </ul></p>
+      <p><codeph><varname>container</varname></codeph> denotes the parent
+        location that holds the files and folders, which is the Containers in
+        the Azure Storage Blobs service.</p>
+      <p><codeph><varname>account</varname></codeph> is the name given for your
+        storage account.</p>
 
       <p>
         For a partitioned table, either specify a separate <codeph>LOCATION</codeph> clause for each new partition,
@@ -288,15 +320,12 @@ adl://<varname>your_account</varname>.azuredatalakestore.net/<varname>rest_of_di
                   >   location 'adl://impalademo.azuredatalakestore.net/dir1/dir2/dir3/t1';
 </codeblock>
 
-      <p>
-        For convenience when working with multiple tables with data files stored in ADLS, you can create a database
-        with a <codeph>LOCATION</codeph> attribute pointing to an ADLS path.
-        Specify a URL of the form <codeph>adl://<varname>store</varname>/<varname>root/path/for/database</varname></codeph>
-        for the <codeph>LOCATION</codeph> attribute of the database.
-        Any tables created inside that database
-        automatically create directories underneath the one specified by the database
-        <codeph>LOCATION</codeph> attribute.
-      </p>
+      <p> For convenience when working with multiple tables with data files
+        stored in ADLS, you can create a database with a
+          <codeph>LOCATION</codeph> attribute pointing to an ADLS path. Specify
+        a URL of the form as shown above. Any tables created inside that
+        database automatically create directories underneath the one specified
+        by the database <codeph>LOCATION</codeph> attribute. </p>
 
       <p>
         The following session creates a database and two partitioned tables residing entirely on ADLS, one

http://git-wip-us.apache.org/repos/asf/impala/blob/030f0ac3/docs/topics/impala_insert.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_insert.xml b/docs/topics/impala_insert.xml
index 7e6ce63..58b5169 100644
--- a/docs/topics/impala_insert.xml
+++ b/docs/topics/impala_insert.xml
@@ -629,7 +629,8 @@ Inserted 2 rows in 0.16s
       <p>See <xref href="../topics/impala_s3.xml#s3"/> for details about reading and writing S3 data with Impala.</p>
 
       <p conref="../shared/impala_common.xml#common/adls_blurb"/>
-      <p conref="../shared/impala_common.xml#common/adls_dml"/>
+      <p conref="../shared/impala_common.xml#common/adls_dml"
+        conrefend="../shared/impala_common.xml#common/adls_dml_end"/>
       <p>See <xref href="../topics/impala_adls.xml#adls"/> for details about reading and writing ADLS data with Impala.</p>
 
       <p conref="../shared/impala_common.xml#common/security_blurb"/>

http://git-wip-us.apache.org/repos/asf/impala/blob/030f0ac3/docs/topics/impala_load_data.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_load_data.xml b/docs/topics/impala_load_data.xml
index 96305a5..f947534 100644
--- a/docs/topics/impala_load_data.xml
+++ b/docs/topics/impala_load_data.xml
@@ -39,12 +39,10 @@ under the License.
 
   <conbody>
 
-    <p>
-      <indexterm audience="hidden">LOAD DATA statement</indexterm>
-      The <codeph>LOAD DATA</codeph> statement streamlines the ETL process for an internal Impala table by moving a
-      data file or all the data files in a directory from an HDFS location into the Impala data directory for that
-      table.
-    </p>
+    <p> The <codeph>LOAD DATA</codeph> statement streamlines the ETL process for
+      an internal Impala table by moving a data file or all the data files in a
+      directory from an HDFS location into the Impala data directory for that
+      table. </p>
 
     <p conref="../shared/impala_common.xml#common/syntax_blurb"/>
 
@@ -240,7 +238,8 @@ Returned 1 row(s) in 0.62s</codeblock>
     <p>See <xref href="../topics/impala_s3.xml#s3"/> for details about reading and writing S3 data with Impala.</p>
 
     <p conref="../shared/impala_common.xml#common/adls_blurb"/>
-    <p conref="../shared/impala_common.xml#common/adls_dml"/>
+    <p conref="../shared/impala_common.xml#common/adls_dml"
+      conrefend="../shared/impala_common.xml#common/adls_dml_end"/>
     <p>See <xref href="../topics/impala_adls.xml#adls"/> for details about reading and writing ADLS data with Impala.</p>
 
     <p conref="../shared/impala_common.xml#common/cancel_blurb_no"/>


[22/33] impala git commit: IMPALA-6323 Allow constant analytic window expressions.

Posted by bo...@apache.org.
IMPALA-6323 Allow constant analytic window expressions.

The constraint imposed by IMPALA-1354 was artificial.
If there are constant "partition by" expressions, simply drop them,
they are no-ops.

Constant "order by" expressions can be ignored as well, though in effect
they should be accounted for as null expressions in the backend, with the
effect that combine all rows in the same window (i.e. no window breaks).

Change-Id: Idf129026c45120e9470df601268863634037908c
Reviewed-on: http://gerrit.cloudera.org:8080/11556
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Michael Ho <kw...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 0bc544685e6d0d13f8800552a62bc35e1ce8e565
Parents: 0a7da0f
Author: Michal Ostrowski <mo...@cloudera.com>
Authored: Fri Sep 28 11:39:11 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 be/src/exec/analytic-eval-node.cc               |  3 +-
 .../apache/impala/analysis/AnalyticExpr.java    | 12 ++------
 .../apache/impala/planner/AnalyticPlanner.java  | 23 ++++++++++++----
 .../impala/analysis/AnalyzeExprsTest.java       | 15 ++++------
 .../queries/PlannerTest/analytic-fns.test       | 29 ++++++++++++++++++++
 .../queries/QueryTest/analytic-fns.test         | 21 ++++++++++++++
 6 files changed, 77 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/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 036cf93..c7bcbda 100644
--- a/be/src/exec/analytic-eval-node.cc
+++ b/be/src/exec/analytic-eval-node.cc
@@ -394,7 +394,8 @@ inline Status AnalyticEvalNode::TryAddResultTupleForPrevRow(
            << " idx=" << stream_idx;
   if (fn_scope_ != ROWS && (next_partition
         || (fn_scope_ == RANGE && window_.__isset.window_end
-            && !PrevRowCompare(order_by_eq_expr_eval_, child_tuple_cmp_row)))) {
+            && !(order_by_eq_expr_eval_ == nullptr ||
+                 PrevRowCompare(order_by_eq_expr_eval_, child_tuple_cmp_row))))) {
     RETURN_IF_ERROR(AddResultTuple(stream_idx - 1));
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/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 d033ee1..76fc81f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
@@ -424,22 +424,14 @@ public class AnalyticExpr extends Expr {
     type_ = getFnCall().getType();
 
     for (Expr e: partitionExprs_) {
-      if (e.isConstant()) {
-        throw new AnalysisException(
-            "Expressions in the PARTITION BY clause must not be constant: "
-              + e.toSql() + " (in " + toSql() + ")");
-      } else if (e.getType().isComplexType()) {
+      if (e.getType().isComplexType()) {
         throw new AnalysisException(String.format("PARTITION BY expression '%s' with " +
             "complex type '%s' is not supported.", e.toSql(),
             e.getType().toSql()));
       }
     }
     for (OrderByElement e: orderByElements_) {
-      if (e.getExpr().isConstant()) {
-        throw new AnalysisException(
-            "Expressions in the ORDER BY clause must not be constant: "
-              + e.getExpr().toSql() + " (in " + toSql() + ")");
-      } else if (e.getExpr().getType().isComplexType()) {
+      if (e.getExpr().getType().isComplexType()) {
         throw new AnalysisException(String.format("ORDER BY expression '%s' with " +
             "complex type '%s' is not supported.", e.getExpr().toSql(),
             e.getExpr().getType().toSql()));

http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
index 90d98fd..3685bf4 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
@@ -82,6 +82,18 @@ public class AnalyticPlanner {
   }
 
   /**
+   * Return true if and only if exprs is non-empty and contains non-constant
+   * expressions.
+   */
+  private boolean activeExprs(List<Expr> exprs) {
+    if (exprs.isEmpty())  return false;
+    for (Expr p: exprs) {
+      if (!p.isConstant()) { return true; }
+    }
+    return false;
+  }
+
+  /**
    * Return plan tree that augments 'root' with plan nodes that implement single-node
    * evaluation of the AnalyticExprs in analyticInfo.
    * This plan takes into account a possible hash partition of its input on
@@ -224,7 +236,7 @@ public class AnalyticPlanner {
     // remove the non-partitioning group from partitionGroups
     PartitionGroup nonPartitioning = null;
     for (PartitionGroup pg: partitionGroups) {
-      if (pg.partitionByExprs.isEmpty()) {
+      if (!activeExprs(pg.partitionByExprs)) {
         nonPartitioning = pg;
         break;
       }
@@ -308,9 +320,10 @@ public class AnalyticPlanner {
     TupleDescriptor bufferedTupleDesc = null;
     // map from input to buffered tuple
     ExprSubstitutionMap bufferedSmap = new ExprSubstitutionMap();
+    boolean activePartition = activeExprs(partitionByExprs);
 
     // sort on partition by (pb) + order by (ob) exprs and create pb/ob predicates
-    if (!partitionByExprs.isEmpty() || !orderByElements.isEmpty()) {
+    if (activePartition || !orderByElements.isEmpty()) {
       // first sort on partitionExprs (direction doesn't matter)
       List<Expr> sortExprs = Lists.newArrayList(partitionByExprs);
       List<Boolean> isAsc =
@@ -337,12 +350,12 @@ public class AnalyticPlanner {
 
       // if this sort group does not have partitioning exprs, we want the sort
       // to be executed like a regular distributed sort
-      if (!partitionByExprs.isEmpty()) sortNode.setIsAnalyticSort(true);
+      if (activePartition) sortNode.setIsAnalyticSort(true);
 
       if (partitionExprs != null) {
         // create required input partition
         DataPartition inputPartition = DataPartition.UNPARTITIONED;
-        if (!partitionExprs.isEmpty()) {
+        if (activePartition) {
           inputPartition = DataPartition.hashPartitioned(partitionExprs);
         }
         sortNode.setInputPartition(inputPartition);
@@ -380,7 +393,7 @@ public class AnalyticPlanner {
       // we need to remap the pb/ob exprs to a) the sort output, b) our buffer of the
       // sort input
       Expr partitionByEq = null;
-      if (!windowGroup.partitionByExprs.isEmpty()) {
+      if (activeExprs(windowGroup.partitionByExprs)) {
         partitionByEq = createNullMatchingEquals(
             Expr.substituteList(windowGroup.partitionByExprs, sortSmap, analyzer_, false),
             sortTupleId, bufferedSmap);

http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/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 1908c21..b547526 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -1005,16 +1005,11 @@ public class AnalyzeExprsTest extends AnalyzerTest {
         "(first_value(cast(int_col AS DECIMAL)) " +
         " over (order by int_col rows between 2 preceding and 1 preceding)) " +
         "from functional.alltypestiny");
-    // IMPALA-1354: Constant expressions in order by and partition by exprs
-    AnalysisError(
-        "select rank() over (order by 1) from functional.alltypestiny",
-        "Expressions in the ORDER BY clause must not be constant: 1");
-    AnalysisError(
-        "select rank() over (partition by 2 order by id) from functional.alltypestiny",
-        "Expressions in the PARTITION BY clause must not be constant: 2");
-    AnalysisError(
-        "select rank() over (partition by 2 order by 1) from functional.alltypestiny",
-        "Expressions in the PARTITION BY clause must not be constant: 2");
+    // IMPALA-6323: Allow constant expressions in analytic window exprs.
+    AnalyzesOk("select rank() over (order by 1) from functional.alltypestiny");
+    AnalyzesOk("select count() over (partition by 2) from functional.alltypestiny");
+    AnalyzesOk(
+        "select rank() over (partition by 2 order by 1) from functional.alltypestiny");
 
     // nested analytic exprs
     AnalysisError(

http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
index f78bfb2..5084d2a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
@@ -2457,3 +2457,32 @@ PLAN-ROOT SINK
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ====
+# IMPALA-6323 Partition by a constant is equivalent to no partitioning.
+select x, count() over(partition by 1) from (VALUES((1 x), (2), (3))) T;
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+01:ANALYTIC
+|  functions: count()
+|  partition by: 1
+|
+00:UNION
+   constant-operands=3
+====
+# IMPALA-6323 Order by a constant is equivalent to no ordering.
+select x, count() over(order by 1) from (VALUES((1 x), (2), (3))) T;
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+02:ANALYTIC
+|  functions: count()
+|  order by: 1 ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|
+01:SORT
+|  order by: 1 ASC
+|
+00:UNION
+   constant-operands=3
+====
+

http://git-wip-us.apache.org/repos/asf/impala/blob/0bc54468/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 0c6207c..fbbd939 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
@@ -2119,3 +2119,24 @@ true,6,2
 true,4,2
 true,2,2
 true,NULL,2
+====
+---- QUERY
+# IMPALA-6323 Partition by a constant is equivalent to no partitioning.
+select x, count() over(partition by 1) from (VALUES((1 x), (2), (3))) T;
+---- TYPES
+TINYINT, BIGINT
+---- RESULTS
+1,3
+2,3
+3,3
+====
+---- QUERY
+# IMPALA-6323 Order by a constant is equivalent to no ordering.
+select x, count() over(order by 1) from (VALUES((1 x), (2), (3))) T;
+---- TYPES
+TINYINT, BIGINT
+---- RESULTS
+1,3
+2,3
+3,3
+====


[24/33] impala git commit: IMPALA-7791: Compute AggregationNode's estimated rows using # instances

Posted by bo...@apache.org.
IMPALA-7791: Compute AggregationNode's estimated rows using # instances

Previously, the AggregationNode calculated the estimated number
of rows based on input cardinality without accounting for the
division of input data across multiple fragment instances. This
bloated up the memory estimates for the node. After this change,
the AggregationNode accounts for the number of fragment instances
while estimating the number of rows per instance. A skew factor of
1.5 was added to account for data skew among multiple fragment
instances. This number was derived using empirical analysis of
real-world and benchmark (tpch, tpcds) queries.

Testing:
Tested queries with changed estimates to avoid cases of
significant underestimation of memory.
Ran front-end and end-to-end tests affected by this change.

Change-Id: I2cb9746fafa3e5952e28caa952837e285bcc22ac
Reviewed-on: http://gerrit.cloudera.org:8080/11854
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: ffdb8a13e4d550f07c8a756625f4df9441b157fc
Parents: 1760339
Author: poojanilangekar <po...@cloudera.com>
Authored: Thu Nov 1 16:31:48 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:40 2018 +0100

----------------------------------------------------------------------
 .../apache/impala/planner/AggregationNode.java  | 17 +++++-
 .../queries/PlannerTest/max-row-size.test       | 20 +++----
 .../PlannerTest/resource-requirements.test      | 58 ++++++++++----------
 .../PlannerTest/spillable-buffer-sizing.test    | 20 +++----
 .../queries/PlannerTest/tpcds-all.test          | 30 +++++-----
 .../queries/PlannerTest/tpch-all.test           | 32 +++++------
 .../queries/PlannerTest/tpch-nested.test        |  2 +-
 7 files changed, 97 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
index 72694ca..c5fe9d1 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -60,6 +60,9 @@ public class AggregationNode extends PlanNode {
   // Conservative minimum size of hash table for low-cardinality aggregations.
   private final static long MIN_HASH_TBL_MEM = 10L * 1024L * 1024L;
 
+  // Default skew factor to account for data skew among fragment instances.
+  private final static double DEFAULT_SKEW_FACTOR = 1.5;
+
   private final MultiAggregateInfo multiAggInfo_;
   private final AggPhase aggPhase_;
 
@@ -474,7 +477,19 @@ public class AggregationNode extends PlanNode {
       // Per-instance cardinality cannot be greater than the total input cardinality.
       long inputCardinality = getChild(0).getCardinality();
       if (inputCardinality != -1) {
-        perInstanceCardinality = Math.min(perInstanceCardinality, inputCardinality);
+        // Calculate the input cardinality distributed across fragment instances.
+        long numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
+        long perInstanceInputCardinality;
+        if (numInstances > 1) {
+          perInstanceInputCardinality =
+              (long) Math.ceil((inputCardinality / numInstances) * DEFAULT_SKEW_FACTOR);
+        } else {
+          // When numInstances is 1 or unknown(-1), perInstanceInputCardinality is the
+          // same as inputCardinality.
+          perInstanceInputCardinality = inputCardinality;
+        }
+        perInstanceCardinality =
+            Math.min(perInstanceCardinality, perInstanceInputCardinality);
       }
       perInstanceDataBytes = (long)Math.ceil(perInstanceCardinality * avgRowSize_);
       perInstanceMemEstimate = (long)Math.max(perInstanceDataBytes *

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
index e1d8119..bc06135 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -257,7 +257,7 @@ select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=120.00MB Threads=4
-Per-Host Resource Estimates: Memory=3.33GB
+Per-Host Resource Estimates: Memory=1.71GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.78MB mem-reservation=0B thread-reservation=1
@@ -270,10 +270,10 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l_partkey,tpch_parquet.lineitem.l_suppkey,tpch_parquet.lineitem.l_linenumber,tpch_parquet.lineitem.l_quantity,tpch_parquet.lineitem.l_extendedprice,tpch_parquet.lineitem.l_discount,tpch_parquet.lineitem.l_tax,tpch_parquet.lineitem.l_returnflag,tpch_parquet.lineitem.l_linestatus,tpch_parquet.lineitem.l_shipdate,tpch_parquet.lineitem.l_commitdate,tpch_parquet.lineitem.l_receiptdate,tpch_parquet.lineitem.l_shipinstruct,tpch_parquet.lineitem.l_shipmode,tpch_parquet.lineitem.l_comment)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.63GB mem-reservation=46.00MB thread-reservation=1
+Per-Host Resources: mem-estimate=837.94MB mem-reservation=46.00MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=46.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=46.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -283,10 +283,10 @@ Per-Host Resources: mem-estimate=1.63GB mem-reservation=46.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.69GB mem-reservation=74.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=907.16MB mem-reservation=74.00MB thread-reservation=2
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 00(GETNEXT)
 |
@@ -307,7 +307,7 @@ from tpch_parquet.lineitem
 group by 1, 2
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=98.00MB Threads=4
-Per-Host Resource Estimates: Memory=503MB
+Per-Host Resource Estimates: Memory=302MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.11MB mem-reservation=0B thread-reservation=1
@@ -320,11 +320,11 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(l_orderkey,l_partkey)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=211.56MB mem-reservation=48.00MB thread-reservation=1
+Per-Host Resources: mem-estimate=110.83MB mem-reservation=48.00MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  output: group_concat:merge(l_linestatus, ',')
 |  group by: l_orderkey, l_partkey
-|  mem-estimate=201.46MB mem-reservation=48.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=100.73MB mem-reservation=48.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=32B cardinality=6001215
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -334,11 +334,11 @@ Per-Host Resources: mem-estimate=211.56MB mem-reservation=48.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=281.46MB mem-reservation=50.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=180.73MB mem-reservation=50.00MB thread-reservation=2
 01:AGGREGATE [STREAMING]
 |  output: group_concat(l_linestatus, ',')
 |  group by: l_orderkey, l_partkey
-|  mem-estimate=201.46MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=100.73MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=32B cardinality=6001215
 |  in pipelines: 00(GETNEXT)
 |

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index 1f41712..b785f32 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -3127,7 +3127,7 @@ PLAN-ROOT SINK
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=123.75MB Threads=12
-Per-Host Resource Estimates: Memory=385MB
+Per-Host Resource Estimates: Memory=376MB
 
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.22MB mem-reservation=0B thread-reservation=1
@@ -3227,7 +3227,7 @@ Per-Host Resources: mem-estimate=109.02MB mem-reservation=43.00MB thread-reserva
 |
 14:AGGREGATE [FINALIZE]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=42.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=70B cardinality=575772
 |  in pipelines: 14(GETNEXT), 01(OPEN)
 |
@@ -3237,10 +3237,10 @@ Per-Host Resources: mem-estimate=109.02MB mem-reservation=43.00MB thread-reserva
 |  in pipelines: 01(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=64.32MB mem-reservation=39.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=55.74MB mem-reservation=39.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 04:AGGREGATE [STREAMING]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=42.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=70B cardinality=575772
 |  in pipelines: 01(GETNEXT)
 |
@@ -3290,8 +3290,8 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=25.00MB thread-reservat
    tuple-ids=0 row-size=78B cardinality=600122
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=243.75MB Threads=13
-Per-Host Resource Estimates: Memory=724MB
+Max Per-Host Resource Reservation: Memory=209.75MB Threads=13
+Per-Host Resource Estimates: Memory=673MB
 
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.44MB mem-reservation=0B thread-reservation=1
@@ -3407,7 +3407,7 @@ Per-Host Resources: mem-estimate=198.00MB mem-reservation=86.00MB thread-reserva
 |
 14:AGGREGATE [FINALIZE]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=42.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=2 row-size=70B cardinality=575772
 |  in pipelines: 14(GETNEXT), 01(OPEN)
 |
@@ -3417,10 +3417,10 @@ Per-Host Resources: mem-estimate=198.00MB mem-reservation=86.00MB thread-reserva
 |  in pipelines: 01(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Host Resources: mem-estimate=113.87MB mem-reservation=75.75MB thread-reservation=2 runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=62.72MB mem-reservation=41.75MB thread-reservation=2 runtime-filters-memory=1.00MB
 04:AGGREGATE [STREAMING]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=42.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=2 row-size=70B cardinality=575772
 |  in pipelines: 01(GETNEXT)
 |
@@ -3609,7 +3609,7 @@ PLAN-ROOT SINK
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=220.38MB Threads=11
-Per-Host Resource Estimates: Memory=597MB
+Per-Host Resource Estimates: Memory=533MB
 
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=33.72KB mem-reservation=0B thread-reservation=1
@@ -3624,7 +3624,7 @@ PLAN-ROOT SINK
 |  in pipelines: 09(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=73.26MB mem-reservation=34.00MB thread-reservation=1
+Per-Host Resources: mem-estimate=44.30MB mem-reservation=34.00MB thread-reservation=1
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |  mem-estimate=9.77KB mem-reservation=0B thread-reservation=0
@@ -3634,7 +3634,7 @@ Per-Host Resources: mem-estimate=73.26MB mem-reservation=34.00MB thread-reservat
 16:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_quantity)
 |  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
-|  mem-estimate=62.96MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=6 row-size=100B cardinality=600122
 |  in pipelines: 16(GETNEXT), 02(OPEN)
 |
@@ -3644,11 +3644,11 @@ Per-Host Resources: mem-estimate=73.26MB mem-reservation=34.00MB thread-reservat
 |  in pipelines: 02(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=184.12MB mem-reservation=116.38MB thread-reservation=1 runtime-filters-memory=3.00MB
+Per-Host Resources: mem-estimate=149.80MB mem-reservation=116.38MB thread-reservation=1 runtime-filters-memory=3.00MB
 08:AGGREGATE [STREAMING]
 |  output: sum(l_quantity)
 |  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
-|  mem-estimate=62.96MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=6 row-size=100B cardinality=600122
 |  in pipelines: 02(GETNEXT)
 |
@@ -3663,7 +3663,7 @@ Per-Host Resources: mem-estimate=184.12MB mem-reservation=116.38MB thread-reserv
 |  |  output: sum:merge(l_quantity)
 |  |  group by: l_orderkey
 |  |  having: sum(l_quantity) > 300
-|  |  mem-estimate=39.36MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=4 row-size=24B cardinality=156344
 |  |  in pipelines: 14(GETNEXT), 03(OPEN)
 |  |
@@ -3760,8 +3760,8 @@ Per-Host Resources: mem-estimate=90.00MB mem-reservation=10.00MB thread-reservat
    tuple-ids=2 row-size=16B cardinality=6001215
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=353.88MB Threads=13
-Per-Host Resource Estimates: Memory=1.03GB
+Max Per-Host Resource Reservation: Memory=268.88MB Threads=13
+Per-Host Resource Estimates: Memory=846MB
 
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=64.19KB mem-reservation=0B thread-reservation=1
@@ -3776,7 +3776,7 @@ PLAN-ROOT SINK
 |  in pipelines: 09(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=6
-Per-Host Resources: mem-estimate=147.13MB mem-reservation=68.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=55.22MB mem-reservation=34.00MB thread-reservation=2
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |  mem-estimate=9.77KB mem-reservation=0B thread-reservation=0
@@ -3786,7 +3786,7 @@ Per-Host Resources: mem-estimate=147.13MB mem-reservation=68.00MB thread-reserva
 16:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_quantity)
 |  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
-|  mem-estimate=62.96MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=6 row-size=100B cardinality=600122
 |  in pipelines: 16(GETNEXT), 02(OPEN)
 |
@@ -3796,11 +3796,11 @@ Per-Host Resources: mem-estimate=147.13MB mem-reservation=68.00MB thread-reserva
 |  in pipelines: 02(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Host Resources: mem-estimate=230.96MB mem-reservation=145.88MB thread-reservation=2 runtime-filters-memory=3.00MB
+Per-Host Resources: mem-estimate=115.30MB mem-reservation=94.88MB thread-reservation=2 runtime-filters-memory=3.00MB
 08:AGGREGATE [STREAMING]
 |  output: sum(l_quantity)
 |  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
-|  mem-estimate=62.96MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=6 row-size=100B cardinality=600122
 |  in pipelines: 02(GETNEXT)
 |
@@ -3823,7 +3823,7 @@ Per-Host Resources: mem-estimate=230.96MB mem-reservation=145.88MB thread-reserv
 |  |  output: sum:merge(l_quantity)
 |  |  group by: l_orderkey
 |  |  having: sum(l_quantity) > 300
-|  |  mem-estimate=39.36MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=4 row-size=24B cardinality=156344
 |  |  in pipelines: 14(GETNEXT), 03(OPEN)
 |  |
@@ -3833,11 +3833,11 @@ Per-Host Resources: mem-estimate=230.96MB mem-reservation=145.88MB thread-reserv
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Host Resources: mem-estimate=254.73MB mem-reservation=84.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=251.55MB mem-reservation=84.00MB thread-reservation=2
 |  04:AGGREGATE [STREAMING]
 |  |  output: sum(l_quantity)
 |  |  group by: l_orderkey
-|  |  mem-estimate=39.36MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=37.77MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=4 row-size=24B cardinality=1563438
 |  |  in pipelines: 03(GETNEXT)
 |  |
@@ -5502,7 +5502,7 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=108.00MB Threads=4
-Per-Host Resource Estimates: Memory=3.32GB
+Per-Host Resource Estimates: Memory=1.70GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
@@ -5516,11 +5516,11 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l_partkey,tpch_parquet.lineitem.l_suppkey,tpch_parquet.lineitem.l_linenumber,tpch_parquet.lineitem.l_quantity,tpch_parquet.lineitem.l_extendedprice,tpch_parquet.lineitem.l_discount,tpch_parquet.lineitem.l_tax,tpch_parquet.lineitem.l_returnflag,tpch_parquet.lineitem.l_linestatus,tpch_parquet.lineitem.l_shipdate,tpch_parquet.lineitem.l_commitdate,tpch_parquet.lineitem.l_receiptdate,tpch_parquet.lineitem.l_shipinstruct,tpch_parquet.lineitem.l_shipmode,tpch_parquet.lineitem.l_comment)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.63GB mem-reservation=34.00MB thread-reservation=1
+Per-Host Resources: mem-estimate=837.94MB mem-reservation=34.00MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
 |  limit: 5
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=5
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -5530,10 +5530,10 @@ Per-Host Resources: mem-estimate=1.63GB mem-reservation=34.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.69GB mem-reservation=74.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=907.16MB mem-reservation=74.00MB thread-reservation=2
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 00(GETNEXT)
 |

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
index 4e92766..ec798e8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -864,7 +864,7 @@ select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=108.00MB Threads=4
-Per-Host Resource Estimates: Memory=3.33GB
+Per-Host Resource Estimates: Memory=1.71GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.78MB mem-reservation=0B thread-reservation=1
@@ -877,10 +877,10 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l_partkey,tpch_parquet.lineitem.l_suppkey,tpch_parquet.lineitem.l_linenumber,tpch_parquet.lineitem.l_quantity,tpch_parquet.lineitem.l_extendedprice,tpch_parquet.lineitem.l_discount,tpch_parquet.lineitem.l_tax,tpch_parquet.lineitem.l_returnflag,tpch_parquet.lineitem.l_linestatus,tpch_parquet.lineitem.l_shipdate,tpch_parquet.lineitem.l_commitdate,tpch_parquet.lineitem.l_receiptdate,tpch_parquet.lineitem.l_shipinstruct,tpch_parquet.lineitem.l_shipmode,tpch_parquet.lineitem.l_comment)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.63GB mem-reservation=34.00MB thread-reservation=1
+Per-Host Resources: mem-estimate=837.94MB mem-reservation=34.00MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -890,10 +890,10 @@ Per-Host Resources: mem-estimate=1.63GB mem-reservation=34.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.69GB mem-reservation=74.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=907.16MB mem-reservation=74.00MB thread-reservation=2
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=827.16MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 00(GETNEXT)
 |
@@ -908,7 +908,7 @@ Per-Host Resources: mem-estimate=1.69GB mem-reservation=74.00MB thread-reservati
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=216.00MB Threads=5
-Per-Host Resource Estimates: Memory=6.65GB
+Per-Host Resource Estimates: Memory=1.81GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.56MB mem-reservation=0B thread-reservation=1
@@ -921,10 +921,10 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(tpch_parquet.lineitem.l_orderkey,tpch_parquet.lineitem.l_partkey,tpch_parquet.lineitem.l_suppkey,tpch_parquet.lineitem.l_linenumber,tpch_parquet.lineitem.l_quantity,tpch_parquet.lineitem.l_extendedprice,tpch_parquet.lineitem.l_discount,tpch_parquet.lineitem.l_tax,tpch_parquet.lineitem.l_returnflag,tpch_parquet.lineitem.l_linestatus,tpch_parquet.lineitem.l_shipdate,tpch_parquet.lineitem.l_commitdate,tpch_parquet.lineitem.l_receiptdate,tpch_parquet.lineitem.l_shipinstruct,tpch_parquet.lineitem.l_shipmode,tpch_parquet.lineitem.l_comment)] hosts=3 instances=6
-Per-Host Resources: mem-estimate=3.25GB mem-reservation=68.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=850.29MB mem-reservation=68.00MB thread-reservation=2
 03:AGGREGATE [FINALIZE]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=413.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -934,10 +934,10 @@ Per-Host Resources: mem-estimate=3.25GB mem-reservation=68.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=3.39GB mem-reservation=148.00MB thread-reservation=2
+Per-Host Resources: mem-estimate=987.16MB mem-reservation=148.00MB thread-reservation=2
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
-|  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=413.58MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |  in pipelines: 00(GETNEXT)
 |

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
index 133613f..095f94d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
@@ -1837,8 +1837,8 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=130.70MB Threads=17
-Per-Host Resource Estimates: Memory=462MB
+Max Per-Host Resource Reservation: Memory=96.70MB Threads=17
+Per-Host Resource Estimates: Memory=428MB
 PLAN-ROOT SINK
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1923,8 +1923,8 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=259.52MB Threads=19
-Per-Host Resource Estimates: Memory=458MB
+Max Per-Host Resource Reservation: Memory=158.52MB Threads=19
+Per-Host Resource Estimates: Memory=362MB
 PLAN-ROOT SINK
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3601,7 +3601,7 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=136.32MB Threads=16
-Per-Host Resource Estimates: Memory=530MB
+Per-Host Resource Estimates: Memory=467MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3684,8 +3684,8 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=272.64MB Threads=17
-Per-Host Resource Estimates: Memory=596MB
+Max Per-Host Resource Reservation: Memory=204.64MB Threads=17
+Per-Host Resource Estimates: Memory=401MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3878,7 +3878,7 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=61.82MB Threads=13
+Max Per-Host Resource Reservation: Memory=57.95MB Threads=13
 Per-Host Resource Estimates: Memory=347MB
 PLAN-ROOT SINK
 |
@@ -3944,7 +3944,7 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000 -> tpcds.store_sales.ss_customer_sk, RF002 -> store_sales.ss_store_sk, RF004 -> store_sales.ss_sold_date_sk, RF006 -> store_sales.ss_hdemo_sk
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=106.64MB Threads=15
+Max Per-Host Resource Reservation: Memory=95.02MB Threads=15
 Per-Host Resource Estimates: Memory=290MB
 PLAN-ROOT SINK
 |
@@ -4117,7 +4117,7 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=53.32MB Threads=13
+Max Per-Host Resource Reservation: Memory=49.45MB Threads=13
 Per-Host Resource Estimates: Memory=335MB
 PLAN-ROOT SINK
 |
@@ -4183,7 +4183,7 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000 -> tpcds.store_sales.ss_customer_sk, RF002 -> store_sales.ss_sold_date_sk, RF004 -> store_sales.ss_store_sk, RF006 -> store_sales.ss_hdemo_sk
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=99.14MB Threads=15
+Max Per-Host Resource Reservation: Memory=87.52MB Threads=15
 Per-Host Resource Estimates: Memory=275MB
 PLAN-ROOT SINK
 |
@@ -4340,7 +4340,7 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF004 -> ss_item_sk
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=54.70MB Threads=10
+Max Per-Host Resource Reservation: Memory=52.76MB Threads=10
 Per-Host Resource Estimates: Memory=285MB
 PLAN-ROOT SINK
 |
@@ -4404,7 +4404,7 @@ PLAN-ROOT SINK
    partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF004 -> ss_item_sk
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=109.39MB Threads=11
+Max Per-Host Resource Reservation: Memory=105.52MB Threads=11
 Per-Host Resource Estimates: Memory=234MB
 PLAN-ROOT SINK
 |
@@ -5350,7 +5350,7 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_item_sk, RF004 -> ss_sold_date_sk
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=393.21MB Threads=29
-Per-Host Resource Estimates: Memory=1.35GB
+Per-Host Resource Estimates: Memory=1.10GB
 PLAN-ROOT SINK
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5536,7 +5536,7 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_item_sk, RF004 -> ss_sold_date_sk
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=752.42MB Threads=33
-Per-Host Resource Estimates: Memory=1.35GB
+Per-Host Resource Estimates: Memory=1.04GB
 PLAN-ROOT SINK
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index 5db8c0c..f6f7672 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
@@ -318,7 +318,7 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> tpch.partsupp.ps_partkey, RF008 -> ps_suppkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=126.81MB Threads=23
+Max Per-Host Resource Reservation: Memory=122.81MB Threads=23
 Per-Host Resource Estimates: Memory=713MB
 PLAN-ROOT SINK
 |
@@ -1149,7 +1149,7 @@ PLAN-ROOT SINK
    runtime filters: RF006 -> l_suppkey, RF008 -> l_orderkey
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=103.95MB Threads=15
-Per-Host Resource Estimates: Memory=723MB
+Per-Host Resource Estimates: Memory=717MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1224,8 +1224,8 @@ PLAN-ROOT SINK
    predicates: l_shipdate <= '1996-12-31', l_shipdate >= '1995-01-01'
    runtime filters: RF006 -> l_suppkey, RF008 -> l_orderkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=194.78MB Threads=17
-Per-Host Resource Estimates: Memory=716MB
+Max Per-Host Resource Reservation: Memory=160.78MB Threads=17
+Per-Host Resource Estimates: Memory=670MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1990,7 +1990,7 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> c_nationkey, RF002 -> c_custkey
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=124.45MB Threads=11
-Per-Host Resource Estimates: Memory=701MB
+Per-Host Resource Estimates: Memory=669MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2047,8 +2047,8 @@ PLAN-ROOT SINK
    predicates: l_returnflag = 'R'
    runtime filters: RF004 -> l_orderkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=231.89MB Threads=13
-Per-Host Resource Estimates: Memory=741MB
+Max Per-Host Resource Reservation: Memory=163.89MB Threads=13
+Per-Host Resource Estimates: Memory=608MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2612,7 +2612,7 @@ PLAN-ROOT SINK
    predicates: NOT o_comment LIKE '%special%requests%'
    runtime filters: RF000 -> o_custkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=77.12MB Threads=9
+Max Per-Host Resource Reservation: Memory=69.38MB Threads=9
 Per-Host Resource Estimates: Memory=334MB
 PLAN-ROOT SINK
 |
@@ -3068,7 +3068,7 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> ps_partkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=101.38MB Threads=11
+Max Per-Host Resource Reservation: Memory=93.62MB Threads=11
 Per-Host Resource Estimates: Memory=393MB
 PLAN-ROOT SINK
 |
@@ -3361,7 +3361,7 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF004 -> l_orderkey
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=220.38MB Threads=11
-Per-Host Resource Estimates: Memory=1.04GB
+Per-Host Resource Estimates: Memory=1005MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3424,8 +3424,8 @@ PLAN-ROOT SINK
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF004 -> l_orderkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=353.88MB Threads=13
-Per-Host Resource Estimates: Memory=1.03GB
+Max Per-Host Resource Reservation: Memory=268.88MB Threads=13
+Per-Host Resource Estimates: Memory=846MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3708,8 +3708,8 @@ PLAN-ROOT SINK
    predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF002 -> tpch.lineitem.l_partkey, RF003 -> tpch.lineitem.l_suppkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=124.63MB Threads=13
-Per-Host Resource Estimates: Memory=677MB
+Max Per-Host Resource Reservation: Memory=107.63MB Threads=13
+Per-Host Resource Estimates: Memory=660MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3776,8 +3776,8 @@ PLAN-ROOT SINK
    predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF002 -> tpch.lineitem.l_partkey, RF003 -> tpch.lineitem.l_suppkey
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=247.39MB Threads=15
-Per-Host Resource Estimates: Memory=691MB
+Max Per-Host Resource Reservation: Memory=163.39MB Threads=15
+Per-Host Resource Estimates: Memory=609MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]

http://git-wip-us.apache.org/repos/asf/impala/blob/ffdb8a13/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
index 7717acf..dfcdc4f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
@@ -1620,7 +1620,7 @@ PLAN-ROOT SINK
    partitions=1/1 files=4 size=288.99MB
    predicates on c_orders: (NOT o_comment LIKE '%special%requests%')
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=96.94MB Threads=5
+Max Per-Host Resource Reservation: Memory=94.94MB Threads=5
 Per-Host Resource Estimates: Memory=548MB
 PLAN-ROOT SINK
 |


[33/33] impala git commit: IMPALA-7148: Make test_profile_fragment_instances() more robust

Posted by bo...@apache.org.
IMPALA-7148: Make test_profile_fragment_instances() more robust

test_profile_fragment_instances() makes assumption about number
of instances of each scan node in the distributed query plan.
The number of instances for each scan nodes can actually vary
depending on how data is loaded and scheduler's decision.

This change relaxes the check for number of instances of each
scan node is a multiple of 3 which is the number of scan nodes
in the plan.

Change-Id: I08b068c21e9637575c85f4d54be9f7c56c106bf1
Reviewed-on: http://gerrit.cloudera.org:8080/11906
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 1d412a09ac807c2f7ab39ac4fdc3e21c29024190
Parents: 59f3f6b
Author: Michael Ho <kw...@cloudera.com>
Authored: Wed Nov 7 14:29:09 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:36 2018 +0100

----------------------------------------------------------------------
 tests/query_test/test_observability.py | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/1d412a09/tests/query_test/test_observability.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_observability.py b/tests/query_test/test_observability.py
index c351e02..2950ee8 100644
--- a/tests/query_test/test_observability.py
+++ b/tests/query_test/test_observability.py
@@ -157,9 +157,11 @@ class TestObservability(ImpalaTestSuite):
         with l as (select * from tpch.lineitem UNION ALL select * from tpch.lineitem)
         select STRAIGHT_JOIN count(*) from (select * from tpch.lineitem a LIMIT 1) a
         join (select * from l LIMIT 2000000) b on a.l_orderkey = -b.l_orderkey;""")
-    # There are 3 scan nodes and each appears in the profile 4 times (for 3 fragment
-    # instances + the averaged fragment).
-    assert results.runtime_profile.count("HDFS_SCAN_NODE") == 12
+    # There are 3 scan nodes and each appears in the profile n+1 times (for n fragment
+    # instances + the averaged fragment). n depends on how data is loaded and scheduler's
+    # decision.
+    n = results.runtime_profile.count("HDFS_SCAN_NODE")
+    assert n > 0 and n % 3 == 0
     # There are 3 exchange nodes and each appears in the profile 2 times (for 1 fragment
     # instance + the averaged fragment).
     assert results.runtime_profile.count("EXCHANGE_NODE") == 6


[23/33] impala git commit: IMPALA-7477: Batch-oriented query set construction

Posted by bo...@apache.org.
IMPALA-7477: Batch-oriented query set construction

Rework the row-by-row construction of query result sets in PlanRootSink
so that it materialises an output column at a time. Make some minor
optimisations like preallocating output vectors and initialising
strings more efficiently.

My intent is both to make this faster and to make the QueryResultSet
interface better before IMPALA-4268 does a bunch of surgery on this
part of the code.

Testing:
Ran core tests.

Perf:
Downloaded tpch_parquet.orders via JDBC driver.
Before: 3.01s, After: 2.57s.

Downloaded l_orderkey from tpch_parquet.lineitem.
Before: 1.21s, After: 1.08s.

Change-Id: I764fa302842438902cd5db2551ec6e3cb77b6874
Reviewed-on: http://gerrit.cloudera.org:8080/11879
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 0f63b2c9f9d62b0d22191f454b672a6047206252
Parents: d8b1e43
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Sun Aug 19 00:48:47 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 be/src/exec/plan-root-sink.cc      |  26 +--
 be/src/exec/plan-root-sink.h       |   4 -
 be/src/service/hs2-util.cc         | 318 ++++++++++++++++++++++++--------
 be/src/service/hs2-util.h          |  15 +-
 be/src/service/query-result-set.cc | 115 +++++++-----
 be/src/service/query-result-set.h  |  14 +-
 6 files changed, 331 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/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 a64dbb9..1f5b2e5 100644
--- a/be/src/exec/plan-root-sink.cc
+++ b/be/src/exec/plan-root-sink.cc
@@ -79,22 +79,11 @@ Status PlanRootSink::Send(RuntimeState* state, RowBatch* batch) {
 
     // Otherwise the consumer is ready. Fill out the rows.
     DCHECK(results_ != nullptr);
-    // List of expr values to hold evaluated rows from the query
-    vector<void*> result_row;
-    result_row.resize(output_exprs_.size());
-
-    // List of scales for floating point values in result_row
-    vector<int> scales;
-    scales.resize(result_row.size());
-
     int num_to_fetch = batch->num_rows() - current_batch_row;
     if (num_rows_requested_ > 0) num_to_fetch = min(num_to_fetch, num_rows_requested_);
-    for (int i = 0; i < num_to_fetch; ++i) {
-      TupleRow* row = batch->GetRow(current_batch_row);
-      GetRowValue(row, &result_row, &scales);
-      RETURN_IF_ERROR(results_->AddOneRow(result_row, scales));
-      ++current_batch_row;
-    }
+    RETURN_IF_ERROR(
+        results_->AddRows(output_expr_evals_, batch, current_batch_row, num_to_fetch));
+    current_batch_row += num_to_fetch;
     // Prevent expr result allocations from accumulating.
     expr_results_pool_->Clear();
     // Signal the consumer.
@@ -146,13 +135,4 @@ Status PlanRootSink::GetNext(
   *eos = sender_state_ == SenderState::EOS;
   return state->GetQueryStatus();
 }
-
-void PlanRootSink::GetRowValue(
-    TupleRow* row, vector<void*>* result, vector<int>* scales) {
-  DCHECK_GE(result->size(), output_expr_evals_.size());
-  for (int i = 0; i < output_expr_evals_.size(); ++i) {
-    (*result)[i] = output_expr_evals_[i]->GetValue(row);
-    (*scales)[i] = output_expr_evals_[i]->output_scale();
-  }
-}
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/be/src/exec/plan-root-sink.h
----------------------------------------------------------------------
diff --git a/be/src/exec/plan-root-sink.h b/be/src/exec/plan-root-sink.h
index 1d64b21..300c993 100644
--- a/be/src/exec/plan-root-sink.h
+++ b/be/src/exec/plan-root-sink.h
@@ -118,10 +118,6 @@ class PlanRootSink : public DataSink {
 
   /// Set by GetNext() to indicate to Send() how many rows it should write to results_.
   int num_rows_requested_ = 0;
-
-  /// Writes a single row into 'result' and 'scales' by evaluating
-  /// output_expr_evals_ over 'row'.
-  void GetRowValue(TupleRow* row, std::vector<void*>* result, std::vector<int>* scales);
 };
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/be/src/service/hs2-util.cc
----------------------------------------------------------------------
diff --git a/be/src/service/hs2-util.cc b/be/src/service/hs2-util.cc
index b856556..66a76bd 100644
--- a/be/src/service/hs2-util.cc
+++ b/be/src/service/hs2-util.cc
@@ -18,9 +18,12 @@
 #include "service/hs2-util.h"
 
 #include "common/logging.h"
+#include "exprs/scalar-expr-evaluator.h"
 #include "runtime/decimal-value.inline.h"
 #include "runtime/raw-value.inline.h"
+#include "runtime/row-batch.h"
 #include "runtime/types.h"
+#include "util/bit-util.h"
 
 #include <gutil/strings/substitute.h>
 
@@ -49,7 +52,9 @@ inline bool GetNullBit(const string& nulls, uint32_t row_idx) {
 
 void impala::StitchNulls(uint32_t num_rows_before, uint32_t num_rows_added,
     uint32_t start_idx, const string& from, string* to) {
-  to->reserve((num_rows_before + num_rows_added + 7) / 8);
+  // Round up to power-of-two to avoid accidentally quadratic behaviour from repeated
+  // small increases in size.
+  to->reserve(BitUtil::RoundUpToPowerOfTwo((num_rows_before + num_rows_added + 7) / 8));
 
   // TODO: This is very inefficient, since we could conceivably go one byte at a time
   // (although the operands should stay live in registers in the loop). However doing this
@@ -118,106 +123,257 @@ void impala::TColumnValueToHS2TColumn(const TColumnValue& col_val,
   SetNullBit(row_idx, is_null, nulls);
 }
 
+// Specialised per-type implementations of ExprValuesToHS2TColumn.
+
+// Helper to reserve space in hs2Vals->values and hs2Vals->nulls for the values that the
+// different implementations of ExprValuesToHS2TColumn will write.
+template <typename T>
+void ReserveSpace(int start_idx, int num_rows, uint32_t output_row_idx, T* hs2Vals) {
+  int64_t num_output_rows = output_row_idx + num_rows - start_idx;
+  int64_t num_null_bytes = BitUtil::RoundUpNumBytes(num_output_rows);
+  // Round up reserve() arguments to power-of-two to avoid accidentally quadratic
+  // behaviour from repeated small increases in size.
+  hs2Vals->values.reserve(BitUtil::RoundUpToPowerOfTwo(num_output_rows));
+  hs2Vals->nulls.reserve(BitUtil::RoundUpToPowerOfTwo(num_null_bytes));
+}
+
+// Implementation for BOOL.
+static void BoolExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->boolVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    BooleanVal val = expr_eval->GetBooleanVal(it.Get());
+    column->boolVal.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->boolVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for TINYINT.
+static void TinyIntExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->byteVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    TinyIntVal val = expr_eval->GetTinyIntVal(it.Get());
+    column->byteVal.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->byteVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for SMALLINT.
+static void SmallIntExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval,
+    RowBatch* batch, int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->i16Val);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    SmallIntVal val = expr_eval->GetSmallIntVal(it.Get());
+    column->i16Val.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->i16Val.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for INT.
+static void IntExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->i32Val);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    IntVal val = expr_eval->GetIntVal(it.Get());
+    column->i32Val.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->i32Val.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for BIGINT.
+static void BigIntExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->i64Val);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    BigIntVal val = expr_eval->GetBigIntVal(it.Get());
+    column->i64Val.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->i64Val.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for FLOAT.
+static void FloatExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->doubleVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    FloatVal val = expr_eval->GetFloatVal(it.Get());
+    column->doubleVal.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->doubleVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for DOUBLE.
+static void DoubleExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->doubleVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    DoubleVal val = expr_eval->GetDoubleVal(it.Get());
+    column->doubleVal.values.push_back(val.val);
+    SetNullBit(output_row_idx, val.is_null, &column->doubleVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for TIMESTAMP.
+static void TimestampExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval,
+    RowBatch* batch, int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->stringVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    TimestampVal val = expr_eval->GetTimestampVal(it.Get());
+    column->stringVal.values.emplace_back();
+    if (!val.is_null) {
+      TimestampValue value = TimestampValue::FromTimestampVal(val);
+      RawValue::PrintValue(
+          &value, TYPE_TIMESTAMP, -1, &(column->stringVal.values.back()));
+    }
+    SetNullBit(output_row_idx, val.is_null, &column->stringVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for STRING and VARCHAR.
+static void StringExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, RowBatch* batch,
+    int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->stringVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    StringVal val = expr_eval->GetStringVal(it.Get());
+    if (val.is_null) {
+      column->stringVal.values.emplace_back();
+    } else {
+      column->stringVal.values.emplace_back(reinterpret_cast<char*>(val.ptr), val.len);
+    }
+    SetNullBit(output_row_idx, val.is_null, &column->stringVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+// Implementation for CHAR.
+static void CharExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval,
+    const TColumnType& type, RowBatch* batch, int start_idx, int num_rows,
+    uint32_t output_row_idx, apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->stringVal);
+  ColumnType char_type = ColumnType::CreateCharType(type.types[0].scalar_type.len);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    StringVal val = expr_eval->GetStringVal(it.Get());
+    if (val.is_null) {
+      column->stringVal.values.emplace_back();
+    } else {
+      column->stringVal.values.emplace_back(
+          reinterpret_cast<const char*>(val.ptr), char_type.len);
+    }
+    SetNullBit(output_row_idx, val.is_null, &column->stringVal.nulls);
+    ++output_row_idx;
+  }
+}
+
+static void DecimalExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval,
+    const TColumnType& type, RowBatch* batch, int start_idx, int num_rows,
+    uint32_t output_row_idx, apache::hive::service::cli::thrift::TColumn* column) {
+  ReserveSpace(start_idx, num_rows, output_row_idx, &column->stringVal);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    DecimalVal val = expr_eval->GetDecimalVal(it.Get());
+    const ColumnType& decimalType = ColumnType::FromThrift(type);
+    if (val.is_null) {
+      column->stringVal.values.emplace_back();
+    } else {
+      switch (decimalType.GetByteSize()) {
+        case 4:
+          column->stringVal.values.emplace_back(
+              Decimal4Value(val.val4).ToString(decimalType));
+          break;
+        case 8:
+          column->stringVal.values.emplace_back(
+              Decimal8Value(val.val8).ToString(decimalType));
+          break;
+        case 16:
+          column->stringVal.values.emplace_back(
+              Decimal16Value(val.val16).ToString(decimalType));
+          break;
+        default:
+          DCHECK(false) << "bad type: " << decimalType;
+      }
+    }
+    SetNullBit(output_row_idx, val.is_null, &column->stringVal.nulls);
+    ++output_row_idx;
+  }
+}
+
 // For V6 and above
-void impala::ExprValueToHS2TColumn(const void* value, const TColumnType& type,
-    uint32_t row_idx, thrift::TColumn* column) {
-  string* nulls;
+void impala::ExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval,
+    const TColumnType& type, RowBatch* batch, int start_idx, int num_rows,
+    uint32_t output_row_idx, apache::hive::service::cli::thrift::TColumn* column) {
+  // Dispatch to a templated function for the loop over rows. This avoids branching on
+  // the type for every row.
+  // TODO: instead of relying on stamped out implementations, we could codegen this loop
+  // to inline the expression evaluation into the loop body.
   switch (type.types[0].scalar_type.type) {
     case TPrimitiveType::NULL_TYPE:
     case TPrimitiveType::BOOLEAN:
-      column->boolVal.values.push_back(
-          value == NULL ? false : *reinterpret_cast<const bool*>(value));
-      nulls = &column->boolVal.nulls;
-      break;
+      BoolExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::TINYINT:
-      column->byteVal.values.push_back(
-          value == NULL ? 0 : *reinterpret_cast<const int8_t*>(value));
-      nulls = &column->byteVal.nulls;
-      break;
+      TinyIntExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::SMALLINT:
-      column->i16Val.values.push_back(
-          value == NULL ? 0 : *reinterpret_cast<const int16_t*>(value));
-      nulls = &column->i16Val.nulls;
-      break;
+      SmallIntExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::INT:
-      column->i32Val.values.push_back(
-          value == NULL ? 0 : *reinterpret_cast<const int32_t*>(value));
-      nulls = &column->i32Val.nulls;
-      break;
+      IntExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::BIGINT:
-      column->i64Val.values.push_back(
-          value == NULL ? 0 : *reinterpret_cast<const int64_t*>(value));
-      nulls = &column->i64Val.nulls;
-      break;
+      BigIntExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::FLOAT:
-      column->doubleVal.values.push_back(
-          value == NULL ? 0.f : *reinterpret_cast<const float*>(value));
-      nulls = &column->doubleVal.nulls;
-      break;
+      FloatExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::DOUBLE:
-      column->doubleVal.values.push_back(
-          value == NULL ? 0.0 : *reinterpret_cast<const double*>(value));
-      nulls = &column->doubleVal.nulls;
-      break;
+      DoubleExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::TIMESTAMP:
-      column->stringVal.values.push_back("");
-      if (value != NULL) {
-        RawValue::PrintValue(value, TYPE_TIMESTAMP, -1,
-            &(column->stringVal.values.back()));
-      }
-      nulls = &column->stringVal.nulls;
-      break;
+      TimestampExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::STRING:
     case TPrimitiveType::VARCHAR:
-      column->stringVal.values.push_back("");
-      if (value != NULL) {
-        const StringValue* str_val = reinterpret_cast<const StringValue*>(value);
-        column->stringVal.values.back().assign(
-            static_cast<char*>(str_val->ptr), str_val->len);
-      }
-      nulls = &column->stringVal.nulls;
-      break;
+      StringExprValuesToHS2TColumn(
+          expr_eval, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::CHAR:
-      column->stringVal.values.push_back("");
-      if (value != NULL) {
-        ColumnType char_type = ColumnType::CreateCharType(type.types[0].scalar_type.len);
-        column->stringVal.values.back().assign(
-            reinterpret_cast<const char*>(value), char_type.len);
-      }
-      nulls = &column->stringVal.nulls;
-      break;
+      CharExprValuesToHS2TColumn(
+          expr_eval, type, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     case TPrimitiveType::DECIMAL: {
-      // HiveServer2 requires decimal to be presented as string.
-      column->stringVal.values.push_back("");
-      const ColumnType& decimalType = ColumnType::FromThrift(type);
-      if (value != NULL) {
-        switch (decimalType.GetByteSize()) {
-          case 4:
-            column->stringVal.values.back() =
-                reinterpret_cast<const Decimal4Value*>(value)->ToString(decimalType);
-            break;
-          case 8:
-            column->stringVal.values.back() =
-                reinterpret_cast<const Decimal8Value*>(value)->ToString(decimalType);
-            break;
-          case 16:
-            column->stringVal.values.back() =
-                reinterpret_cast<const Decimal16Value*>(value)->ToString(decimalType);
-            break;
-          default:
-            DCHECK(false) << "bad type: " << decimalType;
-        }
-      }
-      nulls = &column->stringVal.nulls;
-      break;
+      DecimalExprValuesToHS2TColumn(
+          expr_eval, type, batch, start_idx, num_rows, output_row_idx, column);
+      return;
     }
     default:
       DCHECK(false) << "Unhandled type: "
                     << TypeToString(ThriftToType(type.types[0].scalar_type.type));
-      return;
   }
-
-  SetNullBit(row_idx, (value == NULL), nulls);
 }
 
 // For V1 -> V5

http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/be/src/service/hs2-util.h
----------------------------------------------------------------------
diff --git a/be/src/service/hs2-util.h b/be/src/service/hs2-util.h
index 44ceba6..4f0f973 100644
--- a/be/src/service/hs2-util.h
+++ b/be/src/service/hs2-util.h
@@ -20,16 +20,23 @@
 
 namespace impala {
 
-/// Utility methods for converting from Impala (either an Expr result or a TColumnValue) to
-/// Hive types (either a thrift::TColumnValue (V1->V5) or a TColumn (V6->).
+class RowBatch;
+class ScalarExprEvaluator;
+
+/// Utility methods for converting from Impala (either an Expr result or a TColumnValue)
+/// to Hive types (either a thrift::TColumnValue (V1->V5) or a TColumn (V6->).
 
 /// For V6->
 void TColumnValueToHS2TColumn(const TColumnValue& col_val, const TColumnType& type,
     uint32_t row_idx, apache::hive::service::cli::thrift::TColumn* column);
 
+/// Evaluate 'expr_eval' over the row [start_idx, start_idx + num_rows) from 'batch' into
+/// 'column' with 'type' starting at output_row_idx. The caller is responsible for
+/// calling RuntimeState::GetQueryStatus() to check for expression evaluation errors.
 /// For V6->
-void ExprValueToHS2TColumn(const void* value, const TColumnType& type,
-    uint32_t row_idx, apache::hive::service::cli::thrift::TColumn* column);
+void ExprValuesToHS2TColumn(ScalarExprEvaluator* expr_eval, const TColumnType& type,
+    RowBatch* batch, int start_idx, int num_rows, uint32_t output_row_idx,
+    apache::hive::service::cli::thrift::TColumn* column);
 
 /// For V1->V5
 void TColumnValueToHS2TColumnValue(const TColumnValue& col_val, const TColumnType& type,

http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/be/src/service/query-result-set.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-result-set.cc b/be/src/service/query-result-set.cc
index aacd849..f254176 100644
--- a/be/src/service/query-result-set.cc
+++ b/be/src/service/query-result-set.cc
@@ -20,10 +20,13 @@
 #include <sstream>
 #include <boost/scoped_ptr.hpp>
 
+#include "exprs/scalar-expr-evaluator.h"
 #include "rpc/thrift-util.h"
 #include "runtime/raw-value.h"
+#include "runtime/row-batch.h"
 #include "runtime/types.h"
 #include "service/hs2-util.h"
+#include "util/bit-util.h"
 
 #include "common/names.h"
 
@@ -51,18 +54,19 @@ class AsciiQueryResultSet : public QueryResultSet {
 
   virtual ~AsciiQueryResultSet() {}
 
-  /// Convert one row's expr values stored in 'col_values' to ASCII using "\t" as column
+  /// Evaluate 'expr_evals' over rows in 'batch', convert to ASCII using "\t" as column
   /// delimiter and store it in this result set.
   /// TODO: Handle complex types.
-  virtual Status AddOneRow(const vector<void*>& col_values, const vector<int>& scales);
+  virtual Status AddRows(const vector<ScalarExprEvaluator*>& expr_evals, RowBatch* batch,
+      int start_idx, int num_rows) override;
 
   /// Convert TResultRow to ASCII using "\t" as column delimiter and store it in this
   /// result set.
-  virtual Status AddOneRow(const TResultRow& row);
+  virtual Status AddOneRow(const TResultRow& row) override;
 
-  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows);
-  virtual int64_t ByteSize(int start_idx, int num_rows);
-  virtual size_t size() { return result_set_->size(); }
+  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows) override;
+  virtual int64_t ByteSize(int start_idx, int num_rows) override;
+  virtual size_t size() override { return result_set_->size(); }
 
  private:
   /// Metadata of the result set
@@ -80,18 +84,20 @@ class HS2ColumnarResultSet : public QueryResultSet {
 
   virtual ~HS2ColumnarResultSet() {}
 
-  /// Add a row of expr values
-  virtual Status AddOneRow(const vector<void*>& col_values, const vector<int>& scales);
+  /// Evaluate 'expr_evals' over rows in 'batch' and convert to the HS2 columnar
+  /// representation.
+  virtual Status AddRows(const vector<ScalarExprEvaluator*>& expr_evals, RowBatch* batch,
+      int start_idx, int num_rows) override;
 
   /// Add a row from a TResultRow
-  virtual Status AddOneRow(const TResultRow& row);
+  virtual Status AddOneRow(const TResultRow& row) override;
 
   /// Copy all columns starting at 'start_idx' and proceeding for a maximum of 'num_rows'
   /// from 'other' into this result set
-  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows);
+  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows) override;
 
-  virtual int64_t ByteSize(int start_idx, int num_rows);
-  virtual size_t size() { return num_rows_; }
+  virtual int64_t ByteSize(int start_idx, int num_rows) override;
+  virtual size_t size() override { return num_rows_; }
 
  private:
   /// Metadata of the result set
@@ -119,15 +125,17 @@ class HS2RowOrientedResultSet : public QueryResultSet {
 
   virtual ~HS2RowOrientedResultSet() {}
 
-  /// Convert expr values to HS2 TRow and store it in a TRowSet.
-  virtual Status AddOneRow(const vector<void*>& col_values, const vector<int>& scales);
+  /// Evaluate 'expr_evals' over rows in 'batch' and convert to the HS2 row-oriented
+  /// representation of TRows stored in a TRowSet.
+  virtual Status AddRows(const vector<ScalarExprEvaluator*>& expr_evals, RowBatch* batch,
+      int start_idx, int num_rows) override;
 
   /// Convert TResultRow to HS2 TRow and store it in a TRowSet
-  virtual Status AddOneRow(const TResultRow& row);
+  virtual Status AddOneRow(const TResultRow& row) override;
 
-  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows);
-  virtual int64_t ByteSize(int start_idx, int num_rows);
-  virtual size_t size() { return result_set_->rows.size(); }
+  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows) override;
+  virtual int64_t ByteSize(int start_idx, int num_rows) override;
+  virtual size_t size() override { return result_set_->rows.size(); }
 
  private:
   /// Metadata of the result set
@@ -158,20 +166,34 @@ QueryResultSet* QueryResultSet::CreateHS2ResultSet(
 
 //////////////////////////////////////////////////////////////////////////////////////////
 
-Status AsciiQueryResultSet::AddOneRow(
-    const vector<void*>& col_values, const vector<int>& scales) {
-  int num_col = col_values.size();
+Status AsciiQueryResultSet::AddRows(const vector<ScalarExprEvaluator*>& expr_evals,
+    RowBatch* batch, int start_idx, int num_rows) {
+  DCHECK_GE(batch->num_rows(), start_idx + num_rows);
+  int num_col = expr_evals.size();
   DCHECK_EQ(num_col, metadata_.columns.size());
+  vector<int> scales;
+  scales.reserve(num_col);
+  for (ScalarExprEvaluator* expr_eval : expr_evals) {
+    scales.push_back(expr_eval->output_scale());
+  }
+  // Round up to power-of-two to avoid accidentally quadratic behaviour from repeated
+  // small increases in size.
+  result_set_->reserve(
+      BitUtil::RoundUpToPowerOfTwo(result_set_->size() + num_rows - start_idx));
   stringstream out_stream;
   out_stream.precision(ASCII_PRECISION);
-  for (int i = 0; i < num_col; ++i) {
-    // ODBC-187 - ODBC can only take "\t" as the delimiter
-    out_stream << (i > 0 ? "\t" : "");
-    DCHECK_EQ(1, metadata_.columns[i].columnType.types.size());
-    RawValue::PrintValue(col_values[i],
-        ColumnType::FromThrift(metadata_.columns[i].columnType), scales[i], &out_stream);
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    for (int i = 0; i < num_col; ++i) {
+      // ODBC-187 - ODBC can only take "\t" as the delimiter
+      out_stream << (i > 0 ? "\t" : "");
+      DCHECK_EQ(1, metadata_.columns[i].columnType.types.size());
+      RawValue::PrintValue(expr_evals[i]->GetValue(it.Get()),
+          ColumnType::FromThrift(metadata_.columns[i].columnType), scales[i],
+          &out_stream);
+    }
+    result_set_->push_back(out_stream.str());
+    out_stream.str("");
   }
-  result_set_->push_back(out_stream.str());
   return Status::OK();
 }
 
@@ -263,16 +285,18 @@ HS2ColumnarResultSet::HS2ColumnarResultSet(
   InitColumns();
 }
 
-// Add a row of expr values
-Status HS2ColumnarResultSet::AddOneRow(
-    const vector<void*>& col_values, const vector<int>& scales) {
-  int num_col = col_values.size();
+Status HS2ColumnarResultSet::AddRows(const vector<ScalarExprEvaluator*>& expr_evals,
+    RowBatch* batch, int start_idx, int num_rows) {
+  DCHECK_GE(batch->num_rows(), start_idx + num_rows);
+  int num_col = expr_evals.size();
   DCHECK_EQ(num_col, metadata_.columns.size());
   for (int i = 0; i < num_col; ++i) {
-    ExprValueToHS2TColumn(col_values[i], metadata_.columns[i].columnType, num_rows_,
+    const TColumnType& type = metadata_.columns[i].columnType;
+    ScalarExprEvaluator* expr_eval = expr_evals[i];
+    ExprValuesToHS2TColumn(expr_eval, type, batch, start_idx, num_rows, num_rows_,
         &(result_set_->columns[i]));
   }
-  ++num_rows_;
+  num_rows_ += num_rows;
   return Status::OK();
 }
 
@@ -427,16 +451,21 @@ HS2RowOrientedResultSet::HS2RowOrientedResultSet(
   }
 }
 
-Status HS2RowOrientedResultSet::AddOneRow(
-    const vector<void*>& col_values, const vector<int>& scales) {
-  int num_col = col_values.size();
+Status HS2RowOrientedResultSet::AddRows(const vector<ScalarExprEvaluator*>& expr_evals,
+    RowBatch* batch, int start_idx, int num_rows) {
+  DCHECK_GE(batch->num_rows(), start_idx + num_rows);
+  int num_col = expr_evals.size();
   DCHECK_EQ(num_col, metadata_.columns.size());
-  result_set_->rows.push_back(TRow());
-  TRow& trow = result_set_->rows.back();
-  trow.colVals.resize(num_col);
-  for (int i = 0; i < num_col; ++i) {
-    ExprValueToHS2TColumnValue(
-        col_values[i], metadata_.columns[i].columnType, &(trow.colVals[i]));
+  result_set_->rows.reserve(
+      BitUtil::RoundUpToPowerOfTwo(result_set_->rows.size() + num_rows - start_idx));
+  FOREACH_ROW_LIMIT(batch, start_idx, num_rows, it) {
+    result_set_->rows.push_back(TRow());
+    TRow& trow = result_set_->rows.back();
+    trow.colVals.resize(num_col);
+    for (int i = 0; i < num_col; ++i) {
+      ExprValueToHS2TColumnValue(expr_evals[i]->GetValue(it.Get()),
+          metadata_.columns[i].columnType, &(trow.colVals[i]));
+    }
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0f63b2c9/be/src/service/query-result-set.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-result-set.h b/be/src/service/query-result-set.h
index e0c88d7..fa39d73 100644
--- a/be/src/service/query-result-set.h
+++ b/be/src/service/query-result-set.h
@@ -27,6 +27,9 @@
 
 namespace impala {
 
+class RowBatch;
+class ScalarExprEvaluator;
+
 /// Wraps a client-API specific result representation, and implements the logic required
 /// to translate into that format from Impala's row format.
 ///
@@ -36,12 +39,11 @@ class QueryResultSet {
   QueryResultSet() {}
   virtual ~QueryResultSet() {}
 
-  /// Add a single row to this result set. The row is a vector of pointers to values,
-  /// whose memory belongs to the caller. 'scales' contains the scales for decimal values
-  /// (# of digits after decimal), with -1 indicating no scale specified or the
-  /// corresponding value is not a decimal.
-  virtual Status AddOneRow(
-      const std::vector<void*>& row, const std::vector<int>& scales) = 0;
+  /// Add 'num_rows' rows to the result set, obtained by evaluating 'expr_evals' over
+  /// the rows in 'batch' starting at start_idx. Batch must contain at least
+  /// ('start_idx' + 'num_rows') rows.
+  virtual Status AddRows(const std::vector<ScalarExprEvaluator*>& expr_evals,
+      RowBatch* batch, int start_idx, int num_rows) = 0;
 
   /// Add the TResultRow to this result set. When a row comes from a DDL/metadata
   /// operation, the row in the form of TResultRow.


[09/33] impala git commit: IMPALA-6436: exit instead of abort for catalog startup failure

Posted by bo...@apache.org.
IMPALA-6436: exit instead of abort for catalog startup failure

Rename EXIT_WITH_EXC to ABORT_WITH_EXC to make the behaviour more
obvious at callsites.

Handle exceptions from Catalog constructor by logging the backtrace and
exiting cleanly, rather than aborting. This will prevent generation of a
coredump or minidump.

Testing:
Tested starting the catalogd locally without the HMS running and a
low connection timeout:

  start-impala-cluster.py --catalogd_args=--initial_hms_cnxn_timeout_s=2

Confirmed that the backtrace was logged to catalogd.ERROR and that no
core or minidump was generated.

Change-Id: I4026dccb39843b847426112fc0fe9ba897e48dcc
Reviewed-on: http://gerrit.cloudera.org:8080/11871
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 0d4c6ae0519873dfff3b9859dd17ba25b7419b07
Parents: f90931e
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Nov 2 09:46:29 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/catalog/catalog.cc                 |  4 +--
 be/src/scheduling/request-pool-service.cc | 10 ++++----
 be/src/service/fe-support.cc              |  2 +-
 be/src/service/frontend.cc                |  4 +--
 be/src/util/jni-util.h                    | 35 ++++++++++++++++++++------
 be/src/util/logging-support.cc            |  4 +--
 be/src/util/zip-util.cc                   |  2 +-
 7 files changed, 40 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/catalog/catalog.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog.cc b/be/src/catalog/catalog.cc
index a2d5f4b..e5bf35c 100644
--- a/be/src/catalog/catalog.cc
+++ b/be/src/catalog/catalog.cc
@@ -76,7 +76,7 @@ Catalog::Catalog() {
   JNIEnv* jni_env = getJNIEnv();
   // Create an instance of the java class JniCatalog
   catalog_class_ = jni_env->FindClass("org/apache/impala/service/JniCatalog");
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
 
   uint32_t num_methods = sizeof(methods) / sizeof(methods[0]);
   for (int i = 0; i < num_methods; ++i) {
@@ -87,7 +87,7 @@ Catalog::Catalog() {
   ABORT_IF_ERROR(GetThriftBackendGflags(jni_env, &cfg_bytes));
 
   jobject catalog = jni_env->NewObject(catalog_class_, catalog_ctor_, cfg_bytes);
-  EXIT_IF_EXC(jni_env);
+  CLEAN_EXIT_IF_EXC(jni_env);
   ABORT_IF_ERROR(JniUtil::LocalToGlobalRef(jni_env, catalog, &catalog_));
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/scheduling/request-pool-service.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/request-pool-service.cc b/be/src/scheduling/request-pool-service.cc
index f9fdf88..c3c9750 100644
--- a/be/src/scheduling/request-pool-service.cc
+++ b/be/src/scheduling/request-pool-service.cc
@@ -124,7 +124,7 @@ RequestPoolService::RequestPoolService(MetricGroup* metrics) :
   JNIEnv* jni_env = getJNIEnv();
   request_pool_service_class_ =
     jni_env->FindClass("org/apache/impala/util/RequestPoolService");
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
   uint32_t num_methods = sizeof(methods) / sizeof(methods[0]);
   for (int i = 0; i < num_methods; ++i) {
     ABORT_IF_ERROR(JniUtil::LoadJniMethod(jni_env, request_pool_service_class_,
@@ -133,18 +133,18 @@ RequestPoolService::RequestPoolService(MetricGroup* metrics) :
 
   jstring fair_scheduler_config_path =
       jni_env->NewStringUTF(FLAGS_fair_scheduler_allocation_path.c_str());
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
   jstring llama_site_path =
       jni_env->NewStringUTF(FLAGS_llama_site_path.c_str());
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
 
   jobject request_pool_service = jni_env->NewObject(request_pool_service_class_, ctor_,
       fair_scheduler_config_path, llama_site_path);
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
   ABORT_IF_ERROR(JniUtil::LocalToGlobalRef(jni_env, request_pool_service,
       &request_pool_service_));
   jni_env->CallObjectMethod(request_pool_service_, start_id);
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
 }
 
 Status RequestPoolService::ResolveRequestPool(const TQueryCtx& ctx,

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index 5dd06de..4a25250 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -699,7 +699,7 @@ void InitFeSupport(bool disable_codegen) {
   jclass native_backend_cl = env->FindClass("org/apache/impala/service/FeSupport");
   env->RegisterNatives(native_backend_cl, native_methods,
       sizeof(native_methods) / sizeof(native_methods[0]));
-  EXIT_IF_EXC(env);
+  ABORT_IF_EXC(env);
 }
 
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/service/frontend.cc
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index af45d96..c4ad4c7 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -106,7 +106,7 @@ Frontend::Frontend() {
   JNIEnv* jni_env = getJNIEnv();
   // create instance of java class JniFrontend
   fe_class_ = jni_env->FindClass("org/apache/impala/service/JniFrontend");
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
 
   uint32_t num_methods = sizeof(methods) / sizeof(methods[0]);
   for (int i = 0; i < num_methods; ++i) {
@@ -117,7 +117,7 @@ Frontend::Frontend() {
   ABORT_IF_ERROR(GetThriftBackendGflags(jni_env, &cfg_bytes));
 
   jobject fe = jni_env->NewObject(fe_class_, fe_ctor_, cfg_bytes);
-  EXIT_IF_EXC(jni_env);
+  ABORT_IF_EXC(jni_env);
   ABORT_IF_ERROR(JniUtil::LocalToGlobalRef(jni_env, fe, &fe_));
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/util/jni-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h
index 576f943..4b18f20 100644
--- a/be/src/util/jni-util.h
+++ b/be/src/util/jni-util.h
@@ -59,8 +59,8 @@
 #define THROW_IF_EXC(env, exc_class) \
   do { \
     jthrowable exc = (env)->ExceptionOccurred(); \
-    if (exc != NULL) { \
-      DCHECK((throwable_to_string_id_) != NULL); \
+    if (exc != nullptr) { \
+      DCHECK((throwable_to_string_id_) != nullptr); \
       jstring stack = (jstring) env->CallStaticObjectMethod(JniUtil::jni_util_class(), \
           (JniUtil::throwable_to_stack_trace_id()), exc); \
       jboolean is_copy; \
@@ -75,7 +75,7 @@
 #define RETURN_IF_EXC(env) \
   do { \
     jthrowable exc = (env)->ExceptionOccurred(); \
-    if (exc != NULL) { \
+    if (exc != nullptr) { \
       jstring stack = (jstring) env->CallStaticObjectMethod(JniUtil::jni_util_class(), \
           (JniUtil::throwable_to_stack_trace_id()), exc); \
       jboolean is_copy; \
@@ -86,10 +86,14 @@
     } \
   } while (false)
 
-#define EXIT_IF_EXC(env) \
+// If there's an exception in 'env', log the backtrace at FATAL level and abort the
+// process. This will generate a core dump if core dumps are enabled, so this should
+// generally only be called for internal errors where the coredump is useful for
+// diagnostics.
+#define ABORT_IF_EXC(env) \
   do { \
     jthrowable exc = (env)->ExceptionOccurred(); \
-    if (exc != NULL) { \
+    if (exc != nullptr) { \
       jstring stack = (jstring) env->CallStaticObjectMethod(JniUtil::jni_util_class(), \
           (JniUtil::throwable_to_stack_trace_id()), exc); \
       jboolean is_copy; \
@@ -99,10 +103,25 @@
     } \
   } while (false)
 
+// If there's an exception in 'env', log the backtrace at ERROR level and exit the process
+// cleanly with status 1.
+#define CLEAN_EXIT_IF_EXC(env) \
+  do { \
+    jthrowable exc = (env)->ExceptionOccurred(); \
+    if (exc != nullptr) { \
+      jstring stack = (jstring) env->CallStaticObjectMethod(JniUtil::jni_util_class(), \
+          (JniUtil::throwable_to_stack_trace_id()), exc); \
+      jboolean is_copy; \
+      const char* c_stack = \
+          reinterpret_cast<const char*>((env)->GetStringUTFChars(stack, &is_copy)); \
+      CLEAN_EXIT_WITH_ERROR(c_stack); \
+    } \
+  } while (false)
+
 #define RETURN_ERROR_IF_EXC(env) \
   do { \
     jthrowable exc = (env)->ExceptionOccurred(); \
-    if (exc != NULL) return JniUtil::GetJniExceptionMsg(env);\
+    if (exc != nullptr) return JniUtil::GetJniExceptionMsg(env);\
   } while (false)
 
 /// C linkage for helper functions in hdfsJniHelper.h
@@ -118,8 +137,8 @@ class Status;
 /// the corresponding objects.
 class JniLocalFrame {
  public:
-  JniLocalFrame(): env_(NULL) {}
-  ~JniLocalFrame() { if (env_ != NULL) env_->PopLocalFrame(NULL); }
+  JniLocalFrame(): env_(nullptr) {}
+  ~JniLocalFrame() { if (env_ != nullptr) env_->PopLocalFrame(nullptr); }
 
   JniLocalFrame(JniLocalFrame&& other) noexcept
     : env_(other.env_) {

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/util/logging-support.cc
----------------------------------------------------------------------
diff --git a/be/src/util/logging-support.cc b/be/src/util/logging-support.cc
index 457c8f9..b050f37 100644
--- a/be/src/util/logging-support.cc
+++ b/be/src/util/logging-support.cc
@@ -110,7 +110,7 @@ Webserver::UrlCallback MakeCallback(const F& fnc, bool display_log4j_handlers) {
 void InitDynamicLoggingSupport() {
   JNIEnv* env = getJNIEnv();
   log4j_logger_class_ = env->FindClass("org/apache/impala/util/GlogAppender");
-  EXIT_IF_EXC(env);
+  ABORT_IF_EXC(env);
   JniMethodDescriptor get_log_level_method_desc =
       {"getLogLevel", "([B)Ljava/lang/String;", &get_log_level_method};
   JniMethodDescriptor set_log_level_method_desc =
@@ -240,7 +240,7 @@ void InitJvmLoggingSupport() {
   nm.signature = const_cast<char*>("(ILjava/lang/String;Ljava/lang/String;I)V");
   nm.fnPtr = reinterpret_cast<void*>(::Java_org_apache_impala_util_NativeLogger_Log);
   env->RegisterNatives(native_backend_cl, &nm, 1);
-  EXIT_IF_EXC(env);
+  ABORT_IF_EXC(env);
   InitDynamicLoggingSupport();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0d4c6ae0/be/src/util/zip-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/zip-util.cc b/be/src/util/zip-util.cc
index f6e1f15..43c46df 100644
--- a/be/src/util/zip-util.cc
+++ b/be/src/util/zip-util.cc
@@ -30,7 +30,7 @@ jmethodID ZipUtil::extract_files_method; // ZipUtil.extractFiles()
 void ZipUtil::InitJvm() {
   JNIEnv* env = getJNIEnv();
   zip_util_class_ = env->FindClass("org/apache/impala/util/ZipUtil");
-  EXIT_IF_EXC(env);
+  ABORT_IF_EXC(env);
   JniMethodDescriptor extract_files_method_desc =
       {"extractFiles", "([B)V", &extract_files_method};
 


[02/33] impala git commit: IMPALA-6249: Expose several build flags via web UI

Posted by bo...@apache.org.
IMPALA-6249: Expose several build flags via web UI

Exposes a list of build flags via the impalad web UI. The build flags
can be viewed on the root page under the "Version" section. They can
be accessed via other tests through the debug version of the root page
(e.g. adding &json to the URL). The build flags are listed in a JSON
array so that they can be parsed easily. This should help run Impala
tests against a remote Impala cluster.

The build flags are read in CMakeLists.txt and then stored in
preprocessor variables.

Three build flags are exposed as part of this commit:
- Is_NDEBUG = [true, false]
    - Whether NDEBUG was true or false at compile time
- CMake_Build_Type = [DEBUG, RELEASE, ADDRESS_SANITIZER, TIDY, UBSAN,
  UBSAN_FULL, TSAN, CODE_COVERAGE_RELEASE, CODE_COVERAGE_DEBUG]
    - The value of CMAKE_BUILD_TYPE at compile time
- Library_Link_Type = [DYNAMIC, STATIC]
    - Derived from the compile time value of BUILD_SHARED_LIBS

There are a few other minor changes that are apart of this commit:

* The patch modifies environ.py so that it supports fetching build metadata
for both local and remote clusters.

* The tests under the tests/webserver directory were not being run because
'webserver' was not whitelisted in tests/run-tests.py. This patch fixes
that and addresses several test failures in run-tests.py.

* It reverts part of IMPALA-6947 so that their is no dependency from
start-impala-cluster.py to environ.py. The timeout discussed IMPALA-6947
is now set at compile time.

Testing:

Added new tests to webserver/test_web_pages.py to ensure that the build
flags are being set. Some tests are only run when run against a local
cluster because we have no way of getting the build info from a remote
cluster, whereas local clusters contain a .cmake_build_type file.

Change-Id: I47e3ad4cbf844909bdaf22a6f9d7bd915dce3f19
Reviewed-on: http://gerrit.cloudera.org:8080/11410
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: 9bdb73b188c4c87ee3a71e56aa10de0eba32b086
Parents: c73530d
Author: Sahil Takiar <ta...@gmail.com>
Authored: Wed Sep 5 10:52:49 2018 -0400
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 CMakeLists.txt                                  |  20 +-
 be/src/common/config.h.in                       |   3 +
 be/src/common/global-flags.cc                   |  16 +-
 be/src/util/debug-util.h                        |  19 ++
 be/src/util/default-path-handlers.cc            |  27 +++
 bin/start-impala-cluster.py                     |   4 +-
 tests/common/environ.py                         | 207 +++++++++++++++----
 tests/common/skip.py                            |   8 +-
 tests/conftest.py                               |   4 +-
 .../custom_cluster/test_admission_controller.py |   6 +-
 .../test_automatic_invalidation.py              |   5 +-
 tests/custom_cluster/test_exchange_delays.py    |   4 +-
 tests/custom_cluster/test_metadata_replicas.py  |   1 -
 tests/custom_cluster/test_restart_services.py   |   4 +-
 tests/query_test/test_hdfs_caching.py           |   4 +-
 tests/query_test/test_runtime_filters.py        |   4 +-
 tests/run-tests.py                              |  19 +-
 tests/statestore/test_statestore.py             |   8 +-
 tests/webserver/test_web_pages.py               |  96 +++++++--
 www/root.tmpl                                   |   5 +-
 20 files changed, 368 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 90b4df8..e248540 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -30,6 +30,9 @@ if (NOT DEFINED BUILD_SHARED_LIBS)
   set(BUILD_SHARED_LIBS OFF)
 endif()
 
+# Store BUILD_SHARED_LIBS in a variable so it can be read in config.h.in
+set(IMPALA_BUILD_SHARED_LIBS ${BUILD_SHARED_LIBS})
+
 # Build compile commands database
 set(CMAKE_EXPORT_COMPILE_COMMANDS ON)
 
@@ -47,8 +50,14 @@ endif(NOT CMAKE_BUILD_TYPE)
 STRING (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE)
 
 message(STATUS "Build type is ${CMAKE_BUILD_TYPE}")
-# Write the build type to a file so that tests can determine the current build type.
-file(WRITE "${CMAKE_SOURCE_DIR}/.cmake_build_type" ${CMAKE_BUILD_TYPE})
+
+# Write build flags to a file so that they can be read by tests
+file(WRITE "${CMAKE_SOURCE_DIR}/.cmake_build_type" ${CMAKE_BUILD_TYPE}\n)
+file(APPEND "${CMAKE_SOURCE_DIR}/.cmake_build_type" ${BUILD_SHARED_LIBS}\n)
+
+# Store CMAKE_BUILD_TYPE in a variable so it can be read in config.h.in
+string(REPLACE "_" "-" ESCAPED_CMAKE_BUILD_TYPE ${CMAKE_BUILD_TYPE})
+set(IMPALA_CMAKE_BUILD_TYPE ${ESCAPED_CMAKE_BUILD_TYPE})
 
 set(ENABLE_CODE_COVERAGE false)
 if ("${CMAKE_BUILD_TYPE}" STREQUAL "CODE_COVERAGE_DEBUG")
@@ -61,6 +70,13 @@ endif()
 
 message(STATUS "ENABLE_CODE_COVERAGE: ${ENABLE_CODE_COVERAGE}")
 
+if (ENABLE_CODE_COVERAGE
+    OR "${CMAKE_BUILD_TYPE}" STREQUAL "ADDRESS_SANITIZER"
+    OR "${CMAKE_BUILD_TYPE}" STREQUAL "TSAN"
+    OR "${CMAKE_BUILD_TYPE}" STREQUAL "UBSAN")
+  set (SLOW_BUILD true)
+endif()
+
 # Helper function that given a package name constructs the package_ROOT variable based on
 # the version number extracted from the environment
 function(set_dep_root NAME)

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/be/src/common/config.h.in
----------------------------------------------------------------------
diff --git a/be/src/common/config.h.in b/be/src/common/config.h.in
index f474a9f..43c7e65 100644
--- a/be/src/common/config.h.in
+++ b/be/src/common/config.h.in
@@ -28,5 +28,8 @@
 #cmakedefine HAVE_PIPE2
 #cmakedefine HAVE_MAGIC_H
 #cmakedefine HAVE_SYNC_FILE_RANGE
+#cmakedefine SLOW_BUILD
+#cmakedefine IMPALA_BUILD_SHARED_LIBS @IMPALA_BUILD_SHARED_LIBS@
+#cmakedefine IMPALA_CMAKE_BUILD_TYPE @IMPALA_CMAKE_BUILD_TYPE@
 
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/be/src/common/global-flags.cc
----------------------------------------------------------------------
diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index 0473352..3c73b56 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -161,12 +161,18 @@ DEFINE_int32(kudu_operation_timeout_ms, 3 * 60 * 1000, "Timeout (milliseconds) s
     "all Kudu operations. This must be a positive value, and there is no way to disable "
     "timeouts.");
 
+#ifdef SLOW_BUILD
+static const int32 default_kudu_client_rpc_timeout_ms = 60000;
+#else
+static const int32 default_kudu_client_rpc_timeout_ms = 0;
+#endif
+
 // Timeout (ms) for Kudu rpcs set in the BE on the KuduClient.
-DEFINE_int32(kudu_client_rpc_timeout_ms, 0, "(Advanced) Timeout (milliseconds) set for "
-    "individual Kudu client rpcs. An operation may consist of several rpcs, so this is "
-    "expected to be less than kudu_operation_timeout_ms. This must be a positive value "
-    "or it will be ignored and Kudu's default of 10s will be used. There is no way to "
-    "disable timeouts.");
+DEFINE_int32(kudu_client_rpc_timeout_ms, default_kudu_client_rpc_timeout_ms,
+    "(Advanced) Timeout (milliseconds) set for individual Kudu client rpcs. An operation "
+    "may consist of several rpcs, so this is expected to be less than "
+    "kudu_operation_timeout_ms. This must be a positive value or it will be ignored and "
+    "Kudu's default of 10s will be used. There is no way to disable timeouts.");
 
 DEFINE_int64(inc_stats_size_limit_bytes, 200 * (1LL<<20), "Maximum size of "
     "incremental stats the catalog is allowed to serialize per table. "

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/be/src/util/debug-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index 005cc23..240eab6 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -24,6 +24,7 @@
 
 #include <thrift/protocol/TDebugProtocol.h>
 
+#include "common/config.h"
 #include "gen-cpp/JniCatalog_types.h"
 #include "gen-cpp/Descriptors_types.h"
 #include "gen-cpp/Exprs_types.h"
@@ -97,6 +98,24 @@ bool ParseId(const std::string& s, TUniqueId* id);
 /// This is used to set gflags build version
 std::string GetBuildVersion(bool compact = false);
 
+#ifndef IMPALA_CMAKE_BUILD_TYPE
+static_assert(false, "IMPALA_CMAKE_BUILD_TYPE is not defined");
+#endif
+
+/// Returns the value of CMAKE_BUILD_TYPE used to build the code
+constexpr const char* GetCMakeBuildType() {
+  return AS_STRING(IMPALA_CMAKE_BUILD_TYPE);
+}
+
+/// Returns whether the code was dynamically or statically linked, return
+/// value is either STATIC or DYNAMIC.
+constexpr const char* GetLibraryLinkType() {
+  return AS_STRING(IMPALA_BUILD_SHARED_LIBS)[0] == 'O'
+          && AS_STRING(IMPALA_BUILD_SHARED_LIBS)[1] == 'N' ?
+      "DYNAMIC" :
+      "STATIC";
+}
+
 /// Returns "<program short name> version <GetBuildVersion(compact)>"
 std::string GetVersionString(bool compact = false);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/be/src/util/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index c492d06..df04d92 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -224,11 +224,38 @@ void JmxHandler(const Webserver::ArgumentMap& args, Document* document) {
   }
 }
 
+// Helper function that creates a Value for a given build flag name, value and adds it to
+// an array of build_flags
+void AddBuildFlag(const std::string& flag_name, const std::string& flag_value,
+    Document* document, Value* build_flags) {
+  Value build_type(kObjectType);
+  Value build_type_name(flag_name.c_str(), document->GetAllocator());
+  build_type.AddMember("flag_name", build_type_name, document->GetAllocator());
+  Value build_type_value(flag_value.c_str(), document->GetAllocator());
+  build_type.AddMember("flag_value", build_type_value, document->GetAllocator());
+  build_flags->PushBack(build_type, document->GetAllocator());
+}
+
 namespace impala {
 
 void RootHandler(const Webserver::ArgumentMap& args, Document* document) {
   Value version(GetVersionString().c_str(), document->GetAllocator());
   document->AddMember("version", version, document->GetAllocator());
+
+#ifdef NDEBUG
+  const char* is_ndebug = "true";
+#else
+  const char* is_ndebug = "false";
+#endif
+
+  Value build_flags(kArrayType);
+  AddBuildFlag("is_ndebug", is_ndebug, document, &build_flags);
+  string cmake_build_type(GetCMakeBuildType());
+  replace(cmake_build_type.begin(), cmake_build_type.end(), '-', '_');
+  AddBuildFlag("cmake_build_type", cmake_build_type, document, &build_flags);
+  AddBuildFlag("library_link_type", GetLibraryLinkType(), document, &build_flags);
+  document->AddMember("build_flags", build_flags, document->GetAllocator());
+
   Value cpu_info(CpuInfo::DebugString().c_str(), document->GetAllocator());
   document->AddMember("cpu_info", cpu_info, document->GetAllocator());
   Value mem_info(MemInfo::DebugString().c_str(), document->GetAllocator());

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/bin/start-impala-cluster.py
----------------------------------------------------------------------
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index 7e22ca9..d052650 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -29,7 +29,7 @@ from getpass import getuser
 from time import sleep, time
 from optparse import OptionParser, SUPPRESS_HELP
 from testdata.common import cgroups
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 
 logging.basicConfig(level=logging.ERROR, format="%(asctime)s %(threadName)s: %(message)s",
     datefmt="%H:%M:%S")
@@ -120,7 +120,7 @@ CLUSTER_WAIT_TIMEOUT_IN_SECONDS = 240
 # It is set to a high value to avoid failing if processes are slow to shut down.
 KILL_TIMEOUT_IN_SECONDS = 240
 # For build types like ASAN, modify the default Kudu rpc timeout.
-KUDU_RPC_TIMEOUT = specific_build_type_timeout(0, slow_build_timeout=60000)
+KUDU_RPC_TIMEOUT = build_flavor_timeout(0, slow_build_timeout=60000)
 
 def find_user_processes(binaries):
   """Returns an iterator over all processes owned by the current user with a matching

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/common/environ.py
----------------------------------------------------------------------
diff --git a/tests/common/environ.py b/tests/common/environ.py
index 092c2e0..e20aa98 100644
--- a/tests/common/environ.py
+++ b/tests/common/environ.py
@@ -15,20 +15,23 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import json
 import logging
 import os
 import re
+import requests
 
 LOG = logging.getLogger('tests.common.environ')
 test_start_cluster_args = os.environ.get("TEST_START_CLUSTER_ARGS", "")
 IMPALA_HOME = os.environ.get("IMPALA_HOME", "")
+IMPALA_REMOTE_URL = os.environ.get("IMPALA_REMOTE_URL", "")
 
 # Find the likely BuildType of the running Impala. Assume it's found through the path
 # $IMPALA_HOME/be/build/latest as a fallback.
 build_type_arg_regex = re.compile(r'--build_type=(\w+)', re.I)
 build_type_arg_search_result = re.search(build_type_arg_regex, test_start_cluster_args)
 if build_type_arg_search_result is not None:
-  build_type_dir = build_type_search_result.groups()[0].lower()
+  build_type_dir = build_type_arg_search_result.groups()[0].lower()
 else:
   build_type_dir = 'latest'
 
@@ -36,9 +39,10 @@ else:
 impalad_basedir = \
     os.path.realpath(os.path.join(IMPALA_HOME, 'be/build', build_type_dir)).rstrip('/')
 
-class SpecificImpaladBuildTypes:
+
+class ImpalaBuildFlavors:
   """
-  Represents the possible CMAKE_BUILD_TYPE values. These specific build types are needed
+  Represents the possible CMAKE_BUILD_TYPE values. These build flavors are needed
   by Python test code, e.g. to set different timeouts for different builds. All values
   are lower-cased to enable case-insensitive comparison.
   """
@@ -52,6 +56,8 @@ class SpecificImpaladBuildTypes:
   CODE_COVERAGE_DEBUG = 'code_coverage_debug'
   # ./buildall.sh -release -codecoverage
   CODE_COVERAGE_RELEASE = 'code_coverage_release'
+  # ./buildall.sh -tidy
+  TIDY = 'tidy'
   # ./buildall.sh -tsan
   TSAN = 'tsan'
   # ./buildall.sh -ubsan
@@ -60,82 +66,168 @@ class SpecificImpaladBuildTypes:
   UBSAN_FULL = 'ubsan_full'
 
   VALID_BUILD_TYPES = [ADDRESS_SANITIZER, DEBUG, CODE_COVERAGE_DEBUG, RELEASE,
-      CODE_COVERAGE_RELEASE, TSAN, UBSAN, UBSAN_FULL]
+      CODE_COVERAGE_RELEASE, TIDY, TSAN, UBSAN, UBSAN_FULL]
+
+
+class LinkTypes:
+  """
+  Represents the possible library link type values, either "dynamic" or "static". This
+  value is derived from the cmake value of BUILD_SHARED_LIBS. All values are lower-cased
+  to enable case-insensitive comparison.
+  """
+  # ./buildall.sh
+  STATIC = 'static'
+  # ./buildall.sh -build_shared_libs
+  DYNAMIC = 'dynamic'
+
+  VALID_LINK_TYPES = [STATIC, DYNAMIC]
+
+
+class ImpalaTestClusterFlagsDetector:
+  """
+  Detects the build flags of different types of Impala clusters. Currently supports
+  detecting build flags from either a locally built Impala cluster using a file generated
+  by CMake, or from the Impala web ui, which is useful for detecting flags from a remote
+  Impala cluster. The supported list of build flags is: [CMAKE_BUILD_TYPE,
+  BUILD_SHARED_LIBS]
+  """
 
   @classmethod
-  def detect(cls, impala_build_root):
+  def detect_using_build_root_or_web_ui(cls, impala_build_root):
     """
-    Determine the build type based on the .cmake_build_type file created by
+    Determine the build flags based on the .cmake_build_type file created by
     ${IMPALA_HOME}/CMakeLists.txt. impala_build_root should be the path of the
-    Impala source checkout, i.e. ${IMPALA_HOME}.
+    Impala source checkout, i.e. ${IMPALA_HOME}. If .cmake_build_type is not present,
+    or cannot be read, attempt to detect the build flags from the local web UI using
+    detect_using_web_ui.
     """
-    build_type_path = os.path.join(impala_build_root, ".cmake_build_type")
+    cmake_build_type_path = os.path.join(impala_build_root, ".cmake_build_type")
     try:
-      with open(build_type_path) as build_type_file:
-        build_type = build_type_file.read().strip().lower()
+      with open(cmake_build_type_path) as cmake_build_type_file:
+        build_flags = cmake_build_type_file.readlines()
+        build_type = build_flags[0].strip().lower()
+        build_shared_libs = build_flags[1].strip().lower()
     except IOError:
-      LOG.error("Could not open %s assuming DEBUG", build_type_path)
-      return cls.DEBUG
+      LOG.debug("Unable to read .cmake_build_type file, fetching build flags from " +
+              "web ui on localhost")
+      build_type, build_shared_libs = ImpalaTestClusterFlagsDetector.detect_using_web_ui(
+          "http://localhost:25000")
 
-    if build_type not in cls.VALID_BUILD_TYPES:
+    library_link_type = LinkTypes.STATIC if build_shared_libs == "off"\
+                   else LinkTypes.DYNAMIC
+    ImpalaTestClusterFlagsDetector.validate_build_flags(build_type, library_link_type)
+    return build_type, library_link_type
+
+  @classmethod
+  def detect_using_web_ui(cls, impala_url):
+    """
+    Determine the build type based on the Impala cluster's web UI by using
+    get_build_flags_from_web_ui.
+    """
+    build_flags = ImpalaTestClusterFlagsDetector.get_build_flags_from_web_ui(impala_url)
+    build_type = build_flags['cmake_build_type']
+    library_link_type = build_flags['library_link_type']
+    ImpalaTestClusterFlagsDetector.validate_build_flags(build_type, library_link_type)
+    return build_type, library_link_type
+
+  @classmethod
+  def validate_build_flags(cls, build_type, library_link_type):
+    """
+    Validates that the build flags have valid values.
+    """
+    if build_type not in ImpalaBuildFlavors.VALID_BUILD_TYPES:
       raise Exception("Unknown build type {0}".format(build_type))
+    if library_link_type not in LinkTypes.VALID_LINK_TYPES:
+      raise Exception("Unknown library link type {0}".format(library_link_type))
     LOG.debug("Build type detected: %s", build_type)
-    return build_type
+    LOG.debug("Library link type detected: %s", library_link_type)
+
+  @classmethod
+  def get_build_flags_from_web_ui(cls, impala_url):
+    """
+    Fetches the build flags from the given Impala cluster web UI by parsing the ?json
+    response of the root homepage and looking for the section on build flags. It returns
+    the flags as a dictionary where the key is the flag name.
+    """
+    response = requests.get(impala_url + "/?json")
+    assert response.status_code == requests.codes.ok,\
+            "Offending url: " + impala_url
+    assert "application/json" in response.headers['Content-Type']
+
+    build_flags_json = json.loads(response.text)["build_flags"]
+    build_flags = dict((flag['flag_name'].lower(), flag['flag_value'].lower())
+        for flag in build_flags_json)
+    assert len(build_flags_json) == len(build_flags)  # Ensure there are no collisions
+    return build_flags
 
 
+"""
+Indicates whether we are operating against a locally built Impala cluster or a remote one.
+"""
+(
+  LOCAL_BUILD,
+  REMOTE_BUILD,
+) = xrange(2)
 
-class ImpaladBuild(object):
+
+class ImpalaTestClusterProperties(object):
   """
   Acquires and provides characteristics about the way the Impala under test was compiled
-  and its likely effects on its responsiveness to automated test timings. Currently
-  assumes that the Impala daemon under test was built in our current source checkout.
-  TODO: we could get this information for remote cluster tests if we exposed the build
-  type via a metric or the Impalad web UI.
+  and its likely effects on its responsiveness to automated test timings.
   """
-  def __init__(self, impala_build_root):
-    self._specific_build_type = SpecificImpaladBuildTypes.detect(impala_build_root)
+  def __init__(self, build_flavor, library_link_type, local_or_remote_build):
+    self._build_flavor = build_flavor
+    self._library_link_type = library_link_type
+    self._local_or_remote_build = local_or_remote_build
+
+  @property
+  def build_flavor(self):
+    """
+    Return the correct ImpalaBuildFlavors for the Impala under test.
+    """
+    return self._build_flavor
 
   @property
-  def specific_build_type(self):
+  def library_link_type(self):
     """
-    Return the correct SpecificImpaladBuildTypes for the Impala under test.
+    Return the library link type (either static or dynamic) for the Impala under test.
     """
-    return self._specific_build_type
+    return self._library_link_type
 
   def has_code_coverage(self):
     """
     Return whether the Impala under test was compiled with code coverage enabled.
     """
-    return self.specific_build_type in (SpecificImpaladBuildTypes.CODE_COVERAGE_DEBUG,
-                                        SpecificImpaladBuildTypes.CODE_COVERAGE_RELEASE)
+    return self.build_flavor in (ImpalaBuildFlavors.CODE_COVERAGE_DEBUG,
+                                 ImpalaBuildFlavors.CODE_COVERAGE_RELEASE)
 
   def is_asan(self):
     """
     Return whether the Impala under test was compiled with ASAN.
     """
-    return self.specific_build_type == SpecificImpaladBuildTypes.ADDRESS_SANITIZER
+    return self.build_flavor == ImpalaBuildFlavors.ADDRESS_SANITIZER
 
   def is_tsan(self):
     """
     Return whether the Impala under test was compiled with TSAN.
     """
-    return self.specific_build_type == SpecificImpaladBuildTypes.TSAN
+    return self.build_flavor == ImpalaBuildFlavors.TSAN
 
   def is_ubsan(self):
     """
     Return whether the Impala under test was compiled with UBSAN.
     """
-    return self.specific_build_type == SpecificImpaladBuildTypes.UBSAN
+    return self.build_flavor == ImpalaBuildFlavors.UBSAN
 
   def is_dev(self):
     """
     Return whether the Impala under test is a development build (i.e., any debug or ASAN
     build).
     """
-    return self.specific_build_type in (
-        SpecificImpaladBuildTypes.ADDRESS_SANITIZER, SpecificImpaladBuildTypes.DEBUG,
-        SpecificImpaladBuildTypes.CODE_COVERAGE_DEBUG,
-        SpecificImpaladBuildTypes.TSAN, SpecificImpaladBuildTypes.UBSAN)
+    return self.build_flavor in (
+        ImpalaBuildFlavors.ADDRESS_SANITIZER, ImpalaBuildFlavors.DEBUG,
+        ImpalaBuildFlavors.CODE_COVERAGE_DEBUG, ImpalaBuildFlavors.TSAN,
+        ImpalaBuildFlavors.UBSAN)
 
   def runs_slowly(self):
     """
@@ -144,15 +236,42 @@ class ImpaladBuild(object):
     """
     return self.has_code_coverage() or self.is_asan() or self.is_tsan() or self.is_ubsan()
 
+  def is_statically_linked(self):
+    """
+    Return whether the Impala under test was statically linked during compilation.
+    """
+    return self.build_shared_libs == LinkTypes.STATIC
+
+  def is_dynamically_linked(self):
+    """
+    Return whether the Impala under test was dynamically linked during compilation.
+    """
+    return self.build_shared_libs == LinkTypes.DYNAMIC
+
+  def is_remote_cluster(self):
+    """
+    Return true if the Impala test cluster is running remotely, false otherwise
+    """
+    return self._local_or_remote_build == REMOTE_BUILD
+
 
-IMPALAD_BUILD = ImpaladBuild(IMPALA_HOME)
+if IMPALA_REMOTE_URL:
+  build_flavor, link_type =\
+      ImpalaTestClusterFlagsDetector.detect_using_web_ui(IMPALA_REMOTE_URL)
+  IMPALA_TEST_CLUSTER_PROPERTIES =\
+      ImpalaTestClusterProperties(build_flavor, link_type, REMOTE_BUILD)
+else:
+  build_flavor, link_type =\
+      ImpalaTestClusterFlagsDetector.detect_using_build_root_or_web_ui(IMPALA_HOME)
+  IMPALA_TEST_CLUSTER_PROPERTIES =\
+      ImpalaTestClusterProperties(build_flavor, link_type, LOCAL_BUILD)
 
-def specific_build_type_timeout(
-    default_timeout, slow_build_timeout=None, asan_build_timeout=None,
-    code_coverage_build_timeout=None):
+
+def build_flavor_timeout(default_timeout, slow_build_timeout=None,
+        asan_build_timeout=None, code_coverage_build_timeout=None):
   """
-  Return a test environment-specific timeout based on the sort of
-  SpecificImpalaBuildType under test.
+  Return a test environment-specific timeout based on the sort of ImpalaBuildFlavor under
+  test.
 
   Required parameter: default_timeout - default timeout value. This applies when Impala is
   a standard release or debug build, or if no other timeouts are specified.
@@ -163,7 +282,7 @@ def specific_build_type_timeout(
   "slow". You can use this as a shorthand in lieu of specifying all of the following
   parameters.
 
-  The parameters below correspond to specific build types. These preempt both
+  The parameters below correspond to build flavors. These preempt both
   slow_build_timeout and default_timeout, if the Impala under test is a build of the
   applicable type:
 
@@ -173,13 +292,13 @@ def specific_build_type_timeout(
   (both debug and release code coverage)
   """
 
-  if IMPALAD_BUILD.is_asan() and asan_build_timeout is not None:
+  if IMPALA_TEST_CLUSTER_PROPERTIES.is_asan() and asan_build_timeout is not None:
     timeout_val = asan_build_timeout
-  elif IMPALAD_BUILD.has_code_coverage() and code_coverage_build_timeout is not None:
+  elif IMPALA_TEST_CLUSTER_PROPERTIES.has_code_coverage() and\
+          code_coverage_build_timeout is not None:
     timeout_val = code_coverage_build_timeout
-  elif IMPALAD_BUILD.runs_slowly() and slow_build_timeout is not None:
+  elif IMPALA_TEST_CLUSTER_PROPERTIES.runs_slowly() and slow_build_timeout is not None:
     timeout_val = slow_build_timeout
   else:
     timeout_val = default_timeout
   return timeout_val
-

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/common/skip.py
----------------------------------------------------------------------
diff --git a/tests/common/skip.py b/tests/common/skip.py
index 2f19bf4..41e119a 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -24,7 +24,7 @@ import os
 import pytest
 from functools import partial
 
-from tests.common.environ import IMPALAD_BUILD
+from tests.common.environ import IMPALA_TEST_CLUSTER_PROPERTIES
 from tests.util.filesystem_utils import (
     IS_ABFS,
     IS_ADLS,
@@ -161,8 +161,10 @@ class SkipIfNotHdfsMinicluster:
       reason="Test is tuned for 3-node HDFS minicluster with no EC")
 
 class SkipIfBuildType:
-  not_dev_build = pytest.mark.skipif(not IMPALAD_BUILD.is_dev(),
-      reason="Tests depends on debug build startup option.")
+  not_dev_build = pytest.mark.skipif(not IMPALA_TEST_CLUSTER_PROPERTIES.is_dev(),
+      reason="Test depends on debug build startup option.")
+  remote = pytest.mark.skipif(IMPALA_TEST_CLUSTER_PROPERTIES.is_remote_cluster(),
+      reason="Test depends on running against a local Impala cluster")
 
 class SkipIfEC:
   remote_read = pytest.mark.skipif(IS_EC, reason="EC files are read remotely and "

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/conftest.py
----------------------------------------------------------------------
diff --git a/tests/conftest.py b/tests/conftest.py
index 4e1c837..0fccc5d 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -27,7 +27,7 @@ import logging
 import os
 import pytest
 
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 from common.test_result_verifier import QueryTestResult
 from tests.common.patterns import is_valid_impala_identifier
 from tests.comparison.db_connection import ImpalaConnection
@@ -52,7 +52,7 @@ if FILESYSTEM == 'isilon':
 
 # Timeout each individual test case after 2 hours, or 4 hours for slow builds
 PYTEST_TIMEOUT_S = \
-    specific_build_type_timeout(2 * 60 * 60, slow_build_timeout=4 * 60 * 60)
+    build_flavor_timeout(2 * 60 * 60, slow_build_timeout=4 * 60 * 60)
 
 def pytest_configure(config):
   """ Hook startup of pytest. Sets up log format and per-test timeout. """

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/custom_cluster/test_admission_controller.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_admission_controller.py b/tests/custom_cluster/test_admission_controller.py
index 6b3100f..f716788 100644
--- a/tests/custom_cluster/test_admission_controller.py
+++ b/tests/custom_cluster/test_admission_controller.py
@@ -31,7 +31,7 @@ from time import sleep, time
 from beeswaxd.BeeswaxService import QueryState
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.environ import specific_build_type_timeout, IMPALAD_BUILD
+from tests.common.environ import build_flavor_timeout, IMPALA_TEST_CLUSTER_PROPERTIES
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.resource_pool_config import ResourcePoolConfig
 from tests.common.skip import (
@@ -119,7 +119,7 @@ POOL_NAME = "default-pool"
 
 # Stress test timeout (seconds). The timeout needs to be significantly higher for
 # slow builds like code coverage and ASAN (IMPALA-3790, IMPALA-6241).
-STRESS_TIMEOUT = specific_build_type_timeout(60, slow_build_timeout=600)
+STRESS_TIMEOUT = build_flavor_timeout(60, slow_build_timeout=600)
 
 # The number of queries that can execute concurrently in the pool POOL_NAME.
 MAX_NUM_CONCURRENT_QUERIES = 5
@@ -902,7 +902,7 @@ class TestAdmissionControllerStress(TestAdmissionControllerBase):
 
     # Additional constraints for code coverage jobs and core.
     num_queries = None
-    if IMPALAD_BUILD.has_code_coverage():
+    if IMPALA_TEST_CLUSTER_PROPERTIES.has_code_coverage():
       # Code coverage builds can't handle the increased concurrency.
       num_queries = 15
     elif cls.exploration_strategy() == 'core':

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/custom_cluster/test_automatic_invalidation.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_automatic_invalidation.py b/tests/custom_cluster/test_automatic_invalidation.py
index 019712b..e82217b 100644
--- a/tests/custom_cluster/test_automatic_invalidation.py
+++ b/tests/custom_cluster/test_automatic_invalidation.py
@@ -19,7 +19,7 @@ import os
 import pytest
 import time
 from subprocess import call
-from tests.common.environ import IMPALAD_BUILD
+from tests.common.environ import IMPALA_TEST_CLUSTER_PROPERTIES
 from tests.util.filesystem_utils import IS_HDFS, IS_LOCAL
 
 
@@ -37,7 +37,8 @@ class TestAutomaticCatalogInvalidation(CustomClusterTestSuite):
 
   # The test will run a query and assumes the table is loaded when the query finishes.
   # The timeout should be larger than the time of the query.
-  timeout = 20 if IMPALAD_BUILD.runs_slowly() or (not IS_HDFS and not IS_LOCAL) else 10
+  timeout = 20 if IMPALA_TEST_CLUSTER_PROPERTIES.runs_slowly() or\
+               (not IS_HDFS and not IS_LOCAL) else 10
   timeout_flag = "--invalidate_tables_timeout_s=" + str(timeout)
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/custom_cluster/test_exchange_delays.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_exchange_delays.py b/tests/custom_cluster/test_exchange_delays.py
index 1d7d14b..38dfde1 100644
--- a/tests/custom_cluster/test_exchange_delays.py
+++ b/tests/custom_cluster/test_exchange_delays.py
@@ -17,13 +17,13 @@
 
 import pytest
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 from tests.common.skip import SkipIfBuildType
 from tests.util.filesystem_utils import IS_S3, IS_ADLS, IS_ISILON
 
 # IMPALA-6100: add additional margin for error for slow build types.
 SLOW_BUILD_TIMEOUT=20000
-DELAY_MS = specific_build_type_timeout(10000, slow_build_timeout=SLOW_BUILD_TIMEOUT)
+DELAY_MS = build_flavor_timeout(10000, slow_build_timeout=SLOW_BUILD_TIMEOUT)
 # IMPALA-6381: Isilon can behave as a slow build.
 if IS_ISILON:
   DELAY_MS = SLOW_BUILD_TIMEOUT

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/custom_cluster/test_metadata_replicas.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_metadata_replicas.py b/tests/custom_cluster/test_metadata_replicas.py
index 940f371..ff34cf2 100644
--- a/tests/custom_cluster/test_metadata_replicas.py
+++ b/tests/custom_cluster/test_metadata_replicas.py
@@ -18,7 +18,6 @@
 import pytest
 import re
 from time import sleep
-from tests.common.environ import specific_build_type_timeout
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.skip import (
     SkipIfS3,

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/custom_cluster/test_restart_services.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_restart_services.py b/tests/custom_cluster/test_restart_services.py
index a24149f..e441cbc 100644
--- a/tests/custom_cluster/test_restart_services.py
+++ b/tests/custom_cluster/test_restart_services.py
@@ -22,7 +22,7 @@ import re
 import socket
 import time
 
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 from time import sleep
 
 from impala.error import HiveServer2Error
@@ -50,7 +50,7 @@ class TestRestart(CustomClusterTestSuite):
     # We need to wait for the impalad to register to the new statestored and for a
     # non-empty catalog update from the new statestored. It cannot be expressed with the
     # existing metrics yet so we wait for some time here.
-    wait_time_s = specific_build_type_timeout(60, slow_build_timeout=100)
+    wait_time_s = build_flavor_timeout(60, slow_build_timeout=100)
     sleep(wait_time_s)
     for retry in xrange(wait_time_s):
       try:

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/query_test/test_hdfs_caching.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hdfs_caching.py b/tests/query_test/test_hdfs_caching.py
index 176fe09..64b36da 100644
--- a/tests/query_test/test_hdfs_caching.py
+++ b/tests/query_test/test_hdfs_caching.py
@@ -22,7 +22,7 @@ import re
 import time
 from subprocess import check_call
 
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite, LOG
 from tests.common.skip import SkipIfS3, SkipIfABFS, SkipIfADLS, SkipIfIsilon, \
@@ -339,7 +339,7 @@ def get_num_cache_requests():
     return len(stdout.split('\n'))
 
   # IMPALA-3040: This can take time, especially under slow builds like ASAN.
-  wait_time_in_sec = specific_build_type_timeout(5, slow_build_timeout=20)
+  wait_time_in_sec = build_flavor_timeout(5, slow_build_timeout=20)
   num_stabilization_attempts = 0
   max_num_stabilization_attempts = 10
   new_requests = None

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/query_test/test_runtime_filters.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_runtime_filters.py b/tests/query_test/test_runtime_filters.py
index 547a1ee..f769224 100644
--- a/tests/query_test/test_runtime_filters.py
+++ b/tests/query_test/test_runtime_filters.py
@@ -20,11 +20,11 @@ import pytest
 import re
 import time
 
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfLocal, SkipIfIsilon
 
-WAIT_TIME_MS = specific_build_type_timeout(60000, slow_build_timeout=100000)
+WAIT_TIME_MS = build_flavor_timeout(60000, slow_build_timeout=100000)
 
 # Some of the queries in runtime_filters consume a lot of memory, leading to
 # significant memory reservations in parallel tests.

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/run-tests.py
----------------------------------------------------------------------
diff --git a/tests/run-tests.py b/tests/run-tests.py
index 9aaeaa5..0de9ce9 100755
--- a/tests/run-tests.py
+++ b/tests/run-tests.py
@@ -37,7 +37,17 @@ from _pytest.config import FILE_OR_DIR
 # We whitelist valid test directories. If a new test directory is added, update this.
 VALID_TEST_DIRS = ['failure', 'query_test', 'stress', 'unittests', 'aux_query_tests',
                    'shell', 'hs2', 'catalog_service', 'metadata', 'data_errors',
-                   'statestore', 'infra', 'observability']
+                   'statestore', 'infra', 'observability', 'webserver']
+
+# A list of helper directories that do not contain any tests. The purpose of this
+# additional list is to prevent devs from adding a new test dir, but not adding the
+# new dir to the list of valid test dirs above. All dirs unders tests/ must be placed
+# into one of these lists, otherwise the script will throw an error. This list can be
+# removed once IMPALA-4417 has been resolved.
+TEST_HELPER_DIRS = ['aux_parquet_data_load', 'test-hive-udfs', 'comparison', 'benchmark',
+                     'custom_cluster', 'util', 'experiments', 'verifiers', 'common',
+                     'performance', 'beeswax', 'aux_custom_cluster_tests',
+                     'authorization']
 
 TEST_DIR = os.path.join(os.environ['IMPALA_HOME'], 'tests')
 RESULT_DIR = os.path.join(os.environ['IMPALA_EE_TEST_LOGS_DIR'], 'results')
@@ -201,7 +211,12 @@ def build_ignore_dir_arg_list(valid_dirs):
   code as though it contained tests, resulting in misleading warnings or failures.
   (There is a JIRA filed to restructure this: IMPALA-4417.)
   """
-  subdirs = [subdir for subdir in os.listdir(TEST_DIR) if os.path.isdir(subdir)]
+  subdirs = [subdir for subdir in os.listdir(TEST_DIR)
+      if os.path.isdir(subdir) and not subdir.startswith(".")]
+  for subdir in subdirs:
+      assert subdir in VALID_TEST_DIRS or subdir in TEST_HELPER_DIRS,\
+        "Unexpected test dir '%s' is not in the list of valid or helper test dirs"\
+        % subdir
   ignored_dir_list = []
   for subdir in (set(subdirs) - set(valid_dirs)):
     ignored_dir_list += ['--ignore', subdir]

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/statestore/test_statestore.py
----------------------------------------------------------------------
diff --git a/tests/statestore/test_statestore.py b/tests/statestore/test_statestore.py
index f6a6bb2..a951414 100644
--- a/tests/statestore/test_statestore.py
+++ b/tests/statestore/test_statestore.py
@@ -39,7 +39,7 @@ from StatestoreService.StatestoreSubscriber import TTopicRegistration
 from ErrorCodes.ttypes import TErrorCode
 from Status.ttypes import TStatus
 
-from tests.common.environ import specific_build_type_timeout
+from tests.common.environ import build_flavor_timeout
 
 LOG = logging.getLogger('test_statestore')
 
@@ -69,10 +69,10 @@ DEFAULT_UPDATE_STATE_RESPONSE = TUpdateStateResponse(status=STATUS_OK, topic_upd
                                                      skipped=False)
 
 # IMPALA-3501: the timeout needs to be higher in code coverage builds
-WAIT_FOR_FAILURE_TIMEOUT = specific_build_type_timeout(40, code_coverage_build_timeout=60)
-WAIT_FOR_HEARTBEAT_TIMEOUT = specific_build_type_timeout(
+WAIT_FOR_FAILURE_TIMEOUT = build_flavor_timeout(40, code_coverage_build_timeout=60)
+WAIT_FOR_HEARTBEAT_TIMEOUT = build_flavor_timeout(
     40, code_coverage_build_timeout=60)
-WAIT_FOR_UPDATE_TIMEOUT = specific_build_type_timeout(40, code_coverage_build_timeout=60)
+WAIT_FOR_UPDATE_TIMEOUT = build_flavor_timeout(40, code_coverage_build_timeout=60)
 
 class WildcardServerSocket(TSocket.TSocketBase, TTransport.TServerTransportBase):
   """Specialised server socket that binds to a random port at construction"""

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/tests/webserver/test_web_pages.py
----------------------------------------------------------------------
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index 15d4f8f..0c05546 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -15,14 +15,19 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from tests.common.skip import SkipIf
+from tests.common.environ import IMPALA_TEST_CLUSTER_PROPERTIES
+from tests.common.environ import ImpalaTestClusterFlagsDetector
+from tests.common.skip import SkipIfBuildType
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
 import json
 import requests
+import pytest
+
 
 class TestWebPage(ImpalaTestSuite):
 
+  ROOT_URL = "http://localhost:{0}/"
   GET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/get_java_loglevel"
   SET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/set_java_loglevel"
   RESET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/reset_java_loglevel"
@@ -46,8 +51,55 @@ class TestWebPage(ImpalaTestSuite):
   TEST_PORTS_WITH_SS = ["25000", "25010", "25020"]
   CATALOG_TEST_PORT = ["25020"]
 
+  def test_get_root_url(self):
+    """Tests that the root URL is accessible and loads properly"""
+    self.get_and_check_status(self.ROOT_URL)
+
+  def test_get_build_flags(self):
+    """Tests that the build flags on the root page contain valid values"""
+    for port in self.TEST_PORTS_WITH_SS:
+      build_flags = ImpalaTestClusterFlagsDetector.\
+          get_build_flags_from_web_ui(self.ROOT_URL.format(port))
+
+      assert len(build_flags) == 3
+      assert "is_ndebug" in build_flags
+      assert build_flags["is_ndebug"] in ["true", "false"]
+      assert "cmake_build_type" in build_flags
+      assert build_flags["cmake_build_type"] in ["debug", "release", "address_sanitizer",
+          "tidy", "ubsan", "ubsan_full", "tsan", "code_coverage_release",
+          "code_coverage_debug"]
+      assert "library_link_type" in build_flags
+      assert build_flags["library_link_type"] in ["dynamic", "static"]
+
+  @SkipIfBuildType.remote
+  def test_root_correct_build_flags(self):
+    """Tests that the build flags on the root page contain correct values"""
+    assert not IMPALA_TEST_CLUSTER_PROPERTIES.is_remote_cluster()
+    for port in self.TEST_PORTS_WITH_SS:
+      build_flags = ImpalaTestClusterFlagsDetector.\
+          get_build_flags_from_web_ui(self.ROOT_URL.format(port))
+
+      assert build_flags["cmake_build_type"] ==\
+              IMPALA_TEST_CLUSTER_PROPERTIES.build_flavor
+      assert build_flags["library_link_type"] ==\
+              IMPALA_TEST_CLUSTER_PROPERTIES.library_link_type
+
+  def test_root_consistent_build_flags(self):
+    """Tests that the build flags on the root page contain consistent values"""
+    for port in self.TEST_PORTS_WITH_SS:
+      build_flags = ImpalaTestClusterFlagsDetector.\
+          get_build_flags_from_web_ui(self.ROOT_URL.format(port))
+
+      is_ndebug = build_flags["is_ndebug"] == "true"
+
+      if not is_ndebug:
+        assert not build_flags["cmake_build_type"] in ["release"]
+
+      if build_flags["cmake_build_type"] in ["debug"]:
+        assert not is_ndebug
+
   def test_memz(self):
-    """test /memz at impalad / statestored / catalogd"""
+    """Tests /memz at impalad / statestored / catalogd"""
 
     page = requests.get("http://localhost:25000/memz")
     assert page.status_code == requests.codes.ok
@@ -79,26 +131,31 @@ class TestWebPage(ImpalaTestSuite):
       except ValueError:
         assert False, "Invalid JSON returned from /jmx endpoint: %s" % jmx_json
 
-  def get_and_check_status(self, url, string_to_search = "", ports_to_test = None):
+  def get_and_check_status(self, url, string_to_search="", ports_to_test=None):
     """Helper method that polls a given url and asserts the return code is ok and
     the response contains the input string."""
     if ports_to_test is None:
       ports_to_test = self.TEST_PORTS_WITH_SS
+
+    responses = []
     for port in ports_to_test:
       input_url = url.format(port)
       response = requests.get(input_url)
       assert response.status_code == requests.codes.ok\
           and string_to_search in response.text, "Offending url: " + input_url
+      responses.append(response)
+    return responses
 
-  def get_debug_page(self, page_url):
+  def get_debug_page(self, page_url, port=25000):
     """Returns the content of the debug page 'page_url' as json."""
-    response = self.get_and_check_status(page_url + "?json", ports_to_test=[25000])
-    assert "application/json" in response.headers['Content-Type']
-    return json.loads(response)
+    responses = self.get_and_check_status(page_url + "?json", ports_to_test=[port])
+    assert len(responses) == 1
+    assert "application/json" in responses[0].headers['Content-Type']
+    return json.loads(responses[0].text)
 
-  def get_and_check_status_jvm(self, url, string_to_search = ""):
+  def get_and_check_status_jvm(self, url, string_to_search=""):
     """Calls get_and_check_status() for impalad and catalogd only"""
-    self.get_and_check_status(url, string_to_search,
+    return self.get_and_check_status(url, string_to_search,
                               ports_to_test=self.TEST_PORTS_WITHOUT_SS)
 
   def test_content_type(self):
@@ -151,7 +208,7 @@ class TestWebPage(ImpalaTestSuite):
     self.get_and_check_status(set_glog_url, "v set to 3")
 
     # Try resetting the glog logging defaults again.
-    self.get_and_check_status( self.RESET_GLOG_LOGLEVEL_URL, "v set to ")
+    self.get_and_check_status(self.RESET_GLOG_LOGLEVEL_URL, "v set to ")
 
     # Try to get the log level of an empty class input
     get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class=")
@@ -214,17 +271,19 @@ class TestWebPage(ImpalaTestSuite):
     cancels the query."""
     if query_options:
       self.client.set_configuration(query_options)
-    query_handle =  self.client.execute_async(query)
+    query_handle = self.client.execute_async(query)
     response_json = ""
     try:
-      response = self.get_and_check_status(
+      responses = self.get_and_check_status(
         page_url + "?query_id=%s&json" % query_handle.get_handle().id,
         ports_to_test=[25000])
-      response_json = json.loads(response)
+      assert len(responses) == 1
+      response_json = json.loads(responses[0].text)
     finally:
       self.client.cancel(query_handle)
     return response_json
 
+  @pytest.mark.skip(reason='IMPALA-7625')
   def test_backend_states(self, unique_database):
     """Test that /query_backends returns the list of backend states for DML or queries;
     nothing for DDL statements"""
@@ -240,6 +299,7 @@ class TestWebPage(ImpalaTestSuite):
       else:
         assert 'backend_states' not in response_json
 
+  @pytest.mark.skip(reason='IMPALA-7625')
   def test_backend_instances(self, unique_database):
     """Test that /query_finstances returns the list of fragment instances for DML or
     queries; nothing for DDL statements"""
@@ -255,6 +315,7 @@ class TestWebPage(ImpalaTestSuite):
       else:
         assert 'backend_instances' not in response_json
 
+  @pytest.mark.skip(reason='IMPALA-7625')
   def test_backend_instances_mt_dop(self, unique_database):
     """Test that accessing /query_finstances does not crash the backend when running with
     mt_dop."""
@@ -268,14 +329,15 @@ class TestWebPage(ImpalaTestSuite):
   def test_io_mgr_threads(self):
     """Test that IoMgr threads have readable names. This test assumed that all systems we
     support have a disk called 'sda'."""
-    response = self.get_and_check_status(
+    responses = self.get_and_check_status(
         self.THREAD_GROUP_URL + "?group=disk-io-mgr&json", ports_to_test=[25000])
-    response_json = json.loads(response)
+    assert len(responses) == 1
+    response_json = json.loads(responses[0].text)
     thread_names = [t["name"] for t in response_json['threads']]
-    expected_name_patterns = ["ADLS remote", "S3 remote", "HDFS remote", "sda"]
+    expected_name_patterns = ["ADLS remote", "S3 remote", "HDFS remote"]
     for pattern in expected_name_patterns:
       assert any(pattern in t for t in thread_names), \
-           "Could not find thread matching '%s'" % pattern
+          "Could not find thread matching '%s'" % pattern
 
   def test_krpc_rpcz(self):
     """Test that KRPC metrics are exposed in /rpcz and that they are updated when

http://git-wip-us.apache.org/repos/asf/impala/blob/9bdb73b1/www/root.tmpl
----------------------------------------------------------------------
diff --git a/www/root.tmpl b/www/root.tmpl
index 7d156c4..9d73c41 100644
--- a/www/root.tmpl
+++ b/www/root.tmpl
@@ -32,7 +32,10 @@ under the License.
   {{/impala_server_mode}}
 
   <h2>Vers<span id="v">i</span>on</h2>
-  <pre id="version_pre">{{version}}</pre>
+  <pre id="version_pre">{{version}}
+Build Flags: {{#build_flags}}{{flag_name}}={{flag_value}}  {{/build_flags}}</pre>
+<!-- The space after {{flag_value}} is necessary to add a space between each flag when
+this page is rendered -->
 
   <h2>Process Start Time</h2>
   <pre>{{process_start_time}}</pre>


[08/33] impala git commit: IMPALA-7775: fix some lifecycle issues in statestore/session tests

Posted by bo...@apache.org.
IMPALA-7775: fix some lifecycle issues in statestore/session tests

Background threads from the Statestore's thread pool continued
running in the background and could dereference invalid memory.
We make sure these threads are cleaned up before moving onto
the next test. Note that we don't clean up all background
threads, just the ones that had caused issues here.

I refactored the memory management a bit to put all objects
that we can't safely free into a single ObjectPool.

The statestore tests also had an issue with the lifetime of the
string flags FLAGS_ssl_*_certificate. Those were overwritten
with new values while the thread pool threads were still running,
which could cause use-after-free bugs.

Testing:
Looped the tests under ASAN with the "stress" utility running at the
same time to flush out races.

Ran core tests.

Change-Id: I3b25c8b8a96bfa1183ce273b3bb4debde234dd01
Reviewed-on: http://gerrit.cloudera.org:8080/11864
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: e5d0757983802fbe8a0f473bae8ff2d022fc0117
Parents: 0d4c6ae
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Nov 1 12:00:37 2018 -0700
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:50:23 2018 +0100

----------------------------------------------------------------------
 be/src/service/session-expiry-test.cc | 13 ++++-
 be/src/statestore/statestore-test.cc  | 89 ++++++++++++++++++------------
 be/src/statestore/statestore.cc       | 11 ++++
 be/src/statestore/statestore.h        |  7 ++-
 4 files changed, 81 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e5d07579/be/src/service/session-expiry-test.cc
----------------------------------------------------------------------
diff --git a/be/src/service/session-expiry-test.cc b/be/src/service/session-expiry-test.cc
index 89ae842..7e5b86a 100644
--- a/be/src/service/session-expiry-test.cc
+++ b/be/src/service/session-expiry-test.cc
@@ -44,14 +44,20 @@ DECLARE_int32(beeswax_port);
 // TODO: Come up with a short-running test that confirms a session will keep itself alive
 // that doesn't depend upon being rescheduled in a timely fashion.
 
+// Object pool containing all objects that must live for the duration of the process.
+// E.g. objects that are singletons and never destroyed in a real daemon (so don't support
+// tear-down logic), but which we create multiple times in unit tests. We leak this pool
+// instead of destroying it to avoid destroying the contained objects.
+static ObjectPool* perm_objects;
+
 TEST(SessionTest, TestExpiry) {
   const int NUM_SESSIONS = 5;
   const int MAX_IDLE_TIMEOUT_MS = 4000;
   FLAGS_idle_session_timeout = 1;
   // Skip validation checks for in-process backend.
   FLAGS_abort_on_config_error = false;
-  scoped_ptr<MetricGroup> metrics(new MetricGroup("statestore"));
-  Statestore* statestore = new Statestore(metrics.get());
+  MetricGroup* metrics = perm_objects->Add(new MetricGroup("statestore"));
+  Statestore* statestore = perm_objects->Add(new Statestore(metrics));
   IGNORE_LEAKING_OBJECT(statestore);
   // Pass in 0 to have the statestore use an ephemeral port for the service.
   ABORT_IF_ERROR(statestore->Init(0));
@@ -111,11 +117,14 @@ TEST(SessionTest, TestExpiry) {
   // work). Sleep to allow the threads closing the session to complete before tearing down
   // the server.
   SleepForMs(1000);
+  statestore->ShutdownForTesting();
 }
 
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   InitFeSupport();
+  perm_objects = new ObjectPool;
+  IGNORE_LEAKING_OBJECT(perm_objects);
   return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e5d07579/be/src/statestore/statestore-test.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore-test.cc b/be/src/statestore/statestore-test.cc
index a9ee095..f517a56 100644
--- a/be/src/statestore/statestore-test.cc
+++ b/be/src/statestore/statestore-test.cc
@@ -34,77 +34,94 @@ DECLARE_int32(state_store_port);
 
 namespace impala {
 
+// Object pool containing all objects that must live for the duration of the process.
+// E.g. objects that are singletons and never destroyed in a real daemon (so don't support
+// tear-down logic), but which we create multiple times in unit tests. We leak this pool
+// instead of destroying it to avoid destroying the contained objects.
+static ObjectPool* perm_objects;
+
 TEST(StatestoreTest, SmokeTest) {
   // All allocations done by 'new' to avoid problems shutting down Thrift servers
   // gracefully.
-  scoped_ptr<MetricGroup> metrics(new MetricGroup("statestore"));
-  Statestore* statestore = new Statestore(metrics.get());
+  MetricGroup* metrics = perm_objects->Add(new MetricGroup("statestore"));
+  Statestore* statestore = perm_objects->Add(new Statestore(metrics));
   // Thrift will internally pick an ephemeral port if we pass in 0 as the port.
   int statestore_port = 0;
-  IGNORE_LEAKING_OBJECT(statestore);
   ASSERT_OK(statestore->Init(statestore_port));
 
-  scoped_ptr<MetricGroup> metrics_2(new MetricGroup("statestore_2"));
+  MetricGroup* metrics_2 = perm_objects->Add(new MetricGroup("statestore_2"));
   // Port already in use
-  Statestore* statestore_wont_start = new Statestore(metrics_2.get());
+  Statestore* statestore_wont_start = perm_objects->Add(new Statestore(metrics_2));
   ASSERT_FALSE(statestore_wont_start->Init(statestore->port()).ok());
 
-  StatestoreSubscriber* sub_will_start =
+  StatestoreSubscriber* sub_will_start = perm_objects->Add(
       new StatestoreSubscriber("sub1", MakeNetworkAddress("localhost", 0),
-          MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
-  IGNORE_LEAKING_OBJECT(sub_will_start);
+          MakeNetworkAddress("localhost", statestore->port()), new MetricGroup("")));
   ASSERT_OK(sub_will_start->Start());
 
   // Confirm that a subscriber trying to use an in-use port will fail to start.
-  StatestoreSubscriber* sub_will_not_start = new StatestoreSubscriber("sub3",
-      MakeNetworkAddress("localhost", sub_will_start->heartbeat_port()),
-      MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
-  IGNORE_LEAKING_OBJECT(sub_will_not_start);
+  StatestoreSubscriber* sub_will_not_start = perm_objects->Add(new StatestoreSubscriber(
+      "sub3", MakeNetworkAddress("localhost", sub_will_start->heartbeat_port()),
+      MakeNetworkAddress("localhost", statestore->port()), new MetricGroup("")));
   ASSERT_FALSE(sub_will_not_start->Start().ok());
+
+  statestore->ShutdownForTesting();
 }
 
-TEST(StatestoreSslTest, SmokeTest) {
+// Runs an SSL smoke test with provided parameters.
+void SslSmokeTestHelper(const string& server_ca_certificate,
+    const string& client_ca_certificate, bool sub_should_start) {
   string impala_home(getenv("IMPALA_HOME"));
   stringstream server_cert;
   server_cert << impala_home << "/be/src/testutil/server-cert.pem";
-  FLAGS_ssl_server_certificate = server_cert.str();
-  FLAGS_ssl_client_ca_certificate = server_cert.str();
+  // Override flags for the duration of this test. Modifying them while the statestore
+  // is running is unsafe.
+  FLAGS_ssl_server_certificate = server_ca_certificate;
+  FLAGS_ssl_client_ca_certificate = client_ca_certificate;
   stringstream server_key;
   server_key << impala_home << "/be/src/testutil/server-key.pem";
   FLAGS_ssl_private_key = server_key.str();
 
   // Thrift will internally pick an ephemeral port if we pass in 0 as the port.
   int statestore_port = 0;
-  scoped_ptr<MetricGroup> metrics(new MetricGroup("statestore"));
-  Statestore* statestore = new Statestore(metrics.get());
-  IGNORE_LEAKING_OBJECT(statestore);
+  MetricGroup* metrics = perm_objects->Add(new MetricGroup("statestore"));
+  Statestore* statestore = perm_objects->Add(new Statestore(metrics));
   ASSERT_OK(statestore->Init(statestore_port));
 
-  StatestoreSubscriber* sub_will_start = new StatestoreSubscriber("smoke_sub1",
-      MakeNetworkAddress("localhost", 0),
-      MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
-  IGNORE_LEAKING_OBJECT(sub_will_start);
-  ASSERT_OK(sub_will_start->Start());
+  StatestoreSubscriber* sub = perm_objects->Add(
+      new StatestoreSubscriber("smoke_sub", MakeNetworkAddress("localhost", 0),
+          MakeNetworkAddress("localhost", statestore->port()), new MetricGroup("")));
+  Status sub_status = sub->Start();
+  ASSERT_EQ(sub_should_start, sub_status.ok());
 
-  stringstream invalid_server_cert;
-  invalid_server_cert << impala_home << "/be/src/testutil/invalid-server-cert.pem";
-  FLAGS_ssl_client_ca_certificate = invalid_server_cert.str();
+  statestore->ShutdownForTesting();
+}
 
-  StatestoreSubscriber* sub_will_not_start = new StatestoreSubscriber("smoke_sub2",
-      MakeNetworkAddress("localhost", 0),
-      MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
-  IGNORE_LEAKING_OBJECT(sub_will_not_start);
-  ASSERT_FALSE(sub_will_not_start->Start().ok());
+string GetValidServerCert() {
+  string impala_home(getenv("IMPALA_HOME"));
+  stringstream server_cert;
+  server_cert << impala_home << "/be/src/testutil/server-cert.pem";
+  return server_cert.str();
+}
+
+TEST(StatestoreSslTest, ValidCertSmokeTest) {
+  string valid_cert = GetValidServerCert();
+  SslSmokeTestHelper(valid_cert, valid_cert, true);
 }
 
+TEST(StatestoreSslTest, InvalidCertSmokeTest) {
+  string impala_home(getenv("IMPALA_HOME"));
+  stringstream invalid_server_cert;
+  invalid_server_cert << impala_home << "/be/src/testutil/invalid-server-cert.pem";
+  SslSmokeTestHelper(GetValidServerCert(), invalid_server_cert.str(), false);
 }
 
+} // namespace impala
+
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, false, impala::TestInfo::BE_TEST);
-  int rc = RUN_ALL_TESTS();
-  // IMPALA-5291: statestore services and subscribers may still be running at this point
-  // and accessing global state. Exit without running global destructors to avoid
-  // races with other threads when tearing down the proces.
-  _exit(rc);
+  perm_objects = new ObjectPool;
+  IGNORE_LEAKING_OBJECT(perm_objects);
+  return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e5d07579/be/src/statestore/statestore.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index 4e63dad..70183d7 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -36,6 +36,7 @@
 #include "util/debug-util.h"
 #include "util/logging-support.h"
 #include "util/openssl-util.h"
+#include "util/test-info.h"
 #include "util/time.h"
 #include "util/uid-util.h"
 #include "util/webserver.h"
@@ -1050,3 +1051,13 @@ void Statestore::MainLoop() {
   subscriber_priority_topic_update_threadpool_.Join();
   subscriber_heartbeat_threadpool_.Join();
 }
+
+void Statestore::ShutdownForTesting() {
+  CHECK(TestInfo::is_be_test()) << "Only valid to call in backend tests.";
+  subscriber_topic_update_threadpool_.Shutdown();
+  subscriber_priority_topic_update_threadpool_.Shutdown();
+  subscriber_heartbeat_threadpool_.Shutdown();
+  subscriber_topic_update_threadpool_.Join();
+  subscriber_priority_topic_update_threadpool_.Join();
+  subscriber_heartbeat_threadpool_.Join();
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/e5d07579/be/src/statestore/statestore.h
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.h b/be/src/statestore/statestore.h
index 52f7d68..871494c 100644
--- a/be/src/statestore/statestore.h
+++ b/be/src/statestore/statestore.h
@@ -157,6 +157,11 @@ class Statestore : public CacheLineAligned {
   /// The main processing loop. Runs infinitely.
   void MainLoop();
 
+  /// Shut down some background threads. Only used for testing. Note that this is not
+  /// a clean shutdown because we can't correctly tear down 'thrift_server_', so
+  /// not all background threads are stopped and this object cannot be destroyed.
+  void ShutdownForTesting();
+
   /// Returns the Thrift API interface that proxies requests onto the local Statestore.
   const boost::shared_ptr<StatestoreServiceIf>& thrift_iface() const {
     return thrift_iface_;
@@ -713,6 +718,6 @@ class Statestore : public CacheLineAligned {
   [[noreturn]] void MonitorSubscriberHeartbeat();
 };
 
-}
+} // namespace impala
 
 #endif


[17/33] impala git commit: Bump CDH_BUILD_VERSION to include Hive fix for RELY constraint

Posted by bo...@apache.org.
Bump CDH_BUILD_VERSION to include Hive fix for RELY constraint

This change bumps the CDH_BUILD_VERSION to a version that includes
the fix for a bug that does not correctly propagate "RELY" constraints
from SQL statements in Hive. This is needed for the upcoming patch for
IMPALA-3531.

This new CDH_BUILD_VERSION pulls in newer version of Kudu binaries (v1.9).
Kudu version variables in the impala-config are updated accordingly. Also,
includes a newer toolchain build that has these v1.9 bits.

Testing: DEBUG/Core tests passed.

Change-Id: Ie626882044fea98b81b07caa950b480e16df122d
Reviewed-on: http://gerrit.cloudera.org:8080/11891
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Thomas Marshall <th...@cmu.edu>


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

Branch: refs/heads/branch-3.1.0
Commit: 09dc76327abc441a1b6e1628bad845907f602521
Parents: 14fc20a
Author: Bharath Vissapragada <bh...@cloudera.com>
Authored: Mon Nov 5 11:31:23 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:51:39 2018 +0100

----------------------------------------------------------------------
 bin/impala-config.sh | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/09dc7632/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index c0030ca..32df25d 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -68,7 +68,7 @@ fi
 # moving to a different build of the toolchain, e.g. when a version is bumped or a
 # compile option is changed. The build id can be found in the output of the toolchain
 # build jobs, it is constructed from the build number and toolchain git hash prefix.
-export IMPALA_TOOLCHAIN_BUILD_ID=185-ec90313c05
+export IMPALA_TOOLCHAIN_BUILD_ID=210-5033cfcc59
 # Versions of toolchain dependencies.
 # -----------------------------------
 export IMPALA_AVRO_VERSION=1.7.4-p4
@@ -158,7 +158,7 @@ fi
 : ${CDH_DOWNLOAD_HOST:=native-toolchain.s3.amazonaws.com}
 export CDH_DOWNLOAD_HOST
 export CDH_MAJOR_VERSION=6
-export CDH_BUILD_NUMBER=663310
+export CDH_BUILD_NUMBER=680275
 export IMPALA_HADOOP_VERSION=3.0.0-cdh6.x-SNAPSHOT
 export IMPALA_HBASE_VERSION=2.1.0-cdh6.x-SNAPSHOT
 export IMPALA_HIVE_VERSION=2.1.1-cdh6.x-SNAPSHOT
@@ -167,7 +167,7 @@ export IMPALA_PARQUET_VERSION=1.9.0-cdh6.x-SNAPSHOT
 export IMPALA_AVRO_JAVA_VERSION=1.8.2-cdh6.x-SNAPSHOT
 export IMPALA_LLAMA_MINIKDC_VERSION=1.0.0
 export IMPALA_KITE_VERSION=1.0.0-cdh6.x-SNAPSHOT
-export KUDU_JAVA_VERSION=1.8.0-cdh6.x-SNAPSHOT
+export KUDU_JAVA_VERSION=1.9.0-cdh6.x-SNAPSHOT
 
 # When IMPALA_(CDH_COMPONENT)_URL are overridden, they may contain '$(platform_label)'
 # which will be substituted for the CDH platform label in bootstrap_toolchain.py
@@ -596,10 +596,10 @@ fi
 export KUDU_IS_SUPPORTED
 
 if $USE_CDH_KUDU; then
-  export IMPALA_KUDU_VERSION=${IMPALA_KUDU_VERSION-"1.8.0-cdh6.x-SNAPSHOT"}
+  export IMPALA_KUDU_VERSION=${IMPALA_KUDU_VERSION-"1.9.0-cdh6.x-SNAPSHOT"}
   export IMPALA_KUDU_HOME=${CDH_COMPONENTS_HOME}/kudu-$IMPALA_KUDU_VERSION
 else
-  export IMPALA_KUDU_VERSION=${IMPALA_KUDU_VERSION-"5211897"}
+  export IMPALA_KUDU_VERSION=${IMPALA_KUDU_VERSION-"4ec2598"}
   export IMPALA_KUDU_HOME=${IMPALA_TOOLCHAIN}/kudu-$IMPALA_KUDU_VERSION
 fi
 


[28/33] impala git commit: IMPALA-7528: Fix division by zero when computing cardinalities

Posted by bo...@apache.org.
IMPALA-7528: Fix division by zero when computing cardinalities

This patch fixes a case where there can be a division by zero when
computing cardinalities of many to many joins on NULL columns.

Testing:
Added a planner test case

Change-Id: Ieedd51d3ad6131a4ea2a5883f05104e6a0f2cd14
Reviewed-on: http://gerrit.cloudera.org:8080/11901
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/branch-3.1.0
Commit: d07bc818d6a8af6b2ebc2d1d2672849abc40f51c
Parents: 8dfbe3c
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Tue Nov 6 23:44:19 2018 -0800
Committer: Zoltan Borok-Nagy <bo...@cloudera.com>
Committed: Tue Nov 13 12:52:35 2018 +0100

----------------------------------------------------------------------
 fe/src/main/java/org/apache/impala/planner/JoinNode.java    | 7 ++++++-
 fe/src/test/java/org/apache/impala/planner/PlannerTest.java | 9 +++++++++
 testdata/bin/compute-table-stats.sh                         | 2 +-
 .../functional-planner/queries/PlannerTest/joins.test       | 4 ++--
 4 files changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d07bc818/fe/src/main/java/org/apache/impala/planner/JoinNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index cc50318..96f93cd 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -365,7 +365,12 @@ public abstract class JoinNode extends PlanNode {
       if (slots.lhsNumRows() > lhsCard) lhsAdjNdv *= lhsCard / slots.lhsNumRows();
       double rhsAdjNdv = slots.rhsNdv();
       if (slots.rhsNumRows() > rhsCard) rhsAdjNdv *= rhsCard / slots.rhsNumRows();
-      long joinCard = Math.round((lhsCard / Math.max(lhsAdjNdv, rhsAdjNdv)) * rhsCard);
+      // A lower limit of 1 on the max Adjusted Ndv ensures we don't estimate
+      // cardinality more than the max possible. This also handles the case of
+      // null columns on both sides having an Ndv of zero (which would change
+      // after IMPALA-7310 is fixed).
+      long joinCard = Math.round((lhsCard / Math.max(1, Math.max(lhsAdjNdv, rhsAdjNdv))) *
+          rhsCard);
       if (result == -1) {
         result = joinCard;
       } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/d07bc818/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index 2ec4b15..82235ed 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -703,4 +703,13 @@ public class PlannerTest extends PlannerTestBase {
     assertEquals(HBaseScanNode.memoryEstimateForFetchingColumns(largeColumnList),
         8 * 1024 * 1024);
   }
+
+  @Test
+  public void testNullColumnJoinCardinality() throws ImpalaException {
+    // IMPALA-7565: Make sure there is no division by zero during cardinality calculation
+    // in a many to many join on null columns (ndv = 0).
+    String query = "select * from functional.nulltable t1 "
+        + "inner join [shuffle] functional.nulltable t2 on t1.d = t2.d";
+    checkCardinality(query, 1, 1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/d07bc818/testdata/bin/compute-table-stats.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/compute-table-stats.sh b/testdata/bin/compute-table-stats.sh
index 08c7595..d6e6d22 100755
--- a/testdata/bin/compute-table-stats.sh
+++ b/testdata/bin/compute-table-stats.sh
@@ -33,7 +33,7 @@ COMPUTE_STATS_SCRIPT="${IMPALA_HOME}/tests/util/compute_table_stats.py --impalad
 # Run compute stats over as many of the tables used in the Planner tests as possible.
 ${COMPUTE_STATS_SCRIPT} --db_names=functional\
     --table_names="alltypes,alltypesagg,alltypesaggmultifilesnopart,alltypesaggnonulls,
-    alltypessmall,alltypestiny,jointbl,dimtbl,stringpartitionkey"
+    alltypessmall,alltypestiny,jointbl,dimtbl,stringpartitionkey,nulltable"
 
 # We cannot load HBase on s3 and isilon yet.
 if [ "${TARGET_FILESYSTEM}" = "hdfs" ]; then

http://git-wip-us.apache.org/repos/asf/impala/blob/d07bc818/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
index 4c5d7ab..b681acb 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
@@ -2387,10 +2387,10 @@ PLAN-ROOT SINK
 03:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: t1.d IS DISTINCT FROM t2.d
 |
-|--01:SCAN HDFS [functional.nulltable t2]
+|--00:SCAN HDFS [functional.nulltable t1]
 |     partitions=1/1 files=1 size=18B
 |
-00:SCAN HDFS [functional.nulltable t1]
+01:SCAN HDFS [functional.nulltable t2]
    partitions=1/1 files=1 size=18B
 ====
 # IMPALA-3450: limits on join nodes are reflected in cardinality estimates. The test for