You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2016/09/07 18:40:56 UTC

[1/2] incubator-impala git commit: IMPALA-3567: move ExecOption profile helpers to RuntimeProfile

Repository: incubator-impala
Updated Branches:
  refs/heads/master 39e01abcf -> 25fe78291


IMPALA-3567: move ExecOption profile helpers to RuntimeProfile

This is groundwork for IMPALA-3567, which will move some logic that
previously resided in ExecNodes into DataSinks. We want to report
ExecOption strings consistently in both ExecNodes and DataSinks,
so that logic needs to move to a shared place (e.g. the RuntimeProfile
itself).

I ran the patch through clang-format, which changed line wrapping and
indentation of a few surrounding lines.

Change-Id: I21c1dda8f8a1d92172bf59fbc1070a6834e61913
Reviewed-on: http://gerrit.cloudera.org:8080/4188
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 7194b28029227be0188697cc590b0a0b9e31a31f
Parents: 39e01ab
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Aug 31 13:49:42 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Sep 7 07:22:15 2016 +0000

----------------------------------------------------------------------
 be/src/exec/aggregation-node.cc             |  2 +-
 be/src/exec/blocking-join-node.cc           |  8 ++--
 be/src/exec/exec-node.cc                    | 26 ++---------
 be/src/exec/exec-node.h                     | 22 ----------
 be/src/exec/hash-join-node.cc               | 11 ++---
 be/src/exec/hdfs-scan-node.cc               | 13 +++---
 be/src/exec/partitioned-aggregation-node.cc | 14 +++---
 be/src/exec/partitioned-hash-join-node.cc   | 19 ++++----
 be/src/exec/sort-node.cc                    |  5 +--
 be/src/exec/topn-node.cc                    |  7 +--
 be/src/util/runtime-profile.cc              | 55 ++++++++++++++++--------
 be/src/util/runtime-profile.h               | 30 +++++++++++++
 12 files changed, 114 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/aggregation-node.cc b/be/src/exec/aggregation-node.cc
index eb8d329..2b9550a 100644
--- a/be/src/exec/aggregation-node.cc
+++ b/be/src/exec/aggregation-node.cc
@@ -170,7 +170,7 @@ Status AggregationNode::Prepare(RuntimeState* state) {
       }
     }
   }
-  AddCodegenExecOption(codegen_enabled);
+  runtime_profile()->AddCodegenMsg(codegen_enabled);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/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 ea541c0..2c17d13 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -192,10 +192,10 @@ Status BlockingJoinNode::ConstructBuildAndOpenProbe(RuntimeState* state,
   // returns.
   if (!IsInSubplan() && state->resource_pool()->TryAcquireThreadToken()) {
     Promise<Status> build_side_status;
-    AddRuntimeExecOption("Join Build-Side Prepared Asynchronously");
-    Thread build_thread(node_name_, "build thread",
-        bind(&BlockingJoinNode::ProcessBuildInputAsync, this, state, build_sink,
-          &build_side_status));
+    runtime_profile()->AppendExecOption("Join Build-Side Prepared Asynchronously");
+    Thread build_thread(
+        node_name_, "build thread", bind(&BlockingJoinNode::ProcessBuildInputAsync, this,
+                                        state, build_sink, &build_side_status));
     if (!state->cgroup().empty()) {
       Status status = state->exec_env()->cgroups_mgr()->AssignThreadToCgroup(
           build_thread, state->cgroup());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 937c6f2..0996037 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -198,27 +198,8 @@ void ExecNode::Close(RuntimeState* state) {
   }
 }
 
-void ExecNode::AddRuntimeExecOption(const string& str) {
-  lock_guard<mutex> l(exec_options_lock_);
-  if (runtime_exec_options_.empty()) {
-    runtime_exec_options_ = str;
-  } else {
-    runtime_exec_options_.append(", ");
-    runtime_exec_options_.append(str);
-  }
-  runtime_profile()->AddInfoString("ExecOption", runtime_exec_options_);
-}
-
-void ExecNode::AddCodegenExecOption(bool codegen_enabled, const string& extra_info,
-    const string& extra_label) {
-  string str = codegen_enabled ? "Codegen Enabled" : "Codegen Disabled";
-  if (!extra_info.empty()) str = str + ": " + extra_info;
-  if (!extra_label.empty()) str = extra_label + " " + str;
-  AddRuntimeExecOption(str);
-}
-
-Status ExecNode::CreateTree(RuntimeState* state, const TPlan& plan,
-    const DescriptorTbl& descs, ExecNode** root) {
+Status ExecNode::CreateTree(
+    RuntimeState* state, const TPlan& plan, const DescriptorTbl& descs, ExecNode** root) {
   if (plan.nodes.size() == 0) {
     *root = NULL;
     return Status::OK();
@@ -287,7 +268,8 @@ Status ExecNode::CreateNode(ObjectPool* pool, const TPlanNode& tnode,
       // evaluation whenever possible. Turn codegen on for expr evaluation for
       // the entire fragment.
       if (tnode.hdfs_scan_node.codegen_conjuncts) state->SetCodegenExpr();
-      (*node)->AddCodegenExecOption(state->ShouldCodegenExpr(), "", "Expr Evaluation");
+      (*node)->runtime_profile()->AddCodegenMsg(
+          state->ShouldCodegenExpr(), "", "Expr Evaluation");
       break;
     case TPlanNodeType::HBASE_SCAN_NODE:
       *node = pool->Add(new HBaseScanNode(pool, tnode, descs));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/exec-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index a19b8b6..9283a8b 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -258,12 +258,6 @@ class ExecNode {
   /// MemTracker that should be used for ExprContexts.
   boost::scoped_ptr<MemTracker> expr_mem_tracker_;
 
-  /// Execution options that are determined at runtime.  This is added to the
-  /// runtime profile at Close().  Examples for options logged here would be
-  /// "Codegen Enabled"
-  boost::mutex exec_options_lock_;
-  std::string runtime_exec_options_;
-
   bool is_closed() const { return is_closed_; }
 
   /// Pointer to the containing SubplanNode or NULL if not inside a subplan.
@@ -290,22 +284,6 @@ class ExecNode {
   /// 'phase' must not be INVALID.
   Status ExecDebugAction(TExecNodePhase::type phase, RuntimeState* state);
 
-  /// Appends option to 'runtime_exec_options_'
-  void AddRuntimeExecOption(const std::string& option);
-
-  /// Helper wrapper around AddRuntimeExecOption() for adding "Codegen Enabled" or
-  /// "Codegen Disabled" exec options. If specified, 'extra_info' is appended to the exec
-  /// option, and 'extra_label' is prepended to the exec option.
-  void AddCodegenExecOption(bool codegen_enabled, const string& extra_info = "",
-      const string& extra_label = "");
-
-  /// Helper wrapper that takes a status optionally describing why codegen was
-  /// disabled. 'codegen_status' can be OK.
-  void AddCodegenExecOption(bool codegen_enabled, const Status& codegen_status,
-      const string& extra_label = "") {
-    AddCodegenExecOption(codegen_enabled, codegen_status.GetDetail(), extra_label);
-  }
-
   /// Frees any local allocations made by expr_ctxs_to_free_ and returns the result of
   /// state->CheckQueryState(). Nodes should call this periodically, e.g. once per input
   /// row batch. This should not be called outside the main execution thread.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/hash-join-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hash-join-node.cc b/be/src/exec/hash-join-node.cc
index 60a64be..946ab48 100644
--- a/be/src/exec/hash-join-node.cc
+++ b/be/src/exec/hash-join-node.cc
@@ -173,8 +173,8 @@ Status HashJoinNode::Prepare(RuntimeState* state) {
       }
     }
   }
-  AddCodegenExecOption(build_codegen_enabled, "", "Build Side");
-  AddCodegenExecOption(probe_codegen_enabled, "", "Probe Side");
+  runtime_profile()->AddCodegenMsg(build_codegen_enabled, "", "Build Side");
+  runtime_profile()->AddCodegenMsg(probe_codegen_enabled, "", "Probe Side");
   return Status::OK();
 }
 
@@ -254,13 +254,14 @@ Status HashJoinNode::ProcessBuildInput(RuntimeState* state) {
   if (filters_.size() > 0) {
     int num_enabled_filters = hash_tbl_->AddBloomFilters();
     if (num_enabled_filters == filters_.size()) {
-      AddRuntimeExecOption(Substitute("$0 of $0 Runtime Filter$1 Published",
-              filters_.size(), filters_.size() == 1 ? "" : "s"));
+      runtime_profile()->AppendExecOption(
+          Substitute("$0 of $0 Runtime Filter$1 Published", filters_.size(),
+              filters_.size() == 1 ? "" : "s"));
     } else {
       string exec_option = Substitute("$0 of $1 Runtime Filter$2 Published, $3 Disabled",
           num_enabled_filters, filters_.size(), filters_.size() == 1 ? "" : "s",
           filters_.size() - num_enabled_filters);
-      AddRuntimeExecOption(exec_option);
+      runtime_profile()->AppendExecOption(exec_option);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index 4846004..e0cee4d 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -582,11 +582,11 @@ Status HdfsScanNode::Prepare(RuntimeState* state) {
       file_desc = file_desc_it->second;
     }
 
-    bool expected_local = (*scan_range_params_)[i].__isset.is_remote &&
-        !(*scan_range_params_)[i].is_remote;
+    bool expected_local =
+        (*scan_range_params_)[i].__isset.is_remote && !(*scan_range_params_)[i].is_remote;
     if (expected_local && (*scan_range_params_)[i].volume_id == -1) {
       if (!FLAGS_suppress_unknown_disk_id_warnings && !unknown_disk_id_warned_) {
-        AddRuntimeExecOption("Missing Volume Id");
+        runtime_profile()->AppendExecOption("Missing Volume Id");
         runtime_state()->LogError(ErrorMsg(TErrorCode::HDFS_SCAN_NODE_UNKNOWN_DISK));
         unknown_disk_id_warned_ = true;
       }
@@ -696,9 +696,9 @@ Status HdfsScanNode::Prepare(RuntimeState* state) {
 
     const char* format_name = _THdfsFileFormat_VALUES_TO_NAMES.find(format)->second;
     if (!status.ok()) {
-      AddCodegenExecOption(false, status, format_name);
+      runtime_profile()->AddCodegenMsg(false, status, format_name);
     } else {
-      AddCodegenExecOption(true, status, format_name);
+      runtime_profile()->AddCodegenMsg(true, status, format_name);
       LlvmCodeGen* codegen;
       RETURN_IF_ERROR(runtime_state_->GetCodegen(&codegen));
       codegen->AddFunctionToJit(
@@ -1370,7 +1370,8 @@ void HdfsScanNode::StopAndFinalizeCounters() {
   // Output fraction of scanners with codegen enabled
   int num_enabled = num_scanners_codegen_enabled_.Load();
   int total = num_enabled + num_scanners_codegen_disabled_.Load();
-  AddRuntimeExecOption(Substitute("Codegen enabled: $0 out of $1", num_enabled, total));
+  runtime_profile()->AppendExecOption(
+      Substitute("Codegen enabled: $0 out of $1", num_enabled, total));
 
   if (reader_context_ != NULL) {
     bytes_read_local_->Set(runtime_state_->io_mgr()->bytes_read_local(reader_context_));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/partitioned-aggregation-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-aggregation-node.cc b/be/src/exec/partitioned-aggregation-node.cc
index 02e857b..9c91b07 100644
--- a/be/src/exec/partitioned-aggregation-node.cc
+++ b/be/src/exec/partitioned-aggregation-node.cc
@@ -180,10 +180,10 @@ Status PartitionedAggregationNode::Prepare(RuntimeState* state) {
       ADD_COUNTER(runtime_profile(), "HashBuckets", TUnit::UNIT);
   partitions_created_ =
       ADD_COUNTER(runtime_profile(), "PartitionsCreated", TUnit::UNIT);
-  largest_partition_percent_ = runtime_profile()->AddHighWaterMarkCounter(
-      "LargestPartitionPercent", TUnit::UNIT);
+  largest_partition_percent_ =
+      runtime_profile()->AddHighWaterMarkCounter("LargestPartitionPercent", TUnit::UNIT);
   if (is_streaming_preagg_) {
-    AddRuntimeExecOption("Streaming Preaggregation");
+    runtime_profile()->AppendExecOption("Streaming Preaggregation");
     streaming_timer_ = ADD_TIMER(runtime_profile(), "StreamingTime");
     num_passthrough_rows_ =
         ADD_COUNTER(runtime_profile(), "RowsPassedThrough", TUnit::UNIT);
@@ -279,11 +279,11 @@ Status PartitionedAggregationNode::Prepare(RuntimeState* state) {
   bool codegen_enabled = false;
   Status codegen_status;
   if (state->codegen_enabled()) {
-    codegen_status = is_streaming_preagg_ ? CodegenProcessBatchStreaming()
-                                          : CodegenProcessBatch();
+    codegen_status =
+        is_streaming_preagg_ ? CodegenProcessBatchStreaming() : CodegenProcessBatch();
     codegen_enabled = codegen_status.ok();
   }
-  AddCodegenExecOption(codegen_enabled, codegen_status);
+  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
   return Status::OK();
 }
 
@@ -891,7 +891,7 @@ Status PartitionedAggregationNode::Partition::Spill() {
 
   COUNTER_ADD(parent->num_spilled_partitions_, 1);
   if (parent->num_spilled_partitions_->value() == 1) {
-    parent->AddRuntimeExecOption("Spilled");
+    parent->runtime_profile()->AppendExecOption("Spilled");
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/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 d3aaf3d..d424fe6 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -234,10 +234,12 @@ Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
       insert_codegen_status = codegen_status;
     }
   }
-  AddCodegenExecOption(build_codegen_enabled, codegen_status, "Build Side");
-  AddCodegenExecOption(probe_codegen_enabled, codegen_status, "Probe Side");
-  AddCodegenExecOption(ht_construction_codegen_enabled, codegen_status,
-      "Hash Table Construction");
+  runtime_profile()->AddCodegenMsg(
+      build_codegen_enabled, codegen_status, "Build Side");
+  runtime_profile()->AddCodegenMsg(
+      probe_codegen_enabled, codegen_status, "Probe Side");
+  runtime_profile()->AddCodegenMsg(
+      ht_construction_codegen_enabled, codegen_status, "Hash Table Construction");
   return Status::OK();
 }
 
@@ -434,7 +436,7 @@ Status PartitionedHashJoinNode::Partition::Spill(bool unpin_all_build) {
   if (!is_spilled_) {
     COUNTER_ADD(parent_->num_spilled_partitions_, 1);
     if (parent_->num_spilled_partitions_->value() == 1) {
-      parent_->AddRuntimeExecOption("Spilled");
+      parent_->runtime_profile()->AppendExecOption("Spilled");
     }
   }
 
@@ -559,13 +561,14 @@ void PartitionedHashJoinNode::PublishRuntimeFilters(RuntimeState* state,
 
   if (filters_.size() > 0) {
     if (num_enabled_filters == filters_.size()) {
-      AddRuntimeExecOption(Substitute("$0 of $0 Runtime Filter$1 Published",
-          filters_.size(), filters_.size() == 1 ? "" : "s"));
+      runtime_profile()->AppendExecOption(
+          Substitute("$0 of $0 Runtime Filter$1 Published", filters_.size(),
+              filters_.size() == 1 ? "" : "s"));
     } else {
       string exec_option = Substitute("$0 of $1 Runtime Filter$2 Published, $3 Disabled",
           num_enabled_filters, filters_.size(), filters_.size() == 1 ? "" : "s",
           filters_.size() - num_enabled_filters);
-      AddRuntimeExecOption(exec_option);
+      runtime_profile()->AppendExecOption(exec_option);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/sort-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/sort-node.cc b/be/src/exec/sort-node.cc
index cfcbd33..9271721 100644
--- a/be/src/exec/sort-node.cc
+++ b/be/src/exec/sort-node.cc
@@ -58,10 +58,9 @@ Status SortNode::Prepare(RuntimeState* state) {
     codegen_status = less_than.Codegen(state);
     codegen_enabled = codegen_status.ok();
   }
-  AddCodegenExecOption(codegen_enabled, codegen_status);
+  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
 
-  sorter_.reset(new Sorter(
-      less_than, sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
+  sorter_.reset(new Sorter(less_than, sort_exec_exprs_.sort_tuple_slot_expr_ctxs(),
       &row_descriptor_, mem_tracker(), runtime_profile(), state));
   RETURN_IF_ERROR(sorter_->Init());
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/exec/topn-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/topn-node.cc b/be/src/exec/topn-node.cc
index 29e4e4d..6dd1e34 100644
--- a/be/src/exec/topn-node.cc
+++ b/be/src/exec/topn-node.cc
@@ -114,9 +114,10 @@ Status TopNNode::Prepare(RuntimeState* state) {
     codegen_status.MergeStatus(Codegen(state));
     codegen_enabled = codegen_status.ok();
   }
-  AddCodegenExecOption(codegen_enabled, codegen_status);
-  priority_queue_.reset(new priority_queue<Tuple*, vector<Tuple*>,
-      ComparatorWrapper<TupleRowComparator>>(*tuple_row_less_than_));
+  runtime_profile()->AddCodegenMsg(codegen_enabled, codegen_status);
+  priority_queue_.reset(
+      new priority_queue<Tuple*, vector<Tuple*>, ComparatorWrapper<TupleRowComparator>>(
+          *tuple_row_less_than_));
   materialized_tuple_desc_ = row_descriptor_.tuple_descriptors()[0];
   insert_batch_timer_ = ADD_TIMER(runtime_profile(), "InsertBatchTime");
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/util/runtime-profile.cc
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index d087616..5134804 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -413,6 +413,15 @@ void RuntimeProfile::GetAllChildren(vector<RuntimeProfile*>* children) {
 }
 
 void RuntimeProfile::AddInfoString(const string& key, const string& value) {
+  return AddInfoStringInternal(key, value, false);
+}
+
+void RuntimeProfile::AppendInfoString(const string& key, const string& value) {
+  return AddInfoStringInternal(key, value, true);
+}
+
+void RuntimeProfile::AddInfoStringInternal(
+    const string& key, const string& value, bool append) {
   // Values may contain sensitive data, such as a query.
   const string& info = RedactCopy(value);
   lock_guard<SpinLock> l(info_strings_lock_);
@@ -421,7 +430,11 @@ void RuntimeProfile::AddInfoString(const string& key, const string& value) {
     info_strings_.insert(make_pair(key, info));
     info_strings_display_order_.push_back(key);
   } else {
-    it->second = info;
+    if (append) {
+      it->second += ", " + value;
+    } else {
+      it->second = info;
+    }
   }
 }
 
@@ -432,22 +445,30 @@ const string* RuntimeProfile::GetInfoString(const string& key) const {
   return &it->second;
 }
 
-#define ADD_COUNTER_IMPL(NAME, T) \
-  RuntimeProfile::T* RuntimeProfile::NAME(\
-      const string& name, TUnit::type unit, const string& parent_counter_name) {\
-    DCHECK_EQ(is_averaged_profile_, false);\
-    lock_guard<SpinLock> l(counter_map_lock_);\
-    if (counter_map_.find(name) != counter_map_.end()) {\
-      return reinterpret_cast<T*>(counter_map_[name]);\
-    }\
-    DCHECK(parent_counter_name == ROOT_COUNTER ||\
-           counter_map_.find(parent_counter_name) != counter_map_.end()); \
-    T* counter = pool_->Add(new T(unit));\
-    counter_map_[name] = counter;\
-    set<string>* child_counters =\
-        FindOrInsert(&child_counter_map_, parent_counter_name, set<string>());\
-    child_counters->insert(name);\
-    return counter;\
+void RuntimeProfile::AddCodegenMsg(
+    bool codegen_enabled, const string& extra_info, const string& extra_label) {
+  string str = codegen_enabled ? "Codegen Enabled" : "Codegen Disabled";
+  if (!extra_info.empty()) str = str + ": " + extra_info;
+  if (!extra_label.empty()) str = extra_label + " " + str;
+  AppendExecOption(str);
+}
+
+#define ADD_COUNTER_IMPL(NAME, T)                                                \
+  RuntimeProfile::T* RuntimeProfile::NAME(                                       \
+      const string& name, TUnit::type unit, const string& parent_counter_name) { \
+    DCHECK_EQ(is_averaged_profile_, false);                                      \
+    lock_guard<SpinLock> l(counter_map_lock_);                                   \
+    if (counter_map_.find(name) != counter_map_.end()) {                         \
+      return reinterpret_cast<T*>(counter_map_[name]);                           \
+    }                                                                            \
+    DCHECK(parent_counter_name == ROOT_COUNTER                                   \
+        || counter_map_.find(parent_counter_name) != counter_map_.end());        \
+    T* counter = pool_->Add(new T(unit));                                        \
+    counter_map_[name] = counter;                                                \
+    set<string>* child_counters =                                                \
+        FindOrInsert(&child_counter_map_, parent_counter_name, set<string>());   \
+    child_counters->insert(name);                                                \
+    return counter;                                                              \
   }
 
 ADD_COUNTER_IMPL(AddCounter, Counter);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7194b280/be/src/util/runtime-profile.h
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index e5e4693..77220d8 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -24,6 +24,7 @@
 #include <iostream>
 
 #include "common/atomic.h"
+#include "common/status.h"
 #include "util/spinlock.h"
 
 #include "gen-cpp/RuntimeProfile_types.h"
@@ -191,6 +192,30 @@ class RuntimeProfile {
   /// the value will be updated.
   void AddInfoString(const std::string& key, const std::string& value);
 
+  /// Adds a string to the runtime profile.  If a value already exists for 'key',
+  /// 'value' will be appended to the previous value, with ", " separating them.
+  void AppendInfoString(const std::string& key, const std::string& value);
+
+  /// Helper to append to the "ExecOption" info string.
+  void AppendExecOption(const std::string& option) {
+    AppendInfoString("ExecOption", option);
+  }
+
+  /// Helper to append "Codegen Enabled" or "Codegen Disabled" exec options. If
+  /// specified, 'extra_info' is appended to the exec option, and 'extra_label'
+  /// is prepended to the exec option.
+  void AddCodegenMsg(bool codegen_enabled, const std::string& extra_info = "",
+      const std::string& extra_label = "");
+
+  /// Helper wrapper for AddCodegenMsg() that takes a status instead of a string
+  /// describing why codegen was disabled. 'codegen_status' can be OK whether or
+  /// not 'codegen_enabled' is true (e.g. if codegen is disabled by a query option,
+  /// then no error occurred).
+  void AddCodegenMsg(bool codegen_enabled, const Status& codegen_status,
+      const std::string& extra_label = "") {
+    AddCodegenMsg(codegen_enabled, codegen_status.GetDetail(), extra_label);
+  }
+
   /// Creates and returns a new EventSequence (owned by the runtime
   /// profile) - unless a timer with the same 'key' already exists, in
   /// which case it is returned.
@@ -391,6 +416,11 @@ class RuntimeProfile {
   /// Called recusively.
   void ComputeTimeInProfile(int64_t total_time);
 
+  /// Implementation of AddInfoString() and AppendInfoString(). If 'append' is false,
+  /// implements AddInfoString(), otherwise implements AppendInfoString().
+  void AddInfoStringInternal(
+      const std::string& key, const std::string& value, bool append);
+
   /// Name of the counter maintaining the total time.
   static const std::string TOTAL_TIME_COUNTER_NAME;
   static const std::string LOCAL_TIME_COUNTER_NAME;


[2/2] incubator-impala git commit: IMPALA-3221: Copyright / license audit

Posted by he...@apache.org.
IMPALA-3221: Copyright / license audit

Populates LICENSE.txt with known third-party licenses in the Impala
codebase.

Change-Id: I24a868aec6a4f17f4ccca1b088d2f0de32f75d87
Reviewed-on: http://gerrit.cloudera.org:8080/3995
Reviewed-by: Henry Robinson <he...@cloudera.com>
Tested-by: Henry Robinson <he...@cloudera.com>


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

Branch: refs/heads/master
Commit: 25fe78291b72054d1447fe4ac089c3d683bcf668
Parents: 7194b28
Author: Henry Robinson <he...@cloudera.com>
Authored: Thu Aug 11 16:20:51 2016 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Wed Sep 7 18:39:27 2016 +0000

----------------------------------------------------------------------
 LICENSE.txt                    | 349 +++++++++++++++++++++++++++++++++++-
 be/src/gutil/LICENSE.txt       |  30 ----
 be/src/runtime/string-search.h |  57 +-----
 3 files changed, 354 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/25fe7829/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index d645695..ba2ce24 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -1,4 +1,3 @@
-
                                  Apache License
                            Version 2.0, January 2004
                         http://www.apache.org/licenses/
@@ -200,3 +199,351 @@
    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.
+
+--------------------------------------------------------------------------------
+
+src/be/gutil (some portions): Apache 2.0, and 3-clause BSD
+
+Some portions of this module are derived from code in the Chromium project,
+copyright (c) Google inc and (c) The Chromium Authors and licensed under the
+Apache 2.0 License or the under the 3-clause BSD license:
+
+  Copyright (c) 2013 The Chromium Authors. All rights reserved.
+
+  Redistribution and use in source and binary forms, with or without modification, are
+  permitted provided that the following conditions are met:
+
+     * Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+
+     * Redistributions in binary form must reproduce the above copyright notice, this list
+       of conditions and the following disclaimer in the documentation and/or other
+       materials provided with the distribution.  * Neither the name of Google Inc. nor
+       the names of its contributors may be used to endorse or promote products derived
+       from this software without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+  EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+  MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+  THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT
+  OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+  INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+  LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+src/be/gutil/utf: licensed under the following terms:
+
+  UTF-8 Library
+
+  The authors of this software are Rob Pike and Ken Thompson.
+      Copyright (c) 1998-2002 by Lucent Technologies.
+
+  Permission to use, copy, modify, and distribute this software for any purpose without
+  fee is hereby granted, provided that this entire notice is included in all copies of any
+  software which is or includes a copy or modification of this software and in all copies
+  of the supporting documentation for such software.  THIS SOFTWARE IS BEING PROVIDED "AS
+  IS", WITHOUT ANY EXPRESS OR IMPLIED WARRANTY.  IN PARTICULAR, NEITHER THE AUTHORS NOR
+  LUCENT TECHNOLOGIES MAKE ANY REPRESENTATION OR WARRANTY OF ANY KIND CONCERNING THE
+  MERCHANTABILITY OF THIS SOFTWARE OR ITS FITNESS FOR ANY PARTICULAR PURPOSE.
+
+--------------------------------------------------------------------------------
+
+www/d3.v3.js: BSD 3-clause license
+
+   Copyright (c) 2012, Michael Bostock
+      All rights reserved.
+
+   Redistribution and use in source and binary forms, with or without
+      modification, are permitted provided that the following conditions are met:
+
+   * Redistributions of source code must retain the above copyright notice, this
+      list of conditions and the following disclaimer.
+
+   * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+         and/or other materials provided with the distribution.
+
+   * The name Michael Bostock may not be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+   EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+   MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+   MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+   CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+   OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
+   AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+   EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+www/dagre-d3-min.js: MIT license
+
+  Copyright (c) 2013 Chris Pettitt
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of this
+  software and associated documentation files (the "Software"), to deal in the Software
+  without restriction, including without limitation the rights to use, copy, modify,
+  merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+  permit persons to whom the Software is furnished to do so, subject to the following
+  conditions:
+
+  The above copyright notice and this permission notice shall be included in all copies or
+  substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+  INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+  PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+  LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT
+  OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+  OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+www/jquery/: MIT license
+
+  Copyright 2005, 2014 jQuery Foundation and other contributors,
+    https://jquery.org/
+
+  This software consists of voluntary contributions made by many individuals. For exact
+  contribution history, see the revision history available at
+  https://github.com/jquery/jquery
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of this
+  software and associated documentation files (the "Software"), to deal in the Software
+  without restriction, including without limitation the rights to use, copy, modify,
+  merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+  permit persons to whom the Software is furnished to do so, subject to the following
+  conditions:
+
+  The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+  INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+  PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+  LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT
+  OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+  OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+www/datatables*: MIT license
+
+  Copyright (C) 2008-2016, SpryMedia Ltd.
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of this
+  software and associated documentation files (the "Software"), to deal in the Software
+  without restriction, including without limitation the rights to use, copy, modify,
+  merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+  permit persons to whom the Software is furnished to do so, subject to the following
+  conditions:
+
+  The above copyright notice and this permission notice shall be included in all copies or
+  substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+  INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+  PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+  LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT
+  OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+  OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+Parts of be/src/runtime/string-search.h: Python Software License V2
+
+  Copyright (c) 2001 - 2016 Python Software Foundation; All Rights Reserved
+
+  PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2
+  --------------------------------------------
+
+  1. This LICENSE AGREEMENT is between the Python Software Foundation ("PSF"), and the
+  Individual or Organization ("Licensee") accessing and otherwise using this software
+  ("Python") in source or binary form and its associated documentation.
+
+  2. Subject to the terms and conditions of this License Agreement, PSF hereby grants
+  Licensee a nonexclusive, royalty-free, world-wide license to reproduce, analyze, test,
+  perform and/or display publicly, prepare derivative works, distribute, and otherwise use
+  Python alone or in any derivative version, provided, however, that PSF's License
+  Agreement and PSF's notice of copyright, i.e., "Copyright (c) 2001, 2002, 2003, 2004,
+  2005, 2006 Python Software Foundation; All Rights Reserved" are retained in Python alone
+  or in any derivative version prepared by Licensee.
+
+  3. In the event Licensee prepares a derivative work that is based on or incorporates
+  Python or any part thereof, and wants to make the derivative work available to others as
+  provided herein, then Licensee hereby agrees to include in any such work a brief summary
+  of the changes made to Python.
+
+  4. PSF is making Python available to Licensee on an "AS IS" basis. PSF MAKES NO
+  REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT
+  LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY
+  OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT INFRINGE ANY
+  THIRD PARTY RIGHTS.
+
+  5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON FOR ANY INCIDENTAL,
+  SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR
+  OTHERWISE USING PYTHON, OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY
+  THEREOF.
+
+  6. This License Agreement will automatically terminate upon a material breach of its
+  terms and conditions.
+
+  7. Nothing in this License Agreement shall be deemed to create any relationship of
+  agency, partnership, or joint venture between PSF and Licensee. This License Agreement
+  does not grant permission to use PSF trademarks or trade name in a trademark sense to
+  endorse or promote products or services of Licensee, or any third party.
+
+  8. By copying, installing or otherwise using Python, Licensee agrees to be bound by the
+  terms and conditions of this License Agreement.
+
+--------------------------------------------------------------------------------
+
+Parts of be/src/util/coding-util.cc: Boost Software License V1.0
+
+  Boost Software License - Version 1.0 - August 17th, 2003
+
+  Permission is hereby granted, free of charge, to any person or organization obtaining a
+  copy of the software and accompanying documentation covered by this license (the
+  "Software") to use, reproduce, display, distribute, execute, and transmit the Software,
+  and to prepare derivative works of the Software, and to permit third-parties to whom the
+  Software is furnished to do so, all subject to the following:
+
+  The copyright notices in the Software and this entire statement, including the above
+  license grant, this restriction and the following disclaimer, must be included in all
+  copies of the Software, in whole or in part, and all derivative works of the Software,
+  unless such copies or derivative works are solely in the form of machine-executable
+  object code generated by a source language processor.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+  INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+  PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR ANYONE
+  DISTRIBUTING THE SOFTWARE BE LIABLE FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN
+  CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR
+  THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+www/bootstrap: MIT license
+tests/comparison/leopard/static/css/bootstrap.*: MIT license
+tests/comparison/leopard/static/js/bootstrap.*: MIT license
+
+  The MIT License (MIT)
+
+  Copyright (c) 2011-2016 Twitter, Inc.
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of this
+  software and associated documentation files (the "Software"), to deal in the Software
+  without restriction, including without limitation the rights to use, copy, modify,
+  merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+  permit persons to whom the Software is furnished to do so, subject to the following
+  conditions:
+
+  The above copyright notice and this permission notice shall be included in all copies or
+  substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+  INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+  PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+  LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT
+  OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+  OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+www/highlight: 3-clause BSD
+
+  Copyright (c) 2006, Ivan Sagalaev
+  All rights reserved.
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are met:
+
+      * Redistributions of source code must retain the above copyright
+        notice, this list of conditions and the following disclaimer.
+      * Redistributions in binary form must reproduce the above copyright
+        notice, this list of conditions and the following disclaimer in the
+        documentation and/or other materials provided with the distribution.
+      * Neither the name of highlight.js nor the names of its contributors
+        may be used to endorse or promote products derived from this software
+        without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY EXPRESS OR
+  IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+  MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+  THE REGENTS AND CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+  EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+  SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+  HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR
+  TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+  SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+shell/ext-py/prettytable-0.7.1: 3-clause BSD
+
+  Copyright (c) 2009-2013 Luke Maurits <lu...@maurits.id.au>
+  All rights reserved.
+  With contributions from:
+   * Chris Clark
+   * Christoph Robbert
+   * Klein Stephane
+   * "maartendb"
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are met:
+
+  * Redistributions of source code must retain the above copyright notice,
+    this list of conditions and the following disclaimer.
+  * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+  * The name of the author may not be used to endorse or promote products
+    derived from this software without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+  EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+  MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+  THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT
+  OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+  INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+  LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+shell/ext-py/sqlparse-0.1.14: 3-clause BSD
+
+  Copyright (c) 2009, Andi Albrecht <al...@gmail.com>
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are met:
+
+  * Redistributions of source code must retain the above copyright notice,
+    this list of conditions and the following disclaimer.
+  * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+  * The name of the author may not be used to endorse or promote products
+    derived from this software without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+  EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+  MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+  THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT
+  OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+  INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+  LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+be/src/thirdparty/squeasel: Apache 2.0 license
+be/src/thirdparty/mustache: Apache 2.0 license
+be/src/expr/hll-bias.h: Apache 2.0 license
+shell/ext-py/sasl-0.1.1: Apache 2.0 license

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/25fe7829/be/src/gutil/LICENSE.txt
----------------------------------------------------------------------
diff --git a/be/src/gutil/LICENSE.txt b/be/src/gutil/LICENSE.txt
deleted file mode 100644
index e1ab103..0000000
--- a/be/src/gutil/LICENSE.txt
+++ /dev/null
@@ -1,30 +0,0 @@
-Some portions  copyright (c) Google inc and (c) The Chromium Authors
-and licensed under the Apache 2.0 License or the under the BSD-style Chromium license:
-
-// Copyright (c) 2013 The Chromium Authors. All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//    * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//    * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//    * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/25fe7829/be/src/runtime/string-search.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/string-search.h b/be/src/runtime/string-search.h
index c12ef50..e1a45e0 100644
--- a/be/src/runtime/string-search.h
+++ b/be/src/runtime/string-search.h
@@ -31,58 +31,13 @@ namespace impala {
 /// TODO: This can be sped up with SIDD_CMP_EQUAL_ORDERED or at the very least rewritten
 /// from published algorithms.
 //
-/// This is taken from the python search string function doing string search (substring)
-/// using an optimized boyer-moore-horspool algorithm.
+/// This is based on the Python search string function doing string search
+/// (substring) using an optimized boyer-moore-horspool algorithm.
+
 /// http://hg.python.org/cpython/file/6b6c79eba944/Objects/stringlib/fastsearch.h
-//
-/// PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2
-/// --------------------------------------------
-//
-/// 1. This LICENSE AGREEMENT is between the Python Software Foundation
-/// ("PSF"), and the Individual or Organization ("Licensee") accessing and
-/// otherwise using this software ("Python") in source or binary form and
-/// its associated documentation.
-//
-/// 2. Subject to the terms and conditions of this License Agreement, PSF
-/// hereby grants Licensee a nonexclusive, royalty-free, world-wide
-/// license to reproduce, analyze, test, perform and/or display publicly,
-/// prepare derivative works, distribute, and otherwise use Python
-/// alone or in any derivative version, provided, however, that PSF's
-/// License Agreement and PSF's notice of copyright, i.e., "Copyright (c)
-/// 2001, 2002, 2003, 2004, 2005, 2006 Python Software Foundation; All Rights
-/// Reserved" are retained in Python alone or in any derivative version
-/// prepared by Licensee.
-//
-/// 3. In the event Licensee prepares a derivative work that is based on
-/// or incorporates Python or any part thereof, and wants to make
-/// the derivative work available to others as provided herein, then
-/// Licensee hereby agrees to include in any such work a brief summary of
-/// the changes made to Python.
-//
-/// 4. PSF is making Python available to Licensee on an "AS IS"
-/// basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR
-/// IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND
-/// DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS
-/// FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT
-/// INFRINGE ANY THIRD PARTY RIGHTS.
-//
-/// 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON
-/// FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS
-/// A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON,
-/// OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF.
-//
-/// 6. This License Agreement will automatically terminate upon a material
-/// breach of its terms and conditions.
-//
-/// 7. Nothing in this License Agreement shall be deemed to create any
-/// relationship of agency, partnership, or joint venture between PSF and
-/// Licensee. This License Agreement does not grant permission to use PSF
-/// trademarks or trade name in a trademark sense to endorse or promote
-/// products or services of Licensee, or any third party.
-//
-/// 8. By copying, installing or otherwise using Python, Licensee
-/// agrees to be bound by the terms and conditions of this License
-/// Agreement.
+///
+/// Changes include using our own Bloom implementation, Impala's native StringValue string
+/// type, and removing other search modes (e.g. FAST_COUNT).
 class StringSearch {
 
  public: