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

[01/50] incubator-impala git commit: IMPALA-3081: increase mem limit (again) for TestWideRow

Repository: incubator-impala
Updated Branches:
  refs/heads/master [created] f915d59aa


IMPALA-3081: increase mem limit (again) for TestWideRow

LZO occasionally needs > 80MB to run this query. We still need to
investigate why it needs so much memory (the scan alone takes > 70MB),
but for now bump the mem limit again.

Change-Id: Ifdb7c6d33ab2de0f06e7322aa8f8ba107da84d49
Reviewed-on: http://gerrit.cloudera.org:8080/2602
Reviewed-by: Jim Apple <jb...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: a75d7dd282bd6bdd3b0b82d389403a80c6732feb
Parents: b393729
Author: Skye Wanderman-Milne <sk...@cloudera.com>
Authored: Tue Mar 22 16:16:18 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 23 05:14:29 2016 +0000

----------------------------------------------------------------------
 tests/query_test/test_scanners.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a75d7dd2/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 9da7078..ca34965 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -142,10 +142,10 @@ class TestWideRow(ImpalaTestSuite):
     # We need > 10 MB of memory because we're creating extra buffers:
     # - 10 MB table / 5 MB scan range = 2 scan ranges, each of which may allocate ~20MB
     # - Sync reads will allocate ~5MB of space
-    # The 80MB value used here was determined empirically by raising the limit until the
+    # The 100MB value used here was determined empirically by raising the limit until the
     # query succeeded for all file formats -- I don't know exactly why we need this much.
     # TODO: figure out exact breakdown of memory usage (IMPALA-681)
-    new_vector.get_value('exec_option')['mem_limit'] = 80 * 1024 * 1024
+    new_vector.get_value('exec_option')['mem_limit'] = 100 * 1024 * 1024
     self.run_test_case('QueryTest/wide-row', new_vector)
 
 class TestWideTable(ImpalaTestSuite):


[23/50] incubator-impala git commit: Update another Kudu test to wait for modifying operations

Posted by ta...@apache.org.
Update another Kudu test to wait for modifying operations

I checked that none of the other .test files contain "insert", "update",
or "delete". This should be the last one. Eventually we should have a
solution that provides a more robust way to "read-your-writes".

Change-Id: I6561c32b1558e0a8685392cc5739e8d2e7900e7c
Reviewed-on: http://gerrit.cloudera.org:8080/2680
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 3d5a21c04737d318501f3fe4c80ff7f4e272631a
Parents: 5cd7ada
Author: Casey Ching <ca...@cloudera.com>
Authored: Thu Mar 31 00:38:41 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 31 12:30:19 2016 +0000

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


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3d5a21c0/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index facd476..d5e4555 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -70,7 +70,8 @@ class TestKuduOperations(ImpalaTestSuite):
     self.cleanup_db("kududb_test")
 
   def test_kudu_scan_node(self, vector):
-    self.run_test_case('QueryTest/kudu-scan-node', vector, use_db="functional_kudu")
+    self.run_test_case('QueryTest/kudu-scan-node', vector, use_db="functional_kudu",
+        wait_secs_between_stmts=1)
 
   @pytest.mark.execute_serially
   def test_insert_update_delete(self, vector):


[29/50] incubator-impala git commit: IMPALA-3283: Disable runtime filter tests for local filesystems

Posted by ta...@apache.org.
IMPALA-3283: Disable runtime filter tests for local filesystems

The runtime filter tests assume 3 scans for alltypes* tables. For local
filesystems this isn't a correct assumption. Fixing the tests to be
resilient to different number of scans is hard, and filters aren't
dependent on the filesystem implementation, so let's just disable them.

Change-Id: Ibcd18c7e69355cde70e13e5190ed2503adb7532b
Reviewed-on: http://gerrit.cloudera.org:8080/2688
Tested-by: Internal Jenkins
Reviewed-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/449901fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/449901fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/449901fa

Branch: refs/heads/master
Commit: 449901fac68beff8d41dc185550d634c0cf4fdb5
Parents: 9b51b2b
Author: Henry Robinson <he...@cloudera.com>
Authored: Thu Mar 31 15:10:22 2016 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Tue Apr 5 17:37:32 2016 +0000

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


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/449901fa/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 2205516..b18ff6c 100644
--- a/tests/query_test/test_runtime_filters.py
+++ b/tests/query_test/test_runtime_filters.py
@@ -17,7 +17,9 @@ import time
 
 from tests.common.test_vector import *
 from tests.common.impala_test_suite import *
+from tests.common.skip import SkipIfLocal
 
+@SkipIfLocal.multiple_impalad
 class TestRuntimeFilters(ImpalaTestSuite):
   @classmethod
   def get_workload(cls):
@@ -43,6 +45,7 @@ class TestRuntimeFilters(ImpalaTestSuite):
       "Query took too long (%ss, possibly waiting for missing filters?)" % str(duration)
 
 
+@SkipIfLocal.multiple_impalad
 class TestRuntimeRowFilters(ImpalaTestSuite):
   @classmethod
   def get_workload(cls):


[27/50] incubator-impala git commit: Log EE XUnit test result in IMPALA_HOME/logs/ee_tests/results.

Posted by ta...@apache.org.
Log EE XUnit test result in IMPALA_HOME/logs/ee_tests/results.

I had missed this in my original logs consolidation patch.

This change is needed for Jenkins to pick up the EE test results
for reporting purposes.

Change-Id: I58e6a4a6392223de87ea2ce50a36dd35cafa5b86
Reviewed-on: http://gerrit.cloudera.org:8080/2667
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 3f2840f52869aacaefe840efbbeb9130fb083541
Parents: 2cbd327
Author: Alex Behm <al...@cloudera.com>
Authored: Wed Mar 30 11:24:14 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Apr 1 12:26:02 2016 +0000

----------------------------------------------------------------------
 tests/run-tests.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f2840f5/tests/run-tests.py
----------------------------------------------------------------------
diff --git a/tests/run-tests.py b/tests/run-tests.py
index b76b4e7..815f83b 100755
--- a/tests/run-tests.py
+++ b/tests/run-tests.py
@@ -27,7 +27,7 @@ VALID_TEST_DIRS = ['failure', 'query_test', 'stress', 'unittests', 'aux_query_te
                    'statestore']
 
 TEST_DIR = os.path.join(os.environ['IMPALA_HOME'], 'tests')
-TEST_RESULT_DIR = os.path.join(TEST_DIR, 'results')
+TEST_RESULT_DIR = os.path.join(os.environ['IMPALA_EE_TEST_LOGS_DIR'], 'results')
 
 # Arguments that control output logging. If additional default arguments are needed they
 # should go in the pytest.ini file.


[15/50] incubator-impala git commit: IMPALA-3241: Send dummy filters for non-partitioned joins

Posted by ta...@apache.org.
IMPALA-3241: Send dummy filters for non-partitioned joins

This is the non-partitioned hash-join counterpart for IMPALA-3141,
sending dummy filters when no real filter will be produced.

Tested manually with PHJ disabled.

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


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

Branch: refs/heads/master
Commit: 943b7cc7a7a5090be139f08fa4712c596870e270
Parents: 49a73cd
Author: Henry Robinson <he...@cloudera.com>
Authored: Mon Mar 28 16:22:23 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 30 03:58:33 2016 +0000

----------------------------------------------------------------------
 be/src/exec/hash-join-node.cc | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/943b7cc7/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 7f8f410..fda90d7 100644
--- a/be/src/exec/hash-join-node.cc
+++ b/be/src/exec/hash-join-node.cc
@@ -24,6 +24,7 @@
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
 #include "util/debug-util.h"
+#include "util/bloom-filter.h"
 #include "util/runtime-profile.h"
 
 #include "gen-cpp/PlanNodes_types.h"
@@ -241,6 +242,11 @@ Status HashJoinNode::ConstructBuildSide(RuntimeState* state) {
       AddRuntimeExecOption("Build-Side Runtime-Filter Disabled (FP Rate Too High)");
       VLOG(2) << "Disabling runtime filter build because build table is too large: "
               << hash_tbl_->size();
+      // Send dummy filters to unblock any waiting scans.
+      BOOST_FOREACH(RuntimeFilter* filter, filters_) {
+        state->filter_bank()->UpdateFilterFromLocal(filter->filter_desc().filter_id,
+            BloomFilter::ALWAYS_TRUE_FILTER);
+      }
     }
   }
   return Status::OK();


[28/50] incubator-impala git commit: IMPALA-2835: introduce PARQUET_FALLBACK_SCHEMA_RESOLUTION query option

Posted by ta...@apache.org.
IMPALA-2835: introduce PARQUET_FALLBACK_SCHEMA_RESOLUTION query option

This patch introduces a new query option,
PARQUET_FALLBACK_SCHEMA_RESOLUTION which allows Parquet files' schemas
to be resolved by either name or position.  It's "fallback" because
eventually field IDs will be the primary schema resolution scheme, and
we don't want to create an option that we will have to change the name
of later. The default is still by position. I chose to do a query
option because it will make testing easier and also be easier to
diagnose resolution problems quickly in the field. If users want to
switch the default behavior to be by name (like Hive), they can use
the --default_query_options flag.

This patch also introduces a new test section, SHELL, which can be
used to execute shell commands in a .test file. This is useful for
copying files into test tables.

Change-Id: Id0c715ea23792b2a6872610839a40532aabbb5a6
Reviewed-on: http://gerrit.cloudera.org:8080/2384
Reviewed-by: Skye Wanderman-Milne <sk...@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/9b51b2b6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/9b51b2b6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/9b51b2b6

Branch: refs/heads/master
Commit: 9b51b2b6e64b29d8af5147945e4b0d2679611350
Parents: 3f2840f
Author: Skye Wanderman-Milne <sk...@cloudera.com>
Authored: Wed Mar 30 16:05:25 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Apr 2 04:04:25 2016 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc             |  75 +++++-
 be/src/exec/hdfs-parquet-scanner.h              |  15 +-
 be/src/service/query-options.cc                 |  19 ++
 be/src/service/query-options.h                  |   5 +-
 common/thrift/ImpalaInternalService.thrift      |  10 +
 common/thrift/ImpalaService.thrift              |   4 +
 testdata/parquet_schema_resolution/README       |  12 +
 .../parquet_schema_resolution/switched_map.avsc |   8 +
 .../parquet_schema_resolution/switched_map.json |   4 +
 .../parquet_schema_resolution/switched_map.parq | Bin 0 -> 586 bytes
 .../QueryTest/parquet-resolution-by-name.test   | 234 +++++++++++++++++++
 tests/common/impala_test_suite.py               |  18 +-
 tests/conftest.py                               |   2 +
 tests/query_test/test_scanners.py               |   5 +
 tests/util/test_file_parser.py                  |   2 +-
 15 files changed, 395 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 5adc97f..ad8e360 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -1976,7 +1976,7 @@ Status HdfsParquetScanner::ResolvePathHelper(ArrayEncoding array_encoding,
   for (int i = 0; i < path.size(); ++i) {
     // Advance '*node' if necessary
     if (i == 0 || col_type->type != TYPE_ARRAY || array_encoding == THREE_LEVEL) {
-      *node = NextSchemaNode(path, i, *node, missing_field);
+      *node = NextSchemaNode(col_type, path, i, *node, missing_field);
       if (*missing_field) return Status::OK();
     } else {
       // We just resolved an array, meaning *node is set to the repeated field of the
@@ -2017,22 +2017,79 @@ Status HdfsParquetScanner::ResolvePathHelper(ArrayEncoding array_encoding,
   return Status::OK();
 }
 
-HdfsParquetScanner::SchemaNode* HdfsParquetScanner::NextSchemaNode(const SchemaPath& path,
-    int next_idx, SchemaNode* node, bool* missing_field) {
+HdfsParquetScanner::SchemaNode* HdfsParquetScanner::NextSchemaNode(
+    const ColumnType* col_type, const SchemaPath& path, int next_idx, SchemaNode* node,
+    bool* missing_field) {
   DCHECK_LT(next_idx, path.size());
-  // The first index in a path includes the table's partition keys
-  int file_idx =
-      next_idx == 0 ? path[next_idx] - scan_node_->num_partition_keys() : path[next_idx];
-  if (node->children.size() <= file_idx) {
-    // The selected field is not in the file
+  if (next_idx != 0) DCHECK(col_type != NULL);
+
+  int file_idx;
+  int table_idx = path[next_idx];
+  bool resolve_by_name = state_->query_options().parquet_fallback_schema_resolution ==
+      TParquetFallbackSchemaResolution::NAME;
+  if (resolve_by_name) {
+    if (next_idx == 0) {
+      // Resolve top-level table column by name.
+      DCHECK_LT(table_idx, scan_node_->hdfs_table()->col_descs().size());
+      const string& name = scan_node_->hdfs_table()->col_descs()[table_idx].name();
+      file_idx = FindChildWithName(node, name);
+    } else if (col_type->type == TYPE_STRUCT) {
+      // Resolve struct field by name.
+      DCHECK_LT(table_idx, col_type->field_names.size());
+      const string& name = col_type->field_names[table_idx];
+      file_idx = FindChildWithName(node, name);
+    } else if (col_type->type == TYPE_ARRAY) {
+      // Arrays have only one child in the file.
+      DCHECK_EQ(table_idx, SchemaPathConstants::ARRAY_ITEM);
+      file_idx = table_idx;
+    } else {
+      DCHECK_EQ(col_type->type, TYPE_MAP);
+      // Maps have two values, "key" and "value". These are supposed to be ordered and may
+      // not have the right field names, but try to resolve by name in case they're
+      // switched and otherwise use the order. See
+      // https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps for
+      // more details.
+      DCHECK(table_idx == SchemaPathConstants::MAP_KEY ||
+             table_idx == SchemaPathConstants::MAP_VALUE);
+      const string& name = table_idx == SchemaPathConstants::MAP_KEY ? "key" : "value";
+      file_idx = FindChildWithName(node, name);
+      if (file_idx >= node->children.size()) {
+        // Couldn't resolve by name, fall back to resolution by position.
+        file_idx = table_idx;
+      }
+    }
+  } else {
+    // Resolution by position.
+    DCHECK_EQ(state_->query_options().parquet_fallback_schema_resolution,
+        TParquetFallbackSchemaResolution::POSITION);
+    if (next_idx == 0) {
+      // For top-level columns, the first index in a path includes the table's partition
+      // keys.
+      file_idx = table_idx - scan_node_->num_partition_keys();
+    } else {
+      file_idx = table_idx;
+    }
+  }
+
+  if (file_idx >= node->children.size()) {
     VLOG_FILE << Substitute(
-        "File '$0' does not contain path '$1'", filename(), PrintPath(path));
+        "File '$0' does not contain path '$1' (resolving by $2)", filename(),
+        PrintPath(path), resolve_by_name ? "name" : "position");
     *missing_field = true;
     return NULL;
   }
   return &node->children[file_idx];
 }
 
+int HdfsParquetScanner::FindChildWithName(HdfsParquetScanner::SchemaNode* node,
+    const string& name) {
+  int idx;
+  for (idx = 0; idx < node->children.size(); ++idx) {
+    if (node->children[idx].element->name == name) break;
+  }
+  return idx;
+}
+
 // There are three types of array encodings:
 //
 // 1. One-level encoding

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/be/src/exec/hdfs-parquet-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.h b/be/src/exec/hdfs-parquet-scanner.h
index 1b97363..c238c6a 100644
--- a/be/src/exec/hdfs-parquet-scanner.h
+++ b/be/src/exec/hdfs-parquet-scanner.h
@@ -418,7 +418,7 @@ class HdfsParquetScanner : public HdfsScanner {
   /// Version of the application that wrote this file.
   FileVersion file_version_;
 
-  /// The root schema node for this file
+  /// The root schema node for this file.
   SchemaNode schema_;
 
   /// Scan range for the metadata.
@@ -590,10 +590,15 @@ class HdfsParquetScanner : public HdfsScanner {
 
   /// Helper functions for ResolvePathHelper().
 
-  /// Advances 'node' to one of its children based on path[next_idx]. Returns the child
-  /// node or sets 'missing_field' to true.
-  SchemaNode* NextSchemaNode(const SchemaPath& path, int next_idx, SchemaNode* node,
-    bool* missing_field);
+  /// Advances 'node' to one of its children based on path[next_idx] and
+  /// 'col_type'. 'col_type' is NULL if 'node' is the root node, otherwise it's the type
+  /// associated with 'node'. Returns the child node or sets 'missing_field' to true.
+  SchemaNode* NextSchemaNode(const ColumnType* col_type, const SchemaPath& path,
+      int next_idx, SchemaNode* node, bool* missing_field);
+
+  /// Returns the index of 'node's child with 'name', or the number of children if not
+  /// found.
+  int FindChildWithName(SchemaNode* node, const string& name);
 
   /// The ResolvePathHelper() logic for arrays.
   Status ResolveArray(ArrayEncoding array_encoding, const SchemaPath& path, int idx,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/be/src/service/query-options.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 2b5b879..00f55a3 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -33,6 +33,7 @@ using boost::algorithm::is_any_of;
 using boost::algorithm::token_compress_on;
 using boost::algorithm::split;
 using boost::algorithm::trim;
+using std::to_string;
 using namespace impala;
 using namespace strings;
 
@@ -100,6 +101,9 @@ int GetQueryOptionForKey(const string& key) {
   return -1;
 }
 
+// Note that we allow numerical values for boolean and enum options. This is because
+// TQueryOptionsToMap() will output the numerical values, and we need to parse its output
+// configuration.
 Status impala::SetQueryOption(const string& key, const string& value,
     TQueryOptions* query_options, QueryOptionsMask* set_query_options_mask) {
   int option = GetQueryOptionForKey(key);
@@ -367,6 +371,21 @@ Status impala::SetQueryOption(const string& key, const string& value,
             iequals(value, "true") || iequals(value, "1"));
         break;
       }
+      case TImpalaQueryOptions::PARQUET_FALLBACK_SCHEMA_RESOLUTION: {
+        if (iequals(value, "position") ||
+            iequals(value, to_string(TParquetFallbackSchemaResolution::POSITION))) {
+          query_options->__set_parquet_fallback_schema_resolution(
+              TParquetFallbackSchemaResolution::POSITION);
+        } else if (iequals(value, "name") ||
+                   iequals(value, to_string(TParquetFallbackSchemaResolution::NAME))) {
+          query_options->__set_parquet_fallback_schema_resolution(
+              TParquetFallbackSchemaResolution::NAME);
+        } else {
+          return Status(Substitute("Invalid PARQUET_FALLBACK_SCHEMA_RESOLUTION option: "
+              "'$0'. Valid options are 'POSITION' and 'NAME'.", value));
+        }
+        break;
+      }
       default:
         // We hit this DCHECK(false) if we forgot to add the corresponding entry here
         // when we add a new query option.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/be/src/service/query-options.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 6964583..c445658 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -32,7 +32,7 @@ class TQueryOptions;
 // the DCHECK.
 #define QUERY_OPTS_TABLE\
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),\
-      TImpalaQueryOptions::PARQUET_ANNOTATE_STRINGS_UTF8 + 1);\
+      TImpalaQueryOptions::PARQUET_FALLBACK_SCHEMA_RESOLUTION + 1);\
   QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED)\
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR)\
   QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)\
@@ -74,7 +74,8 @@ class TQueryOptions;
   QUERY_OPT_FN(runtime_filter_wait_time_ms, RUNTIME_FILTER_WAIT_TIME_MS)\
   QUERY_OPT_FN(disable_row_runtime_filtering, DISABLE_ROW_RUNTIME_FILTERING)\
   QUERY_OPT_FN(max_num_runtime_filters, MAX_NUM_RUNTIME_FILTERS)\
-  QUERY_OPT_FN(parquet_annotate_strings_utf8, PARQUET_ANNOTATE_STRINGS_UTF8);
+  QUERY_OPT_FN(parquet_annotate_strings_utf8, PARQUET_ANNOTATE_STRINGS_UTF8)\
+  QUERY_OPT_FN(parquet_fallback_schema_resolution, PARQUET_FALLBACK_SCHEMA_RESOLUTION);
 
 /// Converts a TQueryOptions struct into a map of key, value pairs.
 void TQueryOptionsToMap(const TQueryOptions& query_options,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/common/thrift/ImpalaInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index d3d3080..2af20e3 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -42,6 +42,11 @@ const i32 INVALID_PLAN_NODE_ID = -1
 // Constant default partition ID, must be < 0 to avoid collisions
 const i64 DEFAULT_PARTITION_ID = -1;
 
+enum TParquetFallbackSchemaResolution {
+  POSITION,
+  NAME
+}
+
 // Query options that correspond to ImpalaService.ImpalaQueryOptions, with their
 // respective defaults. Query options can be set in the following ways:
 //
@@ -170,6 +175,11 @@ struct TQueryOptions {
   // This is disabled by default in order to preserve the existing behavior of legacy
   // workloads. In addition, Impala strings are not necessarily UTF8-encoded.
   42: optional bool parquet_annotate_strings_utf8 = false
+
+  // Determines how to resolve Parquet files' schemas in the absence of field IDs (which
+  // is always, since fields IDs are NYI). Valid values are "position" (default) and
+  // "name".
+  43: optional TParquetFallbackSchemaResolution parquet_fallback_schema_resolution = 0
 }
 
 // Impala currently has two types of sessions: Beeswax and HiveServer2

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/common/thrift/ImpalaService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index cbeaa65..3f5273e 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -205,6 +205,10 @@ enum TImpalaQueryOptions {
   // If true, use UTF-8 annotation for string columns. Note that char and varchar columns
   // always use the annotation.
   PARQUET_ANNOTATE_STRINGS_UTF8
+
+  // Determines how to resolve Parquet files' schemas in the absence of field IDs (which
+  // is always, since fields IDs are NYI). Valid values are "position" and "name".
+  PARQUET_FALLBACK_SCHEMA_RESOLUTION
 }
 
 // The summary of an insert.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/testdata/parquet_schema_resolution/README
----------------------------------------------------------------------
diff --git a/testdata/parquet_schema_resolution/README b/testdata/parquet_schema_resolution/README
new file mode 100644
index 0000000..840067f
--- /dev/null
+++ b/testdata/parquet_schema_resolution/README
@@ -0,0 +1,12 @@
+switched_map.parq was generated by modifying parquet-mr to switch the key and value fields
+of map, and then converting switched_map.json to parquet using switched_map.avsc as the
+schema. switched_map.parq has the following schema according to parquet-tools:
+
+message com.cloudera.impala.switched_map {
+  required group int_map (MAP) {
+    repeated group map (MAP_KEY_VALUE) {
+      required int32 value;
+      required binary key (UTF8);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/testdata/parquet_schema_resolution/switched_map.avsc
----------------------------------------------------------------------
diff --git a/testdata/parquet_schema_resolution/switched_map.avsc b/testdata/parquet_schema_resolution/switched_map.avsc
new file mode 100644
index 0000000..8805261
--- /dev/null
+++ b/testdata/parquet_schema_resolution/switched_map.avsc
@@ -0,0 +1,8 @@
+
+{"type": "record",
+ "namespace": "com.cloudera.impala",
+ "name": "switched_map",
+ "fields": [
+     {"name": "int_map", "type": {"type": "map", "values": "int"}}
+ ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/testdata/parquet_schema_resolution/switched_map.json
----------------------------------------------------------------------
diff --git a/testdata/parquet_schema_resolution/switched_map.json b/testdata/parquet_schema_resolution/switched_map.json
new file mode 100644
index 0000000..d6cac13
--- /dev/null
+++ b/testdata/parquet_schema_resolution/switched_map.json
@@ -0,0 +1,4 @@
+[
+  {"int_map": {"a": 1, "b": 2}},
+  {"int_map": {"c": 3}}
+]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/testdata/parquet_schema_resolution/switched_map.parq
----------------------------------------------------------------------
diff --git a/testdata/parquet_schema_resolution/switched_map.parq b/testdata/parquet_schema_resolution/switched_map.parq
new file mode 100644
index 0000000..9306145
Binary files /dev/null and b/testdata/parquet_schema_resolution/switched_map.parq differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/testdata/workloads/functional-query/queries/QueryTest/parquet-resolution-by-name.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/parquet-resolution-by-name.test b/testdata/workloads/functional-query/queries/QueryTest/parquet-resolution-by-name.test
new file mode 100644
index 0000000..c81c468
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/parquet-resolution-by-name.test
@@ -0,0 +1,234 @@
+====
+---- QUERY
+# Create a table and populate with data file
+drop table if exists resolution_by_name_test;
+create table resolution_by_name_test stored as parquet
+as select * from functional_parquet.tinytable;
+select a, b from resolution_by_name_test;
+---- TYPES
+string,string
+---- RESULTS
+'aaaaaaa','bbbbbbb'
+'ccccc','dddd'
+'eeeeeeee','f'
+====
+---- QUERY
+# Rearrange the columns and make sure we can still resolve by name
+alter table resolution_by_name_test replace columns (b string, a string);
+set parquet_fallback_schema_resolution="NAME";
+select a, b from resolution_by_name_test;
+---- TYPES
+string,string
+---- RESULTS
+'aaaaaaa','bbbbbbb'
+'ccccc','dddd'
+'eeeeeeee','f'
+====
+---- QUERY
+# Renaming a column will cause the column to not be resolved
+alter table resolution_by_name_test change a new_a string;
+select new_a from resolution_by_name_test;
+---- TYPES
+string
+---- RESULTS
+'NULL'
+'NULL'
+'NULL'
+====
+---- QUERY
+# Can still resolve by ordinal
+set parquet_fallback_schema_resolution="POSITION";
+select b, new_a from resolution_by_name_test;
+---- TYPES
+string,string
+---- RESULTS
+'aaaaaaa','bbbbbbb'
+'ccccc','dddd'
+'eeeeeeee','f'
+====
+---- QUERY
+# Check that we can parse the integer enum value as well
+set parquet_fallback_schema_resolution=1;
+select new_a from resolution_by_name_test;
+---- TYPES
+string
+---- RESULTS
+'NULL'
+'NULL'
+'NULL'
+====
+---- QUERY
+set parquet_fallback_schema_resolution=0;
+select b, new_a from resolution_by_name_test;
+---- TYPES
+string,string
+---- RESULTS
+'aaaaaaa','bbbbbbb'
+'ccccc','dddd'
+'eeeeeeee','f'
+====
+---- QUERY
+drop table resolution_by_name_test;
+====
+---- QUERY
+# Test nested types resolution
+drop table if exists nested_resolution_by_name_test;
+create table nested_resolution_by_name_test like functional_parquet.complextypestbl;
+====
+---- SHELL
+hadoop fs -cp $FILESYSTEM_PREFIX/test-warehouse/complextypestbl_parquet/nullable.parq \
+$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/nested_resolution_by_name_test/
+hadoop fs -cp $FILESYSTEM_PREFIX/test-warehouse/complextypestbl_parquet/nonnullable.parq \
+$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/nested_resolution_by_name_test/
+====
+---- QUERY
+select id, nested_struct.a, b.item
+from nested_resolution_by_name_test t, t.nested_struct.b
+---- TYPES
+bigint,int,int
+---- RESULTS
+1,1,1
+2,NULL,NULL
+7,7,2
+7,7,3
+7,7,NULL
+8,-1,-1
+====
+---- QUERY
+# Can safely ignore extra fields in nested_struct
+alter table nested_resolution_by_name_test change nested_struct nested_struct
+struct<a:int, b: array<int>>;
+select id, nested_struct.a, b.item
+from nested_resolution_by_name_test t, t.nested_struct.b
+---- TYPES
+bigint,int,int
+---- RESULTS
+1,1,1
+2,NULL,NULL
+7,7,2
+7,7,3
+7,7,NULL
+8,-1,-1
+====
+---- QUERY
+# Rearrange nested_struct's fields and make sure we can still resolve by name
+alter table nested_resolution_by_name_test change nested_struct nested_struct
+struct<b: array<int>, a: int>;
+set parquet_fallback_schema_resolution="name";
+select id, nested_struct.a, b.item
+from nested_resolution_by_name_test t, t.nested_struct.b
+---- TYPES
+bigint,int,int
+---- RESULTS
+1,1,1
+2,NULL,NULL
+7,7,2
+7,7,3
+7,7,NULL
+8,-1,-1
+====
+---- QUERY
+# Can add back a single field
+alter table nested_resolution_by_name_test change nested_struct nested_struct
+struct<b: array<int>, a: int, g: map<string, struct<h: struct<i: array<float>>>>>;
+select id, g.key
+from nested_resolution_by_name_test t, t.nested_struct.g
+---- TYPES
+bigint,string
+---- RESULTS
+1,'foo'
+2,'g1'
+2,'g2'
+2,'g3'
+2,'g4'
+2,'g5'
+5,'foo'
+====
+---- QUERY
+# Add back single more nested field (and remove 'g' field)
+alter table nested_resolution_by_name_test change nested_struct nested_struct
+struct<b: array<int>, a: int, c: struct<d: array<array<struct<f: string>>>>>;
+select tmp.f from nested_resolution_by_name_test.nested_struct.c.d.item tmp;
+---- TYPES
+string
+---- RESULTS
+'aaa'
+'bbb'
+'c'
+'NULL'
+'aaa'
+'NULL'
+'bbb'
+'NULL'
+'c'
+'NULL'
+'NULL'
+'nonnullable'
+====
+---- QUERY
+# Can't rename nested field
+alter table nested_resolution_by_name_test change nested_struct nested_struct
+struct<b: array<int>, a: int, c: struct<d: array<array<struct<renamed: string>>>>>;
+select tmp.renamed from nested_resolution_by_name_test.nested_struct.c.d.item tmp;
+---- TYPES
+string
+---- RESULTS
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+'NULL'
+====
+---- QUERY
+drop table nested_resolution_by_name_test;
+====
+---- QUERY
+# Test switched key/value map fields
+drop table if exists switched_map_fields_resolution_test;
+create table switched_map_fields_resolution_test (int_map map<string,int>)
+stored as parquet;
+====
+---- SHELL
+hadoop fs -copyFromLocal \
+$IMPALA_HOME/testdata/parquet_schema_resolution/switched_map.parq \
+$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/switched_map_fields_resolution_test/
+====
+---- QUERY
+# Switched map fields should be resolvable by name.
+set parquet_fallback_schema_resolution="name";
+select key, value from switched_map_fields_resolution_test.int_map
+---- TYPES
+string,int
+---- RESULTS
+'a',1
+'b',2
+'c',3
+====
+---- QUERY
+# Can't resolve switched map fields by position since types are switched.
+set parquet_fallback_schema_resolution="position";
+select key, value from switched_map_fields_resolution_test.int_map
+---- CATCH
+File '$NAMENODE/test-warehouse/$DATABASE.db/switched_map_fields_resolution_test/
+switched_map.parq' has an incompatible Parquet schema for column
+ '$DATABASE.switched_map_fields_resolution_test.int_map.key'.
+ Column type: STRING, Parquet schema:
+required int32 value [i:0 d:1 r:1]
+====
+---- QUERY
+drop table switched_map_fields_resolution_test
+====
+---- QUERY
+# Check that we handle bad options gracefully
+set parquet_fallback_schema_resolution="FOO"
+---- CATCH
+Invalid PARQUET_FALLBACK_SCHEMA_RESOLUTION option: 'FOO'.
+ Valid options are 'POSITION' and 'NAME'.
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/tests/common/impala_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index 8f8bcce..1bacc16 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -27,6 +27,7 @@ from getpass import getuser
 from functools import wraps
 from impala._thrift_gen.ImpalaService.ttypes import TImpalaQueryOptions
 from random import choice
+from subprocess import check_call
 from tests.common.impala_service import ImpaladService
 from tests.common.impala_connection import ImpalaConnection, create_connection
 from tests.common.test_dimensions import *
@@ -221,6 +222,17 @@ class ImpalaTestSuite(BaseTestSuite):
     sections = self.load_query_test_file(self.get_workload(), test_file_name,
         encoding=encoding)
     for test_section in sections:
+      if 'SHELL' in test_section:
+        assert len(test_section) == 1, \
+          "SHELL test sections can't contain other sections"
+        cmd = test_section['SHELL']\
+          .replace('$FILESYSTEM_PREFIX', FILESYSTEM_PREFIX)\
+          .replace('$IMPALA_HOME', IMPALA_HOME)
+        if use_db: cmd = cmd.replace('$DATABASE', use_db)
+        LOG.info("Shell command: " + cmd)
+        check_call(cmd, shell=True)
+        continue
+
       if 'QUERY' not in test_section:
         assert 0, 'Error in test file %s. Test cases require a -- QUERY section.\n%s' %\
             (test_file_name, pprint.pformat(test_section))
@@ -265,7 +277,11 @@ class ImpalaTestSuite(BaseTestSuite):
               .replace('$FILESYSTEM_PREFIX', FILESYSTEM_PREFIX) \
               .replace('$NAMENODE', NAMENODE) \
               .replace('$IMPALA_HOME', IMPALA_HOME)
-          assert expected_str in str(e)
+          if use_db: expected_str = expected_str.replace('$DATABASE', use_db)
+          # Strip newlines so we can split error message into multiple lines
+          expected_str = expected_str.replace('\n', '')
+          actual_str = str(e).replace('\n', '')
+          assert expected_str in actual_str
           continue
         raise
       finally:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/tests/conftest.py
----------------------------------------------------------------------
diff --git a/tests/conftest.py b/tests/conftest.py
index a9329f3..c6a22c0 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -219,6 +219,8 @@ def unique_database(request, testid_checksum):
                      'characters.'.format(db_name))
 
   def cleanup():
+    # Make sure we don't try to drop the current session database
+    request.instance.execute_query_expect_success(request.instance.client, "use default")
     request.instance.execute_query_expect_success(
         request.instance.client, 'DROP DATABASE `{0}` CASCADE'.format(db_name))
     LOG.info('Dropped database "{0}" for test ID "{1}"'.format(db_name,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 7a4290f..7768a40 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -336,6 +336,11 @@ class TestParquet(ImpalaTestSuite):
     assert c_schema_elt.converted_type == ConvertedType.UTF8
     assert d_schema_elt.converted_type == None
 
+  @SkipIfS3.insert
+  def test_resolution_by_name(self, unique_database, vector):
+    self.run_test_case('QueryTest/parquet-resolution-by-name', vector,
+                       use_db=unique_database)
+
 # We use various scan range lengths to exercise corner cases in the HDFS scanner more
 # thoroughly. In particular, it will exercise:
 # 1. default scan range

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9b51b2b6/tests/util/test_file_parser.py
----------------------------------------------------------------------
diff --git a/tests/util/test_file_parser.py b/tests/util/test_file_parser.py
index 57105b2..f608aa7 100644
--- a/tests/util/test_file_parser.py
+++ b/tests/util/test_file_parser.py
@@ -78,7 +78,7 @@ def parse_query_test_file(file_name, valid_section_names=None, encoding=None):
   section_names = valid_section_names
   if section_names is None:
     section_names = ['QUERY', 'RESULTS', 'TYPES', 'LABELS', 'SETUP', 'CATCH', 'ERRORS',
-                     'USER', 'RUNTIME_PROFILE']
+        'USER', 'RUNTIME_PROFILE', 'SHELL']
   return parse_test_file(file_name, section_names, encoding=encoding,
       skip_unknown_sections=False)
 


[10/50] incubator-impala git commit: IMPALA-3243: Delete unneeded code that doesn't work on RHEL 5

Posted by ta...@apache.org.
IMPALA-3243: Delete unneeded code that doesn't work on RHEL 5

The MaxCPUIndex() interface was added by Kudu, but the code that
initializes it doesn't work on RHEL 5. Impala doesn't depend on
MaxCpuIndex() at all, but the value is calculated and cached at startup
time so it still executes.  So, let's just delete this effectively
dead code that doesn't work on RHEL 5.

Change-Id: I82b71de60e5bf8abec18f981d13528991265908f
Reviewed-on: http://gerrit.cloudera.org:8080/2626
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 5e51f1cd0366bfd590ade2a9c54f0bab942c79ac
Parents: 7e76e92
Author: Dan Hecht <dh...@cloudera.com>
Authored: Thu Mar 24 20:35:48 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Mar 28 20:19:41 2016 +0000

----------------------------------------------------------------------
 be/src/gutil/sysinfo.cc | 33 ---------------------------------
 be/src/gutil/sysinfo.h  |  4 ----
 2 files changed, 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5e51f1cd/be/src/gutil/sysinfo.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/sysinfo.cc b/be/src/gutil/sysinfo.cc
index 60c461b..f9232d1 100644
--- a/be/src/gutil/sysinfo.cc
+++ b/be/src/gutil/sysinfo.cc
@@ -74,7 +74,6 @@ namespace base {
 
 static double cpuinfo_cycles_per_second = 1.0;  // 0.0 might be dangerous
 static int cpuinfo_num_cpus = 1;  // Conservative guess
-static int cpuinfo_max_cpu_index = -1;
 
 void SleepForNanoseconds(int64_t nanoseconds) {
   // Sleep for nanosecond duration
@@ -145,25 +144,6 @@ static bool ReadIntFromFile(const char *file, int *value) {
   return false;
 }
 
-static int ReadMaxCPUIndex() {
-  char buf[1024];
-  CHECK(SlurpSmallTextFile("/sys/devices/system/cpu/present", buf, arraysize(buf)));
-
-  // On a single-core machine, 'buf' will contain the string '0' with a newline.
-  if (strcmp(buf, "0\n") == 0) {
-    return 0;
-  }
-
-  // On multi-core, it will have a CPU range like '0-7'.
-  CHECK_EQ(0, memcmp(buf, "0-", 2)) << "bad list of possible CPUs: " << buf;
-
-  char* max_cpu_str = &buf[2];
-  char* err;
-  int val = strtol(max_cpu_str, &err, 10);
-  CHECK(*err == '\n' || *err == '\0') << "unable to parse max CPU index from: " << buf;
-  return val;
-}
-
 #endif
 
 // WARNING: logging calls back to InitializeSystemInfo() so it must
@@ -306,7 +286,6 @@ static void InitializeSystemInfo() {
   if (num_cpus > 0) {
     cpuinfo_num_cpus = num_cpus;
   }
-  cpuinfo_max_cpu_index = ReadMaxCPUIndex();
 
 #elif defined __FreeBSD__
   // For this sysctl to work, the machine must be configured without
@@ -385,13 +364,6 @@ static void InitializeSystemInfo() {
   // Generic cycles per second counter
   cpuinfo_cycles_per_second = EstimateCyclesPerSecond(1000);
 #endif
-
-  // On platforms where we can't determine the max CPU index, just use the
-  // number of CPUs. This might break if CPUs are taken offline, but
-  // better than a wild guess.
-  if (cpuinfo_max_cpu_index < 0) {
-    cpuinfo_max_cpu_index = cpuinfo_num_cpus - 1;
-  }
 }
 
 double CyclesPerSecond(void) {
@@ -404,9 +376,4 @@ int NumCPUs(void) {
   return cpuinfo_num_cpus;
 }
 
-int MaxCPUIndex(void) {
-  InitializeSystemInfo();
-  return cpuinfo_max_cpu_index;
-}
-
 } // namespace base

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5e51f1cd/be/src/gutil/sysinfo.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/sysinfo.h b/be/src/gutil/sysinfo.h
index ec3abe7..1ce95d0 100644
--- a/be/src/gutil/sysinfo.h
+++ b/be/src/gutil/sysinfo.h
@@ -41,10 +41,6 @@ namespace base {
 // value of sched_getcpu().
 extern int NumCPUs();
 
-// Return the maximum CPU index that may be returned by sched_getcpu(). For example, on
-// an 8-core machine, this will return '7' even if some of the CPUs have been disabled.
-extern int MaxCPUIndex();
-
 void SleepForNanoseconds(int64_t nanoseconds);
 void SleepForMilliseconds(int64_t milliseconds);
 


[42/50] incubator-impala git commit: IMPALA-3269: Remove authz checks on default table location in CTAS queries

Posted by ta...@apache.org.
IMPALA-3269: Remove authz checks on default table location in CTAS queries

Bug: In CreateTableAsSelectStmt.analyze(), we set the default location
of table if the query doesn't explicitly set a table location. However
this is an issue with CTAS with subqueries as they follow a two pass
analysis with the second analyze() call failing with a authz error on
the URI added in the first pass.

Fix: We needn't set the default location explicitly. Metastore
automatically figures it out if it is not set.

Change-Id: I0451586d4994cf9fc8c3dd47c8f3a513067cb2ea
Reviewed-on: http://gerrit.cloudera.org:8080/2664
Reviewed-by: Bharath Vissapragada <bh...@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/49f9559f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/49f9559f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/49f9559f

Branch: refs/heads/master
Commit: 49f9559f204278e39dcd362786adcc7b2f950d9e
Parents: 62bb864
Author: Bharath Vissapragada <bh...@cloudera.com>
Authored: Mon Mar 28 23:37:20 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 .../impala/analysis/CreateTableAsSelectStmt.java        |  7 -------
 .../impala/analysis/CreateTableLikeFileStmt.java        |  2 +-
 .../java/com/cloudera/impala/analysis/ToSqlUtils.java   | 12 ++++++------
 .../com/cloudera/impala/analysis/AuthorizationTest.java |  4 ++++
 .../java/com/cloudera/impala/analysis/ToSqlTest.java    |  4 ++--
 5 files changed, 13 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49f9559f/fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
index 640f92c..15c8a37 100644
--- a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
+++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
@@ -180,13 +180,6 @@ public class CreateTableAsSelectStmt extends StatementBase {
       // user specified a location for the table this will be a no-op.
       msTbl.getSd().setLocation(analyzer.getCatalog().getTablePath(msTbl).toString());
 
-      // If the user didn't specify a table location for the CREATE statement, inject the
-      // location that was calculated in the getTablePath() call. Since this will be the
-      // target location for the INSERT statement, it is important the two match.
-      if (createStmt_.getLocation() == null) {
-        createStmt_.setLocation(new HdfsUri(msTbl.getSd().getLocation()));
-      }
-
       // Create a "temp" table based off the given metastore.api.Table object. Normally,
       // the CatalogService assigns all table IDs, but in this case we need to assign the
       // "temp" table an ID locally. This table ID cannot conflict with any table in the

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49f9559f/fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
index 54cfec0..7676d98 100644
--- a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
+++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
@@ -350,7 +350,7 @@ public class CreateTableLikeFileStmt extends CreateTableStmt {
         getTbl() + " __LIKE_FILEFORMAT__ ",  getComment(), colsSql, partitionColsSql,
         getTblProperties(), getSerdeProperties(), isExternal(), getIfNotExists(),
         getRowFormat(), HdfsFileFormat.fromThrift(getFileFormat()),
-        compression, null, getLocation().toString());
+        compression, null, getLocation());
     s = s.replace("__LIKE_FILEFORMAT__", "LIKE " + schemaFileFormat_ + " " +
         schemaLocation_.toString());
     return s;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49f9559f/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java b/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
index 0b95536..d597266 100644
--- a/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
@@ -127,13 +127,12 @@ public class ToSqlUtils {
     for (ColumnDef col: stmt.getPartitionColumnDefs()) {
       partitionColsSql.add(col.toString());
     }
-    String location = stmt.getLocation() == null ? null : stmt.getLocation().toString();
     // TODO: Pass the correct compression, if applicable.
     return getCreateTableSql(stmt.getDb(), stmt.getTbl(), stmt.getComment(), colsSql,
         partitionColsSql, stmt.getTblProperties(), stmt.getSerdeProperties(),
         stmt.isExternal(), stmt.getIfNotExists(), stmt.getRowFormat(),
         HdfsFileFormat.fromThrift(stmt.getFileFormat()), HdfsCompression.NONE, null,
-        location);
+        stmt.getLocation());
   }
 
   /**
@@ -154,7 +153,7 @@ public class ToSqlUtils {
         innerStmt.getSerdeProperties(), innerStmt.isExternal(),
         innerStmt.getIfNotExists(), innerStmt.getRowFormat(),
         HdfsFileFormat.fromThrift(innerStmt.getFileFormat()), HdfsCompression.NONE, null,
-        innerStmt.getLocation().toString());
+        innerStmt.getLocation());
     return createTableSql + " AS " + stmt.getQueryStmt().toSql();
   }
 
@@ -199,9 +198,10 @@ public class ToSqlUtils {
       // Kudu tables cannot use the Hive DDL syntax for the storage handler
       storageHandlerClassName = null;
     }
+    HdfsUri tableLocation = location == null ? null : new HdfsUri(location);
     return getCreateTableSql(table.getDb().getName(), table.getName(), comment, colsSql,
         partitionColsSql, properties, serdeParameters, isExternal, false, rowFormat,
-        format, compression, storageHandlerClassName, location);
+        format, compression, storageHandlerClassName, tableLocation);
   }
 
   /**
@@ -214,7 +214,7 @@ public class ToSqlUtils {
       Map<String, String> tblProperties, Map<String, String> serdeParameters,
       boolean isExternal, boolean ifNotExists, RowFormat rowFormat,
       HdfsFileFormat fileFormat, HdfsCompression compression, String storageHandlerClass,
-      String location) {
+      HdfsUri location) {
     Preconditions.checkNotNull(tableName);
     StringBuilder sb = new StringBuilder("CREATE ");
     if (isExternal) sb.append("EXTERNAL ");
@@ -279,7 +279,7 @@ public class ToSqlUtils {
       }
     }
     if (location != null) {
-      sb.append("LOCATION '" + location + "'\n");
+      sb.append("LOCATION '" + location.toString() + "'\n");
     }
     if (tblProperties != null && !tblProperties.isEmpty()) {
       sb.append("TBLPROPERTIES " + propertyMapToSql(tblProperties));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49f9559f/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java b/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java
index b456888..499fdf7 100644
--- a/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java
@@ -822,6 +822,10 @@ public class AuthorizationTest {
         "User '%s' does not have privileges to execute 'SELECT' on: " +
         "functional.alltypes");
 
+    // CTAS with a subquery.
+    AuthzOk("create table tpch.new_table as select * from functional.alltypesagg " +
+        "where id < (select max(year) from functional.alltypesagg)");
+
     AuthzError("create table functional.tbl tblproperties('a'='b')" +
         " as select 1",
         "User '%s' does not have privileges to execute 'CREATE' on: " +

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49f9559f/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java b/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java
index 744a8d2..521b19f 100644
--- a/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java
@@ -304,8 +304,8 @@ public class ToSqlTest extends AnalyzerTest {
         "select double_col, int_col from functional.alltypes",
         "default",
         "CREATE TABLE default.p PARTITIONED BY ( int_col ) STORED AS " +
-        "TEXTFILE LOCATION 'hdfs://localhost:20500/test-warehouse/p' " +
-        "AS SELECT double_col, int_col FROM functional.alltypes", true);
+        "TEXTFILE AS SELECT double_col, int_col FROM functional.alltypes",
+        true);
   }
 
   @Test


[18/50] incubator-impala git commit: IMPALA-3072: Allow frame embedding for webserver pages

Posted by ta...@apache.org.
IMPALA-3072: Allow frame embedding for webserver pages

Check that 'X-Frame-Options' HTTP header is set in a webserver replies.
Allow changing the value of the header and test that the value is
changed as server configuration is changed.

Change-Id: I091f00ce62f4ffc58c04459241aeb7a31e104bb4
Reviewed-on: http://gerrit.cloudera.org:8080/2299
Reviewed-by: Henry Robinson <he...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 97b523389e7604183c23574842bd6c35385aa7f5
Parents: 2d0b944
Author: oxpa <ii...@gmail.com>
Authored: Thu Feb 25 09:17:57 2016 +0300
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 30 09:25:33 2016 +0000

----------------------------------------------------------------------
 be/src/util/webserver-test.cc | 22 +++++++++++++++++++---
 be/src/util/webserver.cc      |  8 ++++++--
 2 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/97b52338/be/src/util/webserver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index d119cc1..392334b 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -27,6 +27,7 @@ DECLARE_string(webserver_password_file);
 DECLARE_string(webserver_certificate_file);
 DECLARE_string(webserver_private_key_file);
 DECLARE_string(webserver_private_key_password_cmd);
+DECLARE_string(webserver_x_frame_options);
 
 #include "common/names.h"
 
@@ -296,14 +297,29 @@ TEST(Webserver, NoFrameEmbeddingTest) {
   const string FRAME_TEST_PATH = "/frames_test";
   Webserver webserver(FLAGS_webserver_port);
   Webserver::UrlCallback callback = bind<void>(FrameCallback, _1, _2);
-  webserver.RegisterUrlCallback(FRAME_TEST_PATH, "raw-text.tmpl", callback);
+  webserver.RegisterUrlCallback(FRAME_TEST_PATH, "raw_text.tmpl", callback);
   ASSERT_OK(webserver.Start());
   stringstream contents;
   ASSERT_OK(HttpGet("localhost", FLAGS_webserver_port,
       FRAME_TEST_PATH, &contents, 200));
 
-  // Confirm that the embedded frame isn't rendered
-  ASSERT_TRUE(contents.str().find("Metrics") == string::npos);
+  // Confirm that there is an HTTP header to deny framing
+  ASSERT_FALSE(contents.str().find("X-Frame-Options: DENY") == string::npos);
+}
+TEST(Webserver, FrameAllowEmbeddingTest) {
+  const string FRAME_TEST_PATH = "/frames_test";
+  ScopedFlagSetter<string> webserver_x_frame_options(&FLAGS_webserver_x_frame_options,
+      "ALLOWALL");
+  Webserver webserver(FLAGS_webserver_port);
+  Webserver::UrlCallback callback = bind<void>(FrameCallback, _1, _2);
+  webserver.RegisterUrlCallback(FRAME_TEST_PATH, "raw_text.tmpl", callback);
+  ASSERT_OK(webserver.Start());
+  stringstream contents;
+  ASSERT_OK(HttpGet("localhost", FLAGS_webserver_port,
+      FRAME_TEST_PATH, &contents, 200));
+
+  // Confirm that there is an HTTP header to allow framing
+  ASSERT_FALSE(contents.str().find("X-Frame-Options: ALLOWALL") == string::npos);
 }
 
 const string STRING_WITH_NULL = "123456789\0ABCDE";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/97b52338/be/src/util/webserver.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 8eb05ae..106e37a 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -92,6 +92,9 @@ DEFINE_string(webserver_password_file, "",
     "(Optional) Location of .htpasswd file containing user names and hashed passwords for"
     " debug webserver authentication");
 
+DEFINE_string(webserver_x_frame_options, "DENY", 
+    "webserver will add X-Frame-Options HTTP header with this value");
+
 static const char* DOC_FOLDER = "/www/";
 static const int DOC_FOLDER_LEN = strlen(DOC_FOLDER);
 
@@ -138,11 +141,12 @@ string BuildHeaderString(ResponseCode response, ContentType content_type) {
   static const string RESPONSE_TEMPLATE = "HTTP/1.1 $0 $1\r\n"
       "Content-Type: text/$2\r\n"
       "Content-Length: %d\r\n"
-      "X-Frame-Options: DENY\r\n"
+      "X-Frame-Options: $3\r\n"
       "\r\n";
 
   return Substitute(RESPONSE_TEMPLATE, response, response == OK ? "OK" : "Not found",
-      content_type == HTML ? "html" : "plain");
+      content_type == HTML ? "html" : "plain",
+      FLAGS_webserver_x_frame_options.c_str());
 }
 
 Webserver::Webserver()


[07/50] incubator-impala git commit: IMPALA-2466: Add more tests for the HDFS parquet scanner.

Posted by ta...@apache.org.
IMPALA-2466: Add more tests for the HDFS parquet scanner.

These tests functionally test whether the following type of files
are able to be scanned properly:

1) Add a parquet file with multiple blocks such that each node has to
   scan multiple blocks.
2) Add a parquet file with multiple blocks but only one row group
   that spans the entire file. Only one scan range should do any work
   in this case.

Change-Id: I4faccd9ce3fad42402652c8f17d4e7aa3d593368
Reviewed-on: http://gerrit.cloudera.org:8080/1500
Reviewed-by: Sailesh Mukil <sa...@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/76b67485
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/76b67485
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/76b67485

Branch: refs/heads/master
Commit: 76b674850fa1db36ab6e6acda6d3f958661eefbc
Parents: 0d1eab7
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Fri Oct 2 18:42:34 2015 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Mar 25 13:10:15 2016 +0000

----------------------------------------------------------------------
 testdata/LineItemMultiBlock/README.dox          |   9 ++-
 .../lineitem_one_row_group.parquet              | Bin 0 -> 3144593 bytes
 .../lineitem_sixblocks.parquet                  | Bin 0 -> 5539024 bytes
 testdata/bin/create-load-data.sh                |  12 ++++
 .../functional/functional_schema_template.sql   |  48 ++++++++++++++-
 .../datasets/functional/schema_constraints.csv  |   2 +
 .../queries/QueryTest/parquet.test              |  52 ++++++++++++++--
 tests/query_test/test_scanners.py               |  59 +++++++++++++++----
 8 files changed, 164 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/LineItemMultiBlock/README.dox
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/README.dox b/testdata/LineItemMultiBlock/README.dox
index bc141c3..7608067 100755
--- a/testdata/LineItemMultiBlock/README.dox
+++ b/testdata/LineItemMultiBlock/README.dox
@@ -1,5 +1,6 @@
 This file was created for:
-IMPALA-1881- Maximize data locality when scanning Parquet files with multiple row groups.
+IMPALA-1881: Maximize data locality when scanning Parquet files with multiple row groups.
+IMPALA-2466: Add more tests to the HDFS parquet scanner.
 
 The table lineitem_multiblock is a single parquet file with:
  - A row group size of approximately 12 KB each.
@@ -24,3 +25,9 @@ set parquet.block.size=4086; # This is to set the row group size
 
 insert into functional_parquet.lineitem_multiblock select * from
 tpch.lineitem limit 20000; # We limit to 20000 to keep the size of the table small
+
+'lineitem_sixblocks' was created the same way but with more rows, so that we got more
+blocks.
+
+'lineitem_multiblock_one_row_group' was created similarly but with a much higher
+'parquet.block.size' so that everything fit in one row group.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet b/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet
new file mode 100644
index 0000000..d81e6a5
Binary files /dev/null and b/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet
----------------------------------------------------------------------
diff --git a/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet b/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet
new file mode 100644
index 0000000..34c427b
Binary files /dev/null and b/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index d139dfc..9b2699b 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -304,6 +304,18 @@ function custom-post-load-steps {
   hadoop fs -Ddfs.block.size=1048576 -put -f \
     ${IMPALA_HOME}/testdata/LineItemMultiBlock/000000_0 \
     ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_multiblock_parquet
+
+  # IMPALA-2466: Add more tests to the HDFS Parquet scanner (Added after IMPALA-1881)
+  hadoop fs -mkdir -p /test-warehouse/lineitem_sixblocks_parquet && \
+  hadoop fs -Ddfs.block.size=1048576 -put -f \
+    ${IMPALA_HOME}/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet \
+    /test-warehouse/lineitem_sixblocks_parquet
+
+  # IMPALA-2466: Add more tests to the HDFS Parquet scanner (this has only one row group)
+  hadoop fs -mkdir -p /test-warehouse/lineitem_multiblock_one_row_group_parquet && \
+  hadoop fs -Ddfs.block.size=1048576 -put -f \
+    ${IMPALA_HOME}/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet \
+    /test-warehouse/lineitem_multiblock_one_row_group_parquet
 }
 
 function copy-and-load-ext-data-source {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/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 edf8b69..80487f1 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -1774,8 +1774,52 @@ L_RECEIPTDATE STRING
 L_SHIPINSTRUCT STRING
 L_SHIPMODE STRING
 L_COMMENT STRING
----- ROW_FORMAT
-DELIMITED FIELDS TERMINATED BY '|'
+====
+---- DATASET
+-- IMPALA-2466: Add more tests to the HDFS Parquet scanner
+functional
+---- BASE_TABLE_NAME
+lineitem_sixblocks
+---- COLUMNS
+L_ORDERKEY BIGINT
+L_PARTKEY BIGINT
+L_SUPPKEY BIGINT
+L_LINENUMBER INT
+L_QUANTITY DECIMAL(12,2)
+L_EXTENDEDPRICE DECIMAL(12,2)
+L_DISCOUNT DECIMAL(12,2)
+L_TAX DECIMAL(12,2)
+L_RETURNFLAG STRING
+L_LINESTATUS STRING
+L_SHIPDATE STRING
+L_COMMITDATE STRING
+L_RECEIPTDATE STRING
+L_SHIPINSTRUCT STRING
+L_SHIPMODE STRING
+L_COMMENT STRING
+====
+---- DATASET
+-- IMPALA-2466: Add more tests to the HDFS Parquet scanner (this has only one row group)
+functional
+---- BASE_TABLE_NAME
+lineitem_multiblock_one_row_group
+---- COLUMNS
+L_ORDERKEY BIGINT
+L_PARTKEY BIGINT
+L_SUPPKEY BIGINT
+L_LINENUMBER INT
+L_QUANTITY DECIMAL(12,2)
+L_EXTENDEDPRICE DECIMAL(12,2)
+L_DISCOUNT DECIMAL(12,2)
+L_TAX DECIMAL(12,2)
+L_RETURNFLAG STRING
+L_LINESTATUS STRING
+L_SHIPDATE STRING
+L_COMMITDATE STRING
+L_RECEIPTDATE STRING
+L_SHIPINSTRUCT STRING
+L_SHIPMODE STRING
+L_COMMENT STRING
 ====
 ---- DATASET
 functional

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/datasets/functional/schema_constraints.csv
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index f1f4024..2974942 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -44,6 +44,8 @@ table_name:alltypesagg_hive_13_1, constraint:restrict_to, table_format:parquet/n
 table_name:kite_required_fields, constraint:restrict_to, table_format:parquet/none/none
 table_name:bad_column_metadata, constraint:restrict_to, table_format:parquet/none/none
 table_name:lineitem_multiblock, constraint:restrict_to, table_format:parquet/none/none
+table_name:lineitem_sixblocks, constraint:restrict_to, table_format:parquet/none/none
+table_name:lineitem_multiblock_one_row_group, constraint:restrict_to, table_format:parquet/none/none
 
 # TODO: Support Avro. Data loading currently fails for Avro because complex types
 # cannot be converted to the corresponding Avro types yet.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/testdata/workloads/functional-query/queries/QueryTest/parquet.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/parquet.test b/testdata/workloads/functional-query/queries/QueryTest/parquet.test
index 479e878..e6b4061 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/parquet.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/parquet.test
@@ -53,7 +53,7 @@ SELECT * from bad_magic_number
 File '$NAMENODE/test-warehouse/bad_magic_number_parquet/bad_magic_number.parquet' has an invalid version number: XXXX
 ====
 ---- QUERY
-# count(*) query on parquet file with multiple blocks
+# count(*) query on parquet file with multiple blocks (one block per node)
 SELECT count(*) from lineitem_multiblock
 ---- TYPES
 bigint
@@ -61,7 +61,15 @@ bigint
 20000
 ====
 ---- QUERY
-# Select multiple columns from parquet file with multiple blocks
+# count(*) query on parquet file with more than one block per node
+SELECT count(*) from lineitem_sixblocks
+---- TYPES
+bigint
+---- RESULTS
+40000
+====
+---- QUERY
+# Select multiple columns from parquet file with multiple blocks (one block per node)
 SELECT count(l_comment), min(l_partkey), max(l_linenumber) from lineitem_multiblock;
 ---- TYPES
 bigint, bigint, int
@@ -69,8 +77,44 @@ bigint, bigint, int
 20000,2,7
 ====
 ---- QUERY
-# Test limit queries on parquet with multiple blocks
-select distinct l_orderkey from functional_parquet.lineitem_multiblock where
+# Select multiple columns from parquet file with more than one block per node
+SELECT count(l_comment), min(l_partkey), max(l_linenumber) from lineitem_sixblocks;
+---- TYPES
+bigint, bigint, int
+---- RESULTS
+40000,2,7
+====
+---- QUERY
+# Test limit queries on parquet with multiple blocks (one block per node)
+select distinct l_orderkey from lineitem_multiblock where
+l_orderkey < 5 or l_orderkey > 15000 order by l_orderkey limit 20;
+---- TYPES
+bigint
+---- RESULTS
+1
+2
+3
+4
+15008
+15009
+15010
+15011
+15012
+15013
+15014
+15015
+15040
+15041
+15042
+15043
+15044
+15045
+15046
+15047
+====
+---- QUERY
+# Test limit queries on parquet with more than one block per node
+select distinct l_orderkey from lineitem_sixblocks where
 l_orderkey < 5 or l_orderkey > 15000 order by l_orderkey limit 20;
 ---- TYPES
 bigint

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/76b67485/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index ca34965..7a4290f 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -215,32 +215,69 @@ class TestParquet(ImpalaTestSuite):
   @pytest.mark.execute_serially
   def test_multiple_blocks(self, vector):
     # For IMPALA-1881. The table functional_parquet.lineitem_multiblock has 3 blocks, so
-    # we verify if each impalad reads one block by checking if each impalad reads at
-    # least one row group.
+    # each impalad should read 1 scan range.
     # It needs to execute serially because if there is at a time more, than one query
     # being scheduled, the simple scheduler round robins colocated impalads across
     # all running queries. See IMPALA-2479 for more details.
-    DB_NAME = 'functional_parquet'
-    TABLE_NAME = 'lineitem_multiblock'
-    query = 'select count(l_orderkey) from %s.%s' % (DB_NAME, TABLE_NAME)
+    table_name = 'functional_parquet.lineitem_multiblock'
+    self._multiple_blocks_helper(table_name, 20000, ranges_per_node=1)
+    table_name = 'functional_parquet.lineitem_sixblocks'
+    # 2 scan ranges per node should be created to read 'lineitem_sixblocks' because
+    # there are 6 blocks and 3 scan nodes.
+    self._multiple_blocks_helper(table_name, 40000, ranges_per_node=2)
+
+  @SkipIfS3.hdfs_block_size
+  @SkipIfIsilon.hdfs_block_size
+  @SkipIfLocal.multiple_impalad
+  @pytest.mark.execute_serially
+  def test_multiple_blocks_one_row_group(self, vector):
+    # For IMPALA-1881. The table functional_parquet.lineitem_multiblock_one_row_group has
+    # 3 blocks but only one row group across these blocks. We test to see that only one
+    # scan range reads everything from this row group.
+    table_name = 'functional_parquet.lineitem_multiblock_one_row_group'
+    self._multiple_blocks_helper(
+        table_name, 40000, one_row_group=True, ranges_per_node=1)
+
+  def _multiple_blocks_helper(
+      self, table_name, rows_in_table, one_row_group=False, ranges_per_node=1):
+    """ This function executes a simple SELECT query on a multiblock parquet table and
+    verifies the number of ranges issued per node and verifies that at least one row group
+    was read. If 'one_row_group' is True, then one scan range is expected to read the data
+    from the entire table regardless of the number of blocks. 'ranges_per_node' indicates
+    how many scan ranges we expect to be issued per node. """
+
+    query = 'select count(l_orderkey) from %s' % table_name
     result = self.client.execute(query)
     assert len(result.data) == 1
-    assert result.data[0] == '20000'
+    assert result.data[0] == str(rows_in_table)
 
     runtime_profile = str(result.runtime_profile)
     num_row_groups_list = re.findall('NumRowGroups: ([0-9]*)', runtime_profile)
-    scan_ranges_complete_list = re.findall('ScanRangesComplete: ([0-9]*)', runtime_profile)
+    scan_ranges_complete_list = re.findall(
+        'ScanRangesComplete: ([0-9]*)', runtime_profile)
+    num_rows_read_list = re.findall('RowsRead: [0-9.K]* \(([0-9]*)\)', runtime_profile)
+
     # This will fail if the number of impalads != 3
     # The fourth fragment is the "Averaged Fragment"
     assert len(num_row_groups_list) == 4
     assert len(scan_ranges_complete_list) == 4
+    assert len(num_rows_read_list) == 4
 
+    total_num_row_groups = 0
     # Skip the Averaged Fragment; it comes first in the runtime profile.
     for num_row_groups in num_row_groups_list[1:]:
-      assert int(num_row_groups) > 0
-
-    for scan_ranges_complete in scan_ranges_complete_list[1:]:
-      assert int(scan_ranges_complete) == 1
+      total_num_row_groups += int(num_row_groups)
+      if not one_row_group: assert int(num_row_groups) > 0
+
+    if one_row_group:
+      # If it's the one row group test, only one scan range should read all the data from
+      # that row group.
+      assert total_num_row_groups == 1
+      for rows_read in num_rows_read_list[1:]:
+        if rows_read != '0': assert rows_read == str(rows_in_table)
+
+    for scan_ranges_complete in scan_ranges_complete_list:
+      assert int(scan_ranges_complete) == ranges_per_node
 
   @SkipIfS3.insert
   def test_annotate_utf8_option(self, vector, unique_database):


[14/50] incubator-impala git commit: IMPALA-3249: Failed to mkdirs on core-local-filesystem build.

Posted by ta...@apache.org.
IMPALA-3249: Failed to mkdirs on core-local-filesystem build.

This failure happens on filesystems other than HDFS because as a
part of IMPALA-2466, the $FILESYSTEM_PREFIX was not added to the
new directories that the patch tries to create in create-load-data.

Change-Id: I8de74db93893c5273ccc9c687f608959628f5004
Reviewed-on: http://gerrit.cloudera.org:8080/2644
Reviewed-by: Sailesh Mukil <sa...@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/49a73cd5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/49a73cd5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/49a73cd5

Branch: refs/heads/master
Commit: 49a73cd598890ca9642feea07f81217e67753233
Parents: 39a2818
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Mon Mar 28 11:20:04 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 30 00:03:45 2016 +0000

----------------------------------------------------------------------
 testdata/bin/create-load-data.sh | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/49a73cd5/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index eace01e..136100f 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -306,16 +306,16 @@ function custom-post-load-steps {
     ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_multiblock_parquet
 
   # IMPALA-2466: Add more tests to the HDFS Parquet scanner (Added after IMPALA-1881)
-  hadoop fs -mkdir -p /test-warehouse/lineitem_sixblocks_parquet && \
+  hadoop fs -mkdir -p ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_sixblocks_parquet && \
   hadoop fs -Ddfs.block.size=1048576 -put -f \
     ${IMPALA_HOME}/testdata/LineItemMultiBlock/lineitem_sixblocks.parquet \
-    /test-warehouse/lineitem_sixblocks_parquet
+    ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_sixblocks_parquet
 
   # IMPALA-2466: Add more tests to the HDFS Parquet scanner (this has only one row group)
-  hadoop fs -mkdir -p /test-warehouse/lineitem_multiblock_one_row_group_parquet && \
+  hadoop fs -mkdir -p ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_multiblock_one_row_group_parquet && \
   hadoop fs -Ddfs.block.size=1048576 -put -f \
     ${IMPALA_HOME}/testdata/LineItemMultiBlock/lineitem_one_row_group.parquet \
-    /test-warehouse/lineitem_multiblock_one_row_group_parquet
+    ${FILESYSTEM_PREFIX}/test-warehouse/lineitem_multiblock_one_row_group_parquet
 }
 
 function copy-and-load-ext-data-source {


[22/50] incubator-impala git commit: IMPALA-3194: Allow queries materializing scalar type columns in RC/sequence files

Posted by ta...@apache.org.
IMPALA-3194: Allow queries materializing scalar type columns in RC/sequence files

This commit unblocks queries materializing only scalar typed
columns on tables backed by RC/sequence files containing complex
typed columns. This worked prior to 2.3.0 release.

Change-Id: I3a89b211bdc01f7e07497e293fafd75ccf0500fe
Reviewed-on: http://gerrit.cloudera.org:8080/2580
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 5cd7ada727d04fe56d62ced2e8bfa56f4448ea57
Parents: 2809746
Author: Bharath Vissapragada <bh...@cloudera.com>
Authored: Sun Mar 13 06:17:06 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 31 12:06:57 2016 +0000

----------------------------------------------------------------------
 .../cloudera/impala/catalog/HdfsFileFormat.java | 28 ++++++++++++++------
 .../cloudera/impala/planner/HdfsScanNode.java   | 14 ++++++----
 .../PlannerTest/complex-types-file-formats.test | 23 +++++++++-------
 3 files changed, 43 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5cd7ada7/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java b/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
index 9c883fc..3670aa5 100644
--- a/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
+++ b/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
@@ -35,26 +35,26 @@ public enum HdfsFileFormat {
   RC_FILE("org.apache.hadoop.hive.ql.io.RCFileInputFormat",
       "org.apache.hadoop.hive.ql.io.RCFileOutputFormat",
       "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe",
-      false),
+      false, true),
   TEXT("org.apache.hadoop.mapred.TextInputFormat",
       "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
       "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
-      false),
+      false, false),
   LZO_TEXT("com.hadoop.mapred.DeprecatedLzoTextInputFormat",
       "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
-      "",
-      false),
+      "", false, false),
   SEQUENCE_FILE("org.apache.hadoop.mapred.SequenceFileInputFormat",
       "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat",
-      "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", false),
+      "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", false,
+      true),
   AVRO("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat",
       "org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat",
       "org.apache.hadoop.hive.serde2.avro.AvroSerDe",
-      false),
+      false, false),
   PARQUET("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
       "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
       "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
-      true);
+      true, true);
 
   private final String inputFormat_;
   private final String outputFormat_;
@@ -63,12 +63,18 @@ public enum HdfsFileFormat {
   // Indicates whether we support scanning complex types for this file format.
   private final boolean isComplexTypesSupported_;
 
+  // Indicates whether the file format can skip complex columns in scans and just
+  // materialize scalar typed columns. Ignored if isComplexTypesSupported_ is true.
+  // TODO: Remove this once we support complex types for all file formats.
+  private final boolean canSkipColumnTypes_;
+
   HdfsFileFormat(String inputFormat, String outputFormat, String serializationLib,
-      boolean isComplexTypesSupported) {
+      boolean isComplexTypesSupported, boolean canSkipColumnTypes) {
     inputFormat_ = inputFormat;
     outputFormat_ = outputFormat;
     serializationLib_ = serializationLib;
     isComplexTypesSupported_ = isComplexTypesSupported;
+    canSkipColumnTypes_ = canSkipColumnTypes;
   }
 
   public String inputFormat() { return inputFormat_; }
@@ -235,6 +241,12 @@ public enum HdfsFileFormat {
   public boolean isComplexTypesSupported() { return isComplexTypesSupported_; }
 
   /**
+   * Returns true if this file format can skip complex typed columns and materialize
+   * only scalar typed columns.
+   */
+  public boolean canSkipComplexTypes() { return canSkipColumnTypes_; }
+
+  /**
    * Returns a list with all formats for which isComplexTypesSupported() is true.
    */
   public static List<HdfsFileFormat> complexTypesFormats() {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5cd7ada7/fe/src/main/java/com/cloudera/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/HdfsScanNode.java b/fe/src/main/java/com/cloudera/impala/planner/HdfsScanNode.java
index c6f7722..5edc0dc 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/HdfsScanNode.java
@@ -185,10 +185,11 @@ public class HdfsScanNode extends ScanNode {
     }
     if (firstComplexTypedCol == null) return;
 
-    boolean hasMaterializedSlots = false;
+    boolean referencesComplexTypedCol = false;
     for (SlotDescriptor slotDesc: desc_.getSlots()) {
-      if (slotDesc.isMaterialized()) {
-        hasMaterializedSlots = true;
+      if (!slotDesc.isMaterialized()) continue;
+      if (slotDesc.getType().isComplexType() || slotDesc.getColumn() == null) {
+        referencesComplexTypedCol = true;
         break;
       }
     }
@@ -196,8 +197,11 @@ public class HdfsScanNode extends ScanNode {
     for (HdfsPartition part: partitions_) {
       HdfsFileFormat format = part.getInputFormatDescriptor().getFileFormat();
       if (format.isComplexTypesSupported()) continue;
-      // Allow count(*) and similar queries on RC_FILE with complex types.
-      if (format == HdfsFileFormat.RC_FILE && !hasMaterializedSlots) continue;
+      // If the file format allows querying just scalar typed columns and the query
+      // doesn't materialize any complex typed columns, it is allowed.
+      if (format.canSkipComplexTypes() && !referencesComplexTypedCol) {
+        continue;
+      }
       String errSuffix = String.format(
           "Complex types are supported for these file formats: %s",
           Joiner.on(", ").join(HdfsFileFormat.complexTypesFormats()));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5cd7ada7/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
index 487bb3b..f0431a2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
@@ -38,13 +38,18 @@ select 1 from functional_rc_snap.complextypes_fileformat t, t.a
 not implemented: Scan of table 't' in format 'RC_FILE' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
 Complex types are supported for these file formats: PARQUET.
 ====
-# Complex types are not supported on RC files, even if no complex-typed
-# columns are selected.
-select id from functional_rc_snap.complextypes_fileformat
+select s.f1 from functional_rc_snap.complextypes_fileformat t, t.m
 ---- PLAN
-not implemented: Scan of table 'functional_rc_snap.complextypes_fileformat' in format 'RC_FILE' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
+not implemented: Scan of table 't' in format 'RC_FILE' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
 Complex types are supported for these file formats: PARQUET.
 ====
+# Complex types are not supported on RC files, however queries materializing
+# only scalar type columns are allowed.
+select id from functional_rc_snap.complextypes_fileformat
+---- PLAN
+00:SCAN HDFS [functional_rc_snap.complextypes_fileformat]
+   partitions=1/1 files=1 size=56B
+====
 # Complex types are not supported on RC files but count(*) and similar
 # queries should work.
 select count(*) from functional_rc_snap.complextypes_fileformat
@@ -61,12 +66,12 @@ select s.f1 from functional_seq_snap.complextypes_fileformat t, t.a
 not implemented: Scan of table 't' in format 'SEQUENCE_FILE' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
 Complex types are supported for these file formats: PARQUET.
 ====
-# Complex types are not supported on sequence files, even if no complex-typed
-# columns are selected.
-select 1 from functional_seq_snap.complextypes_fileformat
+# Queries referencing only scalar typed columns on sequence files
+# are allowed.
+select id from functional_seq_snap.complextypes_fileformat
 ---- PLAN
-not implemented: Scan of table 'functional_seq_snap.complextypes_fileformat' in format 'SEQUENCE_FILE' is not supported because the table has a column 's' with a complex type 'STRUCT<f1:STRING,f2:INT>'.
-Complex types are supported for these file formats: PARQUET.
+00:SCAN HDFS [functional_seq_snap.complextypes_fileformat]
+   partitions=1/1 files=1 size=87B
 ====
 # Scanning all partitions fails because there are partitions with a file format for which
 # complex types are not supported. The error message is abbreviated because it is


[03/50] incubator-impala git commit: Add AtomicInt32/AtomicInt64 typedefs

Posted by ta...@apache.org.
Add AtomicInt32/AtomicInt64 typedefs

Only certain primitive types can be used with AtomicInt<>, so let's
treat AtomicInt<> as an implementation detail and instead expose
typedefs for the Atomic integer sizes we support. Convert all decls to
use the new typedefs.

Change-Id: I8b91ba684aabc67ed1721c7b7320aa42049268c8
Reviewed-on: http://gerrit.cloudera.org:8080/2601
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 10e7de7969f96fe3edb00fcab3f7a8c432bd20a0
Parents: c039791
Author: Dan Hecht <dh...@cloudera.com>
Authored: Tue Mar 22 14:24:23 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 23 22:29:18 2016 +0000

----------------------------------------------------------------------
 be/src/common/atomic-test.cc            |  3 ++-
 be/src/common/atomic.h                  | 12 +++++++++++-
 be/src/exec/hdfs-scan-node.h            |  8 ++++----
 be/src/rpc/rpc-trace.h                  |  2 +-
 be/src/runtime/data-stream-recvr.h      |  2 +-
 be/src/runtime/disk-io-mgr-internal.h   | 24 ++++++++++++------------
 be/src/runtime/disk-io-mgr-test.cc      | 18 +++++++++---------
 be/src/runtime/disk-io-mgr.h            |  4 ++--
 be/src/runtime/lib-cache.h              |  2 +-
 be/src/runtime/mem-tracker.cc           |  2 +-
 be/src/runtime/mem-tracker.h            |  2 +-
 be/src/runtime/plan-fragment-executor.h |  2 +-
 be/src/scheduling/query-resource-mgr.cc |  8 ++++----
 be/src/scheduling/query-resource-mgr.h  |  8 ++++----
 be/src/util/counting-barrier.h          |  2 +-
 be/src/util/hdfs-bulk-ops.h             |  2 +-
 be/src/util/internal-queue-test.cc      |  6 +++---
 be/src/util/periodic-counter-updater.h  |  2 +-
 be/src/util/progress-updater.h          |  4 ++--
 be/src/util/runtime-profile.h           |  4 ++--
 20 files changed, 64 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/common/atomic-test.cc
----------------------------------------------------------------------
diff --git a/be/src/common/atomic-test.cc b/be/src/common/atomic-test.cc
index 5715022..4e4559c 100644
--- a/be/src/common/atomic-test.cc
+++ b/be/src/common/atomic-test.cc
@@ -20,9 +20,10 @@
 
 #include "common/names.h"
 
-
 namespace impala {
 
+using namespace internal; // Testing AtomicInt<> directly.
+
 // Simple test to make sure there is no obvious error in the operations.  This is not
 // intended to test the thread safety.
 template<typename T>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/common/atomic.h
----------------------------------------------------------------------
diff --git a/be/src/common/atomic.h b/be/src/common/atomic.h
index 07a9036..b0b8faa 100644
--- a/be/src/common/atomic.h
+++ b/be/src/common/atomic.h
@@ -45,7 +45,10 @@ class AtomicUtil {
   }
 };
 
-/// Atomic integer. 'T' can be either 32-bit or 64-bit signed integer. Each operation
+namespace internal {
+
+/// Atomic integer. This class template should not be used directly; instead use the
+/// typedefs below. 'T' can be either 32-bit or 64-bit signed integer. Each operation
 /// is performed atomically and has a specified memory-ordering semantic:
 ///
 /// Acquire: these operations ensure no later memory access by the same thread can be
@@ -102,6 +105,13 @@ class AtomicInt {
   DISALLOW_COPY_AND_ASSIGN(AtomicInt);
 };
 
+} // namespace internal
+
+/// Supported atomic types. Use these types rather than referring to AtomicInt<>
+/// directly.
+typedef internal::AtomicInt<int32_t> AtomicInt32;
+typedef internal::AtomicInt<int64_t> AtomicInt64;
+
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/exec/hdfs-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.h b/be/src/exec/hdfs-scan-node.h
index 4ddd33d..73b9527 100644
--- a/be/src/exec/hdfs-scan-node.h
+++ b/be/src/exec/hdfs-scan-node.h
@@ -369,7 +369,7 @@ class HdfsScanNode : public ScanNode {
   int64_t scanner_thread_bytes_required_;
 
   /// Number of files that have not been issued from the scanners.
-  AtomicInt<int> num_unqueued_files_;
+  AtomicInt32 num_unqueued_files_;
 
   /// Map of HdfsScanner objects to file types.  Only one scanner object will be
   /// created for each file type.  Objects stored in runtime_state's pool.
@@ -424,12 +424,12 @@ class HdfsScanNode : public ScanNode {
   /// This is the number of io buffers that are owned by the scan node and the scanners.
   /// This is used just to help debug leaked io buffers to determine if the leak is
   /// happening in the scanners vs other parts of the execution.
-  AtomicInt<int> num_owned_io_buffers_;
+  AtomicInt32 num_owned_io_buffers_;
 
   /// Counters which track the number of scanners that have codegen enabled for the
   /// materialize and conjuncts evaluation code paths.
-  AtomicInt<int> num_scanners_codegen_enabled_;
-  AtomicInt<int> num_scanners_codegen_disabled_;
+  AtomicInt32 num_scanners_codegen_enabled_;
+  AtomicInt32 num_scanners_codegen_disabled_;
 
   /// The size of the largest compressed text file to be scanned. This is used to
   /// estimate scanner thread memory usage.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/rpc/rpc-trace.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-trace.h b/be/src/rpc/rpc-trace.h
index aed05f4..5abe0a2 100644
--- a/be/src/rpc/rpc-trace.h
+++ b/be/src/rpc/rpc-trace.h
@@ -84,7 +84,7 @@ class RpcEventHandler : public apache::thrift::TProcessorEventHandler {
     StatsMetric<double>* time_stats;
 
     /// Number of invocations in flight
-    AtomicInt<int32_t> num_in_flight;
+    AtomicInt32 num_in_flight;
   };
 
   /// Map from method name to descriptor

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/data-stream-recvr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-recvr.h b/be/src/runtime/data-stream-recvr.h
index 7879c44..2a3b184 100644
--- a/be/src/runtime/data-stream-recvr.h
+++ b/be/src/runtime/data-stream-recvr.h
@@ -136,7 +136,7 @@ class DataStreamRecvr {
   bool is_merging_;
 
   /// total number of bytes held across all sender queues.
-  AtomicInt<int> num_buffered_bytes_;
+  AtomicInt32 num_buffered_bytes_;
 
   /// Memtracker for batches in the sender queue(s).
   boost::scoped_ptr<MemTracker> mem_tracker_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/disk-io-mgr-internal.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr-internal.h b/be/src/runtime/disk-io-mgr-internal.h
index fe607e7..e58a3f3 100644
--- a/be/src/runtime/disk-io-mgr-internal.h
+++ b/be/src/runtime/disk-io-mgr-internal.h
@@ -212,48 +212,48 @@ class DiskIoMgr::RequestContext {
   RuntimeProfile::Counter* disks_accessed_bitmap_;
 
   /// Total number of bytes read locally, updated at end of each range scan
-  AtomicInt<int64_t> bytes_read_local_;
+  AtomicInt64 bytes_read_local_;
 
   /// Total number of bytes read via short circuit read, updated at end of each range scan
-  AtomicInt<int64_t> bytes_read_short_circuit_;
+  AtomicInt64 bytes_read_short_circuit_;
 
   /// Total number of bytes read from date node cache, updated at end of each range scan
-  AtomicInt<int64_t> bytes_read_dn_cache_;
+  AtomicInt64 bytes_read_dn_cache_;
 
   /// Total number of bytes from remote reads that were expected to be local.
-  AtomicInt<int64_t> unexpected_remote_bytes_;
+  AtomicInt64 unexpected_remote_bytes_;
 
   /// The number of buffers that have been returned to the reader (via GetNext) that the
   /// reader has not returned. Only included for debugging and diagnostics.
-  AtomicInt<int> num_buffers_in_reader_;
+  AtomicInt32 num_buffers_in_reader_;
 
   /// The number of scan ranges that have been completed for this reader.
-  AtomicInt<int> num_finished_ranges_;
+  AtomicInt32 num_finished_ranges_;
 
   /// The number of scan ranges that required a remote read, updated at the end of each
   /// range scan. Only used for diagnostics.
-  AtomicInt<int> num_remote_ranges_;
+  AtomicInt32 num_remote_ranges_;
 
   /// The total number of scan ranges that have not been started. Only used for
   /// diagnostics. This is the sum of all unstarted_scan_ranges across all disks.
-  AtomicInt<int> num_unstarted_scan_ranges_;
+  AtomicInt32 num_unstarted_scan_ranges_;
 
   /// The number of buffers that are being used for this reader. This is the sum
   /// of all buffers in ScanRange queues and buffers currently being read into (i.e. about
   /// to be queued).
-  AtomicInt<int> num_used_buffers_;
+  AtomicInt32 num_used_buffers_;
 
   /// The total number of ready buffers across all ranges.  Ready buffers are buffers
   /// that have been read from disk but not retrieved by the caller.
   /// This is the sum of all queued buffers in all ranges for this reader context.
-  AtomicInt<int> num_ready_buffers_;
+  AtomicInt32 num_ready_buffers_;
 
   /// The total (sum) of queue capacities for finished scan ranges. This value
   /// divided by num_finished_ranges_ is the average for finished ranges and
   /// used to seed the starting queue capacity for future ranges. The assumption
   /// is that if previous ranges were fast, new ones will be fast too. The scan
   /// range adjusts the queue capacity dynamically so a rough approximation will do.
-  AtomicInt<int> total_range_queue_capacity_;
+  AtomicInt32 total_range_queue_capacity_;
 
   /// The initial queue size for new scan ranges. This is always
   /// total_range_queue_capacity_ / num_finished_ranges_ but stored as a separate
@@ -447,7 +447,7 @@ class DiskIoMgr::RequestContext {
     /// entire operation, we need this ref count to keep track of which thread should do
     /// final resource cleanup during cancellation.
     /// Only the thread that sees the count at 0 should do the final cleanup.
-    AtomicInt<int> num_threads_in_op_;
+    AtomicInt32 num_threads_in_op_;
 
     /// Queue of write ranges to process for this disk. A write range is always added
     /// to in_flight_ranges_ in GetNextRequestRange(). There is a separate

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/disk-io-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr-test.cc b/be/src/runtime/disk-io-mgr-test.cc
index 806a036..ee89f56 100644
--- a/be/src/runtime/disk-io-mgr-test.cc
+++ b/be/src/runtime/disk-io-mgr-test.cc
@@ -136,7 +136,7 @@ class DiskIoMgrTest : public testing::Test {
   // Updates num_ranges_processed with the number of ranges seen by this thread.
   static void ScanRangeThread(DiskIoMgr* io_mgr, DiskIoMgr::RequestContext* reader,
       const char* expected_result, int expected_len, const Status& expected_status,
-      int max_ranges, AtomicInt<int>* num_ranges_processed) {
+      int max_ranges, AtomicInt32* num_ranges_processed) {
     int num_ranges = 0;
     while (max_ranges == 0 || num_ranges < max_ranges) {
       DiskIoMgr::ScanRange* range;
@@ -373,7 +373,7 @@ TEST_F(DiskIoMgrTest, SingleReader) {
           }
           ASSERT_OK(io_mgr.AddScanRanges(reader, ranges));
 
-          AtomicInt<int> num_ranges_processed;
+          AtomicInt32 num_ranges_processed;
           thread_group threads;
           for (int i = 0; i < num_read_threads; ++i) {
             threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader, data,
@@ -432,7 +432,7 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
                 stat_val.st_mtime));
           }
         }
-        AtomicInt<int> num_ranges_processed;
+        AtomicInt32 num_ranges_processed;
 
         // Issue first half the scan ranges.
         ASSERT_OK(io_mgr.AddScanRanges(reader, ranges_first_half));
@@ -507,7 +507,7 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
         }
         ASSERT_OK(io_mgr.AddScanRanges(reader, ranges));
 
-        AtomicInt<int> num_ranges_processed;
+        AtomicInt32 num_ranges_processed;
         thread_group threads;
         for (int i = 0; i < 5; ++i) {
           threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader, data,
@@ -570,7 +570,7 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
         }
         ASSERT_OK(io_mgr.AddScanRanges(reader, ranges));
 
-        AtomicInt<int> num_ranges_processed;
+        AtomicInt32 num_ranges_processed;
         int num_succesful_ranges = ranges.size() / 2;
         // Read half the ranges
         for (int i = 0; i < num_succesful_ranges; ++i) {
@@ -637,7 +637,7 @@ TEST_F(DiskIoMgrTest, MemLimits) {
     // Don't return buffers to force memory pressure
     vector<DiskIoMgr::BufferDescriptor*> buffers;
 
-    AtomicInt<int> num_ranges_processed;
+    AtomicInt32 num_ranges_processed;
     ScanRangeThread(&io_mgr, reader, data, strlen(data), Status::MemLimitExceeded(),
         1, &num_ranges_processed);
 
@@ -717,7 +717,7 @@ TEST_F(DiskIoMgrTest, CachedReads) {
     }
     ASSERT_OK(io_mgr.AddScanRanges(reader, ranges));
 
-    AtomicInt<int> num_ranges_processed;
+    AtomicInt32 num_ranges_processed;
     thread_group threads;
     for (int i = 0; i < 5; ++i) {
       threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader, data,
@@ -780,7 +780,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
         while (read_offset < file_size) {
           for (int context_index = 0; context_index < num_contexts; ++context_index) {
             if (++iters % 5000 == 0) LOG(ERROR) << "Starting iteration " << iters;
-            AtomicInt<int> num_ranges_processed;
+            AtomicInt32 num_ranges_processed;
             thread_group threads;
             vector<DiskIoMgr::ScanRange*> ranges;
             int num_scan_ranges = min<int>(num_reads_queued, write_offset - read_offset);
@@ -898,7 +898,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
             ASSERT_OK(io_mgr.AddScanRanges(readers[i], ranges));
           }
 
-          AtomicInt<int> num_ranges_processed;
+          AtomicInt32 num_ranges_processed;
           thread_group threads;
           for (int i = 0; i < NUM_READERS; ++i) {
             for (int j = 0; j < NUM_THREADS_PER_READER; ++j) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/disk-io-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr.h b/be/src/runtime/disk-io-mgr.h
index d695555..b130d52 100644
--- a/be/src/runtime/disk-io-mgr.h
+++ b/be/src/runtime/disk-io-mgr.h
@@ -734,10 +734,10 @@ class DiskIoMgr {
   std::list<BufferDescriptor*> free_buffer_descs_;
 
   /// Total number of allocated buffers, used for debugging.
-  AtomicInt<int> num_allocated_buffers_;
+  AtomicInt32 num_allocated_buffers_;
 
   /// Total number of buffers in readers
-  AtomicInt<int> num_buffers_in_readers_;
+  AtomicInt32 num_buffers_in_readers_;
 
   /// Per disk queues. This is static and created once at Init() time.  One queue is
   /// allocated for each local disk on the system and for each remote filesystem type.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/lib-cache.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h
index 005f166..b36a859 100644
--- a/be/src/runtime/lib-cache.h
+++ b/be/src/runtime/lib-cache.h
@@ -117,7 +117,7 @@ class LibCache {
 
   /// The number of libs that have been copied from HDFS to the local FS.
   /// This is appended to the local fs path to remove collisions.
-  AtomicInt<int64_t> num_libs_copied_;
+  AtomicInt64 num_libs_copied_;
 
   /// Protects lib_cache_. For lock ordering, this lock must always be taken before
   /// the per entry lock.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/mem-tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.cc b/be/src/runtime/mem-tracker.cc
index 1fc462f..e12c946 100644
--- a/be/src/runtime/mem-tracker.cc
+++ b/be/src/runtime/mem-tracker.cc
@@ -39,7 +39,7 @@ MemTracker::RequestTrackersMap MemTracker::request_to_mem_trackers_;
 MemTracker::PoolTrackersMap MemTracker::pool_to_mem_trackers_;
 mutex MemTracker::static_mem_trackers_lock_;
 
-AtomicInt<int64_t> MemTracker::released_memory_since_gc_;
+AtomicInt64 MemTracker::released_memory_since_gc_;
 
 // Name for request pool MemTrackers. '$0' is replaced with the pool name.
 const string REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT = "RequestPool=$0";

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/mem-tracker.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h
index 46f08c3..2fd23a7 100644
--- a/be/src/runtime/mem-tracker.h
+++ b/be/src/runtime/mem-tracker.h
@@ -394,7 +394,7 @@ class MemTracker {
 
   /// Total amount of memory from calls to Release() since the last GC. If this
   /// is greater than GC_RELEASE_SIZE, this will trigger a tcmalloc gc.
-  static AtomicInt<int64_t> released_memory_since_gc_;
+  static AtomicInt64 released_memory_since_gc_;
 
   /// Lock to protect GcMemory(). This prevents many GCs from occurring at once.
   boost::mutex gc_lock_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/runtime/plan-fragment-executor.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/plan-fragment-executor.h b/be/src/runtime/plan-fragment-executor.h
index bb2f3a9..91cece6 100644
--- a/be/src/runtime/plan-fragment-executor.h
+++ b/be/src/runtime/plan-fragment-executor.h
@@ -231,7 +231,7 @@ class PlanFragmentExecutor {
   /// be fired. It is initialized to 0 and atomically swapped to 1 when a completed
   /// fragment report is about to be fired. Used for reducing the probability that a
   /// report is sent twice at the end of the fragment.
-  AtomicInt<int> completed_report_sent_;
+  AtomicInt32 completed_report_sent_;
 
   /// Sampled memory usage at even time intervals.
   RuntimeProfile::TimeSeriesCounter* mem_usage_sampled_counter_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/scheduling/query-resource-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-resource-mgr.cc b/be/src/scheduling/query-resource-mgr.cc
index 6a5f8f6..e1e0581 100644
--- a/be/src/scheduling/query-resource-mgr.cc
+++ b/be/src/scheduling/query-resource-mgr.cc
@@ -97,8 +97,8 @@ void QueryResourceMgr::InitVcoreAcquisition(int32_t init_vcores) {
   // inspects immediately after exiting Expand(), and if true, exits before touching any
   // of the class-wide state (because the destructor may have finished before this point).
 
-  thread_in_expand_.reset(new AtomicInt<int32_t>());
-  early_exit_.reset(new AtomicInt<int32_t>());
+  thread_in_expand_.reset(new AtomicInt32());
+  early_exit_.reset(new AtomicInt32());
   acquire_vcore_thread_.reset(
       new Thread("resource-mgmt", Substitute("acquire-cpu-$0", PrintId(query_id_)),
           bind<void>(mem_fn(&QueryResourceMgr::AcquireVcoreResources), this,
@@ -170,8 +170,8 @@ Status QueryResourceMgr::RequestMemExpansion(int64_t requested_bytes,
 }
 
 void QueryResourceMgr::AcquireVcoreResources(
-    shared_ptr<AtomicInt<int32_t> > thread_in_expand,
-    shared_ptr<AtomicInt<int32_t> > early_exit) {
+    shared_ptr<AtomicInt32> thread_in_expand,
+    shared_ptr<AtomicInt32> early_exit) {
   // Take a copy because we'd like to print it in some cases after the destructor.
   TUniqueId reservation_id = reservation_id_;
   VLOG_QUERY << "Starting Vcore acquisition for: " << reservation_id;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/scheduling/query-resource-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-resource-mgr.h b/be/src/scheduling/query-resource-mgr.h
index 589e327..c6080e8 100644
--- a/be/src/scheduling/query-resource-mgr.h
+++ b/be/src/scheduling/query-resource-mgr.h
@@ -195,12 +195,12 @@ class QueryResourceMgr {
   /// parent QueryResourceMgr has been destroyed.
   /// TODO: Combine with ShouldExit(), and replace with AtomicBool when we have such a
   /// thing.
-  boost::shared_ptr<AtomicInt<int32_t> > early_exit_;
+  boost::shared_ptr<AtomicInt32> early_exit_;
 
   /// Signals to the destructor that the vcore acquisition thread is currently in an
   /// Expand() RPC. If so, the destructor does not need to wait for the acquisition thread
   /// to exit.
-  boost::shared_ptr<AtomicInt<int32_t> > thread_in_expand_;
+  boost::shared_ptr<AtomicInt32> thread_in_expand_;
 
   /// Creates the llama resource for the memory and/or cores specified, associated with
   /// the reservation context.
@@ -209,8 +209,8 @@ class QueryResourceMgr {
   /// Run as a thread owned by acquire_cpu_thread_. Waits for notification from
   /// NotifyThreadUsageChange(), then checks the subscription level to decide if more
   /// VCores are needed, and starts a new expansion request if so.
-  void AcquireVcoreResources(boost::shared_ptr<AtomicInt<int32_t> > thread_in_expand,
-      boost::shared_ptr<AtomicInt<int32_t> > early_exit);
+  void AcquireVcoreResources(boost::shared_ptr<AtomicInt32 > thread_in_expand,
+      boost::shared_ptr<AtomicInt32> early_exit);
 
   /// True if thread:VCore subscription is too high, meaning more VCores are required.
   /// Must be called holding threads_running_ lock.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/counting-barrier.h
----------------------------------------------------------------------
diff --git a/be/src/util/counting-barrier.h b/be/src/util/counting-barrier.h
index 917c8ec..72e7ac9 100644
--- a/be/src/util/counting-barrier.h
+++ b/be/src/util/counting-barrier.h
@@ -43,7 +43,7 @@ class CountingBarrier {
   Promise<bool> promise_;
 
   /// The number of pending notifications remaining.
-  AtomicInt<int32_t> count_;
+  AtomicInt32 count_;
 
   DISALLOW_COPY_AND_ASSIGN(CountingBarrier);
 };

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/hdfs-bulk-ops.h
----------------------------------------------------------------------
diff --git a/be/src/util/hdfs-bulk-ops.h b/be/src/util/hdfs-bulk-ops.h
index f543a06..432029f 100644
--- a/be/src/util/hdfs-bulk-ops.h
+++ b/be/src/util/hdfs-bulk-ops.h
@@ -125,7 +125,7 @@ class HdfsOperationSet {
 
   /// The number of ops remaining to be executed. Used to coordinate between executor
   /// threads so that when all ops are finished, promise_ is signalled.
-  AtomicInt<int64_t> num_ops_;
+  AtomicInt64 num_ops_;
 
   /// HDFS connection shared between all operations. Not owned by this class.
   hdfsFS* hdfs_connection_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/internal-queue-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/internal-queue-test.cc b/be/src/util/internal-queue-test.cc
index 26e91a5..30bb1e8 100644
--- a/be/src/util/internal-queue-test.cc
+++ b/be/src/util/internal-queue-test.cc
@@ -150,7 +150,7 @@ const int VALIDATE_INTERVAL = 10000;
 
 // CHECK() is not thread safe so return the result in *failed.
 void ProducerThread(InternalQueue<IntNode>* queue, int num_inserts,
-    vector<IntNode>* nodes, AtomicInt<int32_t>* counter, bool* failed) {
+    vector<IntNode>* nodes, AtomicInt32* counter, bool* failed) {
   for (int i = 0; i < num_inserts && !*failed; ++i) {
     // Get the next index to queue.
     int32_t value = counter->Add(1) - 1;
@@ -204,7 +204,7 @@ TEST(InternalQueue, TestClear) {
 
 TEST(InternalQueue, TestSingleProducerSingleConsumer) {
   vector<IntNode> nodes;
-  AtomicInt<int32_t> counter;
+  AtomicInt32 counter;
   nodes.resize(1000000);
   vector<int> results;
 
@@ -233,7 +233,7 @@ TEST(InternalQueue, TestMultiProducerMultiConsumer) {
 
   bool failed = false;
   for (int num_producers = 1; num_producers < 5; num_producers += 3) {
-    AtomicInt<int32_t> counter;
+    AtomicInt32 counter;
     const int NUM_CONSUMERS = 4;
     ASSERT_EQ(nodes.size() % NUM_CONSUMERS, 0);
     ASSERT_EQ(nodes.size() % num_producers, 0);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/periodic-counter-updater.h
----------------------------------------------------------------------
diff --git a/be/src/util/periodic-counter-updater.h b/be/src/util/periodic-counter-updater.h
index 660502e..6887a91 100644
--- a/be/src/util/periodic-counter-updater.h
+++ b/be/src/util/periodic-counter-updater.h
@@ -137,7 +137,7 @@ class PeriodicCounterUpdater {
   TimeSeriesCounters time_series_counters_;
 
   /// If 1, tear down the update thread.
-  AtomicInt<int32_t> done_;
+  AtomicInt32 done_;
 
   /// Singleton object that keeps track of all rate counters and the thread
   /// for updating them.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/progress-updater.h
----------------------------------------------------------------------
diff --git a/be/src/util/progress-updater.h b/be/src/util/progress-updater.h
index d938221..774a45a 100644
--- a/be/src/util/progress-updater.h
+++ b/be/src/util/progress-updater.h
@@ -75,10 +75,10 @@ class ProgressUpdater {
   int update_period_;
 
   /// Number of completed work items.
-  AtomicInt<int64_t> num_complete_;
+  AtomicInt64 num_complete_;
 
   /// Percentage when the last output was generated.
-  AtomicInt<int> last_output_percentage_;
+  AtomicInt32 last_output_percentage_;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/10e7de79/be/src/util/runtime-profile.h
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index 6858c5d..6695b65 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -125,7 +125,7 @@ class RuntimeProfile {
    protected:
     friend class RuntimeProfile;
 
-    AtomicInt<int64_t> value_;
+    AtomicInt64 value_;
     TUnit::type unit_;
   };
 
@@ -175,7 +175,7 @@ class RuntimeProfile {
 
     /// The current value of the counter. value_ in the super class represents
     /// the high water mark.
-    AtomicInt<int64_t> current_value_;
+    AtomicInt64 current_value_;
   };
 
   typedef boost::function<int64_t ()> DerivedCounterFunction;


[09/50] incubator-impala git commit: Consolidate test and cluster logs under a single directory.

Posted by ta...@apache.org.
Consolidate test and cluster logs under a single directory.

All logs, test results and SQL files generated during data
loading and testing are now consolidated under a single new
directory $IMPALA_HOME/logs. The goal is to simplify archiving
in Jenkins runs and debugging.

The new structure is as follows:

$IMPALA_HOME/logs/cluster
- logs of Hadoop components and Impala

$IMPALA_HOME/logs/data_loading
- logs and SQL files produced in data loading

$IMPALA_HOME/logs/fe_tests
- logs and test output of Frontend unit tests

$IMPALA_HOME/logs/be_tests
- logs and test output of Backend unit tests

$IMPALA_HOME/logs/ee_tests
- logs and test output of end-to-end tests

$IMPALA_HOME/logs/custom_cluster_tests
- logs and test output of custom cluster tests

I tested this change with a full data load which
was successful.

Change-Id: Ief1f58f3320ec39d31b3c6bc6ef87f58ff7dfdfa
Reviewed-on: http://gerrit.cloudera.org:8080/2456
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 7e76e92bef668b3ad95ef66b5d74a02662e2d1ce
Parents: 35dfbaf
Author: Alex Behm <al...@cloudera.com>
Authored: Fri Jan 15 19:40:31 2016 -0800
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Mar 28 19:23:22 2016 +0000

----------------------------------------------------------------------
 .gitignore                                      |  2 +-
 LOGS.md                                         | 21 ++++++++++++++
 be/CMakeLists.txt                               |  3 +-
 be/src/rpc/authentication-test.cc               |  2 +-
 be/src/util/thread-pool-test.cc                 |  5 ++--
 bin/clean.sh                                    |  3 +-
 bin/impala-config.sh                            | 30 ++++++++++++++++++--
 bin/load-data.py                                | 10 ++++---
 bin/run-all-tests.sh                            |  8 +++---
 bin/run-backend-tests.sh                        |  4 +--
 bin/start-impala-cluster.py                     |  6 +++-
 buildall.sh                                     |  5 ----
 fe/pom.xml                                      |  2 ++
 fe/src/test/resources/hbase-site.xml.template   |  2 +-
 .../resources/hive-log4j.properties.template    |  2 +-
 .../test/resources/mysql-hive-site.xml.template |  2 +-
 .../resources/postgresql-hive-site.xml.template |  2 +-
 testdata/bin/create-load-data.sh                | 12 ++++----
 testdata/bin/generate-schema-statements.py      |  4 +--
 testdata/bin/minikdc_env.sh                     |  2 +-
 testdata/bin/run-all.sh                         | 16 +++++------
 testdata/bin/run-hbase.sh                       |  8 +++---
 testdata/bin/run-hive-server.sh                 |  2 +-
 testdata/cluster/admin                          |  4 +--
 tests/run-custom-cluster-tests.sh               |  9 ++----
 25 files changed, 106 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index c6aaa3a..0987017 100644
--- a/.gitignore
+++ b/.gitignore
@@ -36,7 +36,7 @@ tests/test-hive-udfs/target/
 
 cdh-*-hdfs-data/
 avro_schemas/
-cluster_logs/
+logs/
 
 # Impala toolchain directory
 toolchain/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/LOGS.md
----------------------------------------------------------------------
diff --git a/LOGS.md b/LOGS.md
new file mode 100644
index 0000000..cd43d38
--- /dev/null
+++ b/LOGS.md
@@ -0,0 +1,21 @@
+All logs, test results and SQL files generated during data loading
+and testing are consolidated under $IMPALA_HOME/logs with the
+following directory structure:
+
+$IMPALA_HOME/logs/cluster
+- logs of Hadoop components and Impala
+
+$IMPALA_HOME/logs/data_loading
+- logs and SQL files produced in data loading
+
+$IMPALA_HOME/logs/fe_tests
+- logs and test output of Frontend unit tests
+
+$IMPALA_HOME/logs/be_tests
+- logs and test output of Backend unit tests
+
+$IMPALA_HOME/logs/ee_tests
+- logs and test output of end-to-end tests
+
+$IMPALA_HOME/logs/custom_cluster_tests
+- logs and test output of custom cluster tests

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 7857fbd..64328d0 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -383,7 +383,8 @@ FUNCTION(ADD_BE_TEST TEST_NAME)
   ADD_EXECUTABLE(${TEST_NAME} ${TEST_NAME}.cc)
   TARGET_LINK_LIBRARIES(${TEST_NAME} ${IMPALA_TEST_LINK_LIBS})
   set(CMAKE_EXE_LINKER_FLAGS "--start-group")
-  ADD_TEST(${TEST_NAME} "${BUILD_OUTPUT_ROOT_DIRECTORY}/${DIR_NAME}/${TEST_NAME}")
+  ADD_TEST(${TEST_NAME} "${BUILD_OUTPUT_ROOT_DIRECTORY}/${DIR_NAME}/${TEST_NAME}"
+    -log_dir=$ENV{IMPALA_BE_TEST_LOGS_DIR})
   ADD_DEPENDENCIES(be-test ${TEST_NAME})
 ENDFUNCTION()
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/be/src/rpc/authentication-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/authentication-test.cc b/be/src/rpc/authentication-test.cc
index efccb02..59ecda4 100644
--- a/be/src/rpc/authentication-test.cc
+++ b/be/src/rpc/authentication-test.cc
@@ -174,8 +174,8 @@ TEST(Auth, KerbAndSslEnabled) {
 }
 
 int main(int argc, char** argv) {
-  impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   ::testing::InitGoogleTest(&argc, argv);
+  impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
 
   env_keytab = getenv("KRB5_KTNAME");
   env_princ = getenv("MINIKDC_PRINC_IMPALA");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/be/src/util/thread-pool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/thread-pool-test.cc b/be/src/util/thread-pool-test.cc
index 40c4259..f4541cd 100644
--- a/be/src/util/thread-pool-test.cc
+++ b/be/src/util/thread-pool-test.cc
@@ -18,6 +18,7 @@
 #include <gtest/gtest.h>
 #include <unistd.h>
 
+#include "common/init.h"
 #include "common/logging.h"
 #include "util/thread-pool.h"
 
@@ -67,9 +68,7 @@ TEST(ThreadPoolTest, BasicTest) {
 }
 
 int main(int argc, char** argv) {
-  impala::InitGoogleLoggingSafe(argv[0]);
-  impala::InitThreading();
-  impala::OsInfo::Init();
   ::testing::InitGoogleTest(&argc, argv);
+  impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/clean.sh
----------------------------------------------------------------------
diff --git a/bin/clean.sh b/bin/clean.sh
index 00755b6..5329238 100755
--- a/bin/clean.sh
+++ b/bin/clean.sh
@@ -41,7 +41,8 @@ pushd $IMPALA_FE_DIR
 rm -rf target
 rm -f src/test/resources/{core,hbase,hive}-site.xml
 rm -rf generated-sources/*
-rm -rf ${IMPALA_TEST_CLUSTER_LOG_DIR}/*
+rm -rf ${IMPALA_LOGS_DIR}/*
+mkdir -p $IMPALA_ALL_LOGS_DIRS
 popd
 
 # clean be

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 5b6f673..bbcf2de 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -183,8 +183,34 @@ elif [ "${TARGET_FILESYSTEM}" != "hdfs" ]; then
   return 1
 fi
 
-# Directory where local cluster logs will go when running tests or loading data
-export IMPALA_TEST_CLUSTER_LOG_DIR=${IMPALA_HOME}/cluster_logs
+# Directories where local cluster logs will go when running tests or loading data
+export IMPALA_LOGS_DIR=${IMPALA_HOME}/logs
+export IMPALA_CLUSTER_LOGS_DIR=${IMPALA_LOGS_DIR}/cluster
+export IMPALA_DATA_LOADING_LOGS_DIR=${IMPALA_LOGS_DIR}/data_loading
+export IMPALA_DATA_LOADING_SQL_DIR=${IMPALA_DATA_LOADING_LOGS_DIR}/sql
+export IMPALA_FE_TEST_LOGS_DIR=${IMPALA_LOGS_DIR}/fe_tests
+export IMPALA_BE_TEST_LOGS_DIR=${IMPALA_LOGS_DIR}/be_tests
+export IMPALA_EE_TEST_LOGS_DIR=${IMPALA_LOGS_DIR}/ee_tests
+export IMPALA_CUSTOM_CLUSTER_TEST_LOGS_DIR=${IMPALA_LOGS_DIR}/custom_cluster_tests
+# List of all Impala log dirs and create them.
+export IMPALA_ALL_LOGS_DIRS="${IMPALA_CLUSTER_LOGS_DIR}
+  ${IMPALA_DATA_LOADING_LOGS_DIR} ${IMPALA_DATA_LOADING_SQL_DIR}
+  ${IMPALA_EE_TEST_LOGS_DIR} ${IMPALA_BE_TEST_LOGS_DIR}
+  ${IMPALA_EE_TEST_LOGS_DIR} ${IMPALA_CUSTOM_CLUSTER_TEST_LOGS_DIR}"
+mkdir -p $IMPALA_ALL_LOGS_DIRS
+
+# Create symlinks Testing/Temporary and be/Testing/Temporary that point to the BE test
+# log dir to capture the all logs of BE unit tests. Gtest has Testing/Temporary
+# hardwired in its code, so we cannot change the output dir by configuration.
+# We create two symlinks to capture the logs when running ctest either from
+# ${IMPALA_HOME} or ${IMPALA_HOME}/be.
+rm -rf ${IMPALA_HOME}/Testing
+mkdir -p ${IMPALA_HOME}/Testing
+ln -fs ${IMPALA_BE_TEST_LOGS_DIR} ${IMPALA_HOME}/Testing/Temporary
+rm -rf ${IMPALA_HOME}/be/Testing
+mkdir -p ${IMPALA_HOME}/be/Testing
+ln -fs ${IMPALA_BE_TEST_LOGS_DIR} ${IMPALA_HOME}/be/Testing/Temporary
+
 # Reduce the concurrency for local tests to half the number of cores in the system.
 # Note than nproc may not be available on older distributions (centos5.5)
 if type nproc >/dev/null 2>&1; then

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/load-data.py
----------------------------------------------------------------------
diff --git a/bin/load-data.py b/bin/load-data.py
index 9e83099..b0fb535 100755
--- a/bin/load-data.py
+++ b/bin/load-data.py
@@ -60,7 +60,7 @@ parser.add_option("--principal", default=None, dest="principal",
 
 options, args = parser.parse_args()
 
-DATA_LOAD_DIR = '/tmp/data-load-files'
+SQL_OUTPUT_DIR = os.environ['IMPALA_DATA_LOADING_SQL_DIR']
 WORKLOAD_DIR = options.workload_dir
 DATASET_DIR = options.dataset_dir
 TESTDATA_BIN_DIR = os.path.join(os.environ['IMPALA_HOME'], 'testdata/bin')
@@ -257,9 +257,11 @@ if __name__ == "__main__":
     start_time = time.time()
     dataset = get_dataset_for_workload(workload)
     generate_schema_statements(workload)
-    assert os.path.isdir(os.path.join(DATA_LOAD_DIR, dataset)), ("Data loading files "
-        "do not exist for (%s)" % dataset)
-    os.chdir(os.path.join(DATA_LOAD_DIR, dataset))
+    sql_dir = os.path.join(SQL_OUTPUT_DIR, dataset)
+    assert os.path.isdir(sql_dir),\
+      ("Could not find the generated SQL files for loading dataset '%s'.\
+        \nExpected to find the SQL files in: %s" % (dataset, sql_dir))
+    os.chdir(os.path.join(SQL_OUTPUT_DIR, dataset))
     copy_avro_schemas_to_hdfs(AVRO_SCHEMA_DIR)
     dataset_dir_contents = os.listdir(os.getcwd())
     load_file_substr = "%s-%s" % (workload, options.exploration_strategy)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/run-all-tests.sh
----------------------------------------------------------------------
diff --git a/bin/run-all-tests.sh b/bin/run-all-tests.sh
index 67dda02..1023fee 100755
--- a/bin/run-all-tests.sh
+++ b/bin/run-all-tests.sh
@@ -77,8 +77,8 @@ do
   esac
 done
 
-LOG_DIR=${IMPALA_TEST_CLUSTER_LOG_DIR}/query_tests
-mkdir -p ${LOG_DIR}
+# For logging when using run-step.
+LOG_DIR=${IMPALA_EE_TEST_LOGS_DIR}
 
 # Enable core dumps
 ulimit -c unlimited
@@ -96,7 +96,7 @@ do
   TEST_RET_CODE=0
 
   run-step "Starting Impala cluster" start-impala-cluster.log \
-      ${IMPALA_HOME}/bin/start-impala-cluster.py --log_dir=${LOG_DIR} \
+      ${IMPALA_HOME}/bin/start-impala-cluster.py --log_dir=${IMPALA_EE_TEST_LOGS_DIR} \
       ${TEST_START_CLUSTER_ARGS}
 
   if [[ "$BE_TEST" == true ]]; then
@@ -157,7 +157,7 @@ do
   if [[ "$JDBC_TEST" == true ]]; then
     # Run the JDBC tests with background loading disabled. This is interesting because
     # it requires loading missing table metadata.
-    ${IMPALA_HOME}/bin/start-impala-cluster.py --log_dir=${LOG_DIR} \
+    ${IMPALA_HOME}/bin/start-impala-cluster.py --log_dir=${IMPALA_EE_TEST_LOGS_DIR} \
         --catalogd_args=--load_catalog_in_background=false \
         ${TEST_START_CLUSTER_ARGS}
     pushd ${IMPALA_FE_DIR}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/run-backend-tests.sh
----------------------------------------------------------------------
diff --git a/bin/run-backend-tests.sh b/bin/run-backend-tests.sh
index d5c317a..400ee1e 100755
--- a/bin/run-backend-tests.sh
+++ b/bin/run-backend-tests.sh
@@ -4,9 +4,7 @@
 set -euo pipefail
 trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
 
-# Prepare output directory
-mkdir -p $IMPALA_TEST_CLUSTER_LOG_DIR/be_test
-export GTEST_OUTPUT="xml:$IMPALA_TEST_CLUSTER_LOG_DIR/be_test/"
+export GTEST_OUTPUT="xml:$IMPALA_BE_TEST_LOGS_DIR/"
 
 : ${SKIP_BE_TEST_PATTERN:=}
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/bin/start-impala-cluster.py
----------------------------------------------------------------------
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index d3c7b77..1302a66 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -51,7 +51,7 @@ parser.add_option("-r", "--restart_impalad_only", dest="restart_impalad_only",
                   help="Restarts only the impalad processes")
 parser.add_option("--in-process", dest="inprocess", action="store_true", default=False,
                   help="Start all Impala backends and state store in a single process.")
-parser.add_option("--log_dir", dest="log_dir", default="/tmp",
+parser.add_option("--log_dir", dest="log_dir", default=os.environ['IMPALA_CLUSTER_LOGS_DIR'],
                   help="Directory to store output logs to.")
 parser.add_option("-v", "--verbose", dest="verbose", action="store_true", default=False,
                   help="Prints all output to stderr/stdout.")
@@ -327,6 +327,10 @@ if __name__ == "__main__":
     print 'Please specify a cluster size >= 0'
     sys.exit(1)
 
+  if not os.path.isdir(options.log_dir):
+    print 'Log dir does not exist or is not a directory: %s' % options.log_dir
+    sys.exit(1)
+
   # Kill existing cluster processes based on the current configuration.
   if options.restart_impalad_only:
     if options.inprocess:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/buildall.sh
----------------------------------------------------------------------
diff --git a/buildall.sh b/buildall.sh
index 8f1afe7..e61ccb1 100755
--- a/buildall.sh
+++ b/buildall.sh
@@ -298,11 +298,6 @@ ${IMPALA_HOME}/shell/make_shell_tarball.sh
 echo "Creating test tarball"
 ${IMPALA_HOME}/tests/make_test_tarball.sh
 
-# Create subdirectories for the test and data loading impalad logs.
-mkdir -p ${IMPALA_TEST_CLUSTER_LOG_DIR}/query_tests
-mkdir -p ${IMPALA_TEST_CLUSTER_LOG_DIR}/fe_tests
-mkdir -p ${IMPALA_TEST_CLUSTER_LOG_DIR}/data_loading
-
 if [ $FORMAT_CLUSTER -eq 1 ]; then
   $IMPALA_HOME/testdata/bin/run-all.sh -format
 elif [ $TESTDATA_ACTION -eq 1 ] || [ $TESTS_ACTION -eq 1 ]; then

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/fe/pom.xml
----------------------------------------------------------------------
diff --git a/fe/pom.xml b/fe/pom.xml
index 1d36475..982e05d 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -26,6 +26,7 @@
   <name>Cloudera Impala Query Engine Frontend</name>
 
   <properties>
+    <surefire.reports.dir>${env.IMPALA_LOGS_DIR}/fe_tests</surefire.reports.dir>
     <test.hive.testdata>${project.basedir}/../testdata/target/AllTypes.txt</test.hive.testdata>
     <backend.library.path>${env.IMPALA_HOME}/be/build/debug/service:${env.IMPALA_HOME}/be/build/release/service</backend.library.path>
     <beeswax_port>21000</beeswax_port>
@@ -459,6 +460,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <version>2.18</version>
         <configuration>
+          <reportsDirectory>${surefire.reports.dir}</reportsDirectory>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <argLine>-Djava.library.path=${java.library.path}:${backend.library.path}</argLine>
           <systemProperties>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/fe/src/test/resources/hbase-site.xml.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/hbase-site.xml.template b/fe/src/test/resources/hbase-site.xml.template
index 5059fa5..7a72924 100644
--- a/fe/src/test/resources/hbase-site.xml.template
+++ b/fe/src/test/resources/hbase-site.xml.template
@@ -53,7 +53,7 @@
 
   <property>
     <name>hbase.zookeeper.property.dataDir</name>
-    <value>${IMPALA_TEST_CLUSTER_LOG_DIR}/zoo</value>
+    <value>${IMPALA_CLUSTER_LOGS_DIR}/zoo</value>
   </property>
 
   <!-- BEGIN Kerberos settings -->

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/fe/src/test/resources/hive-log4j.properties.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/hive-log4j.properties.template b/fe/src/test/resources/hive-log4j.properties.template
index 2728977..e847e38 100644
--- a/fe/src/test/resources/hive-log4j.properties.template
+++ b/fe/src/test/resources/hive-log4j.properties.template
@@ -1,6 +1,6 @@
 # Define some default values that can be overridden by system properties
 hive.root.logger=INFO,DRFA
-hive.log.dir=${IMPALA_TEST_CLUSTER_LOG_DIR}/hive
+hive.log.dir=${IMPALA_CLUSTER_LOGS_DIR}/hive
 hive.log.file=hive.log
 
 # Define the root logger to the system property "hadoop.root.logger".

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/fe/src/test/resources/mysql-hive-site.xml.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/mysql-hive-site.xml.template b/fe/src/test/resources/mysql-hive-site.xml.template
index bab3eb2..add10dc 100644
--- a/fe/src/test/resources/mysql-hive-site.xml.template
+++ b/fe/src/test/resources/mysql-hive-site.xml.template
@@ -96,7 +96,7 @@
 <property>
   <!--  Location of Hive per-query log files of the form: hive_job_log_<hive_query_id>.txt -->
   <name>hive.querylog.location</name>
-  <value>${IMPALA_TEST_CLUSTER_LOG_DIR}/hive</value>
+  <value>${IMPALA_CLUSTER_LOGS_DIR}/hive</value>
 </property>
 
 <!--property>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/fe/src/test/resources/postgresql-hive-site.xml.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/postgresql-hive-site.xml.template b/fe/src/test/resources/postgresql-hive-site.xml.template
index e416635..7e21433 100644
--- a/fe/src/test/resources/postgresql-hive-site.xml.template
+++ b/fe/src/test/resources/postgresql-hive-site.xml.template
@@ -99,7 +99,7 @@
 <property>
   <!--  Location of Hive per-query log files of the form: hive_job_log_<hive_query_id>.txt -->
   <name>hive.querylog.location</name>
-  <value>${IMPALA_TEST_CLUSTER_LOG_DIR}/hive</value>
+  <value>${IMPALA_CLUSTER_LOGS_DIR}/hive</value>
 </property>
 
 <!--property>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index 9b2699b..6323e62 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -35,8 +35,8 @@ SKIP_SNAPSHOT_LOAD=0
 SNAPSHOT_FILE=""
 LOAD_DATA_ARGS=""
 JDBC_URL="jdbc:hive2://localhost:11050/default;"
-LOG_DIR=${IMPALA_TEST_CLUSTER_LOG_DIR}/data_loading
-mkdir -p ${LOG_DIR}
+# For logging when using run-step.
+LOG_DIR=${IMPALA_DATA_LOADING_LOGS_DIR}
 
 while [ -n "$*" ]
 do
@@ -141,7 +141,7 @@ function load-data {
     ARGS+=("--force")
     echo "Force loading."
   fi
-  LOG_FILE=${LOG_DIR}/data-load-${WORKLOAD}-${EXPLORATION_STRATEGY}.log
+  LOG_FILE=${IMPALA_DATA_LOADING_LOGS_DIR}/data-load-${WORKLOAD}-${EXPLORATION_STRATEGY}.log
   echo "$MSG. Logging to ${LOG_FILE}"
   # Use unbuffered logging by executing with -u
   if ! impala-python -u ${IMPALA_HOME}/bin/load-data.py ${ARGS[@]} &> ${LOG_FILE}; then
@@ -162,7 +162,7 @@ function cache-test-tables {
 }
 
 function load-aux-workloads {
-  LOG_FILE=${LOG_DIR}/data-load-auxiliary-workloads-core.log
+  LOG_FILE=${IMPALA_DATA_LOADING_LOGS_DIR}/data-load-auxiliary-workloads-core.log
   rm -f $LOG_FILE
   # Load all the auxiliary workloads (if any exist)
   if [ -d ${IMPALA_AUX_WORKLOAD_DIR} ] && [ -d ${IMPALA_AUX_DATASET_DIR} ]; then
@@ -339,8 +339,8 @@ else
   START_CLUSTER_ARGS="-s 3 ${START_CLUSTER_ARGS}"
 fi
 run-step "Starting Impala cluster" start-impala-cluster.log \
-    ${IMPALA_HOME}/bin/start-impala-cluster.py --log_dir=${LOG_DIR} \
-    ${START_CLUSTER_ARGS}
+    ${IMPALA_HOME}/bin/start-impala-cluster.py \
+      --log_dir=${IMPALA_DATA_LOADING_LOGS_DIR} ${START_CLUSTER_ARGS}
 # The hdfs environment script sets up kms (encryption) and cache pools (hdfs caching).
 # On a non-hdfs filesystem, we don't test encryption or hdfs caching, so this setup is not
 # needed.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/generate-schema-statements.py
----------------------------------------------------------------------
diff --git a/testdata/bin/generate-schema-statements.py b/testdata/bin/generate-schema-statements.py
index f4b95a0..f06eb52 100755
--- a/testdata/bin/generate-schema-statements.py
+++ b/testdata/bin/generate-schema-statements.py
@@ -73,9 +73,9 @@ if options.workload is None:
   parser.print_help()
   sys.exit(1)
 
-DATA_LOAD_DIR = '/tmp/data-load-files'
 WORKLOAD_DIR = os.path.join(os.environ['IMPALA_HOME'], 'testdata', 'workloads')
 DATASET_DIR = os.path.join(os.environ['IMPALA_HOME'], 'testdata', 'datasets')
+SQL_OUTPUT_DIR = os.environ['IMPALA_DATA_LOADING_SQL_DIR']
 AVRO_SCHEMA_DIR = "avro_schemas"
 IMPALA_SUPPORTED_INSERT_FORMATS = ['parquet', 'hbase', 'text', 'kudu']
 
@@ -690,7 +690,7 @@ if __name__ == "__main__":
 
   target_dataset = test_vectors[0].dataset
   print 'Target Dataset: ' + target_dataset
-  dataset_load_dir = os.path.join(DATA_LOAD_DIR, target_dataset)
+  dataset_load_dir = os.path.join(SQL_OUTPUT_DIR, target_dataset)
   # If the directory containing the sql files does not exist, create it. Else nuke all the
   # files corresponding to the current workload.
   try:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/minikdc_env.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/minikdc_env.sh b/testdata/bin/minikdc_env.sh
index 70c4985..3c41770 100644
--- a/testdata/bin/minikdc_env.sh
+++ b/testdata/bin/minikdc_env.sh
@@ -51,7 +51,7 @@ export MINIKDC_PRINC_USER=${USER}/localhost@${MINIKDC_REALM}
 export MINIKDC_PRINC_LLAM=llama/localhost@${MINIKDC_REALM}
 
 # Basic directory setup:
-MINIKDC_SCRATCH_ROOT=${MINIKDC_SCRATCH_ROOT-${IMPALA_TEST_CLUSTER_LOG_DIR}}
+MINIKDC_SCRATCH_ROOT=${MINIKDC_SCRATCH_ROOT-${IMPALA_CLUSTER_LOGS_DIR}}
 export MINIKDC_WD=${MINIKDC_SCRATCH_ROOT}/minikdc-workdir
 
 # The one big keytab created by the minikdc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/run-all.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-all.sh b/testdata/bin/run-all.sh
index 34cac08..2df6618 100755
--- a/testdata/bin/run-all.sh
+++ b/testdata/bin/run-all.sh
@@ -31,7 +31,7 @@ fi
 
 # Kill and clean data for a clean start.
 echo "Killing running services..."
-$IMPALA_HOME/testdata/bin/kill-all.sh &>${IMPALA_TEST_CLUSTER_LOG_DIR}/kill-all.log
+$IMPALA_HOME/testdata/bin/kill-all.sh &>${IMPALA_CLUSTER_LOGS_DIR}/kill-all.log
 
 # Starts up a mini-cluster which includes:
 # - HDFS with 3 DNs
@@ -41,25 +41,25 @@ if [[ ${DEFAULT_FS} == "hdfs://localhost:20500" ]]; then
   echo "Starting all cluster services..."
   echo " --> Starting mini-DFS cluster"
   $IMPALA_HOME/testdata/bin/run-mini-dfs.sh ${HDFS_FORMAT_CLUSTER-} 2>&1 | \
-      tee ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-mini-dfs.log
+      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-mini-dfs.log
 
   echo " --> Starting HBase"
   $IMPALA_HOME/testdata/bin/run-hbase.sh 2>&1 | \
-      tee ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-hbase.log
+      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-hbase.log
 
   echo " --> Starting Hive Server and Metastore Service"
   $IMPALA_HOME/testdata/bin/run-hive-server.sh 2>&1 | \
-      tee ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-hive-server.log
+      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-hive-server.log
 
   echo " --> Starting the Sentry Policy Server"
   $IMPALA_HOME/testdata/bin/run-sentry-service.sh > \
-      ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-sentry-service.log 2>&1
+      ${IMPALA_CLUSTER_LOGS_DIR}/run-sentry-service.log 2>&1
 
 elif [[ ${DEFAULT_FS} == "${LOCAL_FS}" ]]; then
   # When the local file system is used as default, we only start the Hive metastore.
   # Impala can run locally without additional services.
   $IMPALA_HOME/testdata/bin/run-hive-server.sh -only_metastore 2>&1 | \
-      tee ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-hive-server.log
+      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-hive-server.log
 else
   # With Isilon, we only start the Hive metastore and Sentry Policy Server.
   #   - HDFS is not started becuase Isilon is used as the defaultFs in core-site
@@ -72,9 +72,9 @@ else
   # TODO: Figure out how to start YARN, LLAMA and Hive with a different defaultFs.
   echo " --> Starting Hive Metastore Service"
   $IMPALA_HOME/testdata/bin/run-hive-server.sh -only_metastore 2>&1 | \
-      tee ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-hive-server.log
+      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-hive-server.log
 
   echo " --> Starting the Sentry Policy Server"
   $IMPALA_HOME/testdata/bin/run-sentry-service.sh > \
-      ${IMPALA_TEST_CLUSTER_LOG_DIR}/run-sentry-service.log 2>&1
+      ${IMPALA_CLUSTER_LOGS_DIR}/run-sentry-service.log 2>&1
 fi

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/run-hbase.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-hbase.sh b/testdata/bin/run-hbase.sh
index 4dd5b76..2d76a69 100755
--- a/testdata/bin/run-hbase.sh
+++ b/testdata/bin/run-hbase.sh
@@ -7,7 +7,7 @@ trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)
 CLUSTER_BIN=${IMPALA_HOME}/testdata/bin
 HBASE_JAAS_CLIENT=${HBASE_CONF_DIR}/hbase-jaas-client.conf
 HBASE_JAAS_SERVER=${HBASE_CONF_DIR}/hbase-jaas-server.conf
-HBASE_LOGDIR=${IMPALA_TEST_CLUSTER_LOG_DIR}/hbase
+HBASE_LOGDIR=${IMPALA_CLUSTER_LOGS_DIR}/hbase
 
 # Kill and clean data for a clean start.
 ${CLUSTER_BIN}/kill-hbase.sh > /dev/null 2>&1
@@ -22,10 +22,10 @@ export HBASE_LOG_DIR=${HBASE_LOGDIR}
 export HBASE_PID_DIR=${HBASE_LOGDIR}
 EOF
 
-# Put zookeeper things in the cluster_logs/zoo directory.
+# Put zookeeper things in the logs/cluster/zoo directory.
 # (See hbase.zookeeper.property.dataDir in hbase-site.xml)
-rm -rf ${IMPALA_TEST_CLUSTER_LOG_DIR}/zoo
-mkdir -p ${IMPALA_TEST_CLUSTER_LOG_DIR}/zoo
+rm -rf ${IMPALA_CLUSTER_LOGS_DIR}/zoo
+mkdir -p ${IMPALA_CLUSTER_LOGS_DIR}/zoo
 mkdir -p ${HBASE_LOGDIR}
 
 if ${CLUSTER_DIR}/admin is_kerberized; then

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/bin/run-hive-server.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-hive-server.sh b/testdata/bin/run-hive-server.sh
index 23df556..86832cb 100755
--- a/testdata/bin/run-hive-server.sh
+++ b/testdata/bin/run-hive-server.sh
@@ -9,7 +9,7 @@ trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)
 HIVE_SERVER_PORT=10000
 export HIVE_SERVER2_THRIFT_PORT=11050
 HIVE_METASTORE_PORT=9083
-LOGDIR=${IMPALA_HOME}/cluster_logs/hive
+LOGDIR=${IMPALA_CLUSTER_LOGS_DIR}/hive
 HIVES2_TRANSPORT="plain_sasl"
 METASTORE_TRANSPORT="buffered"
 ONLY_METASTORE=0

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/testdata/cluster/admin
----------------------------------------------------------------------
diff --git a/testdata/cluster/admin b/testdata/cluster/admin
index ba5f975..286d526 100755
--- a/testdata/cluster/admin
+++ b/testdata/cluster/admin
@@ -82,7 +82,7 @@ if [[ "$CDH_MAJOR_VERSION" -ge 5 ]]; then
   EMPTY_NODE_DIRS+=" /var/log/llama"
 fi
 
-EASY_ACCESS_LOG_DIR="$IMPALA_HOME/cluster_logs"
+EASY_ACCESS_LOG_DIR="$IMPALA_CLUSTER_LOGS_DIR"
 MINIKDC_INIT=${IMPALA_HOME}/testdata/bin/minikdc.sh
 
 if $IS_OSX; then
@@ -246,7 +246,7 @@ function create_cluster {
     if [[ ! -e "$EASY_ACCESS_LOG_LINK" ]]; then
       mkdir -p "$EASY_ACCESS_LOG_DIR"
       ln -s "$NODE_DIR/var/log" "$EASY_ACCESS_LOG_DIR"
-      mv "$IMPALA_HOME/cluster_logs/log" "$EASY_ACCESS_LOG_LINK"
+      mv "$IMPALA_CLUSTER_LOGS_DIR/log" "$EASY_ACCESS_LOG_LINK"
     fi
 
     # Template population

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7e76e92b/tests/run-custom-cluster-tests.sh
----------------------------------------------------------------------
diff --git a/tests/run-custom-cluster-tests.sh b/tests/run-custom-cluster-tests.sh
index 9339894..0131546 100755
--- a/tests/run-custom-cluster-tests.sh
+++ b/tests/run-custom-cluster-tests.sh
@@ -23,17 +23,14 @@ trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)
 # TODO: Combine with run-process-failure-tests.sh
 export HEAPCHECK=
 
-RESULTS_DIR=${IMPALA_HOME}/tests/custom_cluster/results
-mkdir -p ${RESULTS_DIR}
-LOG_DIR=${IMPALA_TEST_CLUSTER_LOG_DIR}/custom_cluster/
-mkdir -p ${LOG_DIR}
-
 AUX_CUSTOM_DIR=""
 if [ -n ${IMPALA_AUX_TEST_HOME} ]; then
     AUX_CUSTOM_DIR=${IMPALA_AUX_TEST_HOME}/tests/aux_custom_cluster_tests/
 fi
 
-export LOG_DIR
+export LOG_DIR=${IMPALA_CUSTOM_CLUSTER_TEST_LOGS_DIR}
+RESULTS_DIR=${IMPALA_CUSTOM_CLUSTER_TEST_LOGS_DIR}/results
+mkdir -p ${RESULTS_DIR}
 
 # KERBEROS TODO We'll want to pass kerberos status in here.
 cd ${IMPALA_HOME}/tests


[38/50] incubator-impala git commit: IMPALA-3285: Fix ASAN failure in webserver-test

Posted by ta...@apache.org.
IMPALA-3285: Fix ASAN failure in webserver-test

This patch fixes a test ASAN issue where a RapidJson value was 'moved',
rather than copied, into a Document object; the original was the deleted
and the moved version became invalid.

Change-Id: Ib0c71163af14c6f18e785938712caf193a393794
Reviewed-on: http://gerrit.cloudera.org:8080/2737
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: b085e31ff415e5eea56fa7dc6d35b508ed313e7c
Parents: 9bb1b8a
Author: Henry Robinson <he...@cloudera.com>
Authored: Fri Apr 8 13:10:40 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:43 2016 -0700

----------------------------------------------------------------------
 be/src/util/webserver-test.cc | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b085e31f/be/src/util/webserver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index 392334b..094587c 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -289,8 +289,9 @@ TEST(Webserver, DirectoryListingDisabledTest) {
 }
 
 void FrameCallback(const Webserver::ArgumentMap& args, Document* document) {
-  const string& contents = "<frameset cols='50%,50%'><frame src='/metrics'></frameset>";
-  document->AddMember("contents", contents.c_str(), document->GetAllocator());
+  const string contents = "<frameset cols='50%,50%'><frame src='/metrics'></frameset>";
+  Value value(contents.c_str(), document->GetAllocator());
+  document->AddMember("contents", value, document->GetAllocator());
 }
 
 TEST(Webserver, NoFrameEmbeddingTest) {


[02/50] incubator-impala git commit: IMPALA-2973: Loosen bound on join timer test

Posted by ta...@apache.org.
IMPALA-2973: Loosen bound on join timer test

Code coverage builds are slower than release or debug builds. This
patch gives test_hash_join_timer extra time so this test doesn't cause
code coverage builds don't fail in Jenkins.

Change-Id: I5598e073d779f744d79c5292e80a8ed8f6aa9548
Reviewed-on: http://gerrit.cloudera.org:8080/2608
Reviewed-by: Michael Brown <mi...@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/c0397911
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/c0397911
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/c0397911

Branch: refs/heads/master
Commit: c0397911b5aec4733b284c9c76fc554db64ffbef
Parents: a75d7dd
Author: Jim Apple <jb...@cloudera.com>
Authored: Wed Mar 23 10:25:25 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 23 22:25:13 2016 +0000

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


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0397911/tests/query_test/test_hash_join_timer.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hash_join_timer.py b/tests/query_test/test_hash_join_timer.py
index abbcab5..1979759 100644
--- a/tests/query_test/test_hash_join_timer.py
+++ b/tests/query_test/test_hash_join_timer.py
@@ -44,7 +44,9 @@ class TestHashJoinTimer(ImpalaTestSuite):
               " where a.id>b.id and a.id=99",
               "NESTED LOOP JOIN"]
              ]
-  HASH_JOIN_UPPER_BOUND_MS = 1000
+  # IMPALA-2973: For non-code-coverage builds, 1000 milliseconds are sufficent, but more
+  # time is needed in code-coverage builds.
+  HASH_JOIN_UPPER_BOUND_MS = 2000
   # IMPALA-2973: Temporary workaround: when timers are using Linux COARSE clockid_t, very
   # short times may be measured as zero.
   HASH_JOIN_LOWER_BOUND_MS = 0


[47/50] incubator-impala git commit: IMPALA-3276: consistently handle pin failure in BTS::PrepareForRead()

Posted by ta...@apache.org.
IMPALA-3276: consistently handle pin failure in BTS::PrepareForRead()

BufferedTupleStream::PrepareForRead() can fail in all scenarios except
when the stream is all pinned. The basic problem is that reservations
in the BufferedBlockMgr are not dependable since they can be
overcommitted.

This patch changes the interface so that all callers have to handle pin
failures and updates the callsites.

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

Branch: refs/heads/master
Commit: 36c294b55e64b6b9dd1c0fca30205a05db24b120
Parents: 92fafa1
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Mar 30 13:52:44 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 be/src/exec/analytic-eval-node.cc            | 10 +++++-
 be/src/exec/partitioned-hash-join-node.cc    | 41 ++++++++++++++++++++---
 be/src/runtime/buffered-tuple-stream-test.cc | 24 +++++++++----
 be/src/runtime/buffered-tuple-stream.cc      |  7 ++--
 be/src/runtime/buffered-tuple-stream.h       |  7 ++--
 5 files changed, 70 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/36c294b5/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 94725ae..9442a78 100644
--- a/be/src/exec/analytic-eval-node.cc
+++ b/be/src/exec/analytic-eval-node.cc
@@ -190,7 +190,15 @@ Status AnalyticEvalNode::Open(RuntimeState* state) {
       state->block_mgr(), client_, false /* use_initial_small_buffers */,
       true /* read_write */);
   RETURN_IF_ERROR(input_stream_->Init(id(), runtime_profile(), true));
-  RETURN_IF_ERROR(input_stream_->PrepareForRead(true));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(input_stream_->PrepareForRead(true, &got_read_buffer));
+  if (!got_read_buffer) {
+    Status status = Status::MemLimitExceeded();
+    status.AddDetail("Failed to acquire initial read buffer for analytic function "
+        "evaluation. Reducing query concurrency or increasing the memory limit may "
+        "help this query to complete successfully.");
+    return status;
+  }
 
   DCHECK_EQ(evaluators_.size(), fn_ctxs_.size());
   for (int i = 0; i < evaluators_.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/36c294b5/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 151b0c6..b22dff1 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -41,6 +41,10 @@
 DEFINE_bool(enable_phj_probe_side_filtering, true,
     "Enables pushing PHJ build side filters to probe side");
 
+const string PREPARE_FOR_READ_FAILED_ERROR_MSG = "Failed to acquire initial read buffer "
+    "for stream in hash join node $0. Reducing query concurrency or increasing the "
+    "memory limit may help this query to complete successfully.";
+
 using namespace impala;
 using namespace llvm;
 using namespace strings;
@@ -433,7 +437,9 @@ Status PartitionedHashJoinNode::Partition::BuildHashTableInternal(
   // We got the buffers we think we will need, try to build the hash table.
   RETURN_IF_ERROR(build_rows_->PinStream(false, built));
   if (!*built) return Status::OK();
-  RETURN_IF_ERROR(build_rows_->PrepareForRead(false));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(build_rows_->PrepareForRead(false, &got_read_buffer));
+  DCHECK(got_read_buffer) << "Stream was already pinned.";
 
   RowBatch batch(parent_->child(1)->row_desc(), state->batch_size(),
       parent_->mem_tracker());
@@ -623,7 +629,14 @@ Status PartitionedHashJoinNode::ProcessBuildInput(RuntimeState* state, int level
   if (input_partition_ != NULL) {
     DCHECK(input_partition_->build_rows() != NULL);
     DCHECK_EQ(input_partition_->build_rows()->blocks_pinned(), 0) << NodeDebugString();
-    RETURN_IF_ERROR(input_partition_->build_rows()->PrepareForRead(true));
+    bool got_read_buffer;
+    RETURN_IF_ERROR(
+        input_partition_->build_rows()->PrepareForRead(true, &got_read_buffer));
+    if (!got_read_buffer) {
+      Status status = Status::MemLimitExceeded();
+      status.AddDetail(Substitute(PREPARE_FOR_READ_FAILED_ERROR_MSG, id_));
+      return status;
+    }
   }
 
   for (int i = 0; i < PARTITION_FANOUT; ++i) {
@@ -788,7 +801,13 @@ Status PartitionedHashJoinNode::PrepareNextPartition(RuntimeState* state) {
   DCHECK(input_partition_->is_spilled());
 
   // Reserve one buffer to read the probe side.
-  RETURN_IF_ERROR(input_partition_->probe_rows()->PrepareForRead(true));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(input_partition_->probe_rows()->PrepareForRead(true, &got_read_buffer));
+  if (!got_read_buffer) {
+    Status status = Status::MemLimitExceeded();
+    status.AddDetail(Substitute(PREPARE_FOR_READ_FAILED_ERROR_MSG, id_));
+    return status;
+  }
   ht_ctx_->set_level(input_partition_->level_);
 
   int64_t mem_limit = mem_tracker()->SpareCapacity();
@@ -1045,7 +1064,13 @@ void PartitionedHashJoinNode::OutputUnmatchedBuild(RowBatch* out_batch) {
 
 Status PartitionedHashJoinNode::PrepareNullAwareNullProbe() {
   DCHECK_EQ(null_probe_output_idx_, -1);
-  RETURN_IF_ERROR(null_probe_rows_->PrepareForRead(true));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(null_probe_rows_->PrepareForRead(true, &got_read_buffer));
+  if (!got_read_buffer) {
+    Status status = Status::MemLimitExceeded();
+    status.AddDetail(Substitute(PREPARE_FOR_READ_FAILED_ERROR_MSG, id_));
+    return status;
+  }
   DCHECK_EQ(probe_batch_->num_rows(), 0);
   probe_batch_pos_ = 0;
   null_probe_output_idx_ = 0;
@@ -1120,7 +1145,13 @@ Status PartitionedHashJoinNode::PrepareNullAwarePartition() {
   if (!got_rows) return NullAwareAntiJoinError(true);
 
   // Initialize the streams for read.
-  RETURN_IF_ERROR(probe_stream->PrepareForRead(true));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(probe_stream->PrepareForRead(true, &got_read_buffer));
+  if (!got_read_buffer) {
+    Status status = Status::MemLimitExceeded();
+    status.AddDetail(Substitute(PREPARE_FOR_READ_FAILED_ERROR_MSG, id_));
+    return status;
+  }
   probe_batch_pos_ = 0;
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/36c294b5/be/src/runtime/buffered-tuple-stream-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream-test.cc b/be/src/runtime/buffered-tuple-stream-test.cc
index 0323cb8..c1633a9 100644
--- a/be/src/runtime/buffered-tuple-stream-test.cc
+++ b/be/src/runtime/buffered-tuple-stream-test.cc
@@ -314,7 +314,9 @@ class SimpleTupleStreamTest : public testing::Test {
       batch->Reset();
     }
 
-    ASSERT_OK(stream.PrepareForRead(false));
+    bool got_read_buffer;
+    ASSERT_OK(stream.PrepareForRead(false, &got_read_buffer));
+    ASSERT_TRUE(got_read_buffer);
 
     // Read all the rows back
     vector<T> results;
@@ -333,7 +335,9 @@ class SimpleTupleStreamTest : public testing::Test {
           client_, small_buffers == 0,  // initial small buffers
           true); // read_write
       ASSERT_OK(stream.Init(-1, NULL, true));
-      ASSERT_OK(stream.PrepareForRead(true));
+      bool got_read_buffer;
+      ASSERT_OK(stream.PrepareForRead(true, &got_read_buffer));
+      ASSERT_TRUE(got_read_buffer);
       if (unpin_stream) ASSERT_OK(stream.UnpinStream());
 
       vector<int> results;
@@ -575,7 +579,9 @@ void SimpleTupleStreamTest::TestUnpinPin(bool varlen_data) {
   int read_iters = 3;
   for (int i = 0; i < read_iters; ++i) {
     bool delete_on_read = i == read_iters - 1;
-    ASSERT_OK(stream.PrepareForRead(delete_on_read));
+    bool got_read_buffer;
+    ASSERT_OK(stream.PrepareForRead(delete_on_read, &got_read_buffer));
+    ASSERT_TRUE(got_read_buffer);
 
     if (varlen_data) {
       vector<StringValue> results;
@@ -690,7 +696,9 @@ TEST_F(SimpleTupleStreamTest, StringsOutsideStream) {
   for (int delete_on_read = 0; delete_on_read <= 1; ++delete_on_read) {
     // Keep stream in memory and test we can read ok.
     vector<StringValue> results;
-    stream.PrepareForRead(delete_on_read);
+    bool got_read_buffer;
+    ASSERT_OK(stream.PrepareForRead(delete_on_read, &got_read_buffer));
+    ASSERT_TRUE(got_read_buffer);
     ReadValues(&stream, string_desc_, &results);
     VerifyResults<StringValue>(*string_desc_, results, rows_added, false);
   }
@@ -860,7 +868,9 @@ TEST_F(MultiTupleStreamTest, MultiTupleAllocateRow) {
   for (int i = 0; i < 3; ++i) {
     bool delete_on_read = i == 2;
     vector<StringValue> results;
-    stream.PrepareForRead(delete_on_read);
+    bool got_read_buffer;
+    stream.PrepareForRead(delete_on_read, &got_read_buffer);
+    ASSERT_TRUE(got_read_buffer);
     ReadValues(&stream, string_desc_, &results);
     VerifyResults<StringValue>(*string_desc_, results, rows_added, false);
   }
@@ -1021,7 +1031,9 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) {
   }
 
   // Read back and verify data.
-  stream.PrepareForRead(false);
+  bool got_read_buffer;
+  stream.PrepareForRead(false, &got_read_buffer);
+  ASSERT_TRUE(got_read_buffer);
   strings_index = 0;
   array_len_index = 0;
   bool eos = false;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/36c294b5/be/src/runtime/buffered-tuple-stream.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream.cc b/be/src/runtime/buffered-tuple-stream.cc
index 6a487bc..86eedc4 100644
--- a/be/src/runtime/buffered-tuple-stream.cc
+++ b/be/src/runtime/buffered-tuple-stream.cc
@@ -398,7 +398,6 @@ Status BufferedTupleStream::PrepareForRead(bool delete_on_read, bool* got_buffer
       bool current_pinned;
       RETURN_IF_ERROR((*it)->Pin(&current_pinned));
       if (!current_pinned) {
-        DCHECK(got_buffer != NULL) << "Should have reserved enough blocks";
         *got_buffer = false;
         return Status::OK();
       }
@@ -419,7 +418,7 @@ Status BufferedTupleStream::PrepareForRead(bool delete_on_read, bool* got_buffer
   rows_returned_ = 0;
   read_block_idx_ = 0;
   delete_on_read_ = delete_on_read;
-  if (got_buffer != NULL) *got_buffer = true;
+  *got_buffer = true;
   return Status::OK();
 }
 
@@ -502,7 +501,9 @@ int BufferedTupleStream::ComputeNumNullIndicatorBytes(int block_size) const {
 Status BufferedTupleStream::GetRows(scoped_ptr<RowBatch>* batch, bool* got_rows) {
   RETURN_IF_ERROR(PinStream(false, got_rows));
   if (!*got_rows) return Status::OK();
-  RETURN_IF_ERROR(PrepareForRead(false));
+  bool got_read_buffer;
+  RETURN_IF_ERROR(PrepareForRead(false, &got_read_buffer));
+  DCHECK(got_read_buffer) << "Stream was pinned";
   batch->reset(
       new RowBatch(desc_, num_rows(), block_mgr_->get_tracker(block_mgr_client_)));
   bool eos = false;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/36c294b5/be/src/runtime/buffered-tuple-stream.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-tuple-stream.h b/be/src/runtime/buffered-tuple-stream.h
index 75a69be..23c09a9 100644
--- a/be/src/runtime/buffered-tuple-stream.h
+++ b/be/src/runtime/buffered-tuple-stream.h
@@ -256,10 +256,9 @@ class BufferedTupleStream {
   /// begin reading. Otherwise this must be called after the last AddRow() and
   /// before GetNext().
   /// delete_on_read: Blocks are deleted after they are read.
-  /// If got_buffer is NULL, this function will fail (with a bad status) if no buffer
-  /// is available. If got_buffer is non-null, this function will not fail on OOM and
-  /// *got_buffer is true if a buffer was pinned.
-  Status PrepareForRead(bool delete_on_read, bool* got_buffer = NULL);
+  /// got_buffer: set to true if the first read block was successfully pinned, or
+  ///     false if the block could not be pinned and no error was encountered.
+  Status PrepareForRead(bool delete_on_read, bool* got_buffer);
 
   /// Pins all blocks in this stream and switches to pinned mode.
   /// If there is not enough memory, *pinned is set to false and the stream is unmodified.


[44/50] incubator-impala git commit: IMPALA-3148. Fix selectivity computation for pushed Kudu predicates

Posted by ta...@apache.org.
IMPALA-3148. Fix selectivity computation for pushed Kudu predicates

This follows up on a TODO from the Kudu merge and also fixes a bug:
IMPALA-976 changed the computation of selectivities for a combined
list of conjuncts to better handle expressions with no selectivity
estimate. The Kudu implementation was forked from before this change
and thus did not have an equivalent change.

This refactors the algorithm to a new static method and calls it from
both PlanNode and KuduScanNode so that the selectivity estimate
behavior is the same regardless of whether Kudu can evaluate the
predicate server-side.

Todd tested this on TPCH 3TB and verified that the plans are reasonable
now where they used to be nonsense.

Change-Id: Id507077b577ed5804fc80517f33ea185f2bff41a
Reviewed-on: http://gerrit.cloudera.org:8080/2628
Reviewed-by: Casey Ching <ca...@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/4bdd0b97
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/4bdd0b97
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/4bdd0b97

Branch: refs/heads/master
Commit: 4bdd0b976dc80f64ebd1f2c37ad03b4ec661cf02
Parents: 86fd262
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Mar 24 22:53:33 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 .../cloudera/impala/planner/KuduScanNode.java   | 18 ++---
 .../com/cloudera/impala/planner/PlanNode.java   | 10 ++-
 .../queries/PlannerTest/kudu-selectivity.test   | 71 +++++++++++++++++++-
 3 files changed, 80 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4bdd0b97/fe/src/main/java/com/cloudera/impala/planner/KuduScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/KuduScanNode.java b/fe/src/main/java/com/cloudera/impala/planner/KuduScanNode.java
index 558c282..ee5c318 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/KuduScanNode.java
@@ -29,6 +29,7 @@ import com.cloudera.impala.common.ImpalaRuntimeException;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import org.kududb.client.KuduClient;
 import org.kududb.client.LocatedTablet;
 import org.slf4j.Logger;
@@ -151,20 +152,11 @@ public class KuduScanNode extends ScanNode {
     }
   }
 
-  /**
-   * TODO(kudu-merge): IMPALA-3148 - Update selectivity computation.
-   */
   @Override
   protected double computeSelectivity() {
-    double baseSelectivity = super.computeSelectivity();
-    // The kuduConjuncts_ are not part of the selectivity calculation of the PlanNode
-    // superclass. Adjust the selectivity to account for predicates that are
-    // pushed down to Kudu.
-    for (Expr e : kuduConjuncts_) {
-      if (baseSelectivity < 0) continue;
-      baseSelectivity *= e.getSelectivity();
-    }
-    return baseSelectivity;
+    List<Expr> allConjuncts = Lists.newArrayList(
+        Iterables.concat(conjuncts_, kuduConjuncts_));
+    return computeCombinedSelectivity(allConjuncts);
   }
 
   @Override
@@ -201,7 +193,7 @@ public class KuduScanNode extends ScanNode {
         }
         if (!kuduConjuncts_.isEmpty()) {
           result.append(detailPrefix + "kudu predicates: " + getExplainString(
-              kuduConjuncts_));
+              kuduConjuncts_) + "\n");
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4bdd0b97/fe/src/main/java/com/cloudera/impala/planner/PlanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/PlanNode.java b/fe/src/main/java/com/cloudera/impala/planner/PlanNode.java
index 1169190..643ca01 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/PlanNode.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/PlanNode.java
@@ -491,13 +491,13 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
    * The second issue is addressed by an exponential backoff when multiplying each
    * additional selectivity into the final result.
    */
-  protected double computeSelectivity() {
+  static protected double computeCombinedSelectivity(List<Expr> conjuncts) {
     // Collect all estimated selectivities.
     List<Double> selectivities = Lists.newArrayList();
-    for (Expr e: conjuncts_) {
+    for (Expr e: conjuncts) {
       if (e.hasSelectivity()) selectivities.add(e.getSelectivity());
     }
-    if (selectivities.size() != conjuncts_.size()) {
+    if (selectivities.size() != conjuncts.size()) {
       // Some conjuncts have no estimated selectivity. Use a single default
       // representative selectivity for all those conjuncts.
       selectivities.add(Expr.DEFAULT_SELECTIVITY);
@@ -515,6 +515,10 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     return Math.max(0.0, Math.min(1.0, result));
   }
 
+  protected double computeSelectivity() {
+    return computeCombinedSelectivity(conjuncts_);
+  }
+
   // Convert this plan node into msg (excluding children), which requires setting
   // the node type and the node-specific field.
   protected abstract void toThrift(TPlanNode msg);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4bdd0b97/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index 0062fd5..fe1cb29 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -2,7 +2,30 @@ select * from functional_kudu.zipcode_incomes where id = '8600000US00601'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED]
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
-     kudu predicates: id = '8600000US00601'     hosts=3 per-host-mem=unavailable
+     kudu predicates: id = '8600000US00601'
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=1
+---- DISTRIBUTEDPLAN
+F01:PLAN FRAGMENT [UNPARTITIONED]
+  01:EXCHANGE [UNPARTITIONED]
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=1
+
+F00:PLAN FRAGMENT [RANDOM]
+  DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     kudu predicates: id = '8600000US00601'
+     hosts=3 per-host-mem=0B
+     tuple-ids=0 row-size=124B cardinality=1
+====
+# The cardinality from "zip = '2'" should dominate.
+select * from functional_kudu.zipcode_incomes where id != '1' and zip = '2'
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     predicates: id != '1'
+     kudu predicates: zip = '2'
+     hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=1
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED]
@@ -13,6 +36,48 @@ F01:PLAN FRAGMENT [UNPARTITIONED]
 F00:PLAN FRAGMENT [RANDOM]
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
-     kudu predicates: id = '8600000US00601'     hosts=3 per-host-mem=0B
+     predicates: id != '1'
+     kudu predicates: zip = '2'
+     hosts=3 per-host-mem=0B
      tuple-ids=0 row-size=124B cardinality=1
-====
\ No newline at end of file
+====
+select * from functional_kudu.zipcode_incomes where id > '1' and zip > '2'
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     predicates: id > '1', zip > '2'
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=3317
+---- DISTRIBUTEDPLAN
+F01:PLAN FRAGMENT [UNPARTITIONED]
+  01:EXCHANGE [UNPARTITIONED]
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=3317
+
+F00:PLAN FRAGMENT [RANDOM]
+  DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     predicates: id > '1', zip > '2'
+     hosts=3 per-host-mem=0B
+     tuple-ids=0 row-size=124B cardinality=3317
+====
+select * from functional_kudu.zipcode_incomes where id = '1' or id = '2'
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     predicates: id = '1' OR id = '2'
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=2
+---- DISTRIBUTEDPLAN
+F01:PLAN FRAGMENT [UNPARTITIONED]
+  01:EXCHANGE [UNPARTITIONED]
+     hosts=3 per-host-mem=unavailable
+     tuple-ids=0 row-size=124B cardinality=2
+
+F00:PLAN FRAGMENT [RANDOM]
+  DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  00:SCAN KUDU [functional_kudu.zipcode_incomes]
+     predicates: id = '1' OR id = '2'
+     hosts=3 per-host-mem=0B
+     tuple-ids=0 row-size=124B cardinality=2
+====


[30/50] incubator-impala git commit: Disable Kudu on SLES and Debian

Posted by ta...@apache.org.
Disable Kudu on SLES and Debian

Kudu won't work on these platforms yet.

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

Branch: refs/heads/master
Commit: 5703f4e0c6916fbd474f72e4a0ecb011fc5e0927
Parents: 449901f
Author: casey <ca...@cloudera.com>
Authored: Tue Apr 5 10:43:54 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Apr 5 22:25:05 2016 +0000

----------------------------------------------------------------------
 bin/impala-config.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5703f4e0/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index d8b1dde..d21b153 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -107,7 +107,7 @@ if [[ -z $KUDU_BUILD_DIR ]]; then
     DISTRO_VERSION=$(tr -d ' \n' <<< "$DISTRO_VERSION" | cut -d. -f1 | tr "A-Z" "a-z")
     case "$DISTRO_VERSION" in
       # "enterprise" is Oracle
-      centos5 | debian6 | enterprise*5 | redhat*5 | suse*11 | ubuntu*12)
+      centos5 | debian* | enterprise*5 | redhat*5 | suse* | ubuntu*12)
           KUDU_IS_SUPPORTED=false;;
     esac
   fi


[40/50] incubator-impala git commit: IMPALA-3324: Hive server does not start for S3 builds.

Posted by ta...@apache.org.
IMPALA-3324: Hive server does not start for S3 builds.

The hive server does not start for S3 builds because HDFS is marked
as an unsupported service in testdata/cluster/admin; and so HDFS is
not started at all, and so the Hive server is unable to start as well.
Due to this, all our S3 builds fail.
Currently our S3 builds need HDFS to run correctly.

(This has to be reverted once IMPALA-1850 goes in, because then S3 can
run as a default FS without HDFS)

Change-Id: Ibda9dc3ef895c2aa4d39eb5694ac5f2dbd83bee4
Reviewed-on: http://gerrit.cloudera.org:8080/2741
Reviewed-by: Casey Ching <ca...@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/86fd262d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/86fd262d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/86fd262d

Branch: refs/heads/master
Commit: 86fd262dc91a638ddc18272c6c9f3d7b8e93b887
Parents: b085e31
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Fri Apr 8 16:00:31 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:43 2016 -0700

----------------------------------------------------------------------
 testdata/cluster/admin | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/86fd262d/testdata/cluster/admin
----------------------------------------------------------------------
diff --git a/testdata/cluster/admin b/testdata/cluster/admin
index 692f0e5..e68abab 100755
--- a/testdata/cluster/admin
+++ b/testdata/cluster/admin
@@ -49,7 +49,8 @@ TEMPLATE_SUFFIX=".tmpl"
 # Each process should be marked with this so a "pkill -f" can be done to nuke everything.
 export KILL_CLUSTER_MARKER=IBelongToTheMiniCluster
 
-if [[ "$TARGET_FILESYSTEM" == "hdfs" ]]; then
+if [[ "$TARGET_FILESYSTEM" == "hdfs" || "$TARGET_FILESYSTEM" == "s3" ]]; then
+  #TODO: Remove S3 from this condition once IMPALA-1850 goes in.
   # The check above indicates that the regular mini-cluster is in use.
   SUPPORTED_SERVICES=(hdfs yarn llama kms)
 else


[34/50] incubator-impala git commit: IMPALA-3301: TestParquet::test_resolution_by_name fails on legacy join/agg nodes

Posted by ta...@apache.org.
IMPALA-3301: TestParquet::test_resolution_by_name fails on legacy join/agg nodes

Change-Id: I7ce4c1f28966a1b14c25df228338ef4db6851a5b
Reviewed-on: http://gerrit.cloudera.org:8080/2723
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 4208fdacc1db755562cb952d82f197f3bed740db
Parents: 7dd4998
Author: Skye Wanderman-Milne <sk...@cloudera.com>
Authored: Tue Apr 5 15:27:49 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:02:35 2016 -0700

----------------------------------------------------------------------
 tests/query_test/test_scanners.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4208fdac/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 7768a40..036496c 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -337,6 +337,7 @@ class TestParquet(ImpalaTestSuite):
     assert d_schema_elt.converted_type == None
 
   @SkipIfS3.insert
+  @SkipIfOldAggsJoins.nested_types
   def test_resolution_by_name(self, unique_database, vector):
     self.run_test_case('QueryTest/parquet-resolution-by-name', vector,
                        use_db=unique_database)


[39/50] incubator-impala git commit: Kudu: Disable fsnyc in the mini-cluster

Posted by ta...@apache.org.
Kudu: Disable fsnyc in the mini-cluster

The Kudu team recommended disabling this for testing purposes. This
should help with timeouts in cloud machines (ec2/gce). Disabling
fsyncs could lead to data loss if the system crashed before the OS had a
chance to write the data to disk. Our test setups don't need that level
of reliability.

Change-Id: I72fd85ce5c4bc71f071b854ea6a9ebe60fc1305f
Reviewed-on: http://gerrit.cloudera.org:8080/2734
Reviewed-by: Casey Ching <ca...@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/9bb1b8a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/9bb1b8a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/9bb1b8a3

Branch: refs/heads/master
Commit: 9bb1b8a3660f5b76f92670d6e0273ca49f41e3b9
Parents: 8311f5e
Author: Casey Ching <ca...@cloudera.com>
Authored: Thu Apr 7 19:54:11 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:43 2016 -0700

----------------------------------------------------------------------
 testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl  | 5 +++++
 testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl | 5 +++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9bb1b8a3/testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl b/testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl
index 17b8d0d..3984d43 100644
--- a/testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl
+++ b/testdata/cluster/node_templates/cdh5/etc/kudu/master.conf.tmpl
@@ -1,3 +1,8 @@
 -fs_wal_dir=${NODE_DIR}/var/lib/kudu/master/wal
 -fs_data_dirs=${NODE_DIR}/var/lib/kudu/master/data
 -log_dir=${NODE_DIR}/var/log/kudu/master
+
+# fsync is disabled for additional speed. Sometimes operations are slow on EC2/GCE test
+# machines. Some data loss could occur if the system crashes before the OS has a chance
+# to flush data to disk but that is acceptable for development purposes.
+-never_fsync

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9bb1b8a3/testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl
----------------------------------------------------------------------
diff --git a/testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl b/testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl
index a39a38e..2625b8a 100644
--- a/testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl
+++ b/testdata/cluster/node_templates/cdh5/etc/kudu/tserver.conf.tmpl
@@ -3,3 +3,8 @@
 -fs_wal_dir=${NODE_DIR}/var/lib/kudu/ts/wal
 -fs_data_dirs=${NODE_DIR}/var/lib/kudu/ts/data
 -log_dir=${NODE_DIR}/var/log/kudu/ts
+
+# fsync is disabled for additional speed. Sometimes operations are slow on EC2/GCE test
+# machines. Some data loss could occur if the system crashes before the OS has a chance
+# to flush data to disk but that is acceptable for development purposes.
+-never_fsync


[35/50] incubator-impala git commit: IMPALA-3309: Remove reference to Hive JDBC version from pom.xml

Posted by ta...@apache.org.
IMPALA-3309: Remove reference to Hive JDBC version from pom.xml

Change-Id: I62331f4d52f696ebf6f84da2c96b826f4faeda51
Reviewed-on: http://gerrit.cloudera.org:8080/2722
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 0c4dc965a4f2d90dcbcd504400fe3a592fd16384
Parents: 4208fda
Author: Charlie Helin <ch...@cloudera.com>
Authored: Tue Apr 5 18:01:13 2016 -0400
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:02:35 2016 -0700

----------------------------------------------------------------------
 fe/pom.xml | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0c4dc965/fe/pom.xml
----------------------------------------------------------------------
diff --git a/fe/pom.xml b/fe/pom.xml
index 982e05d..82f820c 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -38,8 +38,6 @@
     <hbase.version>${env.IMPALA_HBASE_VERSION}</hbase.version>
     <parquet.version>${env.IMPALA_PARQUET_VERSION}</parquet.version>
     <impala.extdatasrc.api.version>1.0-SNAPSHOT</impala.extdatasrc.api.version>
-    <!-- Impala does not currently support the Hive .13 JDBC driver -->
-    <hive-jdbc.version>0.13.1-cdh5.2.0-SNAPSHOT</hive-jdbc.version>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <kudu.version>${env.KUDU_JAVA_VERSION}</kudu.version>
   </properties>
@@ -249,7 +247,7 @@
     <dependency>
       <groupId>org.apache.hive</groupId>
       <artifactId>hive-jdbc</artifactId>
-      <version>${hive-jdbc.version}</version>
+      <version>${hive.version}</version>
       <scope>test</scope>
     </dependency>
 


[45/50] incubator-impala git commit: Remove conjuncts_passed arg in various Parquet scanner functions.

Posted by ta...@apache.org.
Remove conjuncts_passed arg in various Parquet scanner functions.

The argument is not used anywhere, so let's remove it.

Change-Id: Iff511d6c4ba309eead6222701c17dee1b909a638
Reviewed-on: http://gerrit.cloudera.org:8080/2732
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 344fbc096644a5d06753aedeafe36eca82910cbc
Parents: ee8c309
Author: Alex Behm <al...@cloudera.com>
Authored: Thu Apr 7 16:24:31 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc | 71 ++++++++++++--------------------
 1 file changed, 27 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/344fbc09/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index a910243..9e18c66 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -253,10 +253,6 @@ class HdfsParquetScanner::ColumnReader {
   /// set, the query is cancelled, or the scan node limit was reached. Otherwise returns
   /// true.
   ///
-  /// *conjuncts_passed is an in/out parameter. ReadValue() sets to false if the current
-  /// row should be filtered based on this column's value. If already false on input, the
-  /// row has already been filtered and ReadValue() only needs to advance the value.
-  ///
   /// NextLevels() must be called on this reader before calling ReadValue() for the first
   /// time. This is to initialize the current value that ReadValue() will read.
   ///
@@ -266,12 +262,11 @@ class HdfsParquetScanner::ColumnReader {
   /// TODO: another option is to materialize col by col for the entire row batch in
   /// one call.  e.g. MaterializeCol would write out 1024 values.  Our row batches
   /// are currently dense so we'll need to figure out something there.
-  virtual bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) = 0;
+  virtual bool ReadValue(MemPool* pool, Tuple* tuple) = 0;
 
   /// Same as ReadValue() but does not advance repetition level. Only valid for columns not
   /// in collections.
-  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple,
-      bool* conjuncts_passed) = 0;
+  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple) = 0;
 
   /// Advances this column reader's def and rep levels to the next logical value, i.e. to
   /// the next scalar value or the beginning of the next collection, without attempting to
@@ -375,11 +370,11 @@ class HdfsParquetScanner::CollectionColumnReader :
 
   /// Materializes CollectionValue into tuple slot (if materializing) and advances to next
   /// value.
-  virtual bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed);
+  virtual bool ReadValue(MemPool* pool, Tuple* tuple);
 
   /// Same as ReadValue but does not advance repetition level. Only valid for columns not
   /// in collections.
-  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed);
+  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple);
 
   /// Advances all child readers to the beginning of the next collection and updates this
   /// reader's state.
@@ -409,7 +404,7 @@ class HdfsParquetScanner::CollectionColumnReader :
   /// Returns false if execution should be aborted for some reason, e.g. parse_error_ is
   /// set, the query is cancelled, or the scan node limit was reached. Otherwise returns
   /// true.
-  inline bool ReadSlot(void* slot, MemPool* pool, bool* conjuncts_passed);
+  inline bool ReadSlot(void* slot, MemPool* pool);
 };
 
 /// Reader for a single column from the parquet file.  It's associated with a
@@ -565,7 +560,7 @@ class HdfsParquetScanner::BaseScalarColumnReader :
   /// set, the query is cancelled, or the scan node limit was reached. Otherwise returns
   /// true.
   template <bool IN_COLLECTION>
-  inline bool ReadSlot(void* slot, MemPool* pool, bool* conjuncts_passed);
+  inline bool ReadSlot(void* slot, MemPool* pool);
 };
 
 /// Per column type reader. If MATERIALIZED is true, the column values are materialized
@@ -605,17 +600,17 @@ class HdfsParquetScanner::ScalarColumnReader :
 
   virtual ~ScalarColumnReader() { }
 
-  virtual bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
-    return ReadValue<true>(pool, tuple, conjuncts_passed);
+  virtual bool ReadValue(MemPool* pool, Tuple* tuple) {
+    return ReadValue<true>(pool, tuple);
   }
 
-  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
-    return ReadValue<false>(pool, tuple, conjuncts_passed);
+  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple) {
+    return ReadValue<false>(pool, tuple);
   }
 
  protected:
   template <bool IN_COLLECTION>
-  inline bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
+  inline bool ReadValue(MemPool* pool, Tuple* tuple) {
     // NextLevels() should have already been called and def and rep levels should be in
     // valid range.
     DCHECK_GE(rep_level_, 0);
@@ -628,8 +623,7 @@ class HdfsParquetScanner::ScalarColumnReader :
     if (!MATERIALIZED) {
       return NextLevels<IN_COLLECTION>();
     } else if (def_level_ >= max_def_level()) {
-      return ReadSlot<IN_COLLECTION>(tuple->GetSlot(tuple_offset_), pool,
-          conjuncts_passed);
+      return ReadSlot<IN_COLLECTION>(tuple->GetSlot(tuple_offset_), pool);
     } else {
       // Null value
       tuple->SetNull(null_indicator_offset_);
@@ -682,7 +676,7 @@ class HdfsParquetScanner::ScalarColumnReader :
   /// set, the query is cancelled, or the scan node limit was reached. Otherwise returns
   /// true.
   template <bool IN_COLLECTION>
-  inline bool ReadSlot(void* slot, MemPool* pool, bool* conjuncts_passed) {
+  inline bool ReadSlot(void* slot, MemPool* pool) {
     T val;
     T* val_ptr = NeedsConversion() ? &val : reinterpret_cast<T*>(slot);
     if (page_encoding_ == parquet::Encoding::PLAIN_DICTIONARY) {
@@ -794,13 +788,12 @@ class HdfsParquetScanner::BoolColumnReader :
 
   virtual ~BoolColumnReader() { }
 
-  virtual bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
-    return ReadValue<true>(pool, tuple, conjuncts_passed);
+  virtual bool ReadValue(MemPool* pool, Tuple* tuple) {
+    return ReadValue<true>(pool, tuple);
   }
 
-  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple,
-      bool* conjuncts_passed) {
-    return ReadValue<false>(pool, tuple, conjuncts_passed);
+  virtual bool ReadNonRepeatedValue(MemPool* pool, Tuple* tuple) {
+    return ReadValue<false>(pool, tuple);
   }
 
  protected:
@@ -825,7 +818,7 @@ class HdfsParquetScanner::BoolColumnReader :
 
  private:
   template<bool IN_COLLECTION>
-  inline bool ReadValue(MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
+  inline bool ReadValue(MemPool* pool, Tuple* tuple) {
     DCHECK(slot_desc_ != NULL);
     // Def and rep levels should be in valid range.
     DCHECK_GE(rep_level_, 0);
@@ -836,8 +829,7 @@ class HdfsParquetScanner::BoolColumnReader :
         "Caller should have called NextLevels() until we are ready to read a value";
 
     if (def_level_ >= max_def_level()) {
-      return ReadSlot<IN_COLLECTION>(tuple->GetSlot(tuple_offset_), pool,
-          conjuncts_passed);
+      return ReadSlot<IN_COLLECTION>(tuple->GetSlot(tuple_offset_), pool);
     } else {
       // Null value
       tuple->SetNull(null_indicator_offset_);
@@ -852,7 +844,7 @@ class HdfsParquetScanner::BoolColumnReader :
   /// set, the query is cancelled, or the scan node limit was reached. Otherwise returns
   /// true.
   template <bool IN_COLLECTION>
-  inline bool ReadSlot(void* slot, MemPool* pool, bool* conjuncts_passed)  {
+  inline bool ReadSlot(void* slot, MemPool* pool)  {
     if (!bool_values_.GetValue(1, reinterpret_cast<bool*>(slot))) {
       parent_->parse_status_ = Status("Invalid bool column.");
       return false;
@@ -1345,8 +1337,7 @@ bool HdfsParquetScanner::CollectionColumnReader::NextLevels() {
   return true;
 }
 
-bool HdfsParquetScanner::CollectionColumnReader::ReadValue(
-    MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
+bool HdfsParquetScanner::CollectionColumnReader::ReadValue(MemPool* pool, Tuple* tuple) {
   DCHECK_GE(rep_level_, 0);
   DCHECK_GE(def_level_, 0);
   DCHECK_GE(def_level_, def_level_of_immediate_repeated_ancestor()) <<
@@ -1355,7 +1346,7 @@ bool HdfsParquetScanner::CollectionColumnReader::ReadValue(
   if (tuple_offset_ == -1) {
     return CollectionColumnReader::NextLevels();
   } else if (def_level_ >= max_def_level()) {
-    return ReadSlot(tuple->GetSlot(tuple_offset_), pool, conjuncts_passed);
+    return ReadSlot(tuple->GetSlot(tuple_offset_), pool);
   } else {
     // Null value
     tuple->SetNull(null_indicator_offset_);
@@ -1364,19 +1355,14 @@ bool HdfsParquetScanner::CollectionColumnReader::ReadValue(
 }
 
 bool HdfsParquetScanner::CollectionColumnReader::ReadNonRepeatedValue(
-    MemPool* pool, Tuple* tuple, bool* conjuncts_passed) {
-  return CollectionColumnReader::ReadValue(pool, tuple, conjuncts_passed);
+    MemPool* pool, Tuple* tuple) {
+  return CollectionColumnReader::ReadValue(pool, tuple);
 }
 
-// TODO for 2.3: test query where *conjuncts_passed == false
-bool HdfsParquetScanner::CollectionColumnReader::ReadSlot(
-    void* slot, MemPool* pool, bool* conjuncts_passed) {
+bool HdfsParquetScanner::CollectionColumnReader::ReadSlot(void* slot, MemPool* pool) {
   DCHECK(!children_.empty());
   DCHECK_LE(rep_level_, new_collection_rep_level());
 
-  // TODO: do something with conjuncts_passed? We still need to "read" the value in order
-  // to advance children_ but we don't need to materialize the collection.
-
   // Recursively read the collection into a new CollectionValue.
   CollectionValue* coll_slot = reinterpret_cast<CollectionValue*>(slot);
   *coll_slot = CollectionValue();
@@ -1716,7 +1702,6 @@ inline bool HdfsParquetScanner::ReadRow(const vector<ColumnReader*>& column_read
     Tuple* tuple, MemPool* pool, bool* materialize_tuple) {
   DCHECK(!column_readers.empty());
   bool continue_execution = true;
-  bool conjuncts_passed = true;
   int size = column_readers.size();
   for (int c = 0; c < size; ++c) {
     ColumnReader* col_reader = column_readers[c];
@@ -1724,15 +1709,14 @@ inline bool HdfsParquetScanner::ReadRow(const vector<ColumnReader*>& column_read
       DCHECK(*materialize_tuple);
       DCHECK(col_reader->pos_slot_desc() == NULL);
       // We found a value, read it
-      continue_execution = col_reader->ReadNonRepeatedValue(pool, tuple,
-          &conjuncts_passed);
+      continue_execution = col_reader->ReadNonRepeatedValue(pool, tuple);
     } else if (*materialize_tuple) {
       // All column readers for this tuple should a value to materialize.
       FILE_CHECK_GE(col_reader->def_level(),
                     col_reader->def_level_of_immediate_repeated_ancestor());
       // Fill in position slot if applicable
       if (col_reader->pos_slot_desc() != NULL) col_reader->ReadPosition(tuple);
-      continue_execution = col_reader->ReadValue(pool, tuple, &conjuncts_passed);
+      continue_execution = col_reader->ReadValue(pool, tuple);
     } else {
       // A containing repeated field is empty or NULL
       FILE_CHECK_LT(col_reader->def_level(),
@@ -1741,7 +1725,6 @@ inline bool HdfsParquetScanner::ReadRow(const vector<ColumnReader*>& column_read
     }
     if (UNLIKELY(!continue_execution)) break;
   }
-  *materialize_tuple &= conjuncts_passed;
 
   if (!IN_COLLECTION && *materialize_tuple) {
     TupleRow* tuple_row_mem = reinterpret_cast<TupleRow*>(&tuple);


[43/50] incubator-impala git commit: Add -release switch to buildall.sh help, change coverage options.

Posted by ta...@apache.org.
Add -release switch to buildall.sh help, change coverage options.

This change documents the -release switch. It also removes the _debug
and _release suffixes from -codecoverage_* and determines them from
the presence of -release. On top of that it adds sanity checks to the
specified options.

Change-Id: Id69791264cb2d9e0ffe96a7ac5aabc34a553a7be
Reviewed-on: http://gerrit.cloudera.org:8080/2043
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: a65ffda542e6efd8f44cb116912260f056d6f141
Parents: 49f9559
Author: Lars Volker <lv...@cloudera.com>
Authored: Thu Feb 4 11:17:09 2016 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 buildall.sh | 46 +++++++++++++++++++++++++++++++++-------------
 1 file changed, 33 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a65ffda5/buildall.sh
----------------------------------------------------------------------
diff --git a/buildall.sh b/buildall.sh
index 862e629..eeb8927 100755
--- a/buildall.sh
+++ b/buildall.sh
@@ -46,11 +46,13 @@ IMPALA_KERBERIZE=0
 SNAPSHOT_FILE=
 METASTORE_SNAPSHOT_FILE=
 MAKE_IMPALA_ARGS=""
+BUILD_COVERAGE=0
+BUILD_ASAN=0
 
 # Defaults that can be picked up from the environment, but are overridable through the
 # commandline.
 : ${EXPLORATION_STRATEGY:=core}
-: ${TARGET_BUILD_TYPE:=Debug}
+: ${CMAKE_BUILD_TYPE:=Debug}
 
 # parse command line options
 # TODO: We have to change this to use getopts, or something more sensible.
@@ -89,17 +91,14 @@ do
     -format_sentry_policy_db)
       FORMAT_SENTRY_POLICY_DB=1
       ;;
-    -codecoverage_debug)
-      TARGET_BUILD_TYPE=CODE_COVERAGE_DEBUG
+    -release)
+      CMAKE_BUILD_TYPE=Release
       ;;
-    -codecoverage_release)
-      TARGET_BUILD_TYPE=CODE_COVERAGE_RELEASE
+    -codecoverage)
+      BUILD_COVERAGE=1
       ;;
     -asan)
-      TARGET_BUILD_TYPE=ADDRESS_SANITIZER
-      ;;
-    -release)
-      TARGET_BUILD_TYPE=Release
+      BUILD_ASAN=1
       ;;
     -testpairwise)
       EXPLORATION_STRATEGY=pairwise
@@ -149,9 +148,9 @@ do
       echo "[-format_cluster] : Format the minicluster [Default: False]"
       echo "[-format_metastore] : Format the metastore db [Default: False]"
       echo "[-format_sentry_policy_db] : Format the Sentry policy db [Default: False]"
-      echo "[-codecoverage_release] : Release code coverage build"
-      echo "[-codecoverage_debug] : Debug code coverage build"
-      echo "[-asan] : Build with address sanitizer"
+      echo "[-release] : Release build [Default: debug]"
+      echo "[-codecoverage] : Build with code coverage [Default: False]"
+      echo "[-asan] : Address sanitizer build [Default: False]"
       echo "[-skiptests] : Skips execution of all tests"
       echo "[-notests] : Skips building and execution of all tests"
       echo "[-testpairwise] : Sun tests in 'pairwise' mode (increases"\
@@ -195,6 +194,27 @@ Examples of common tasks:
   shift;
 done
 
+# Adjust CMAKE_BUILD_TYPE for ASAN and code coverage, if necessary.
+if [[ ${BUILD_COVERAGE} -eq 1 ]]; then
+  case ${CMAKE_BUILD_TYPE} in
+    Debug)
+      CMAKE_BUILD_TYPE=CODE_COVERAGE_DEBUG
+      ;;
+    Release)
+      CMAKE_BUILD_TYPE=CODE_COVERAGE_RELEASE
+      ;;
+  esac
+fi
+if [[ ${BUILD_ASAN} -eq 1 ]]; then
+  # The next check also catches cases where BUILD_COVERAGE=1, which is not supported
+  # together with BUILD_ASAN=1.
+  if [[ "${CMAKE_BUILD_TYPE}" != "Debug" ]]; then
+    echo "Address sanitizer build not supported for build type: ${CMAKE_BUILD_TYPE}"
+    exit 1
+  fi
+  CMAKE_BUILD_TYPE=ADDRESS_SANITIZER
+fi
+
 # If we aren't kerberized then we certainly don't need to talk about
 # re-sourcing impala-config.
 if [ ${IMPALA_KERBERIZE} -eq 0 ]; then
@@ -264,7 +284,7 @@ if [ $METASTORE_SNAPSHOT_FILE ]; then
 fi
 
 # build common and backend
-MAKE_IMPALA_ARGS="${MAKE_IMPALA_ARGS} -build_type=${TARGET_BUILD_TYPE}"
+MAKE_IMPALA_ARGS="${MAKE_IMPALA_ARGS} -build_type=${CMAKE_BUILD_TYPE}"
 echo "Calling make_impala.sh ${MAKE_IMPALA_ARGS}"
 $IMPALA_HOME/bin/make_impala.sh ${MAKE_IMPALA_ARGS}
 


[37/50] incubator-impala git commit: Update python impala cluster script for docker compatibility

Posted by ta...@apache.org.
Update python impala cluster script for docker compatibility

The python scripts looked at all processes even those not owned by the
current user which includes docker processes. The fix is to only
consider processes owned by the current user.

Some unused imports were also removed.

Change-Id: I8ae7c4a5f3c6ce7d2838f9f722d5341c585cf562
Reviewed-on: http://gerrit.cloudera.org:8080/1629
Reviewed-by: Casey Ching <ca...@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/8311f5e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/8311f5e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/8311f5e3

Branch: refs/heads/master
Commit: 8311f5e30afd29598e733f500406a8de9eaace3b
Parents: e956a6b
Author: casey <ca...@cloudera.com>
Authored: Sun Dec 13 20:39:18 2015 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:43 2016 -0700

----------------------------------------------------------------------
 tests/common/impala_cluster.py | 19 ++++++++++---------
 1 file changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8311f5e3/tests/common/impala_cluster.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_cluster.py b/tests/common/impala_cluster.py
index 6fbc0f1..4aac17c 100644
--- a/tests/common/impala_cluster.py
+++ b/tests/common/impala_cluster.py
@@ -14,20 +14,15 @@
 #
 # Basic object model of an Impala cluster (set of Impala processes).
 #
-import json
 import logging
-import os
 import psutil
 import socket
-import sys
-import urllib
 
-from collections import defaultdict
-from HTMLParser import HTMLParser
-from random import choice, shuffle
+from getpass import getuser
+from random import choice
 from tests.common.impala_service import *
 from tests.util.shell_util import exec_process_async, exec_process
-from time import sleep, time
+from time import sleep
 
 logging.basicConfig(level=logging.ERROR, format='%(threadName)s: %(message)s')
 LOG = logging.getLogger('impala_cluster')
@@ -97,7 +92,6 @@ class ImpalaCluster(object):
     impalads = list()
     statestored = list()
     catalogd = None
-    # TODO: Consider using process_iter() here
     for pid in psutil.get_pid_list():
       try:
         process = psutil.Process(pid)
@@ -105,6 +99,13 @@ class ImpalaCluster(object):
         # A process from get_pid_list() no longer exists, continue.
         LOG.info(e)
         continue
+      try:
+        if process.username != getuser():
+          continue
+      except KeyError, e:
+        if "uid not found" in str(e):
+          continue
+        raise
       if process.name == 'impalad' and len(process.cmdline) >= 1:
         impalads.append(ImpaladProcess(process.cmdline))
       elif process.name == 'statestored' and len(process.cmdline) >= 1:


[12/50] incubator-impala git commit: IMPALA-2888: StatestoreSslTest.SmokeTest failed

Posted by ta...@apache.org.
IMPALA-2888: StatestoreSslTest.SmokeTest failed

The StatestoreTest.SmokeTest and StatestoreSslTest.SmokeTest both use
ephemeral ports for the statestore subscribers without checking first
if they are unused or not. This is most likely the reason behind these
tests failing.
This patch includes a new util function which checks for and returns
unused ephemeral ports, which the tests now use.

Change-Id: I51b7a2c609d3a5ae6f7d32ef38db89efe71a882b
Reviewed-on: http://gerrit.cloudera.org:8080/2630
Reviewed-by: Sailesh Mukil <sa...@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/2b47c5a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/2b47c5a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/2b47c5a3

Branch: refs/heads/master
Commit: 2b47c5a3bf454aca199cf0a9b2397b540b6cdf0d
Parents: b2ccb17
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Fri Mar 25 10:22:53 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Mar 29 02:04:02 2016 +0000

----------------------------------------------------------------------
 be/src/statestore/statestore-test.cc  | 17 ++++++++++----
 be/src/testutil/in-process-servers.cc | 37 +++++++++++++++++++-----------
 be/src/util/network-util.cc           | 27 ++++++++++++++++++++++
 be/src/util/network-util.h            |  3 +++
 4 files changed, 66 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2b47c5a3/be/src/statestore/statestore-test.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore-test.cc b/be/src/statestore/statestore-test.cc
index 2338f8b..d0b7d88 100644
--- a/be/src/statestore/statestore-test.cc
+++ b/be/src/statestore/statestore-test.cc
@@ -41,14 +41,17 @@ TEST(StatestoreTest, SmokeTest) {
       new InProcessStatestore(ips->port(), ips->port() + 10);
   ASSERT_FALSE(statestore_wont_start->Start().ok());
 
+  int subscriber_port = FindUnusedEphemeralPort();
+  ASSERT_NE(subscriber_port, -1) << "Could not find unused port";
+
   StatestoreSubscriber* sub_will_start = new StatestoreSubscriber("sub1",
-      MakeNetworkAddress("localhost", ips->port() + 20),
+      MakeNetworkAddress("localhost", subscriber_port),
       MakeNetworkAddress("localhost", ips->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("sub2",
-      MakeNetworkAddress("localhost", ips->port() + 20),
+      MakeNetworkAddress("localhost", subscriber_port),
       MakeNetworkAddress("localhost", ips->port()), new MetricGroup(""));
   ASSERT_FALSE(sub_will_not_start->Start().ok());
 }
@@ -66,16 +69,22 @@ TEST(StatestoreSslTest, SmokeTest) {
   InProcessStatestore* statestore =  InProcessStatestore::StartWithEphemeralPorts();
   if (statestore == NULL) FAIL() << "Unable to start Statestore";
 
+  int subscriber_port = FindUnusedEphemeralPort();
+  ASSERT_NE(subscriber_port, -1) << "Could not find unused port";
+
   StatestoreSubscriber* sub_will_start = new StatestoreSubscriber("smoke_sub1",
-      MakeNetworkAddress("localhost", statestore->port() + 10),
+      MakeNetworkAddress("localhost", subscriber_port),
       MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
   ASSERT_OK(sub_will_start->Start());
 
   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();
+  int another_subscriber_port = FindUnusedEphemeralPort();
+  ASSERT_NE(another_subscriber_port, -1) << "Could not find unused port";
+
   StatestoreSubscriber* sub_will_not_start = new StatestoreSubscriber("smoke_sub2",
-      MakeNetworkAddress("localhost", statestore->port() + 20),
+      MakeNetworkAddress("localhost", another_subscriber_port),
       MakeNetworkAddress("localhost", statestore->port()), new MetricGroup(""));
   ASSERT_FALSE(sub_will_not_start->Start().ok());
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2b47c5a3/be/src/testutil/in-process-servers.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/in-process-servers.cc b/be/src/testutil/in-process-servers.cc
index 016aece..a502f26 100644
--- a/be/src/testutil/in-process-servers.cc
+++ b/be/src/testutil/in-process-servers.cc
@@ -36,19 +36,24 @@ using namespace apache::thrift;
 using namespace impala;
 using boost::shared_ptr;
 
-/// Pick a random port in the range of ephemeral ports
-/// https://tools.ietf.org/html/rfc6335
-static uint32_t RandomEphemeralPort() {
-  static uint32_t LOWER = 49152, UPPER = 65000;
-  return LOWER + rand() % (UPPER - LOWER);
-}
-
 InProcessImpalaServer* InProcessImpalaServer::StartWithEphemeralPorts(
     const string& statestore_host, int statestore_port) {
-  for (uint32_t tries = 0; tries < 10; ++tries) {
-    uint32_t p = RandomEphemeralPort();
-    uint32_t backend_port = p, subscriber_port = ++p, webserver_port = ++p,
-        beeswax_port = ++p, hs2_port = ++p;
+  for (int tries = 0; tries < 10; ++tries) {
+    int backend_port = FindUnusedEphemeralPort();
+    if (backend_port == -1) continue;
+
+    int subscriber_port = FindUnusedEphemeralPort();
+    if (subscriber_port == -1) continue;
+
+    int webserver_port = FindUnusedEphemeralPort();
+    if (webserver_port == -1) continue;
+
+    int beeswax_port = FindUnusedEphemeralPort();
+    if (beeswax_port == -1) continue;
+
+    int hs2_port = FindUnusedEphemeralPort();
+    if (hs2_port == -1) continue;
+
     InProcessImpalaServer* impala =
         new InProcessImpalaServer("localhost", backend_port, subscriber_port,
             webserver_port, statestore_host, statestore_port);
@@ -122,9 +127,13 @@ Status InProcessImpalaServer::Join() {
 }
 
 InProcessStatestore* InProcessStatestore::StartWithEphemeralPorts() {
-  for (uint32_t tries = 0; tries < 10; ++tries) {
-    uint32_t p = RandomEphemeralPort();
-    uint32_t statestore_port = p, webserver_port = ++p;
+  for (int tries = 0; tries < 10; ++tries) {
+    int statestore_port = FindUnusedEphemeralPort();
+    if (statestore_port == -1) continue;
+
+    int webserver_port = FindUnusedEphemeralPort();
+    if (webserver_port == -1) continue;
+
     InProcessStatestore* ips = new InProcessStatestore(statestore_port, webserver_port);
     if (ips->Start().ok()) return ips;
     delete ips;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2b47c5a3/be/src/util/network-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/network-util.cc b/be/src/util/network-util.cc
index ea18110..edfc953 100644
--- a/be/src/util/network-util.cc
+++ b/be/src/util/network-util.cc
@@ -32,6 +32,7 @@
 
 using boost::algorithm::is_any_of;
 using boost::algorithm::split;
+using std::random_device;
 
 #ifdef __APPLE__
 // OS X does not seem to have a similar limitation as Linux and thus the
@@ -140,4 +141,30 @@ ostream& operator<<(ostream& out, const TNetworkAddress& hostport) {
   return out;
 }
 
+/// Pick a random port in the range of ephemeral ports
+/// https://tools.ietf.org/html/rfc6335
+int FindUnusedEphemeralPort() {
+  static uint32_t LOWER = 49152, UPPER = 65000;
+  random_device rd;
+  srand(rd());
+
+  int sockfd = socket(AF_INET, SOCK_STREAM, 0);
+  if (sockfd < 0) return -1;
+  struct sockaddr_in server_address;
+  bzero(reinterpret_cast<char*>(&server_address), sizeof(server_address));
+  server_address.sin_family = AF_INET;
+  server_address.sin_addr.s_addr = INADDR_ANY;
+  for (uint32_t tries = 0; tries < 10; ++tries) {
+    int port = LOWER + rand() % (UPPER - LOWER);
+    server_address.sin_port = htons(port);
+    if (bind(sockfd, reinterpret_cast<struct sockaddr*>(&server_address),
+        sizeof(server_address)) == 0) {
+      close(sockfd);
+      return port;
+    }
+  }
+  close(sockfd);
+  return -1;
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2b47c5a3/be/src/util/network-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/network-util.h b/be/src/util/network-util.h
index 4e923d7..e2cb88a 100644
--- a/be/src/util/network-util.h
+++ b/be/src/util/network-util.h
@@ -49,4 +49,7 @@ std::string TNetworkAddressToString(const TNetworkAddress& address);
 /// Prints a hostport as ipaddress:port
 std::ostream& operator<<(std::ostream& out, const TNetworkAddress& hostport);
 
+/// Returns a ephemeral port that is unused when this function executes. Returns -1 on an
+/// error or if a free ephemeral port can't be found after 10 tries.
+int FindUnusedEphemeralPort();
 }


[36/50] incubator-impala git commit: Remove generated server.xml file

Posted by ta...@apache.org.
Remove generated server.xml file

Change-Id: I5325e043c6edd956f03f95a7f5dabd2ba0a72cd1
Reviewed-on: http://gerrit.cloudera.org:8080/2730
Reviewed-by: Casey Ching <ca...@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/e956a6be
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/e956a6be
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/e956a6be

Branch: refs/heads/master
Commit: e956a6be0ad993d4c88c124a858a862002e36cec
Parents: 9d43aac
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Thu Apr 7 10:29:31 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:35 2016 -0700

----------------------------------------------------------------------

----------------------------------------------------------------------



[25/50] incubator-impala git commit: Remove make_test_tarball.sh

Posted by ta...@apache.org.
Remove make_test_tarball.sh

As far as I know nothing actually uses the "test tarball". For some
reason building it take a minute or so on my computer. If it's not used,
then it seems best to just get rid of it.

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

Branch: refs/heads/master
Commit: 52841302de23e5cb08e6eba1aa95175d45fe72c2
Parents: 40117bf
Author: casey <ca...@cloudera.com>
Authored: Thu Mar 31 13:40:15 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Apr 1 01:26:44 2016 +0000

----------------------------------------------------------------------
 buildall.sh                |   3 -
 tests/make_test_tarball.sh | 120 ----------------------------------------
 2 files changed, 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/52841302/buildall.sh
----------------------------------------------------------------------
diff --git a/buildall.sh b/buildall.sh
index e61ccb1..862e629 100755
--- a/buildall.sh
+++ b/buildall.sh
@@ -295,9 +295,6 @@ popd
 echo "Creating shell tarball"
 ${IMPALA_HOME}/shell/make_shell_tarball.sh
 
-echo "Creating test tarball"
-${IMPALA_HOME}/tests/make_test_tarball.sh
-
 if [ $FORMAT_CLUSTER -eq 1 ]; then
   $IMPALA_HOME/testdata/bin/run-all.sh -format
 elif [ $TESTDATA_ACTION -eq 1 ] || [ $TESTS_ACTION -eq 1 ]; then

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/52841302/tests/make_test_tarball.sh
----------------------------------------------------------------------
diff --git a/tests/make_test_tarball.sh b/tests/make_test_tarball.sh
deleted file mode 100755
index 2bc2dbe..0000000
--- a/tests/make_test_tarball.sh
+++ /dev/null
@@ -1,120 +0,0 @@
-#!/bin/bash
-# Copyright 2012 Cloudera Inc.
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# This script makes a tarball of the Python-based tests and benchmark scripts
-# that can be unpacked and run out-of-the-box. The final tarball is left in
-# ${IMPALA_HOME}/tests/build. Assumes Impala and /thirdparty have been built
-# built prior to running this script.
-
-set -euo pipefail
-trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
-
-if [ "x${IMPALA_HOME}" == "x" ]; then
-  echo "\$IMPALA_HOME must be set"
-  exit 1
-fi
-
-# Parse the version info
-IMPALA_VERSION_INFO_FILE=${IMPALA_HOME}/bin/version.info
-
-if [ ! -f ${IMPALA_VERSION_INFO_FILE} ]; then
-  echo "No version.info file found. Generating new version info"
-  ${IMPALA_HOME}/bin/save-version.sh
-else
-  echo "Using existing version.info file."
-fi
-
-VERSION=$(grep "VERSION: " ${IMPALA_VERSION_INFO_FILE} | awk '{print $2}')
-cat ${IMPALA_VERSION_INFO_FILE}
-
-TMP_ROOT_DIR=$(mktemp -dt "impala_test_tmp.XXXXXX")
-TARBALL_ROOT=${TMP_ROOT_DIR}/impala-tests-${VERSION}
-OUTPUT_DIR=${IMPALA_HOME}/tests/build
-
-echo "Root of temp output dir: ${TMP_ROOT_DIR}"
-
-echo "Creating required directories"
-mkdir -p ${OUTPUT_DIR}
-mkdir -p ${TARBALL_ROOT}/bin
-mkdir -p ${TARBALL_ROOT}/testdata
-mkdir -p ${TARBALL_ROOT}/thirdparty
-
-echo "Generating environment setup script"
-cat > ${TARBALL_ROOT}/bin/set-env.sh <<EOF
-# Copyright 2014 Cloudera Inc.
-#
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-if [ -z \${IMPALA_HOME} ]; then
-  export IMPALA_HOME=\`pwd\`
-fi
-
-export IMPALA_WORKLOAD_DIR=\${IMPALA_HOME}/testdata/workloads
-export IMPALA_DATASET_DIR=\${IMPALA_HOME}/testdata/datasets
-export THRIFT_HOME=\${IMPALA_HOME}/thirdparty/thrift-${IMPALA_THRIFT_VERSION}
-
-# Set these based on your cluster environment
-# export HIVE_HOME=/usr/lib/hive/
-# export HIVE_CONF_DIR=/etc/hive/conf
-# export HADOOP_HOME=/usr/lib/hadoop/
-# export HADOOP_CONF_DIR=/etc/hadoop/conf
-# export HBASE_HOME=/usr/lib/hbase/
-# export HBASE_CONF_DIR=/etc/hbase/conf
-# export PATH=$HBASE_HOME/bin:$PATH
-
-# Build the Python path
-. \${IMPALA_HOME}/bin/set-pythonpath.sh
-
-echo IMPALA_HOME=\${IMPALA_HOME}
-echo PYTHONPATH=\${PYTHONPATH}
-echo THRIFT_HOME=\${THRIFT_HOME}
-EOF
-
-echo "Copying required files and dependencies"
-cp ${IMPALA_VERSION_INFO_FILE} ${TARBALL_ROOT}/bin/
-cp ${IMPALA_HOME}/bin/load-data.py ${TARBALL_ROOT}/bin/
-cp ${IMPALA_HOME}/bin/run-workload.py ${TARBALL_ROOT}/bin/
-cp ${IMPALA_HOME}/bin/set-pythonpath.sh ${TARBALL_ROOT}/bin/
-cp -a ${IMPALA_HOME}/testdata/workloads/ ${TARBALL_ROOT}/testdata/
-cp -a ${IMPALA_HOME}/testdata/datasets/ ${TARBALL_ROOT}/testdata/
-cp -a ${IMPALA_HOME}/testdata/bin/ ${TARBALL_ROOT}/testdata/
-cp -a ${IMPALA_HOME}/testdata/avro_schema_resolution/ ${TARBALL_ROOT}/testdata/
-cp -a ${IMPALA_HOME}/tests/ ${TARBALL_ROOT}/
-cp -a ${IMPALA_HOME}/shell/ ${TARBALL_ROOT}/
-
-# Bundle thrift
-cp -a ${THRIFT_HOME} ${TARBALL_ROOT}/thirdparty/thrift-${IMPALA_THRIFT_VERSION}
-
-echo "Making tarball in ${TMP_ROOT_DIR} and copying to: ${OUTPUT_DIR}"
-pushd ${TMP_ROOT_DIR} 2>&1 > /dev/null
-# Exclude dirs that are not needed.
-rm -rf ${TMP_ROOT_DIR}/impala-tests-${VERSION}/tests/results/*
-rm -rf ${TMP_ROOT_DIR}/impala-tests-${VERSION}/tests/build/*
-tar czf ${TMP_ROOT_DIR}/impala-tests-${VERSION}.tar.gz ./impala-tests-${VERSION}/\
-    --exclude="*.pyc" || popd 2>&1 > /dev/null
-cp ${TMP_ROOT_DIR}/impala-tests-${VERSION}.tar.gz ${OUTPUT_DIR}
-
-echo "Cleaning up ${TMP_ROOT_DIR}"
-rm -rf ${TMP_ROOT_DIR}


[19/50] incubator-impala git commit: IMPALA-3238: avoid log spam for very large hash tables

Posted by ta...@apache.org.
IMPALA-3238: avoid log spam for very large hash tables

Before this change, every time the limit was hit in ConsumeMemory() was
hit, a warning message with stack trace was logged. The streaming preagg
calls this once per partition per batch, resulting in an enormous volume
of logging.

The fix is to log the warning only once for each client (i.e. each exec
node).

We need a very large data set to reproduce this error. To test it
manually I lowered the threshold to 1 << 19, ran 'select distinct
l_orderkey from tpch_20_parquet', and confirmed that only two warning
messages per query appeared in the logs (one per agg node).

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

Branch: refs/heads/master
Commit: ce9284454b6c3ff382ca5f509f75697a528b14a4
Parents: 97b5233
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Mar 29 14:37:11 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Wed Mar 30 15:42:42 2016 +0000

----------------------------------------------------------------------
 be/src/runtime/buffered-block-mgr.cc | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ce928445/be/src/runtime/buffered-block-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/buffered-block-mgr.cc b/be/src/runtime/buffered-block-mgr.cc
index 17d344b..aecd8a2 100644
--- a/be/src/runtime/buffered-block-mgr.cc
+++ b/be/src/runtime/buffered-block-mgr.cc
@@ -55,7 +55,8 @@ struct BufferedBlockMgr::Client {
         num_reserved_buffers_(num_reserved_buffers),
         tolerates_oversubscription_(tolerates_oversubscription),
         num_tmp_reserved_buffers_(0),
-        num_pinned_buffers_(0) {
+        num_pinned_buffers_(0),
+        logged_large_allocation_warning_(false) {
     DCHECK(tracker != NULL);
   }
 
@@ -95,6 +96,10 @@ struct BufferedBlockMgr::Client {
   /// Number of buffers pinned by this client.
   int num_pinned_buffers_;
 
+  /// Whether a warning about a large allocation has been made for this client. Used
+  /// to avoid producing excessive log messages.
+  bool logged_large_allocation_warning_;
+
   void PinBuffer(BufferDescriptor* buffer) {
     DCHECK(buffer != NULL);
     if (buffer->len == mgr_->max_block_size()) {
@@ -301,8 +306,13 @@ bool BufferedBlockMgr::ConsumeMemory(Client* client, int64_t size) {
   // Workaround IMPALA-1619. Return immediately if the allocation size will cause
   // an arithmetic overflow.
   if (UNLIKELY(size >= (1LL << 31))) {
-    LOG(WARNING) << "Trying to allocate memory >=2GB (" << size << ")B."
-                 << GetStackTrace();
+    // IMPALA-3238: don't repeatedly log warning when bumping up against this limit for
+    // large hash tables.
+    if (!client->logged_large_allocation_warning_) {
+      LOG(WARNING) << "Trying to allocate memory >=2GB (" << size << ")B."
+                   << GetStackTrace();
+      client->logged_large_allocation_warning_ = true;
+    }
     return false;
   }
   int buffers_needed = BitUtil::Ceil(size, max_block_size());


[33/50] incubator-impala git commit: IMPALA-2399: Memory limit checks for all scanners.

Posted by ta...@apache.org.
IMPALA-2399: Memory limit checks for all scanners.

This change replaces all instances of MemPool::Allocate() in
avro, text, hbase scanners with MemPool::TryAllocate().

HdfsAvroScanner::MaterializeTuple() has been converted to return
a boolean in case of memory allocation failure. The codegen'ed
version of MaterializeTuple() will also return a boolean. In the
future, we should consider returning Status directly but that will
be more involved and best left as a separate change.

Change-Id: I3e5a56501967a58513888917db5ce66dec4fb5ce
Reviewed-on: http://gerrit.cloudera.org:8080/2568
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 7050294215cdd81963aa815f69a573421a05ab3e
Parents: 0c4dc96
Author: Michael Ho <kw...@cloudera.com>
Authored: Thu Feb 11 15:32:04 2016 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:02:35 2016 -0700

----------------------------------------------------------------------
 be/src/exec/base-sequence-scanner.cc |   3 +-
 be/src/exec/data-source-scan-node.cc |  16 +++-
 be/src/exec/hbase-table-scanner.cc   |  52 +++++++++---
 be/src/exec/hbase-table-scanner.h    |  14 +--
 be/src/exec/hdfs-avro-scanner-ir.cc  |  36 +++++---
 be/src/exec/hdfs-avro-scanner.cc     | 136 +++++++++++++++++++++---------
 be/src/exec/hdfs-avro-scanner.h      |  36 +++++---
 be/src/exec/hdfs-parquet-scanner.cc  |  11 +--
 be/src/exec/hdfs-scanner.cc          |   8 +-
 be/src/exec/hdfs-sequence-scanner.cc |   4 +
 be/src/exec/hdfs-text-scanner.cc     |  21 +++--
 be/src/exec/hdfs-text-scanner.h      |  11 +--
 be/src/exec/scanner-context.cc       |  10 +--
 be/src/exec/text-converter.h         |   2 +-
 be/src/exec/text-converter.inline.h  |  15 +++-
 be/src/runtime/string-buffer.h       |  38 ++++++---
 16 files changed, 285 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/base-sequence-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/base-sequence-scanner.cc b/be/src/exec/base-sequence-scanner.cc
index 9145403..f88717f 100644
--- a/be/src/exec/base-sequence-scanner.cc
+++ b/be/src/exec/base-sequence-scanner.cc
@@ -98,7 +98,8 @@ void BaseSequenceScanner::Close() {
   // Verify all resources (if any) have been transferred.
   DCHECK_EQ(data_buffer_pool_.get()->total_allocated_bytes(), 0);
   DCHECK_EQ(context_->num_completed_io_buffers(), 0);
-  if (!only_parsing_header_) {
+  // 'header_' can be NULL if HdfsScanNode::CreateAndPrepareScanner() failed.
+  if (!only_parsing_header_ && header_ != NULL) {
     scan_node_->RangeComplete(file_format(), header_->compression_type);
   }
   HdfsScanner::Close();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/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 c865234..2c52c11 100644
--- a/be/src/exec/data-source-scan-node.cc
+++ b/be/src/exec/data-source-scan-node.cc
@@ -55,6 +55,8 @@ const string ERROR_INVALID_TIMESTAMP = "Data source returned invalid timestamp d
     "This likely indicates a problem with the data source library.";
 const string ERROR_INVALID_DECIMAL = "Data source returned invalid decimal data. "
     "This likely indicates a problem with the data source library.";
+const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate "
+    "$1 bytes for $2.";
 
 // Size of an encoded TIMESTAMP
 const size_t TIMESTAMP_SIZE = sizeof(int64_t) + sizeof(int32_t);
@@ -210,7 +212,12 @@ Status DataSourceScanNode::MaterializeNextRow(MemPool* tuple_pool) {
           }
           const string& val = col.string_vals[val_idx];
           size_t val_size = val.size();
-          char* buffer = reinterpret_cast<char*>(tuple_pool->Allocate(val_size));
+          char* buffer = reinterpret_cast<char*>(tuple_pool->TryAllocate(val_size));
+          if (UNLIKELY(buffer == NULL)) {
+            string details = Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow",
+                val_size, "string slot");
+            return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size);
+          }
           memcpy(buffer, val.data(), val_size);
           reinterpret_cast<StringValue*>(slot)->ptr = buffer;
           reinterpret_cast<StringValue*>(slot)->len = val_size;
@@ -300,7 +307,12 @@ Status DataSourceScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, boo
   // create new tuple buffer for row_batch
   MemPool* tuple_pool = row_batch->tuple_data_pool();
   int tuple_buffer_size = row_batch->MaxTupleBufferSize();
-  void* tuple_buffer = tuple_pool->Allocate(tuple_buffer_size);
+  void* tuple_buffer = tuple_pool->TryAllocate(tuple_buffer_size);
+  if (UNLIKELY(tuple_buffer == NULL)) {
+    string details = Substitute(ERROR_MEM_LIMIT_EXCEEDED, "GetNext",
+        tuple_buffer_size, "tuple");
+    return tuple_pool->mem_tracker()->MemLimitExceeded(state, details, tuple_buffer_size);
+  }
   tuple_ = reinterpret_cast<Tuple*>(tuple_buffer);
   ExprContext** ctxs = &conjunct_ctxs_[0];
   int num_ctxs = conjunct_ctxs_.size();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hbase-table-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hbase-table-scanner.cc b/be/src/exec/hbase-table-scanner.cc
index 93cad54..cd622bc 100644
--- a/be/src/exec/hbase-table-scanner.cc
+++ b/be/src/exec/hbase-table-scanner.cc
@@ -28,6 +28,7 @@
 #include "common/names.h"
 
 using namespace impala;
+using namespace strings;
 
 jclass HBaseTableScanner::scan_cl_ = NULL;
 jclass HBaseTableScanner::resultscanner_cl_ = NULL;
@@ -70,6 +71,9 @@ jobject HBaseTableScanner::empty_row_ = NULL;
 jobject HBaseTableScanner::must_pass_all_op_ = NULL;
 jobjectArray HBaseTableScanner::compare_ops_ = NULL;
 
+const string HBASE_MEM_LIMIT_EXCEEDED = "HBaseTableScanner::$0() failed to "
+    "allocate $1 bytes for $2.";
+
 void HBaseTableScanner::ScanRange::DebugString(int indentation_level,
     stringstream* out) {
   *out << string(indentation_level * 2, ' ');
@@ -573,53 +577,77 @@ inline void HBaseTableScanner::WriteTupleSlot(const SlotDescriptor* slot_desc,
   BitUtil::ByteSwap(slot, data, slot_desc->type().GetByteSize());
 }
 
-inline void HBaseTableScanner::GetRowKey(JNIEnv* env, jobject cell,
+inline Status HBaseTableScanner::GetRowKey(JNIEnv* env, jobject cell,
     void** data, int* length) {
   int offset = env->CallIntMethod(cell, cell_get_row_offset_id_);
   *length = env->CallShortMethod(cell, cell_get_row_length_id_);
   jbyteArray jdata =
       (jbyteArray) env->CallObjectMethod(cell, cell_get_row_array_);
-  *data = value_pool_->Allocate(*length);
+  *data = value_pool_->TryAllocate(*length);
+  if (UNLIKELY(*data == NULL)) {
+    string details = Substitute(HBASE_MEM_LIMIT_EXCEEDED, "GetRowKey",
+        *length, "row array");
+    return value_pool_->mem_tracker()->MemLimitExceeded(state_, details, *length);
+  }
   env->GetByteArrayRegion(jdata, offset, *length, reinterpret_cast<jbyte*>(*data));
   COUNTER_ADD(scan_node_->bytes_read_counter(), *length);
+  return Status::OK();
 }
 
-inline void HBaseTableScanner::GetFamily(JNIEnv* env, jobject cell,
+inline Status HBaseTableScanner::GetFamily(JNIEnv* env, jobject cell,
     void** data, int* length) {
   int offset = env->CallIntMethod(cell, cell_get_family_offset_id_);
   *length = env->CallShortMethod(cell, cell_get_family_length_id_);
   jbyteArray jdata =
       (jbyteArray) env->CallObjectMethod(cell, cell_get_family_array_);
-  *data = value_pool_->Allocate(*length);
+  *data = value_pool_->TryAllocate(*length);
+  if (UNLIKELY(*data == NULL)) {
+    string details = Substitute(HBASE_MEM_LIMIT_EXCEEDED, "GetFamily",
+        *length, "family array");
+    return value_pool_->mem_tracker()->MemLimitExceeded(state_, details, *length);
+  }
   env->GetByteArrayRegion(jdata, offset, *length, reinterpret_cast<jbyte*>(*data));
   COUNTER_ADD(scan_node_->bytes_read_counter(), *length);
+  return Status::OK();
 }
 
-inline void HBaseTableScanner::GetQualifier(JNIEnv* env, jobject cell,
+inline Status HBaseTableScanner::GetQualifier(JNIEnv* env, jobject cell,
     void** data, int* length) {
   int offset = env->CallIntMethod(cell, cell_get_qualifier_offset_id_);
   *length = env->CallIntMethod(cell, cell_get_qualifier_length_id_);
   jbyteArray jdata =
       (jbyteArray) env->CallObjectMethod(cell, cell_get_qualifier_array_);
-  *data = value_pool_->Allocate(*length);
+  *data = value_pool_->TryAllocate(*length);
+  if (UNLIKELY(*data == NULL)) {
+    string details = Substitute(HBASE_MEM_LIMIT_EXCEEDED, "GetQualifier",
+        *length, "qualifier array");
+    return value_pool_->mem_tracker()->MemLimitExceeded(state_, details, *length);
+  }
   env->GetByteArrayRegion(jdata, offset, *length, reinterpret_cast<jbyte*>(*data));
   COUNTER_ADD(scan_node_->bytes_read_counter(), *length);
+  return Status::OK();
 }
 
-inline void HBaseTableScanner::GetValue(JNIEnv* env, jobject cell,
+inline Status HBaseTableScanner::GetValue(JNIEnv* env, jobject cell,
     void** data, int* length) {
   int offset = env->CallIntMethod(cell, cell_get_value_offset_id_);
   *length = env->CallIntMethod(cell, cell_get_value_length_id_);
   jbyteArray jdata =
       (jbyteArray) env->CallObjectMethod(cell, cell_get_value_array_);
-  *data = value_pool_->Allocate(*length);
+  *data = value_pool_->TryAllocate(*length);
+  if (UNLIKELY(*data == NULL)) {
+    string details = Substitute(HBASE_MEM_LIMIT_EXCEEDED, "GetValue",
+        *length, "value array");
+    return value_pool_->mem_tracker()->MemLimitExceeded(state_, details, *length);
+  }
   env->GetByteArrayRegion(jdata, offset, *length, reinterpret_cast<jbyte*>(*data));
   COUNTER_ADD(scan_node_->bytes_read_counter(), *length);
+  return Status::OK();
 }
 
 Status HBaseTableScanner::GetRowKey(JNIEnv* env, void** key, int* key_length) {
   jobject cell = env->GetObjectArrayElement(cells_, 0);
-  GetRowKey(env, cell, key, key_length);
+  RETURN_IF_ERROR(GetRowKey(env, cell, key, key_length));
   RETURN_ERROR_IF_EXC(env);
   return Status::OK();
 }
@@ -650,7 +678,7 @@ Status HBaseTableScanner::GetCurrentValue(JNIEnv* env, const string& family,
     // Check family. If it doesn't match, we have a NULL value.
     void* family_data;
     int family_length;
-    GetFamily(env, cell, &family_data, &family_length);
+    RETURN_IF_ERROR(GetFamily(env, cell, &family_data, &family_length));
     if (CompareStrings(family, family_data, family_length) != 0) {
       *is_null = true;
       return Status::OK();
@@ -659,13 +687,13 @@ Status HBaseTableScanner::GetCurrentValue(JNIEnv* env, const string& family,
     // Check qualifier. If it doesn't match, we have a NULL value.
     void* qualifier_data;
     int qualifier_length;
-    GetQualifier(env, cell, &qualifier_data, &qualifier_length);
+    RETURN_IF_ERROR(GetQualifier(env, cell, &qualifier_data, &qualifier_length));
     if (CompareStrings(qualifier, qualifier_data, qualifier_length) != 0) {
       *is_null = true;
       return Status::OK();
     }
   }
-  GetValue(env, cell, data, length);
+  RETURN_IF_ERROR(GetValue(env, cell, data, length));
   *is_null = false;
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hbase-table-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hbase-table-scanner.h b/be/src/exec/hbase-table-scanner.h
index 994a036..52e5da7 100644
--- a/be/src/exec/hbase-table-scanner.h
+++ b/be/src/exec/hbase-table-scanner.h
@@ -271,18 +271,20 @@ class HBaseTableScanner {
   Status InitScanRange(JNIEnv* env, jbyteArray start_bytes, jbyteArray end_bytes);
 
   /// Copies the row key of cell into value_pool_ and returns it via *data and *length.
-  inline void GetRowKey(JNIEnv* env, jobject cell, void** data, int* length);
+  /// Returns error status if memory limit is exceeded.
+  inline Status GetRowKey(JNIEnv* env, jobject cell, void** data, int* length);
 
   /// Copies the column family of cell into value_pool_ and returns it
-  /// via *data and *length.
-  inline void GetFamily(JNIEnv* env, jobject cell, void** data, int* length);
+  /// via *data and *length. Returns error status if memory limit is exceeded.
+  inline Status GetFamily(JNIEnv* env, jobject cell, void** data, int* length);
 
   /// Copies the column qualifier of cell into value_pool_ and returns it
-  /// via *data and *length.
-  inline void GetQualifier(JNIEnv* env, jobject cell, void** data, int* length);
+  /// via *data and *length. Returns error status if memory limit is exceeded.
+  inline Status GetQualifier(JNIEnv* env, jobject cell, void** data, int* length);
 
   /// Copies the value of cell into value_pool_ and returns it via *data and *length.
-  inline void GetValue(JNIEnv* env, jobject cell, void** data, int* length);
+  /// Returns error status if memory limit is exceeded.
+  inline Status GetValue(JNIEnv* env, jobject cell, void** data, int* length);
 
   /// Returns the current value of cells_[cell_index_] in *data and *length
   /// if its family/qualifier match the given family/qualifier.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-avro-scanner-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner-ir.cc b/be/src/exec/hdfs-avro-scanner-ir.cc
index a84fe7b..f3ebb97 100644
--- a/be/src/exec/hdfs-avro-scanner-ir.cc
+++ b/be/src/exec/hdfs-avro-scanner-ir.cc
@@ -19,15 +19,18 @@
 #include "runtime/string-value.inline.h"
 
 using namespace impala;
+using namespace strings;
 
 // Functions in this file are cross-compiled to IR with clang.
 
 int HdfsAvroScanner::DecodeAvroData(int max_tuples, MemPool* pool, uint8_t** data,
-                                    Tuple* tuple, TupleRow* tuple_row) {
+    Tuple* tuple, TupleRow* tuple_row) {
   int num_to_commit = 0;
   for (int i = 0; i < max_tuples; ++i) {
     InitTuple(template_tuple_, tuple);
-    MaterializeTuple(*avro_header_->schema.get(), pool, data, tuple);
+    if (UNLIKELY(!MaterializeTuple(*avro_header_->schema.get(), pool, data, tuple))) {
+      return 0;
+    }
     tuple_row->SetTuple(scan_node_->tuple_idx(), tuple);
     if (EvalConjuncts(tuple_row)) {
       ++num_to_commit;
@@ -50,7 +53,7 @@ bool HdfsAvroScanner::ReadUnionType(int null_union_position, uint8_t** data) {
 }
 
 void HdfsAvroScanner::ReadAvroBoolean(PrimitiveType type, uint8_t** data, bool write_slot,
-                                      void* slot, MemPool* pool) {
+    void* slot, MemPool* pool) {
   if (write_slot) {
     DCHECK_EQ(type, TYPE_BOOLEAN);
     *reinterpret_cast<bool*>(slot) = *reinterpret_cast<bool*>(*data);
@@ -59,7 +62,7 @@ void HdfsAvroScanner::ReadAvroBoolean(PrimitiveType type, uint8_t** data, bool w
 }
 
 void HdfsAvroScanner::ReadAvroInt32(PrimitiveType type, uint8_t** data, bool write_slot,
-                                    void* slot, MemPool* pool) {
+    void* slot, MemPool* pool) {
   int32_t val = ReadWriteUtil::ReadZInt(data);
   if (write_slot) {
     if (type == TYPE_INT) {
@@ -77,7 +80,7 @@ void HdfsAvroScanner::ReadAvroInt32(PrimitiveType type, uint8_t** data, bool wri
 }
 
 void HdfsAvroScanner::ReadAvroInt64(PrimitiveType type, uint8_t** data, bool write_slot,
-                                    void* slot, MemPool* pool) {
+    void* slot, MemPool* pool) {
   int64_t val = ReadWriteUtil::ReadZLong(data);
   if (write_slot) {
     if (type == TYPE_BIGINT) {
@@ -93,7 +96,7 @@ void HdfsAvroScanner::ReadAvroInt64(PrimitiveType type, uint8_t** data, bool wri
 }
 
 void HdfsAvroScanner::ReadAvroFloat(PrimitiveType type, uint8_t** data, bool write_slot,
-                                    void* slot, MemPool* pool) {
+    void* slot, MemPool* pool) {
   if (write_slot) {
     float val = *reinterpret_cast<float*>(*data);
     if (type == TYPE_FLOAT) {
@@ -108,7 +111,7 @@ void HdfsAvroScanner::ReadAvroFloat(PrimitiveType type, uint8_t** data, bool wri
 }
 
 void HdfsAvroScanner::ReadAvroDouble(PrimitiveType type, uint8_t** data, bool write_slot,
-                                     void* slot, MemPool* pool) {
+    void* slot, MemPool* pool) {
   if (write_slot) {
     DCHECK_EQ(type, TYPE_DOUBLE);
     *reinterpret_cast<double*>(slot) = *reinterpret_cast<double*>(*data);
@@ -117,7 +120,7 @@ void HdfsAvroScanner::ReadAvroDouble(PrimitiveType type, uint8_t** data, bool wr
 }
 
 void HdfsAvroScanner::ReadAvroVarchar(PrimitiveType type, int max_len, uint8_t** data,
-                                     bool write_slot, void* slot, MemPool* pool) {
+    bool write_slot, void* slot, MemPool* pool) {
   int64_t len = ReadWriteUtil::ReadZLong(data);
   if (write_slot) {
     DCHECK(type == TYPE_VARCHAR);
@@ -130,8 +133,8 @@ void HdfsAvroScanner::ReadAvroVarchar(PrimitiveType type, int max_len, uint8_t**
   *data += len;
 }
 
-void HdfsAvroScanner::ReadAvroChar(PrimitiveType type, int max_len, uint8_t** data,
-                                   bool write_slot, void* slot, MemPool* pool) {
+bool HdfsAvroScanner::ReadAvroChar(PrimitiveType type, int max_len, uint8_t** data,
+    bool write_slot, void* slot, MemPool* pool) {
   int64_t len = ReadWriteUtil::ReadZLong(data);
   if (write_slot) {
     DCHECK(type == TYPE_CHAR);
@@ -139,7 +142,13 @@ void HdfsAvroScanner::ReadAvroChar(PrimitiveType type, int max_len, uint8_t** da
     int str_len = std::min(static_cast<int>(len), max_len);
     if (ctype.IsVarLenStringType()) {
       StringValue* sv = reinterpret_cast<StringValue*>(slot);
-      sv->ptr = reinterpret_cast<char*>(pool->Allocate(max_len));
+      sv->ptr = reinterpret_cast<char*>(pool->TryAllocate(max_len));
+      if (UNLIKELY(sv->ptr == NULL)) {
+        string details = Substitute("HdfsAvroScanner::ReadAvroChar() failed to allocate"
+            "$0 bytes for char slot.", max_len);
+        parse_status_ = pool->mem_tracker()->MemLimitExceeded(state_, details, max_len);
+        return false;
+      }
       sv->len = max_len;
       memcpy(sv->ptr, *data, str_len);
       StringValue::PadWithSpaces(sv->ptr, max_len, str_len);
@@ -149,10 +158,11 @@ void HdfsAvroScanner::ReadAvroChar(PrimitiveType type, int max_len, uint8_t** da
     }
   }
   *data += len;
+  return true;
 }
 
 void HdfsAvroScanner::ReadAvroString(PrimitiveType type, uint8_t** data,
-                                     bool write_slot, void* slot, MemPool* pool) {
+    bool write_slot, void* slot, MemPool* pool) {
   int64_t len = ReadWriteUtil::ReadZLong(data);
   if (write_slot) {
     DCHECK(type == TYPE_STRING);
@@ -164,7 +174,7 @@ void HdfsAvroScanner::ReadAvroString(PrimitiveType type, uint8_t** data,
 }
 
 void HdfsAvroScanner::ReadAvroDecimal(int slot_byte_size, uint8_t** data,
-                                      bool write_slot, void* slot, MemPool* pool) {
+    bool write_slot, void* slot, MemPool* pool) {
   int64_t len = ReadWriteUtil::ReadZLong(data);
   if (write_slot) {
     // Decimals are encoded as big-endian integers. Copy the decimal into the most

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-avro-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.cc b/be/src/exec/hdfs-avro-scanner.cc
index ddcec6b..060ec05 100644
--- a/be/src/exec/hdfs-avro-scanner.cc
+++ b/be/src/exec/hdfs-avro-scanner.cc
@@ -47,6 +47,9 @@ const string HdfsAvroScanner::AVRO_NULL_CODEC("null");
 const string HdfsAvroScanner::AVRO_SNAPPY_CODEC("snappy");
 const string HdfsAvroScanner::AVRO_DEFLATE_CODEC("deflate");
 
+const string AVRO_MEM_LIMIT_EXCEEDED = "HdfsAvroScanner::$0() failed to allocate "
+    "$1 bytes for $2.";
+
 #define RETURN_IF_FALSE(x) if (UNLIKELY(!(x))) return parse_status_
 
 HdfsAvroScanner::HdfsAvroScanner(HdfsScanNode* scan_node, RuntimeState* state)
@@ -496,6 +499,7 @@ Status HdfsAvroScanner::ProcessRange() {
           num_to_commit = DecodeAvroData(max_tuples, pool, &data, tuple, tuple_row);
         }
       }
+      RETURN_IF_ERROR(parse_status_);
       RETURN_IF_ERROR(CommitRows(num_to_commit));
       num_records -= max_tuples;
       COUNTER_ADD(scan_node_->rows_read_counter(), max_tuples);
@@ -512,7 +516,7 @@ Status HdfsAvroScanner::ProcessRange() {
   return Status::OK();
 }
 
-void HdfsAvroScanner::MaterializeTuple(const AvroSchemaElement& record_schema,
+bool HdfsAvroScanner::MaterializeTuple(const AvroSchemaElement& record_schema,
     MemPool* pool, uint8_t** data, Tuple* tuple) {
   DCHECK_EQ(record_schema.schema->type, AVRO_RECORD);
   BOOST_FOREACH(const AvroSchemaElement& element, record_schema.children) {
@@ -555,7 +559,11 @@ void HdfsAvroScanner::MaterializeTuple(const AvroSchemaElement& record_schema,
         if (slot_desc != NULL && slot_desc->type().type == TYPE_VARCHAR) {
           ReadAvroVarchar(slot_type, slot_desc->type().len, data, write_slot, slot, pool);
         } else if (slot_desc != NULL && slot_desc->type().type == TYPE_CHAR) {
-          ReadAvroChar(slot_type, slot_desc->type().len, data, write_slot, slot, pool);
+          if (UNLIKELY(!ReadAvroChar(slot_type, slot_desc->type().len, data, write_slot,
+                           slot, pool))) {
+            DCHECK(!parse_status_.ok());
+            return false;
+          }
         } else {
           ReadAvroString(slot_type, data, write_slot, slot, pool);
         }
@@ -576,53 +584,86 @@ void HdfsAvroScanner::MaterializeTuple(const AvroSchemaElement& record_schema,
         DCHECK(false) << "Unsupported SchemaElement: " << type;
     }
   }
+  return true;
 }
 
 // This function produces a codegen'd function equivalent to MaterializeTuple() but
 // optimized for the table schema. Via helper functions CodegenReadRecord() and
 // CodegenReadScalar(), it eliminates the conditionals necessary when interpreting the
 // type of each element in the schema, instead generating code to handle each element in
-// the schema. Example output:
+// the schema. Example output with tpch.region:
 //
-// define void @MaterializeTuple(%"class.impala::HdfsAvroScanner"* %this,
+// define i1 @MaterializeTuple(%"class.impala::HdfsAvroScanner"* %this,
+//     %"struct.impala::AvroSchemaElement"* %record_schema,
 //     %"class.impala::MemPool"* %pool, i8** %data, %"class.impala::Tuple"* %tuple) {
 // entry:
-//   %tuple_ptr = bitcast %"class.impala::Tuple"* %tuple to { i8, i32 }*
+//   %tuple_ptr = bitcast %"class.impala::Tuple"* %tuple to { i8, i32,
+//       %"struct.impala::StringValue", %"struct.impala::StringValue" }*
 //   %is_not_null = call i1 @_ZN6impala15HdfsAvroScanner13ReadUnionTypeEiPPh(
 //       %"class.impala::HdfsAvroScanner"* %this, i32 1, i8** %data)
 //   br i1 %is_not_null, label %read_field, label %null_field
 //
 // read_field:                                       ; preds = %entry
-//   %slot = getelementptr inbounds { i8, i32 }* %tuple_ptr, i32 0, i32 1
+//   %slot = getelementptr inbounds { i8, i32, %"struct.impala::StringValue",
+//       %"struct.impala::StringValue" }* %tuple_ptr, i32 0, i32 1
 //   %opaque_slot = bitcast i32* %slot to i8*
 //   call void
-//    @_ZN6impala15HdfsAvroScanner13ReadAvroInt32ENS_13PrimitiveTypeEPPhPvPNS_7MemPoolE(
-//        %"class.impala::HdfsAvroScanner"* %this, i32 5, i8** %data,
+//    @_ZN6impala15HdfsAvroScanner13ReadAvroInt32ENS_13PrimitiveTypeEPPhbPvPNS_7MemPoolE(
+//        %"class.impala::HdfsAvroScanner"* %this, i32 5, i8** %data, i1 true,
 //        i8* %opaque_slot, %"class.impala::MemPool"* %pool)
-//   br label %endif
+//   br label %end_field
 //
 // null_field:                                       ; preds = %entry
-//   call void @SetNull({ i8, i32 }* %tuple_ptr)
-//   br label %endif
+//   call void @SetNull({ i8, i32, %"struct.impala::StringValue",
+//       %"struct.impala::StringValue" }* %tuple_ptr)
+//   br label %end_field
+//
+// end_field:                                        ; preds = %read_field, %null_field
+//  %is_not_null4 = call i1 @_ZN6impala15HdfsAvroScanner13ReadUnionTypeEiPPh(
+//      %"class.impala::HdfsAvroScanner"* %this, i32 1, i8** %data)
+//  br i1 %is_not_null4, label %read_field1, label %null_field3
 //
-// endif:                                            ; preds = %null_field, %read_field
-//   %is_not_null4 = call i1 @_ZN6impala15HdfsAvroScanner13ReadUnionTypeEiPPh(
+// read_field1:                                      ; preds = %end_field
+//  %slot5 = getelementptr inbounds { i8, i32, %"struct.impala::StringValue",
+//      %"struct.impala::StringValue" }* %tuple_ptr, i32 0, i32 2
+//  %opaque_slot6 = bitcast %"struct.impala::StringValue"* %slot5 to i8*
+//  call void
+//   @_ZN6impala15HdfsAvroScanner14ReadAvroStringENS_13PrimitiveTypeEPPhbPvPNS_7MemPoolE(
+//       %"class.impala::HdfsAvroScanner"* %this, i32 10, i8** %data, i1 true,
+//       i8* %opaque_slot6, %"class.impala::MemPool"* %pool)
+//  br label %end_field2
+//
+// null_field3:                                      ; preds = %end_field
+//   call void @SetNull1({ i8, i32, %"struct.impala::StringValue",
+//       %"struct.impala::StringValue" }* %tuple_ptr)
+//   br label %end_field2
+//
+// end_field2:                                       ; preds = %read_field1, %null_field3
+//   %is_not_null10 = call i1 @_ZN6impala15HdfsAvroScanner13ReadUnionTypeEiPPh(
 //       %"class.impala::HdfsAvroScanner"* %this, i32 1, i8** %data)
-//   br i1 %is_not_null4, label %read_field1, label %null_field2
+//   br i1 %is_not_null10, label %read_field7, label %null_field9
 //
-// read_field1:                                      ; preds = %endif
+// read_field7:                                      ; preds = %end_field2
+//   %slot11 = getelementptr inbounds { i8, i32, %"struct.impala::StringValue",
+//       %"struct.impala::StringValue" }* %tuple_ptr, i32 0, i32 3
+//   %opaque_slot12 = bitcast %"struct.impala::StringValue"* %slot11 to i8*
 //   call void
-//    @_ZN6impala15HdfsAvroScanner15ReadAvroBooleanENS_13PrimitiveTypeEPPhPvPNS_7MemPoolE(
-//        %"class.impala::HdfsAvroScanner"* %this, i32 0, i8** %data,
-//        i8* null, %"class.impala::MemPool"* %pool)
-//   br label %endif3
+//    @_ZN6impala15HdfsAvroScanner14ReadAvroStringENS_13PrimitiveTypeEPPhbPvPNS_7MemPoolE(
+//        %"class.impala::HdfsAvroScanner"* %this, i32 10, i8** %data, i1 true,
+//        i8* %opaque_slot12, %"class.impala::MemPool"* %pool)
+//   br label %end_field8
+//
+// null_field9:                                      ; preds = %end_field2
+//   call void @SetNull2({ i8, i32, %"struct.impala::StringValue",
+//       %"struct.impala::StringValue" }* %tuple_ptr)
+//   br label %end_field8
 //
-// null_field2:                                      ; preds = %endif
-//   br label %endif3
+// end_field8:                                       ; preds = %read_field7, %null_field9
+//   ret i1 true
 //
-// endif3:                                           ; preds = %null_field2, %read_field1
-//   ret void
-// }
+// bail_out:                                         ; No predecessors!
+//   ret i1 false                                    // used only if there is CHAR.
+//}
 Function* HdfsAvroScanner::CodegenMaterializeTuple(
     HdfsScanNode* node, LlvmCodeGen* codegen) {
   LLVMContext& context = codegen->context();
@@ -644,7 +685,7 @@ Function* HdfsAvroScanner::CodegenMaterializeTuple(
   Type* mempool_type = PointerType::get(codegen->GetType(MemPool::LLVM_CLASS_NAME), 0);
   Type* schema_element_type = codegen->GetPtrType(AvroSchemaElement::LLVM_CLASS_NAME);
 
-  LlvmCodeGen::FnPrototype prototype(codegen, "MaterializeTuple", codegen->void_type());
+  LlvmCodeGen::FnPrototype prototype(codegen, "MaterializeTuple", codegen->boolean_type());
   prototype.AddArgument(LlvmCodeGen::NamedVariable("this", this_ptr_type));
   prototype.AddArgument(LlvmCodeGen::NamedVariable("record_schema", schema_element_type));
   prototype.AddArgument(LlvmCodeGen::NamedVariable("pool", mempool_type));
@@ -661,24 +702,33 @@ Function* HdfsAvroScanner::CodegenMaterializeTuple(
 
   Value* tuple_val = builder.CreateBitCast(opaque_tuple_val, tuple_ptr_type, "tuple_ptr");
 
+  // Create a bail out block to handle decoding failures.
+  BasicBlock* bail_out_block = BasicBlock::Create(context, "bail_out", fn, NULL);
+
   Status status = CodegenReadRecord(
-      SchemaPath(), node->avro_schema(), node, codegen, &builder, fn, NULL, this_val,
-      pool_val, tuple_val, data_val);
+      SchemaPath(), node->avro_schema(), node, codegen, &builder, fn, bail_out_block,
+      bail_out_block, this_val, pool_val, tuple_val, data_val);
   if (!status.ok()) {
     VLOG_QUERY << status.GetDetail();
     fn->eraseFromParent();
     return NULL;
   }
 
-  builder.SetInsertPoint(&fn->back());
-  builder.CreateRetVoid();
+  // Returns true on successful decoding.
+  builder.CreateRet(codegen->true_value());
+
+  // Returns false on decoding errors.
+  builder.SetInsertPoint(bail_out_block);
+  builder.CreateRet(codegen->false_value());
+
   return codegen->FinalizeFunction(fn);
 }
 
 Status HdfsAvroScanner::CodegenReadRecord(
     const SchemaPath& path, const AvroSchemaElement& record, HdfsScanNode* node,
     LlvmCodeGen* codegen, void* void_builder, Function* fn, BasicBlock* insert_before,
-    Value* this_val, Value* pool_val, Value* tuple_val, Value* data_val) {
+    BasicBlock* bail_out, Value* this_val, Value* pool_val, Value* tuple_val,
+    Value* data_val) {
   DCHECK_EQ(record.schema->type, AVRO_RECORD);
   LLVMContext& context = codegen->context();
   LlvmCodeGen::LlvmBuilder* builder =
@@ -742,21 +792,22 @@ Status HdfsAvroScanner::CodegenReadRecord(
       BasicBlock* insert_before_block =
           (null_block != NULL) ? null_block : end_field_block;
       RETURN_IF_ERROR(CodegenReadRecord(new_path, *field, node, codegen, builder, fn,
-          insert_before_block, this_val, pool_val, tuple_val, data_val));
+          insert_before_block, bail_out, this_val, pool_val, tuple_val, data_val));
     } else {
-      RETURN_IF_ERROR(CodegenReadScalar(
-          *field, slot_desc, codegen, builder, this_val, pool_val, tuple_val, data_val));
+      RETURN_IF_ERROR(CodegenReadScalar(*field, slot_desc, codegen, builder,
+          end_field_block, bail_out, this_val, pool_val, tuple_val, data_val));
     }
     builder->CreateBr(end_field_block);
 
-    // Set insertion point for next field
+    // Set insertion point for next field.
     builder->SetInsertPoint(end_field_block);
   }
   return Status::OK();
 }
 
 Status HdfsAvroScanner::CodegenReadScalar(const AvroSchemaElement& element,
-    SlotDescriptor* slot_desc, LlvmCodeGen* codegen, void* void_builder, Value* this_val,
+    SlotDescriptor* slot_desc, LlvmCodeGen* codegen, void* void_builder,
+    BasicBlock* end_field_block, BasicBlock* bail_out_block, Value* this_val,
     Value* pool_val, Value* tuple_val, Value* data_val) {
   LlvmCodeGen::LlvmBuilder* builder =
       reinterpret_cast<LlvmCodeGen::LlvmBuilder*>(void_builder);
@@ -781,10 +832,13 @@ Status HdfsAvroScanner::CodegenReadScalar(const AvroSchemaElement& element,
     case AVRO_BYTES:
       if (slot_desc != NULL && slot_desc->type().type == TYPE_VARCHAR) {
         read_field_fn = codegen->GetFunction(IRFunction::READ_AVRO_VARCHAR, false);
+      } else if (slot_desc != NULL && slot_desc->type().type == TYPE_CHAR) {
+        read_field_fn = codegen->GetFunction(IRFunction::READ_AVRO_CHAR, false);
       } else {
         read_field_fn = codegen->GetFunction(IRFunction::READ_AVRO_STRING, false);
       }
       break;
+    // TODO: Add AVRO_DECIMAL here.
     default:
       return Status(Substitute(
           "Failed to codegen MaterializeTuple() due to unsupported type: $0",
@@ -811,13 +865,19 @@ Status HdfsAvroScanner::CodegenReadScalar(const AvroSchemaElement& element,
   }
 
   // NOTE: ReadAvroVarchar/Char has different signature than rest of read functions
-  if ((slot_desc != NULL) &&
+  if (slot_desc != NULL &&
       (slot_desc->type().type == TYPE_VARCHAR || slot_desc->type().type == TYPE_CHAR)) {
-    // Need to pass an extra argument (the length) to the codegen function
+    // Need to pass an extra argument (the length) to the codegen function.
     Value* fixed_len = builder->getInt32(slot_desc->type().len);
     Value* read_field_args[] = {this_val, slot_type_val, fixed_len, data_val,
                                 write_slot_val, opaque_slot_val, pool_val};
-    builder->CreateCall(read_field_fn, read_field_args);
+    if (slot_desc->type().type == TYPE_VARCHAR) {
+      builder->CreateCall(read_field_fn, read_field_args);
+    } else {
+      // ReadAvroChar() returns false if allocation from MemPool fails.
+      Value* ret_val = builder->CreateCall(read_field_fn, read_field_args);
+      builder->CreateCondBr(ret_val, end_field_block, bail_out_block);
+    }
   } else {
     Value* read_field_args[] =
         {this_val, slot_type_val, data_val, write_slot_val, opaque_slot_val, pool_val};

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-avro-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-avro-scanner.h b/be/src/exec/hdfs-avro-scanner.h
index 15e046f..682ba04 100644
--- a/be/src/exec/hdfs-avro-scanner.h
+++ b/be/src/exec/hdfs-avro-scanner.h
@@ -176,8 +176,10 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   int DecodeAvroData(int max_tuples, MemPool* pool, uint8_t** data,
       Tuple* tuple, TupleRow* tuple_row);
 
-  /// Materializes a single tuple from serialized record data.
-  void MaterializeTuple(const AvroSchemaElement& record_schema, MemPool* pool,
+  /// Materializes a single tuple from serialized record data. Will return false and set
+  /// error in parse_status_ if memory limit is exceeded when allocating new char buffer.
+  /// See comments below for ReadAvroChar().
+  bool MaterializeTuple(const AvroSchemaElement& record_schema, MemPool* pool,
       uint8_t** data, Tuple* tuple);
 
   /// Produces a version of DecodeAvroData that uses codegen'd instead of interpreted
@@ -190,7 +192,7 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   /// schema.
   /// TODO: Codegen a function for each unique file schema.
   static llvm::Function* CodegenMaterializeTuple(HdfsScanNode* node,
-                                                 LlvmCodeGen* codegen);
+      LlvmCodeGen* codegen);
 
   /// Used by CodegenMaterializeTuple to recursively create the IR for reading an Avro
   /// record.
@@ -200,21 +202,23 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   /// - builder: used to insert the IR, starting at the current insert point. The insert
   ///     point will be left at the end of the record but before the 'insert_before'
   ///     block.
-  /// - insert_before: the block to insert any new blocks directly before. NULL if blocks
-  ///     should be inserted at the end of fn. (This could theoretically be inferred from
-  ///     builder's insert point, but I can't figure out how to get the successor to a
-  ///     basic block.)
+  /// - insert_before: the block to insert any new blocks directly before. This is either
+  ///     the bail_out block or some basic blocks before that.
+  /// - bail_out: the block to jump to if anything fails. This is used in particular by
+  ///     ReadAvroChar() which can exceed memory limit during allocation from MemPool.
   /// - this_val, pool_val, tuple_val, data_val: arguments to MaterializeTuple()
   static Status CodegenReadRecord(
       const SchemaPath& path, const AvroSchemaElement& record, HdfsScanNode* node,
       LlvmCodeGen* codegen, void* builder, llvm::Function* fn,
-      llvm::BasicBlock* insert_before, llvm::Value* this_val, llvm::Value* pool_val,
-      llvm::Value* tuple_val, llvm::Value* data_val);
+      llvm::BasicBlock* insert_before, llvm::BasicBlock* bail_out, llvm::Value* this_val,
+      llvm::Value* pool_val, llvm::Value* tuple_val, llvm::Value* data_val);
 
   /// Creates the IR for reading an Avro scalar at builder's current insert point.
   static Status CodegenReadScalar(const AvroSchemaElement& element,
-    SlotDescriptor* slot_desc, LlvmCodeGen* codegen, void* builder, llvm::Value* this_val,
-    llvm::Value* pool_val, llvm::Value* tuple_val, llvm::Value* data_val);
+      SlotDescriptor* slot_desc, LlvmCodeGen* codegen, void* void_builder,
+      llvm::BasicBlock* end_field_block, llvm::BasicBlock* bail_out_block,
+      llvm::Value* this_val, llvm::Value* pool_val, llvm::Value* tuple_val,
+      llvm::Value* data_val);
 
   /// The following are cross-compiled functions for parsing a serialized Avro primitive
   /// type and writing it to a slot. They can also be used for skipping a field without
@@ -225,6 +229,10 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   /// - type: The type of the slot. (This is necessary because there is not a 1:1 mapping
   ///         between Avro types and Impala's primitive types.)
   /// - pool: MemPool for string data.
+  ///
+  /// ReadAvroChar() will return false and set error in parse_status_ if memory limit
+  /// is exceeded when allocating the new char buffer. It returns true otherwise.
+  ///
   void ReadAvroBoolean(
       PrimitiveType type, uint8_t** data, bool write_slot, void* slot, MemPool* pool);
   void ReadAvroInt32(
@@ -238,11 +246,11 @@ class HdfsAvroScanner : public BaseSequenceScanner {
   void ReadAvroVarchar(
       PrimitiveType type, int max_len, uint8_t** data, bool write_slot, void* slot,
       MemPool* pool);
-  void ReadAvroChar(
+  bool ReadAvroChar(
       PrimitiveType type, int max_len, uint8_t** data, bool write_slot, void* slot,
       MemPool* pool);
-  void ReadAvroString( PrimitiveType type, uint8_t** data, bool write_slot, void* slot,
-      MemPool* pool);
+  void ReadAvroString(
+      PrimitiveType type, uint8_t** data, bool write_slot, void* slot, MemPool* pool);
 
   /// Same as the above functions, except takes the size of the decimal slot (i.e. 4, 8, or
   /// 16) instead of the type (which should be TYPE_DECIMAL). The slot size is passed

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index ad8e360..a910243 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -176,7 +176,8 @@ DiskIoMgr::ScanRange* HdfsParquetScanner::FindFooterSplit(HdfsFileDesc* file) {
 
 namespace impala {
 
-const string PARQUET_MEM_LIMIT_EXCEEDED = "$0 failed to allocate $1 bytes for $2.";
+const string PARQUET_MEM_LIMIT_EXCEEDED = "HdfsParquetScanner::$0() failed to allocate "
+    "$1 bytes for $2.";
 
 HdfsParquetScanner::HdfsParquetScanner(HdfsScanNode* scan_node, RuntimeState* state)
     : HdfsScanner(scan_node, state),
@@ -750,7 +751,7 @@ bool HdfsParquetScanner::ScalarColumnReader<StringValue, true>::ConvertSlot(
   if (slot_desc()->type().IsVarLenStringType()) {
     sv.ptr = reinterpret_cast<char*>(pool->TryAllocate(len));
     if (UNLIKELY(sv.ptr == NULL)) {
-      string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ConvertSlot()",
+      string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ConvertSlot",
           len, "StringValue");
       parent_->parse_status_ =
           pool->mem_tracker()->MemLimitExceeded(parent_->state_, details, len);
@@ -1132,7 +1133,7 @@ Status HdfsParquetScanner::BaseScalarColumnReader::ReadDataPage() {
       if (decompressor_.get() != NULL) {
         dict_values = parent_->dictionary_pool_->TryAllocate(uncompressed_size);
         if (UNLIKELY(dict_values == NULL)) {
-          string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage()",
+          string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage",
               uncompressed_size, "dictionary");
           return parent_->dictionary_pool_->mem_tracker()->MemLimitExceeded(
               parent_->state_, details, uncompressed_size);
@@ -1147,7 +1148,7 @@ Status HdfsParquetScanner::BaseScalarColumnReader::ReadDataPage() {
         // more data) to a new buffer
         dict_values = parent_->dictionary_pool_->TryAllocate(data_size);
         if (UNLIKELY(dict_values == NULL)) {
-          string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage()",
+          string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage",
               data_size, "dictionary");
           return parent_->dictionary_pool_->mem_tracker()->MemLimitExceeded(
               parent_->state_, details, data_size);
@@ -1184,7 +1185,7 @@ Status HdfsParquetScanner::BaseScalarColumnReader::ReadDataPage() {
       uint8_t* decompressed_buffer =
           decompressed_data_pool_->TryAllocate(uncompressed_size);
       if (UNLIKELY(decompressed_buffer == NULL)) {
-        string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage()",
+        string details = Substitute(PARQUET_MEM_LIMIT_EXCEEDED, "ReadDataPage",
             uncompressed_size, "decompressed data");
         return decompressed_data_pool_->mem_tracker()->MemLimitExceeded(
             parent_->state_, details, uncompressed_size);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scanner.cc b/be/src/exec/hdfs-scanner.cc
index 7c0fd9a..a26a575 100644
--- a/be/src/exec/hdfs-scanner.cc
+++ b/be/src/exec/hdfs-scanner.cc
@@ -115,8 +115,8 @@ Status HdfsScanner::InitializeWriteTuplesFn(HdfsPartitionDescriptor* partition,
     THdfsFileFormat::type type, const string& scanner_name) {
   if (!scan_node_->tuple_desc()->string_slots().empty()
       && partition->escape_char() != '\0') {
-    // Cannot use codegen if there are strings slots and we need to
-    // compact (i.e. copy) the data.
+    // Codegen currently doesn't emit call to MemPool::TryAllocate() so skip codegen if
+    // there are strings slots and we need to compact (i.e. copy) the data.
     scan_node_->IncNumScannersCodegenDisabled();
     return Status::OK();
   }
@@ -181,8 +181,8 @@ Status HdfsScanner::CommitRows(int num_rows) {
     RETURN_IF_ERROR(StartNewRowBatch());
   }
   if (context_->cancelled()) return Status::CANCELLED;
-  // TODO: Replace with GetQueryStatus().
-  RETURN_IF_ERROR(state_->CheckQueryState());
+  // Check for UDF errors.
+  RETURN_IF_ERROR(state_->GetQueryStatus());
   // Free local expr allocations for this thread
   HdfsScanNode::ConjunctsMap::const_iterator iter = scanner_conjuncts_map_.begin();
   for (; iter != scanner_conjuncts_map_.end(); ++iter) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-sequence-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-sequence-scanner.cc b/be/src/exec/hdfs-sequence-scanner.cc
index 3bed4e3..168cdd1 100644
--- a/be/src/exec/hdfs-sequence-scanner.cc
+++ b/be/src/exec/hdfs-sequence-scanner.cc
@@ -252,6 +252,10 @@ Status HdfsSequenceScanner::ProcessDecompressedBlock() {
   // Call jitted function if possible
   int tuples_returned;
   if (write_tuples_fn_ != NULL) {
+    // HdfsScanner::InitializeWriteTuplesFn() will skip codegen if there are string slots
+    // and escape characters. TextConverter::WriteSlot() will be used instead.
+    DCHECK(scan_node_->tuple_desc()->string_slots().empty() ||
+        delimited_text_parser_->escape_char() == '\0');
     // last argument: seq always starts at record_location[0]
     tuples_returned = write_tuples_fn_(this, pool, tuple_row,
         batch_->row_byte_size(), &field_locations_[0], num_to_process,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-text-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-text-scanner.cc b/be/src/exec/hdfs-text-scanner.cc
index 69abebc..9572260 100644
--- a/be/src/exec/hdfs-text-scanner.cc
+++ b/be/src/exec/hdfs-text-scanner.cc
@@ -35,6 +35,7 @@ using boost::algorithm::ends_with;
 using boost::algorithm::to_lower;
 using namespace impala;
 using namespace llvm;
+using namespace strings;
 
 const char* HdfsTextScanner::LLVM_CLASS_NAME = "class.impala::HdfsTextScanner";
 
@@ -359,7 +360,7 @@ Status HdfsTextScanner::ProcessRange(int* num_tuples, bool past_scan_range) {
       // There can be one partial tuple which returned no more fields from this buffer.
       DCHECK_LE(*num_tuples, num_fields + 1);
       if (!boundary_column_.Empty()) {
-        CopyBoundaryField(&field_locations_[0], pool);
+        RETURN_IF_ERROR(CopyBoundaryField(&field_locations_[0], pool));
         boundary_column_.Clear();
       }
       num_tuples_materialized = WriteFields(pool, tuple_row_mem, num_fields, *num_tuples);
@@ -379,14 +380,14 @@ Status HdfsTextScanner::ProcessRange(int* num_tuples, bool past_scan_range) {
     // Save contents that are split across buffers if we are going to return this column
     if (col_start != byte_buffer_ptr_ && delimited_text_parser_->ReturnCurrentColumn()) {
       DCHECK_EQ(byte_buffer_ptr_, byte_buffer_end_);
-      boundary_column_.Append(col_start, byte_buffer_ptr_ - col_start);
+      RETURN_IF_ERROR(boundary_column_.Append(col_start, byte_buffer_ptr_ - col_start));
       char* last_row = NULL;
       if (*num_tuples == 0) {
         last_row = batch_start_ptr_;
       } else {
         last_row = row_end_locations_[*num_tuples - 1] + 1;
       }
-      boundary_row_.Append(last_row, byte_buffer_ptr_ - last_row);
+      RETURN_IF_ERROR(boundary_row_.Append(last_row, byte_buffer_ptr_ - last_row));
     }
     COUNTER_ADD(scan_node_->rows_read_counter(), *num_tuples);
 
@@ -718,6 +719,10 @@ int HdfsTextScanner::WriteFields(MemPool* pool, TupleRow* tuple_row,
     int tuples_returned = 0;
     // Call jitted function if possible
     if (write_tuples_fn_ != NULL) {
+      // HdfsScanner::InitializeWriteTuplesFn() will skip codegen if there are string
+      // slots and escape characters. TextConverter::WriteSlot() will be used instead.
+      DCHECK(scan_node_->tuple_desc()->string_slots().empty() ||
+          delimited_text_parser_->escape_char() == '\0');
       tuples_returned = write_tuples_fn_(this, pool, tuple_row,
           batch_->row_byte_size(), fields, num_tuples, max_added_tuples,
           scan_node_->materialized_slots().size(), num_tuples_processed);
@@ -751,15 +756,21 @@ int HdfsTextScanner::WriteFields(MemPool* pool, TupleRow* tuple_row,
   return num_tuples_materialized;
 }
 
-void HdfsTextScanner::CopyBoundaryField(FieldLocation* data, MemPool* pool) {
+Status HdfsTextScanner::CopyBoundaryField(FieldLocation* data, MemPool* pool) {
   bool needs_escape = data->len < 0;
   int copy_len = needs_escape ? -data->len : data->len;
   int total_len = copy_len + boundary_column_.Size();
-  char* str_data = reinterpret_cast<char*>(pool->Allocate(total_len));
+  char* str_data = reinterpret_cast<char*>(pool->TryAllocate(total_len));
+  if (UNLIKELY(str_data == NULL)) {
+    string details = Substitute("HdfsTextScanner::CopyBoundaryField() failed to allocate "
+        "$0 bytes.", total_len);
+    return pool->mem_tracker()->MemLimitExceeded(state_, details, total_len);
+  }
   memcpy(str_data, boundary_column_.str().ptr, boundary_column_.Size());
   memcpy(str_data + boundary_column_.Size(), data->start, copy_len);
   data->start = str_data;
   data->len = needs_escape ? -total_len : total_len;
+  return Status::OK();
 }
 
 int HdfsTextScanner::WritePartialTuple(FieldLocation* fields,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/hdfs-text-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-text-scanner.h b/be/src/exec/hdfs-text-scanner.h
index 50758ae..804c9c3 100644
--- a/be/src/exec/hdfs-text-scanner.h
+++ b/be/src/exec/hdfs-text-scanner.h
@@ -116,11 +116,12 @@ class HdfsTextScanner : public HdfsScanner {
       int64_t* decompressed_len, bool *eosr);
 
   /// Prepends field data that was from the previous file buffer (This field straddled two
-  /// file buffers).  'data' already contains the pointer/len from the current file buffer,
-  /// boundary_column_ contains the beginning of the data from the previous file
-  /// buffer. This function will allocate a new string from the tuple pool, concatenate the
-  /// two pieces and update 'data' to contain the new pointer/len.
-  void CopyBoundaryField(FieldLocation* data, MemPool* pool);
+  /// file buffers). 'data' already contains the pointer/len from the current file buffer,
+  /// boundary_column_ contains the beginning of the data from the previous file buffer.
+  /// This function will allocate a new string from the tuple pool, concatenate the
+  /// two pieces and update 'data' to contain the new pointer/len. Return error status if
+  /// memory limit is exceeded when allocating a new string.
+  Status CopyBoundaryField(FieldLocation* data, MemPool* pool);
 
   /// Writes the intermediate parsed data into slots, outputting
   /// tuples to row_batch as they complete.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/scanner-context.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.cc b/be/src/exec/scanner-context.cc
index 040362b..897f8c5 100644
--- a/be/src/exec/scanner-context.cc
+++ b/be/src/exec/scanner-context.cc
@@ -118,7 +118,7 @@ void ScannerContext::Stream::ReleaseCompletedResources(RowBatch* batch, bool don
 }
 
 Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
-  if (parent_->cancelled()) return Status::CANCELLED;
+  if (UNLIKELY(parent_->cancelled())) return Status::CANCELLED;
 
   // io_buffer_ should only be null the first time this is called
   DCHECK(io_buffer_ != NULL ||
@@ -178,7 +178,7 @@ Status ScannerContext::Stream::GetBuffer(bool peek, uint8_t** out_buffer, int64_
   *len = 0;
   if (eosr()) return Status::OK();
 
-  if (parent_->cancelled()) {
+  if (UNLIKELY(parent_->cancelled())) {
     DCHECK(*out_buffer == NULL);
     return Status::CANCELLED;
   }
@@ -243,11 +243,11 @@ Status ScannerContext::Stream::GetBytesInternal(int64_t requested_len,
   while (requested_len > boundary_buffer_bytes_left_ + io_buffer_bytes_left_) {
     // We need to fetch more bytes. Copy the end of the current buffer and fetch the next
     // one.
-    boundary_buffer_->Append(io_buffer_pos_, io_buffer_bytes_left_);
+    RETURN_IF_ERROR(boundary_buffer_->Append(io_buffer_pos_, io_buffer_bytes_left_));
     boundary_buffer_bytes_left_ += io_buffer_bytes_left_;
 
     RETURN_IF_ERROR(GetNextBuffer());
-    if (parent_->cancelled()) return Status::CANCELLED;
+    if (UNLIKELY(parent_->cancelled())) return Status::CANCELLED;
 
     if (io_buffer_bytes_left_ == 0) {
       // No more bytes (i.e. EOF)
@@ -267,7 +267,7 @@ Status ScannerContext::Stream::GetBytesInternal(int64_t requested_len,
     output_buffer_pos_ = &io_buffer_pos_;
     output_buffer_bytes_left_ = &io_buffer_bytes_left_;
   } else {
-    boundary_buffer_->Append(io_buffer_pos_, num_bytes);
+    RETURN_IF_ERROR(boundary_buffer_->Append(io_buffer_pos_, num_bytes));
     boundary_buffer_bytes_left_ += num_bytes;
     boundary_buffer_pos_ = reinterpret_cast<uint8_t*>(boundary_buffer_->str().ptr) +
                            boundary_buffer_->Size() - boundary_buffer_bytes_left_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/text-converter.h
----------------------------------------------------------------------
diff --git a/be/src/exec/text-converter.h b/be/src/exec/text-converter.h
index 266e74d..3651146 100644
--- a/be/src/exec/text-converter.h
+++ b/be/src/exec/text-converter.h
@@ -49,7 +49,7 @@ class TextConverter {
   /// and writes the result into the tuples's slot.
   /// copy_string indicates whether we need to make a separate copy of the string data:
   /// For regular unescaped strings, we point to the original data in the file_buf_.
-  /// For regular escaped strings, we copy an its unescaped string into a separate buffer
+  /// For regular escaped strings, we copy its unescaped string into a separate buffer
   /// and point to it.
   /// If the string needs to be copied, the memory is allocated from 'pool', otherwise
   /// 'pool' is unused.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/exec/text-converter.inline.h
----------------------------------------------------------------------
diff --git a/be/src/exec/text-converter.inline.h b/be/src/exec/text-converter.inline.h
index 7b429fa..d50b66f 100644
--- a/be/src/exec/text-converter.inline.h
+++ b/be/src/exec/text-converter.inline.h
@@ -33,7 +33,7 @@
 namespace impala {
 
 /// Note: this function has a codegen'd version.  Changing this function requires
-/// corresponding changes to CodegenWriteSlot.
+/// corresponding changes to CodegenWriteSlot().
 inline bool TextConverter::WriteSlot(const SlotDescriptor* slot_desc, Tuple* tuple,
     const char* data, int len, bool copy_string, bool need_escape, MemPool* pool) {
   if ((len == 0 && !slot_desc->type().IsStringType()) || data == NULL) {
@@ -59,7 +59,7 @@ inline bool TextConverter::WriteSlot(const SlotDescriptor* slot_desc, Tuple* tup
       if (type.type == TYPE_VARCHAR || type.type == TYPE_CHAR) buffer_len = type.len;
 
       bool reuse_data = type.IsVarLenStringType() &&
-                        !(len != 0 && (copy_string || need_escape));
+          !(len != 0 && (copy_string || need_escape));
       if (type.type == TYPE_CHAR) reuse_data &= (buffer_len <= len);
 
       StringValue str;
@@ -67,9 +67,18 @@ inline bool TextConverter::WriteSlot(const SlotDescriptor* slot_desc, Tuple* tup
       if (reuse_data) {
         str.ptr = const_cast<char*>(data);
       } else {
+        // The codegen version of this code (generated by CodegenWriteSlot()) doesn't
+        // include this path. In other words, 'reuse_data' will always be true in the
+        // codegen version:
+        // 1. CodegenWriteSlot() doesn't yet support slot of TYPE_CHAR
+        // 2. HdfsScanner::InitializeWriteTuplesFn() will not codegen if there is
+        //    any escape character.
+        // 3. HdfsScanner::WriteCompleteTuple() always calls this function with
+        //    'copy_string' == false.
         str.ptr = type.IsVarLenStringType() ?
-            reinterpret_cast<char*>(pool->Allocate(buffer_len)) :
+            reinterpret_cast<char*>(pool->TryAllocate(buffer_len)) :
             reinterpret_cast<char*>(slot);
+        if (UNLIKELY(str.ptr == NULL)) return false;
         if (need_escape) {
           UnescapeString(data, str.ptr, &str.len, buffer_len);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/70502942/be/src/runtime/string-buffer.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/string-buffer.h b/be/src/runtime/string-buffer.h
index 1787b12..c8f79df 100644
--- a/be/src/runtime/string-buffer.h
+++ b/be/src/runtime/string-buffer.h
@@ -16,9 +16,12 @@
 #ifndef IMPALA_RUNTIME_STRING_BUFFER_H
 #define IMPALA_RUNTIME_STRING_BUFFER_H
 
+#include "common/status.h"
 #include "runtime/mem-pool.h"
 #include "runtime/string-value.h"
 
+using namespace strings;
+
 namespace impala {
 
 /// Dynamic-sizable string (similar to std::string) but without as many
@@ -30,7 +33,7 @@ namespace impala {
 class StringBuffer {
  public:
   /// C'tor for StringBuffer.  Memory backing the string will be allocated from
-  /// the pool as necessary.  Can optionally be initialized from a StringValue.
+  /// the pool as necessary. Can optionally be initialized from a StringValue.
   StringBuffer(MemPool* pool, StringValue* str = NULL)
       : pool_(pool), buffer_size_(0) {
     DCHECK(pool_ != NULL);
@@ -41,24 +44,24 @@ class StringBuffer {
   }
 
   /// Append 'str' to the current string, allocating a new buffer as necessary.
-  void Append(const char* str, int len) {
+  /// Return error status if memory limit is exceeded.
+  Status Append(const char* str, int len) {
     int new_len = len + string_value_.len;
-    if (new_len > buffer_size_) {
-      GrowBuffer(new_len);
-    }
+    if (new_len > buffer_size_) RETURN_IF_ERROR(GrowBuffer(new_len));
     memcpy(string_value_.ptr + string_value_.len, str, len);
     string_value_.len = new_len;
+    return Status::OK();
   }
 
   /// TODO: switch everything to uint8_t?
-  void Append(const uint8_t* str, int len) {
-    Append(reinterpret_cast<const char*>(str), len);
+  Status Append(const uint8_t* str, int len) {
+    return Append(reinterpret_cast<const char*>(str), len);
   }
 
-  /// Assigns contents to StringBuffer
-  void Assign(const char* str, int len) {
+  /// Assigns contents to StringBuffer. Return error status if memory limit is exceeded.
+  Status Assign(const char* str, int len) {
     Clear();
-    Append(str, len);
+    return Append(str, len);
   }
 
   /// Clear the underlying StringValue.  The allocated buffer can be reused.
@@ -94,16 +97,23 @@ class StringBuffer {
 
  private:
   /// Grows the buffer backing the string to be at least new_size, copying over the
-  /// previous string data into the new buffer.
-  void GrowBuffer(int new_len) {
+  /// previous string data into the new buffer. Return error status if memory limit
+  /// is exceeded.
+  Status GrowBuffer(int new_len) {
     // TODO: Release/reuse old buffers somehow
     buffer_size_ = std::max(buffer_size_ * 2, new_len);
     DCHECK_LE(buffer_size_, StringValue::MAX_LENGTH);
-    char* new_buffer = reinterpret_cast<char*>(pool_->Allocate(buffer_size_));
-    if (string_value_.len > 0) {
+    char* new_buffer = reinterpret_cast<char*>(pool_->TryAllocate(buffer_size_));
+    if (UNLIKELY(new_buffer == NULL)) {
+      string details = Substitute("StringBuffer failed to grow buffer by $0 bytes.",
+          buffer_size_);
+      return pool_->mem_tracker()->MemLimitExceeded(NULL, details, buffer_size_);
+    }
+    if (LIKELY(string_value_.len > 0)) {
       memcpy(new_buffer, string_value_.ptr, string_value_.len);
     }
     string_value_.ptr = new_buffer;
+    return Status::OK();
   }
 
   MemPool* pool_;


[06/50] incubator-impala git commit: IMPALA-3141: Send dummy filters when filter production is disabled

Posted by ta...@apache.org.
IMPALA-3141: Send dummy filters when filter production is disabled

The PHJ may disable runtime filter production for one of several
reasons, including a predicted high false-positive rate. If the filters
are not produced, any scans will wait for their entire timeout before
continuing.

This patch changes the filter logic to always send a filter, even if one
wasn't actually produced by the PHJ. To preserve correctness, that
filter must contain every element of the set. Such a filter is
represented by (BloomFilter*)NULL. This allows us to make no changes to
RuntimeFilter::Eval(), which already returns true if the member Bloom
filter is NULL.

In RPCs, a new field is added to TBloomFilter to identify filters that
are always true.

The HdfsParquetScanner checks to see if filters would always return true
for any element, and disables them if so.

There is some miscellaneous cleanup in this patch, particularly the
removal of unused members in BloomFilter.

This patch has been manually tested on queries that would otherwise take
a long time to time-out. A unit test was added to ensure that queries do
not wait.

Change-Id: I04b3e6542651c1e7b77a9bab01d0e3d9506af42f
Reviewed-on: http://gerrit.cloudera.org:8080/2475
Tested-by: Internal Jenkins
Reviewed-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/0d1eab7a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/0d1eab7a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/0d1eab7a

Branch: refs/heads/master
Commit: 0d1eab7a9ebc7e4653f9043ef90672c6f3088c3b
Parents: c06912e
Author: Henry Robinson <he...@cloudera.com>
Authored: Fri Mar 4 16:08:58 2016 -0800
Committer: Henry Robinson <he...@cloudera.com>
Committed: Thu Mar 24 23:17:50 2016 +0000

----------------------------------------------------------------------
 be/src/benchmarks/bloom-filter-benchmark.cc     |  7 +-
 be/src/exec/blocking-join-node.cc               |  2 +-
 be/src/exec/blocking-join-node.h                |  8 +-
 be/src/exec/hash-join-node.cc                   |  4 +-
 be/src/exec/hdfs-parquet-scanner.cc             |  7 +-
 be/src/exec/hdfs-scan-node.cc                   |  2 +-
 be/src/exec/partitioned-hash-join-node.cc       | 43 +++++-----
 be/src/exec/partitioned-hash-join-node.h        |  2 +-
 be/src/runtime/coordinator.cc                   | 39 +++++----
 be/src/runtime/runtime-filter.cc                | 86 +++++++++-----------
 be/src/runtime/runtime-filter.h                 | 22 +++--
 be/src/runtime/runtime-filter.inline.h          |  6 +-
 be/src/util/bloom-filter-test.cc                | 24 +++---
 be/src/util/bloom-filter.cc                     | 37 ++++-----
 be/src/util/bloom-filter.h                      | 30 +++----
 be/src/util/cpu-info.cc                         | 31 +++++--
 be/src/util/cpu-info.h                          | 15 +++-
 common/thrift/ImpalaInternalService.thrift      |  4 +
 common/thrift/PlanNodes.thrift                  |  4 -
 .../cloudera/impala/planner/HashJoinNode.java   |  7 --
 .../cloudera/impala/planner/PlanFragment.java   | 70 +---------------
 .../queries/QueryTest/runtime_filters_wait.test | 22 +++++
 22 files changed, 226 insertions(+), 246 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/benchmarks/bloom-filter-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/bloom-filter-benchmark.cc b/be/src/benchmarks/bloom-filter-benchmark.cc
index e574dbe..7374ed4 100644
--- a/be/src/benchmarks/bloom-filter-benchmark.cc
+++ b/be/src/benchmarks/bloom-filter-benchmark.cc
@@ -99,7 +99,7 @@ namespace initialize {
 void Benchmark(int batch_size, void* data) {
   int * d = reinterpret_cast<int*>(data);
   for (int i = 0; i < batch_size; ++i) {
-    BloomFilter bf(*d, nullptr, nullptr);
+    BloomFilter bf(*d);
   }
 }
 
@@ -109,8 +109,7 @@ void Benchmark(int batch_size, void* data) {
 namespace insert {
 
 struct TestData {
-  explicit TestData(int log_heap_size)
-      : bf(log_heap_size, nullptr, nullptr), data(1ull << 20) {
+  explicit TestData(int log_heap_size) : bf(log_heap_size), data(1ull << 20) {
     for (size_t i = 0; i < data.size(); ++i) {
       data[i] = MakeRand();
     }
@@ -137,7 +136,7 @@ namespace find {
 
 struct TestData {
   TestData(int log_heap_size, size_t size)
-      : bf(log_heap_size, nullptr, nullptr),
+      : bf(log_heap_size),
         vec_mask((1ull << static_cast<int>(floor(log2(size))))-1),
         present(size),
         absent(size),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 c07a856..952c608 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -42,7 +42,7 @@ BlockingJoinNode::BlockingJoinNode(const string& node_name, const TJoinOp::type
     probe_batch_pos_(-1),
     current_probe_row_(NULL),
     semi_join_staging_row_(NULL),
-    can_add_runtime_filters_(false) {
+    runtime_filters_enabled_(false) {
 }
 
 Status BlockingJoinNode::Init(const TPlanNode& tnode, RuntimeState* state) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 6f18d2b..cbd3b1e 100644
--- a/be/src/exec/blocking-join-node.h
+++ b/be/src/exec/blocking-join-node.h
@@ -93,8 +93,12 @@ class BlockingJoinNode : public ExecNode {
 
   /// If true, this node can build filters from the build side that can be used elsewhere
   /// in the plan to eliminate rows early.
-  /// Note that we disable probe filters if we are inside a subplan.
-  bool can_add_runtime_filters_;
+  /// Filters might be disabled during execution in several cases, including if we are
+  /// inside a subplan or the false-positive rate would be too high.
+  /// TODO: Consider moving into FilterContext, which will allow us to track enabled state
+  /// per-filter, and also alows us to move this state into only HJ nodes that support
+  /// filter production.
+  bool runtime_filters_enabled_;
 
   RuntimeProfile::Counter* build_timer_;   // time to prepare build side
   RuntimeProfile::Counter* probe_timer_;   // time to process the probe (left child) batch

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 2193c9e..7f8f410 100644
--- a/be/src/exec/hash-join-node.cc
+++ b/be/src/exec/hash-join-node.cc
@@ -58,7 +58,7 @@ HashJoinNode::HashJoinNode(
   match_one_build_ = (join_op_ == TJoinOp::LEFT_SEMI_JOIN);
   match_all_build_ =
     (join_op_ == TJoinOp::RIGHT_OUTER_JOIN || join_op_ == TJoinOp::FULL_OUTER_JOIN);
-  can_add_runtime_filters_ = FLAGS_enable_probe_side_filtering;
+  runtime_filters_enabled_ = FLAGS_enable_probe_side_filtering;
 }
 
 Status HashJoinNode::Init(const TPlanNode& tnode, RuntimeState* state) {
@@ -233,7 +233,7 @@ Status HashJoinNode::ConstructBuildSide(RuntimeState* state) {
   // We only do this if the build side is sufficiently small.
   // TODO: Better heuristic? Currently we simply compare the size of the HT with a
   // constant value.
-  if (can_add_runtime_filters_) {
+  if (runtime_filters_enabled_) {
     if (!state->filter_bank()->ShouldDisableFilter(hash_tbl_->size())) {
       AddRuntimeExecOption("Build-Side Filter Built");
       hash_tbl_->AddBloomFilters();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 5aa67fa..5adc97f 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -877,7 +877,7 @@ Status HdfsParquetScanner::Prepare(ScannerContext* context) {
   for (int i = 0; i < context->filter_ctxs().size(); ++i) {
     const FilterContext* ctx = &context->filter_ctxs()[i];
     DCHECK(ctx->filter != NULL);
-    filter_ctxs_.push_back(ctx);
+    if (!ctx->filter->AlwaysTrue()) filter_ctxs_.push_back(ctx);
   }
   filter_stats_.resize(filter_ctxs_.size());
   return Status::OK();
@@ -1748,18 +1748,19 @@ inline bool HdfsParquetScanner::ReadRow(const vector<ColumnReader*>& column_read
     for (int i = 0; i < num_filters; ++i) {
       LocalFilterStats* stats = &filter_stats_[i];
       if (!stats->enabled) continue;
+      const RuntimeFilter* filter = filter_ctxs_[i]->filter;
       ++stats->total_possible;
       // Check filter effectiveness every ROWS_PER_FILTER_SELECTIVITY_CHECK rows.
       if (UNLIKELY(
           !(stats->total_possible & (ROWS_PER_FILTER_SELECTIVITY_CHECK - 1)))) {
         double reject_ratio = stats->rejected / static_cast<double>(stats->considered);
-        if (reject_ratio < FLAGS_parquet_min_filter_reject_ratio) {
+        if (filter->AlwaysTrue() ||
+            reject_ratio < FLAGS_parquet_min_filter_reject_ratio) {
           stats->enabled = 0;
           continue;
         }
       }
       ++stats->considered;
-      const RuntimeFilter* filter = filter_ctxs_[i]->filter;
       void* e = filter_ctxs_[i]->expr->GetValue(tuple_row_mem);
       if (!filter->Eval<void>(e, filter_ctxs_[i]->expr->root()->type())) {
         ++stats->rejected;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 c43bdcb..5947ac1 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -1136,7 +1136,7 @@ bool HdfsScanNode::PartitionPassesFilterPredicates(int32_t partition_id,
 
     // Not quite right because bitmap could arrive after Eval(), but we're ok with
     // off-by-one errors.
-    bool processed = ctx.filter->GetBloomFilter();
+    bool processed = ctx.filter->HasBloomFilter();
     bool passed_filter = ctx.filter->Eval<void>(e, ctx.expr->root()->type());
     ctx.stats->IncrCounters(stats_name, 1, processed, !passed_filter);
     if (!passed_filter)  return false;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 b796acf..151b0c6 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -66,7 +66,7 @@ PartitionedHashJoinNode::PartitionedHashJoinNode(
     process_probe_batch_fn_level0_(NULL) {
   memset(hash_tbls_, 0, sizeof(HashTable*) * PARTITION_FANOUT);
 
-  can_add_runtime_filters_ =
+  runtime_filters_enabled_ =
       FLAGS_enable_phj_probe_side_filtering && tnode.runtime_filters.size() > 0;
 }
 
@@ -128,7 +128,7 @@ Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
 
   // Disable probe-side filters if we are inside a subplan because no node
   // inside the subplan can use them.
-  if (IsInSubplan()) can_add_runtime_filters_ = false;
+  if (IsInSubplan()) runtime_filters_enabled_ = false;
 
   RETURN_IF_ERROR(BlockingJoinNode::Prepare(state));
   runtime_state_ = state;
@@ -400,10 +400,10 @@ Status PartitionedHashJoinNode::Partition::Spill(bool unpin_all_build) {
     }
   }
 
-  if (parent_->can_add_runtime_filters_) {
+  if (parent_->runtime_filters_enabled_) {
     // Disabling runtime filter push down because not all rows will be included in the
     // filter due to a spilled partition.
-    parent_->can_add_runtime_filters_ = false;
+    parent_->runtime_filters_enabled_ = false;
     parent_->AddRuntimeExecOption("Build-Side Runtime-Filter Disabled (Spilling)");
     VLOG(2) << "Disabling runtime filter construction because a partition will spill.";
   }
@@ -505,7 +505,7 @@ not_built:
 }
 
 bool PartitionedHashJoinNode::AllocateRuntimeFilters(RuntimeState* state) {
-  if (!can_add_runtime_filters_) return false;
+  if (!runtime_filters_enabled_) return false;
   DCHECK(ht_ctx_.get() != NULL);
   for (int i = 0; i < filters_.size(); ++i) {
     filters_[i].local_bloom_filter = state->filter_bank()->AllocateScratchBloomFilter();
@@ -513,19 +513,20 @@ bool PartitionedHashJoinNode::AllocateRuntimeFilters(RuntimeState* state) {
   return true;
 }
 
-bool PartitionedHashJoinNode::PublishRuntimeFilters(RuntimeState* state) {
-  if (can_add_runtime_filters_) {
-    // Add all the bloom filters to the runtime state.
-    // TODO: DCHECK(!is_in_subplan());
+void PartitionedHashJoinNode::PublishRuntimeFilters(RuntimeState* state) {
+  if (runtime_filters_enabled_) {
     AddRuntimeExecOption("Build-Side Runtime-Filter Produced");
-    BOOST_FOREACH(const FilterContext& ctx, filters_) {
-      if (ctx.local_bloom_filter == NULL) continue;
-      state->filter_bank()->UpdateFilterFromLocal(ctx.filter->filter_desc().filter_id,
-          ctx.local_bloom_filter);
-    }
-    return true;
   }
-  return false;
+
+  // Add all the bloom filters to the runtime state. If runtime filters are disabled,
+  // publish a complete Bloom filter (which rejects no values) to allow plan nodes that
+  // are waiting for these filters to make progress.
+  BOOST_FOREACH(const FilterContext& ctx, filters_) {
+    BloomFilter* filter = runtime_filters_enabled_ ?
+        ctx.local_bloom_filter : BloomFilter::ALWAYS_TRUE_FILTER;
+    state->filter_bank()->UpdateFilterFromLocal(
+        ctx.filter->filter_desc().filter_id, filter);
+  }
 }
 
 bool PartitionedHashJoinNode::AppendRowStreamFull(BufferedTupleStream* stream,
@@ -1195,7 +1196,7 @@ Status PartitionedHashJoinNode::BuildHashTables(RuntimeState* state) {
   DCHECK_EQ(hash_partitions_.size(), PARTITION_FANOUT);
 
   // Decide whether probe filters will be built.
-  if (input_partition_ == NULL && can_add_runtime_filters_) {
+  if (input_partition_ == NULL && runtime_filters_enabled_) {
     uint64_t num_build_rows = 0;
     BOOST_FOREACH(Partition* partition, hash_partitions_) {
       DCHECK(!partition->is_spilled()) << "Runtime filters enabled despite spilling";
@@ -1209,10 +1210,10 @@ Status PartitionedHashJoinNode::BuildHashTables(RuntimeState* state) {
     // TODO: Better heuristic.
     if (state->filter_bank()->ShouldDisableFilter(num_build_rows)) {
       AddRuntimeExecOption("Build-Side Runtime-Filter Disabled (FP Rate Too High)");
-      can_add_runtime_filters_ = false;
+      runtime_filters_enabled_ = false;
     }
   } else {
-    can_add_runtime_filters_ = false;
+    runtime_filters_enabled_ = false;
   }
 
   // First loop over the partitions and build hash tables for the partitions that did
@@ -1227,13 +1228,13 @@ Status PartitionedHashJoinNode::BuildHashTables(RuntimeState* state) {
     if (!partition->is_spilled()) {
       bool built = false;
       DCHECK(partition->build_rows()->is_pinned());
-      RETURN_IF_ERROR(partition->BuildHashTable(state, &built, can_add_runtime_filters_));
+      RETURN_IF_ERROR(partition->BuildHashTable(state, &built, runtime_filters_enabled_));
       // If we did not have enough memory to build this hash table, we need to spill this
       // partition (clean up the hash table, unpin build).
       if (!built) RETURN_IF_ERROR(partition->Spill(true));
     }
 
-    DCHECK(!can_add_runtime_filters_ || !partition->is_spilled())
+    DCHECK(!runtime_filters_enabled_ || !partition->is_spilled())
         << "Runtime filters enabled despite spilling";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/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 e03f05d..119dd61 100644
--- a/be/src/exec/partitioned-hash-join-node.h
+++ b/be/src/exec/partitioned-hash-join-node.h
@@ -257,7 +257,7 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   bool AllocateRuntimeFilters(RuntimeState* state);
 
   /// Publish the runtime filters to the fragment-local RuntimeFilterBank.
-  bool PublishRuntimeFilters(RuntimeState* state);
+  void PublishRuntimeFilters(RuntimeState* state);
 
   /// Codegen function to create output row. Assumes that the probe row is non-NULL.
   Status CodegenCreateOutputRow(LlvmCodeGen* codegen, llvm::Function** fn);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index b970333..bb49166 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -638,8 +638,9 @@ string Coordinator::FilterDebugString() {
     row.push_back(state.desc.is_bound_by_partition_columns ? "true" : "false");
 
     if (filter_mode_ == TRuntimeFilterMode::GLOBAL) {
-      row.push_back(Substitute("$0 ($1)", state.pending_count,
-              state.src_fragment_instance_idxs.size()));
+      int pending_count = state.completion_time != 0L ? 0 : state.pending_count;
+      row.push_back(Substitute("$0 ($1)", pending_count,
+          state.src_fragment_instance_idxs.size()));
       if (state.first_arrival_time == 0L) {
         row.push_back("N/A");
       } else {
@@ -1966,7 +1967,6 @@ void Coordinator::UpdateFilter(const TUpdateFilterParams& params) {
   // Make a 'master' copy that will be shared by all concurrent delivery RPC attempts.
   shared_ptr<TPublishFilterParams> rpc_params(new TPublishFilterParams());
   unordered_set<int32_t> target_fragment_instance_idxs;
-  unique_ptr<BloomFilter> bloom_filter(new BloomFilter(params.bloom_filter, NULL, NULL));
   {
     lock_guard<SpinLock> l(filter_lock_);
     FilterRoutingTable::iterator it = filter_routing_table_.find(params.filter_id);
@@ -1978,36 +1978,39 @@ void Coordinator::UpdateFilter(const TUpdateFilterParams& params) {
     DCHECK(!state->desc.has_local_target)
         << "Coordinator received filter that has local target";
 
-    // Receiving unnecessary updates for a broadcast.
-    if (state->pending_count == 0) {
-      DCHECK(state->desc.is_broadcast_join)
-          << "Received more updates than expected for partition filter: "
-          << params.filter_id;
-      return;
-    }
+    // Check if the filter has already been sent, which could happen in two cases: if one
+    // local filter had always_true set - no point waiting for other local filters that
+    // can't affect the aggregated global filter, or if this is a broadcast join, and
+    // another local filter was already received.
+    if (state->pending_count == 0) return;
+    DCHECK_EQ(state->completion_time, 0L);
     if (state->first_arrival_time == 0L) {
       state->first_arrival_time = query_events_->ElapsedTime();
     }
-    --state->pending_count;
 
     if (filter_updates_received_->value() == 0) {
       query_events_->MarkEvent("First dynamic filter received");
     }
     filter_updates_received_->Add(1);
-    if (state->bloom_filter == NULL) {
-      state->bloom_filter =
-          obj_pool()->Add(bloom_filter.release());
+    if (params.bloom_filter.always_true) {
+      state->bloom_filter = NULL;
+      state->pending_count = 0;
     } else {
-      // TODO: Implement BloomFilter::Or(const ThriftBloomFilter&)
-      state->bloom_filter->Or(*bloom_filter);
+      if (state->bloom_filter == NULL) {
+        state->bloom_filter = obj_pool()->Add(new BloomFilter(params.bloom_filter));
+      } else {
+        // TODO: Implement BloomFilter::Or(const ThriftBloomFilter&)
+        state->bloom_filter->Or(BloomFilter(params.bloom_filter));
+      }
+      if (--state->pending_count > 0) return;
     }
 
-    if (state->pending_count > 0) return;
     // No more filters are pending on this filter ID. Create a distribution payload and
     // offer it to the queue.
+    DCHECK_EQ(state->pending_count, 0);
     state->completion_time = query_events_->ElapsedTime();
     target_fragment_instance_idxs = state->target_fragment_instance_idxs;
-    state->bloom_filter->ToThrift(&rpc_params->bloom_filter);
+    BloomFilter::ToThrift(state->bloom_filter, &rpc_params->bloom_filter);
   }
 
   rpc_params->filter_id = params.filter_id;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/runtime/runtime-filter.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter.cc b/be/src/runtime/runtime-filter.cc
index fee8566..675e0f3 100644
--- a/be/src/runtime/runtime-filter.cc
+++ b/be/src/runtime/runtime-filter.cc
@@ -49,7 +49,7 @@ RuntimeFilterBank::RuntimeFilterBank(const TQueryCtx& query_ctx, RuntimeState* s
 RuntimeFilter* RuntimeFilterBank::RegisterFilter(const TRuntimeFilterDesc& filter_desc,
     bool is_producer) {
   RuntimeFilter* ret = obj_pool_.Add(new RuntimeFilter(filter_desc));
-  lock_guard<SpinLock> l(runtime_filter_lock_);
+  lock_guard<mutex> l(runtime_filter_lock_);
   if (is_producer) {
     DCHECK(produced_filters_.find(filter_desc.filter_id) == produced_filters_.end());
     produced_filters_[filter_desc.filter_id] = ret;
@@ -88,7 +88,7 @@ void RuntimeFilterBank::UpdateFilterFromLocal(uint32_t filter_id,
   TUpdateFilterParams params;
   bool has_local_target = false;
   {
-    lock_guard<SpinLock> l(runtime_filter_lock_);
+    lock_guard<mutex> l(runtime_filter_lock_);
     RuntimeFilterMap::iterator it = produced_filters_.find(filter_id);
     DCHECK(it != produced_filters_.end()) << "Tried to update unregistered filter: "
                                           << filter_id;
@@ -101,72 +101,65 @@ void RuntimeFilterBank::UpdateFilterFromLocal(uint32_t filter_id,
     // side.
     RuntimeFilter* filter;
     {
-      lock_guard<SpinLock> l(runtime_filter_lock_);
+      lock_guard<mutex> l(runtime_filter_lock_);
       RuntimeFilterMap::iterator it = consumed_filters_.find(filter_id);
       if (it == consumed_filters_.end()) return;
       filter = it->second;
       // Check if the filter already showed up.
-      if (filter->GetBloomFilter() != NULL) return;
-    }
-    // TODO: Avoid need for this copy.
-    BloomFilter* copy = AllocateScratchBloomFilter();
-    if (copy == NULL) return;
-    copy->Or(*bloom_filter);
-    {
-      // Take lock only to ensure no race with PublishGlobalFilter() - there's no need for
-      // coordination with readers of the filter.
-      lock_guard<SpinLock> l(runtime_filter_lock_);
-      if (filter->GetBloomFilter() == NULL) {
-        filter->SetBloomFilter(copy);
-        state_->runtime_profile()->AddInfoString(
-            Substitute("Filter $0 arrival", filter_id),
-            PrettyPrinter::Print(filter->arrival_delay(), TUnit::TIME_MS));
-      }
+      DCHECK(!filter->HasBloomFilter());
     }
+    filter->SetBloomFilter(bloom_filter);
+    state_->runtime_profile()->AddInfoString(
+        Substitute("Filter $0 arrival", filter_id),
+        PrettyPrinter::Print(filter->arrival_delay(), TUnit::TIME_MS));
   } else if (state_->query_options().runtime_filter_mode == TRuntimeFilterMode::GLOBAL) {
-      bloom_filter->ToThrift(&params.bloom_filter);
-      params.filter_id = filter_id;
-      params.query_id = query_ctx_.query_id;
+    BloomFilter::ToThrift(bloom_filter, &params.bloom_filter);
+    params.filter_id = filter_id;
+    params.query_id = query_ctx_.query_id;
 
-      ExecEnv::GetInstance()->rpc_pool()->Offer(bind<void>(
-          SendFilterToCoordinator, query_ctx_.coord_address, params,
-          ExecEnv::GetInstance()->impalad_client_cache()));
+    ExecEnv::GetInstance()->rpc_pool()->Offer(bind<void>(
+        SendFilterToCoordinator, query_ctx_.coord_address, params,
+        ExecEnv::GetInstance()->impalad_client_cache()));
   }
 }
 
 void RuntimeFilterBank::PublishGlobalFilter(uint32_t filter_id,
     const TBloomFilter& thrift_filter) {
-  lock_guard<SpinLock> l(runtime_filter_lock_);
+  lock_guard<mutex> l(runtime_filter_lock_);
   if (closed_) return;
   RuntimeFilterMap::iterator it = consumed_filters_.find(filter_id);
   DCHECK(it != consumed_filters_.end()) << "Tried to publish unregistered filter: "
                                         << filter_id;
-  if (it->second->filter_desc().is_broadcast_join &&
-      it->second->GetBloomFilter() != NULL) {
-    // Already showed up from local filter.
-    return;
+  if (thrift_filter.always_true) {
+    it->second->SetBloomFilter(BloomFilter::ALWAYS_TRUE_FILTER);
+  } else {
+    uint32_t required_space =
+        BloomFilter::GetExpectedHeapSpaceUsed(thrift_filter.log_heap_space);
+    // Silently fail to publish the filter (replacing it with a 0-byte complete one) if
+    // there's not enough memory for it.
+    if (!state_->query_mem_tracker()->TryConsume(required_space)) {
+      VLOG_QUERY << "No memory for global filter: " << filter_id
+                 << " (fragment instance: " << state_->fragment_instance_id() << ")";
+      it->second->SetBloomFilter(BloomFilter::ALWAYS_TRUE_FILTER);
+    } else {
+      BloomFilter* bloom_filter = obj_pool_.Add(new BloomFilter(thrift_filter));
+      DCHECK_EQ(required_space, bloom_filter->GetHeapSpaceUsed());
+      memory_allocated_->Add(bloom_filter->GetHeapSpaceUsed());
+      it->second->SetBloomFilter(bloom_filter);
+    }
   }
-  uint32_t required_space =
-      BloomFilter::GetExpectedHeapSpaceUsed(thrift_filter.log_heap_space);
-  // Silently fail to publish the filter if there's not enough memory for it.
-  if (!state_->query_mem_tracker()->TryConsume(required_space)) return;
-  BloomFilter* bloom_filter = obj_pool_.Add(new BloomFilter(thrift_filter, NULL, NULL));
-  DCHECK_EQ(required_space, bloom_filter->GetHeapSpaceUsed());
-  memory_allocated_->Add(bloom_filter->GetHeapSpaceUsed());
-  it->second->SetBloomFilter(bloom_filter);
   state_->runtime_profile()->AddInfoString(Substitute("Filter $0 arrival", filter_id),
       PrettyPrinter::Print(it->second->arrival_delay(), TUnit::TIME_MS));
 }
 
 BloomFilter* RuntimeFilterBank::AllocateScratchBloomFilter() {
-  lock_guard<SpinLock> l(runtime_filter_lock_);
+  lock_guard<mutex> l(runtime_filter_lock_);
   if (closed_) return NULL;
 
   // Track required space
   uint32_t required_space = BloomFilter::GetExpectedHeapSpaceUsed(log_filter_size_);
   if (!state_->query_mem_tracker()->TryConsume(required_space)) return NULL;
-  BloomFilter* bloom_filter =
-      obj_pool_.Add(new BloomFilter(log_filter_size_, NULL, NULL));
+  BloomFilter* bloom_filter = obj_pool_.Add(new BloomFilter(log_filter_size_));
   DCHECK_EQ(required_space, bloom_filter->GetHeapSpaceUsed());
   memory_allocated_->Add(bloom_filter->GetHeapSpaceUsed());
   return bloom_filter;
@@ -178,18 +171,17 @@ bool RuntimeFilterBank::ShouldDisableFilter(uint64_t max_ndv) {
 }
 
 void RuntimeFilterBank::Close() {
-  lock_guard<SpinLock> l(runtime_filter_lock_);
+  lock_guard<mutex> l(runtime_filter_lock_);
   closed_ = true;
   obj_pool_.Clear();
   state_->query_mem_tracker()->Release(memory_allocated_->value());
 }
 
 bool RuntimeFilter::WaitForArrival(int32_t timeout_ms) const {
-  if (GetBloomFilter() != NULL) return true;
-  while ((MonotonicMillis() - registration_time_) < timeout_ms) {
+  do {
+    if (HasBloomFilter()) return true;
     SleepForMs(SLEEP_PERIOD_MS);
-    if (GetBloomFilter() != NULL) return true;
-  }
+  } while ((MonotonicMillis() - registration_time_) < timeout_ms);
 
-  return false;
+  return HasBloomFilter();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/runtime/runtime-filter.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter.h b/be/src/runtime/runtime-filter.h
index 34aa476..e41c97a 100644
--- a/be/src/runtime/runtime-filter.h
+++ b/be/src/runtime/runtime-filter.h
@@ -44,8 +44,8 @@ class RuntimeState;
 /// fragments update the bloom filters by calling UpdateFilterFromLocal()
 /// (UpdateFilterFromLocal() may only be called once per filter ID per filter bank). The
 /// bloom_filter that is passed into UpdateFilterFromLocal() must have been allocated by
-/// AllocateScratchBloomFilter(); this allows RuntimeFilterBank to manage all memory
-/// associated with filters.
+/// AllocateScratchBloomFilter() (or be NULL); this allows RuntimeFilterBank to manage all
+/// memory associated with filters.
 ///
 /// Filters are aggregated at the coordinator, and then made available to consumers after
 /// PublishGlobalFilter() has been called.
@@ -65,7 +65,8 @@ class RuntimeFilterBank {
   RuntimeFilter* RegisterFilter(const TRuntimeFilterDesc& filter_desc, bool is_producer);
 
   /// Updates a filter's bloom_filter with 'bloom_filter' which has been produced by some
-  /// operator in the local fragment instance.
+  /// operator in the local fragment instance. 'bloom_filter' may be NULL, representing a
+  /// full filter that contains all elements.
   void UpdateFilterFromLocal(uint32_t filter_id, BloomFilter* bloom_filter);
 
   /// Makes a bloom_filter (aggregated globally from all producer fragments) available for
@@ -104,7 +105,7 @@ class RuntimeFilterBank {
   const TQueryCtx query_ctx_;
 
   /// Lock protecting produced_filters_ and consumed_filters_.
-  SpinLock runtime_filter_lock_;
+  boost::mutex runtime_filter_lock_;
 
   /// Map from filter id to a RuntimeFilter.
   typedef boost::unordered_map<uint32_t, RuntimeFilter*> RuntimeFilterMap;
@@ -147,8 +148,8 @@ class RuntimeFilter {
     registration_time_ = MonotonicMillis();
   }
 
-  /// Returns NULL if no calls to SetBloomFilter() have been made yet.
-  const BloomFilter* GetBloomFilter() const { return bloom_filter_; }
+  /// Returns true if SetBloomFilter() has been called.
+  bool HasBloomFilter() const { return arrival_time_ != 0; }
 
   const TRuntimeFilterDesc& filter_desc() const { return filter_desc_; }
 
@@ -176,11 +177,16 @@ class RuntimeFilter {
   /// false otherwise.
   bool WaitForArrival(int32_t timeout_ms) const;
 
+  /// Returns true if the filter returns true for all elements, i.e. Eval(v) returns true
+  /// for all v.
+  inline bool AlwaysTrue() const;
+
   /// Frequency with which to check for filter arrival in WaitForArrival()
   static const int SLEEP_PERIOD_MS;
 
  private:
-  /// Membership bloom_filter.
+  /// Membership bloom_filter. May be NULL even after arrival_time_ is set. This is a
+  /// compact way of representing a full Bloom filter that contains every element.
   BloomFilter* bloom_filter_;
 
   /// Descriptor of the filter.
@@ -189,7 +195,7 @@ class RuntimeFilter {
   /// Time, in ms, that the filter was registered.
   int64_t registration_time_;
 
-  /// Time, in ms, that the global fiter arrived.
+  /// Time, in ms, that the global fiter arrived. Set in SetBloomFilter().
   int64_t arrival_time_;
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/runtime/runtime-filter.inline.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter.inline.h b/be/src/runtime/runtime-filter.inline.h
index 6cae673..f7141a8 100644
--- a/be/src/runtime/runtime-filter.inline.h
+++ b/be/src/runtime/runtime-filter.inline.h
@@ -27,7 +27,7 @@
 namespace impala {
 
 inline const RuntimeFilter* RuntimeFilterBank::GetRuntimeFilter(uint32_t filter_id) {
-  boost::lock_guard<SpinLock> l(runtime_filter_lock_);
+  boost::lock_guard<boost::mutex> l(runtime_filter_lock_);
   RuntimeFilterMap::iterator it = consumed_filters_.find(filter_id);
   if (it == consumed_filters_.end()) return NULL;
   return it->second;
@@ -53,6 +53,10 @@ inline bool RuntimeFilter::Eval(T* val, const ColumnType& col_type) const {
   return bloom_filter_->Find(h);
 }
 
+inline bool RuntimeFilter::AlwaysTrue() const  {
+  return HasBloomFilter() && bloom_filter_ == BloomFilter::ALWAYS_TRUE_FILTER;
+}
+
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/util/bloom-filter-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter-test.cc b/be/src/util/bloom-filter-test.cc
index e9dcca8..d8b5818 100644
--- a/be/src/util/bloom-filter-test.cc
+++ b/be/src/util/bloom-filter-test.cc
@@ -38,7 +38,7 @@ namespace impala {
 // We can construct (and destruct) Bloom filters with different spaces.
 TEST(BloomFilter, Constructor) {
   for (int i = 0; i < 30; ++i) {
-    BloomFilter bf(i, nullptr, nullptr);
+    BloomFilter bf(i);
   }
 }
 
@@ -46,7 +46,7 @@ TEST(BloomFilter, Constructor) {
 TEST(BloomFilter, Insert) {
   srand(0);
   for (int i = 13; i < 17; ++i) {
-    BloomFilter bf(i, nullptr, nullptr);
+    BloomFilter bf(i);
     for (int k = 0; k < (1 << 15); ++k) {
       bf.Insert(MakeRand());
     }
@@ -57,7 +57,7 @@ TEST(BloomFilter, Insert) {
 TEST(BloomFilter, Find) {
   srand(0);
   for (int i = 13; i < 17; ++i) {
-    BloomFilter bf(i, nullptr, nullptr);
+    BloomFilter bf(i);
     for (int k = 0; k < (1 << 15); ++k) {
       const uint64_t to_insert = MakeRand();
       bf.Insert(to_insert);
@@ -71,7 +71,7 @@ TEST(BloomFilter, CumulativeFind) {
   srand(0);
   for (int i = 5; i < 11; ++i) {
     std::vector<uint32_t> inserted;
-    BloomFilter bf(i, nullptr, nullptr);
+    BloomFilter bf(i);
     for (int k = 0; k < (1 << 10); ++k) {
       const uint32_t to_insert = MakeRand();
       inserted.push_back(to_insert);
@@ -104,7 +104,7 @@ TEST(BloomFilter, FindInvalid) {
       double fpp = 1.0 / (1 << log_fpp);
       const size_t ndv = 1 << log_ndv;
       const int log_heap_space = BloomFilter::MinLogSpace(ndv, fpp);
-      BloomFilter bf(log_heap_space, nullptr, nullptr);
+      BloomFilter bf(log_heap_space);
       // Fill up a BF with exactly as much ndv as we planned for it:
       for (size_t i = 0; i < ndv; ++i) {
         bf.Insert(shuffled_insert[i]);
@@ -187,7 +187,7 @@ TEST(BloomFilter, MinSpaceForFpp) {
 }
 
 TEST(BloomFilter, Thrift) {
-  BloomFilter bf(BloomFilter::MinLogSpace(100, 0.01), NULL, NULL);
+  BloomFilter bf(BloomFilter::MinLogSpace(100, 0.01));
   for (int i = 0; i < 10; ++i) bf.Insert(i);
   // Check no unexpected new false positives.
   set<int> missing_ints;
@@ -196,16 +196,20 @@ TEST(BloomFilter, Thrift) {
   }
 
   TBloomFilter to_thrift;
-  bf.ToThrift(&to_thrift);
+  BloomFilter::ToThrift(&bf, &to_thrift);
+  EXPECT_EQ(to_thrift.always_true, false);
 
-  BloomFilter from_thrift(to_thrift, NULL, NULL);
+  BloomFilter from_thrift(to_thrift);
   for (int i = 0; i < 10; ++i) ASSERT_TRUE(from_thrift.Find(i));
   for (int missing: missing_ints) ASSERT_FALSE(from_thrift.Find(missing));
+
+  BloomFilter::ToThrift(NULL, &to_thrift);
+  EXPECT_EQ(to_thrift.always_true, true);
 }
 
 TEST(BloomFilter, Or) {
-  BloomFilter bf1(BloomFilter::MinLogSpace(100, 0.01), NULL, NULL);
-  BloomFilter bf2(BloomFilter::MinLogSpace(100, 0.01), NULL, NULL);
+  BloomFilter bf1(BloomFilter::MinLogSpace(100, 0.01));
+  BloomFilter bf2(BloomFilter::MinLogSpace(100, 0.01));
   for (int i = 60; i < 80; ++i) bf2.Insert(i);
 
   for (int i = 0; i < 10; ++i) bf1.Insert(i);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/util/bloom-filter.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.cc b/be/src/util/bloom-filter.cc
index a58e0fa..8a37a76 100644
--- a/be/src/util/bloom-filter.cc
+++ b/be/src/util/bloom-filter.cc
@@ -26,31 +26,22 @@ using namespace std;
 
 namespace impala {
 
-BloomFilter::BloomFilter(const int log_heap_space, RuntimeState* state,
-    BufferedBlockMgr::Client* client)
-    :  // Since log_heap_space is in bytes, we need to convert it to cache lines. There
-       // are 64 = 2^6 bytes in a cache line.
+BloomFilter* BloomFilter::ALWAYS_TRUE_FILTER = NULL;
+
+BloomFilter::BloomFilter(const int log_heap_space)
+    : // Since log_heap_space is in bytes, we need to convert it to cache lines. There
+      // are 64 = 2^6 bytes in a cache line.
       log_num_buckets_(std::max(1, log_heap_space - LOG_BUCKET_WORD_BITS)),
       // Don't use log_num_buckets_ if it will lead to undefined behavior by a shift
       // that is too large.
       directory_mask_((1ull << std::min(63, log_num_buckets_)) - 1),
-      directory_(NULL),
-      state_(state),
-      client_(client) {
+      directory_(NULL) {
   // Since we use 32 bits in the arguments of Insert() and Find(), log_num_buckets_
   // must be limited.
   DCHECK(log_num_buckets_ <= 32)
       << "Bloom filter too large. log_heap_space: " << log_heap_space;
-  DCHECK_EQ(client_ == NULL, state_ == NULL);
   // Each bucket has 64 = 2^6 bytes:
   const size_t alloc_size = directory_size();
-  if (state_) {
-    const bool consume_success = state_->block_mgr()->ConsumeMemory(client_, alloc_size);
-    DCHECK(consume_success) << "ConsumeMemory failed. log_heap_space: "
-                            << log_heap_space
-                            << " log_num_buckets_: " << log_num_buckets_
-                            << " alloc_size: " << alloc_size;
-  }
   const int malloc_failed =
       posix_memalign(reinterpret_cast<void**>(&directory_), 64, alloc_size);
   DCHECK_EQ(malloc_failed, 0) << "Malloc failed. log_heap_space: " << log_heap_space
@@ -59,16 +50,14 @@ BloomFilter::BloomFilter(const int log_heap_space, RuntimeState* state,
   memset(directory_, 0, alloc_size);
 }
 
-BloomFilter::BloomFilter(const TBloomFilter& thrift, RuntimeState* state,
-    BufferedBlockMgr::Client* client)
-    : BloomFilter(thrift.log_heap_space, state, client) {
+BloomFilter::BloomFilter(const TBloomFilter& thrift)
+    : BloomFilter(thrift.log_heap_space) {
   DCHECK_EQ(thrift.directory.size(), directory_size());
   memcpy(directory_, &thrift.directory[0], thrift.directory.size());
 }
 
 BloomFilter::~BloomFilter() {
   if (directory_) {
-    if (state_) state_->block_mgr()->ReleaseMemory(client_, directory_size());
     free(directory_);
     directory_ = NULL;
   }
@@ -78,6 +67,16 @@ void BloomFilter::ToThrift(TBloomFilter* thrift) const {
   thrift->log_heap_space = log_num_buckets_ + LOG_BUCKET_BYTE_SIZE;
   string tmp(reinterpret_cast<const char*>(directory_), directory_size());
   thrift->directory.swap(tmp);
+  thrift->always_true = false;
+}
+
+void BloomFilter::ToThrift(const BloomFilter* filter, TBloomFilter* thrift) {
+  DCHECK(thrift != NULL);
+  if (filter == NULL) {
+    thrift->always_true = true;
+    return;
+  }
+  filter->ToThrift(thrift);
 }
 
 void BloomFilter::Or(const BloomFilter& other) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/util/bloom-filter.h
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index edb9083..32f6ee6 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -27,8 +27,6 @@
 
 namespace impala {
 
-class RuntimeState;
-
 /// A BloomFilter stores sets of items and offers a query operation indicating whether or
 /// not that item is in the set.  BloomFilters use much less space than other compact data
 /// structures, but they are less accurate: for a small percentage of elements, the query
@@ -50,17 +48,17 @@ class RuntimeState;
 /// bits).
 class BloomFilter {
  public:
-  /// Consumes at most (1 << log_heap_space) bytes on the heap. If state is non-NULL,
-  /// client is also non-NULL and the constructor and destructor call
-  /// BufferedBlockMgr::ConsumeMemory() and ReleaseMemory().
-  BloomFilter(
-      const int log_heap_space, RuntimeState* state, BufferedBlockMgr::Client* client);
-  BloomFilter(const TBloomFilter& thrift, RuntimeState* state,
-      BufferedBlockMgr::Client* client);
+  /// Consumes at most (1 << log_heap_space) bytes on the heap.
+  BloomFilter(const int log_heap_space);
+  BloomFilter(const TBloomFilter& thrift);
   ~BloomFilter();
 
-  /// Serializes this filter as Thrift.
-  void ToThrift(TBloomFilter* thrift) const;
+  /// Representation of a filter which allows all elements to pass.
+  static BloomFilter* ALWAYS_TRUE_FILTER;
+
+  /// Converts 'filter' to its corresponding Thrift representation. If the first argument
+  /// is NULL, it is interpreted as a complete filter which contains all elements.
+  static void ToThrift(const BloomFilter* filter, TBloomFilter* thrift);
 
   /// Adds an element to the BloomFilter. The function used to generate 'hash' need not
   /// have good uniformity, but it should have low collision probability. For instance, if
@@ -126,17 +124,13 @@ class BloomFilter {
   typedef BucketWord Bucket[BUCKET_WORDS];
   Bucket* directory_;
 
-  /// Used only for tracking memory. If both are non-NULL,
-  /// BufferedBlockMgr::{Acquire,Release}Memory() are called when this object allocates
-  /// and frees heap memory. These objects pointed to by state_ and client_ are not owned
-  /// by this BloomFilter.
-  RuntimeState* const state_;
-  BufferedBlockMgr::Client* const client_;
-
   int64_t directory_size() const {
     return 1uLL << (log_num_buckets_ + LOG_BUCKET_BYTE_SIZE);
   }
 
+  /// Serializes this filter as Thrift.
+  void ToThrift(TBloomFilter* thrift) const;
+
   DISALLOW_COPY_AND_ASSIGN(BloomFilter);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 2e1af59..5a7d2bc 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -21,6 +21,7 @@
 #include <boost/algorithm/string.hpp>
 #include <iostream>
 #include <fstream>
+#include <gutil/strings/substitute.h>
 #include <mmintrin.h>
 #include <sstream>
 #include <stdlib.h>
@@ -34,6 +35,7 @@
 using boost::algorithm::contains;
 using boost::algorithm::trim;
 using std::max;
+using strings::Substitute;
 
 DECLARE_bool(abort_on_config_error);
 DEFINE_int32(num_cores, 0, "(Advanced) If > 0, it sets the number of cores available to"
@@ -46,6 +48,7 @@ bool CpuInfo::initialized_ = false;
 int64_t CpuInfo::hardware_flags_ = 0;
 int64_t CpuInfo::original_hardware_flags_;
 long CpuInfo::cache_sizes_[L3_CACHE + 1];
+long CpuInfo::cache_line_sizes_[L3_CACHE + 1];
 int64_t CpuInfo::cycles_per_ms_;
 int CpuInfo::num_cores_ = 1;
 string CpuInfo::model_name_ = "unknown";
@@ -124,11 +127,21 @@ void CpuInfo::Init() {
   for (size_t i = 0; i < 3; ++i) {
     cache_sizes_[i] = data[i];
   }
+  size_t linesize;
+  size_t sizeof_linesize = sizeof(linesize);
+  sysctlbyname("hw.cachelinesize", &linesize, &sizeof_linesize, NULL, 0);
+  for (size_t i = 0; i < 3; ++i) cache_line_sizes_[i] = linesize;
 #else
   // Call sysconf to query for the cache sizes
   cache_sizes_[0] = sysconf(_SC_LEVEL1_DCACHE_SIZE);
   cache_sizes_[1] = sysconf(_SC_LEVEL2_CACHE_SIZE);
   cache_sizes_[2] = sysconf(_SC_LEVEL3_CACHE_SIZE);
+
+  cache_line_sizes_[0] = sysconf(_SC_LEVEL1_DCACHE_LINESIZE);
+  // See bloom-filter.cc for one dependency.
+  DCHECK_EQ(cache_line_sizes_[0], 64) << "Impala expects 64-byte L1 cache lines";
+  cache_line_sizes_[1] = sysconf(_SC_LEVEL2_CACHE_LINESIZE);
+  cache_line_sizes_[2] = sysconf(_SC_LEVEL3_CACHE_LINESIZE);
 #endif
 
   if (max_mhz != 0) {
@@ -171,15 +184,21 @@ void CpuInfo::EnableFeature(long flag, bool enable) {
 string CpuInfo::DebugString() {
   DCHECK(initialized_);
   stringstream stream;
-  int64_t L1 = CacheSize(L1_CACHE);
-  int64_t L2 = CacheSize(L2_CACHE);
-  int64_t L3 = CacheSize(L3_CACHE);
+  string L1 = Substitute("L1 Cache: $0 (Line: $1)",
+      PrettyPrinter::Print(CacheSize(L1_CACHE), TUnit::BYTES),
+      PrettyPrinter::Print(CacheLineSize(L1_CACHE), TUnit::BYTES));
+  string L2 = Substitute("L1 Cache: $0 (Line: $1)",
+      PrettyPrinter::Print(CacheSize(L2_CACHE), TUnit::BYTES),
+      PrettyPrinter::Print(CacheLineSize(L2_CACHE), TUnit::BYTES));
+  string L3 = Substitute("L1 Cache: $0 (Line: $1)",
+      PrettyPrinter::Print(CacheSize(L3_CACHE), TUnit::BYTES),
+      PrettyPrinter::Print(CacheLineSize(L3_CACHE), TUnit::BYTES));
   stream << "Cpu Info:" << endl
          << "  Model: " << model_name_ << endl
          << "  Cores: " << num_cores_ << endl
-         << "  L1 Cache: " << PrettyPrinter::Print(L1, TUnit::BYTES) << endl
-         << "  L2 Cache: " << PrettyPrinter::Print(L2, TUnit::BYTES) << endl
-         << "  L3 Cache: " << PrettyPrinter::Print(L3, TUnit::BYTES) << endl
+         << "  " << L1 << endl
+         << "  " << L2 << endl
+         << "  " << L3 << endl
          << "  Hardware Supports:" << endl;
   for (int i = 0; i < num_flags; ++i) {
     if (IsSupported(flag_mappings[i].flag)) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/be/src/util/cpu-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index 860498e..c3597f3 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -53,7 +53,7 @@ class CpuInfo {
     DCHECK(initialized_);
     return hardware_flags_;
   }
-  
+
   /// Returns whether of not the cpu supports this flag
   inline static bool IsSupported(long flag) {
     DCHECK(initialized_);
@@ -70,6 +70,12 @@ class CpuInfo {
     return cache_sizes_[level];
   }
 
+  /// Returns the size of a line in the cache at this level.
+  static long CacheLineSize(CacheLevel level) {
+    DCHECK(initialized_);
+    return cache_line_sizes_[level];
+  }
+
   /// Returns the number of cpu cycles per millisecond
   static int64_t cycles_per_ms() {
     DCHECK(initialized_);
@@ -77,13 +83,13 @@ class CpuInfo {
   }
 
   /// Returns the number of cores (including hyper-threaded) on this machine.
-  static int num_cores() { 
+  static int num_cores() {
     DCHECK(initialized_);
-    return num_cores_; 
+    return num_cores_;
   }
 
   /// Returns the model name of the cpu (e.g. Intel i7-2600)
-  static std::string model_name() { 
+  static std::string model_name() {
     DCHECK(initialized_);
     return model_name_;
   }
@@ -95,6 +101,7 @@ class CpuInfo {
   static int64_t hardware_flags_;
   static int64_t original_hardware_flags_;
   static long cache_sizes_[L3_CACHE + 1];
+  static long cache_line_sizes_[L3_CACHE + 1];
   static int64_t cycles_per_ms_;
   static int num_cores_;
   static std::string model_name_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/common/thrift/ImpalaInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index 70fd2f1..d3d3080 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -556,6 +556,10 @@ struct TBloomFilter {
   // string for efficiency of (de)serialisation. See BloomFilter::Bucket and
   // BloomFilter::directory_.
   2: binary directory
+
+  // If true, this filter allows all elements to pass (i.e. its selectivity is 1). If
+  // true, 'directory' and 'log_heap_space' are not meaningful.
+  4: required bool always_true
 }
 
 struct TUpdateFilterResult {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/common/thrift/PlanNodes.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift
index b71291e..eabf171 100644
--- a/common/thrift/PlanNodes.thrift
+++ b/common/thrift/PlanNodes.thrift
@@ -253,10 +253,6 @@ struct THashJoinNode {
 
   // non equi-join predicates
   3: optional list<Exprs.TExpr> other_join_conjuncts
-
-  // If true, this join node can (but may choose not to) generate slot filters
-  // after constructing the build side that can be applied to the probe side.
-  4: optional bool add_probe_filters
 }
 
 struct TNestedLoopJoinNode {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/fe/src/main/java/com/cloudera/impala/planner/HashJoinNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/HashJoinNode.java b/fe/src/main/java/com/cloudera/impala/planner/HashJoinNode.java
index 645d39c..1caae3b 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/HashJoinNode.java
@@ -46,11 +46,6 @@ import com.google.common.collect.Lists;
 public class HashJoinNode extends JoinNode {
   private final static Logger LOG = LoggerFactory.getLogger(HashJoinNode.class);
 
-  // If true, this node can add filters for the probe side that can be generated
-  // after reading the build side. This can be very helpful if the join is selective and
-  // there are few build rows.
-  private boolean addProbeFilters_;
-
   public HashJoinNode(
       PlanNode outer, PlanNode inner, DistributionMode distrMode, JoinOperator joinOp,
       List<BinaryPredicate> eqJoinConjuncts, List<Expr> otherJoinConjuncts) {
@@ -61,7 +56,6 @@ public class HashJoinNode extends JoinNode {
   }
 
   public List<BinaryPredicate> getEqJoinConjuncts() { return eqJoinConjuncts_; }
-  public void setAddProbeFilters(boolean b) { addProbeFilters_ = b; }
 
   @Override
   public void init(Analyzer analyzer) throws ImpalaException {
@@ -137,7 +131,6 @@ public class HashJoinNode extends JoinNode {
     for (Expr e: otherJoinConjuncts_) {
       msg.hash_join_node.addToOther_join_conjuncts(e.treeToThrift());
     }
-    msg.hash_join_node.setAdd_probe_filters(addProbeFilters_);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/fe/src/main/java/com/cloudera/impala/planner/PlanFragment.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/planner/PlanFragment.java b/fe/src/main/java/com/cloudera/impala/planner/PlanFragment.java
index 53ca1f3..f702271 100644
--- a/fe/src/main/java/com/cloudera/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/com/cloudera/impala/planner/PlanFragment.java
@@ -127,8 +127,6 @@ public class PlanFragment {
    */
   public void finalize(Analyzer analyzer)
       throws InternalException, NotImplementedException {
-    if (planRoot_ != null) computeCanAddSlotFilters(planRoot_);
-
     if (destNode_ != null) {
       Preconditions.checkState(sink_ == null);
       // we're streaming to an exchange node
@@ -180,73 +178,7 @@ public class PlanFragment {
     return dataPartition_ == DataPartition.UNPARTITIONED ? 1 : planRoot_.getNumNodes();
   }
 
-  /**
-   * Returns true and sets node.canAddPredicate, if we can add single-slot filters at
-   * execution time (i.e. after Prepare() to the plan tree rooted at this node.
-   * That is, 'node' can add filters that can be evaluated at nodes below.
-   *
-   * We compute this by walking the tree bottom up.
-   *
-   * TODO: move this to PlanNode.init() which is normally responsible for computing
-   * internal state of PlanNodes. We cannot do this currently since we need the
-   * distrubutionMode() set on HashJoin nodes. Once we call init() properly for
-   * repartitioned joins, this logic can move to init().
-   */
-  private boolean computeCanAddSlotFilters(PlanNode node) {
-    if (node instanceof HashJoinNode) {
-      HashJoinNode hashJoinNode = (HashJoinNode)node;
-      boolean childResult = computeCanAddSlotFilters(node.getChild(0));
-      if (!childResult) return false;
-      if (hashJoinNode.getJoinOp().equals(JoinOperator.FULL_OUTER_JOIN) ||
-          hashJoinNode.getJoinOp().equals(JoinOperator.LEFT_OUTER_JOIN) ||
-          hashJoinNode.getJoinOp().equals(JoinOperator.LEFT_ANTI_JOIN) ||
-          hashJoinNode.getJoinOp().equals(JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN)) {
-        // It is not correct to push through an outer or anti join on the probe side.
-        // We cannot filter those rows out.
-        return false;
-      }
-      // We can't push down predicates for partitioned joins yet.
-      // TODO: this can be hugely helpful to avoid network traffic. Implement this.
-      if (hashJoinNode.getDistributionMode() == DistributionMode.PARTITIONED) {
-        return false;
-      }
-
-      List<BinaryPredicate> joinConjuncts = hashJoinNode.getEqJoinConjuncts();
-      // We can only add these filters for conjuncts of the form:
-      // <probe_slot> = *. If the hash join has any equal join conjuncts in this form,
-      // mark the hash join node.
-      for (Expr c: joinConjuncts) {
-        if (c.getChild(0) instanceof SlotRef) {
-          hashJoinNode.setAddProbeFilters(true);
-          break;
-        }
-      }
-      // Even if this join cannot add predicates, return true so the parent node can.
-      return true;
-    } else if (node instanceof HdfsScanNode) {
-      // Since currently only the Parquet scanner employs the slot filter optimization,
-      // we enable it only if the majority format is Parquet. Otherwise we are adding
-      // the overhead of creating the SlotFilters in the build side in queries not on
-      // Parquet data.
-      // TODO: Modify the other scanners to exploit the slot filter optimization.
-      HdfsScanNode scanNode = (HdfsScanNode) node;
-      Preconditions.checkNotNull(scanNode.desc_);
-      Preconditions.checkNotNull(scanNode.desc_.getTable() instanceof HdfsTable);
-      HdfsTable table = (HdfsTable) scanNode.desc_.getTable();
-      if (table.getMajorityFormat() == HdfsFileFormat.PARQUET) {
-        return true;
-      } else {
-        return false;
-      }
-    } else {
-      for (PlanNode child : node.getChildren()) {
-        computeCanAddSlotFilters(child);
-      }
-      return false;
-    }
-  }
-
-  /**
+ /**
    * Estimates the per-node number of distinct values of exprs based on the data
    * partition of this fragment and its number of nodes. Returns -1 for an invalid
    * estimate, e.g., because getNumDistinctValues() failed on one of the exprs.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d1eab7a/testdata/workloads/functional-query/queries/QueryTest/runtime_filters_wait.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/runtime_filters_wait.test b/testdata/workloads/functional-query/queries/QueryTest/runtime_filters_wait.test
index 73ea1ae..f2f49f6 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/runtime_filters_wait.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/runtime_filters_wait.test
@@ -14,3 +14,25 @@ on p.month = b.int_col and b.month = 1 and b.string_col = "1"
 ---- RESULTS
 620
 ====
+
+
+---- QUERY
+####################################################
+# Regression test for IMPALA-3141: Disabled filters should send dummy filters
+# to unblock waiters. Filters are disabled by setting the filter size to be very
+# small, and having a large enough build input to trigger the false-positive check.
+# Note: build side has a low NDV, so if the FP estimate changes to take NDV into
+# account this test will need to be updated.
+####################################################
+
+SET RUNTIME_FILTER_WAIT_TIME_MS=600000;
+SET RUNTIME_FILTER_MODE=GLOBAL;
+set RUNTIME_BLOOM_FILTER_SIZE=4096;
+select STRAIGHT_JOIN count(*) from alltypes a JOIN [BROADCAST]
+  (select c.id from alltypes c CROSS JOIN alltypes d LIMIT 40000) b
+  on a.id = b.id
+---- RESULTS
+40000
+---- RUNTIME_PROFILE
+row_regex: .*Build-Side Runtime-Filter Disabled \(FP Rate Too High\).*
+====


[24/50] incubator-impala git commit: Allow switching between custom and bootstrapped toolchain

Posted by ta...@apache.org.
Allow switching between custom and bootstrapped toolchain

If SKIP_TOOLCHAIN_BOOTSTRAP is set, toolchain bootstrap is skipped. This
means that even if you are running on a supported OS, your custom-built
toolchain artifacts will always be used.

Also use Ubuntu 14.04 toolchain artifacts for Ubuntu 15.10.
I have been using the artifacts locally for a while and it has been
working fine.

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

Branch: refs/heads/master
Commit: 40117bf7258f00d156b5a458c6949041aa076118
Parents: 3d5a21c
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Mar 30 09:03:56 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 31 22:01:01 2016 +0000

----------------------------------------------------------------------
 CMakeLists.txt             | 20 ++++++++++++--------
 bin/bootstrap_toolchain.py |  3 ++-
 2 files changed, 14 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/40117bf7/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index f1a8e01..595b082 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -18,15 +18,19 @@ if (NOT $ENV{DISABLE_IMPALA_TOOLCHAIN})
   # Enable toolchain variable if the environment is setup
   set(IMPALA_TOOLCHAIN ON)
   message(STATUS "Toolchain build.")
-  # Download any missing toolchain dependencies. If this fails, fail the build.
-  set(BOOTSTRAP_CMD "$ENV{IMPALA_HOME}/bin/bootstrap_toolchain.py")
-  # Download and unpack the dependencies
-  message(STATUS "Downloading and extracting dependencies.")
-  execute_process(COMMAND ${BOOTSTRAP_CMD} RESULT_VARIABLE BOOTSTRAP_RESULT)
-  if (${BOOTSTRAP_RESULT} EQUAL 0)
-    message(STATUS "Toolchain bootstrap complete.")
+  if ("$ENV{SKIP_TOOLCHAIN_BOOTSTRAP}" STREQUAL "true")
+    message(STATUS "SKIP_TOOLCHAIN_BOOTSTRAP is true, skipping toolchain bootstrap.")
   else()
-    message(FATAL_ERROR "Toolchain bootstrap failed.")
+    # Download any missing toolchain dependencies. If this fails, fail the build.
+    set(BOOTSTRAP_CMD "$ENV{IMPALA_HOME}/bin/bootstrap_toolchain.py")
+    # Download and unpack the dependencies
+    message(STATUS "Downloading and extracting dependencies.")
+    execute_process(COMMAND ${BOOTSTRAP_CMD} RESULT_VARIABLE BOOTSTRAP_RESULT)
+    if (${BOOTSTRAP_RESULT} EQUAL 0)
+      message(STATUS "Toolchain bootstrap complete.")
+    else()
+      message(FATAL_ERROR "Toolchain bootstrap failed.")
+    endif()
   endif()
 else()
   set(IMPALA_TOOLCHAIN OFF)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/40117bf7/bin/bootstrap_toolchain.py
----------------------------------------------------------------------
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index e12eef4..16f2e1b 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -46,7 +46,8 @@ OS_MAPPING = {
   "suselinux12": "ec2-package-sles-12",
   "ubuntu12.04" : "ec2-package-ubuntu-12-04",
   "ubuntu14.04" : "ec2-package-ubuntu-14-04",
-  "ubuntu15.04" : "ec2-package-ubuntu-14-04"
+  "ubuntu15.04" : "ec2-package-ubuntu-14-04",
+  "ubuntu15.10" : "ec2-package-ubuntu-14-04",
 }
 
 def try_get_release_label():


[31/50] incubator-impala git commit: IMPALA-2645: Set mechUsing only on success

Posted by ta...@apache.org.
IMPALA-2645: Set mechUsing only on success

Better to check success status from sasl_client_start() before using
returned structure with auth mechanism. According to the API
documentation, its value is guaranteed only after success (no error, or
SASL_CONTINUE).

Change-Id: I71e5bb754866003e2a282ce3eeb8d4e6eac0573d
Reviewed-on: http://gerrit.cloudera.org:8080/2687
Tested-by: Internal Jenkins
Reviewed-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/7dd4998b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/7dd4998b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/7dd4998b

Branch: refs/heads/master
Commit: 7dd4998b4fc142425080fa1cce07dcf36df24c1d
Parents: 5703f4e
Author: František Dvořák <va...@civ.zcu.cz>
Authored: Thu Mar 31 23:43:37 2016 +0200
Committer: Henry Robinson <he...@cloudera.com>
Committed: Tue Apr 5 22:26:16 2016 +0000

----------------------------------------------------------------------
 be/src/transport/TSasl.cpp | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/7dd4998b/be/src/transport/TSasl.cpp
----------------------------------------------------------------------
diff --git a/be/src/transport/TSasl.cpp b/be/src/transport/TSasl.cpp
index 31e1e1a..3108dff 100644
--- a/be/src/transport/TSasl.cpp
+++ b/be/src/transport/TSasl.cpp
@@ -148,7 +148,9 @@ uint8_t* TSaslClient::evaluateChallengeOrResponse(
           &outlen,   /* filled in on success */
           (const char**)&mechUsing);
     clientStarted = true;
-    chosenMech = mechUsing;
+    if (result == SASL_OK || result == SASL_CONTINUE) {
+      chosenMech = mechUsing;
+    }
   } else {
     if (len  > 0) {
       result=sasl_client_step(conn,  /* our context */


[05/50] incubator-impala git commit: IMPALA-3226: Increase timeout for runtime filter tests

Posted by ta...@apache.org.
IMPALA-3226: Increase timeout for runtime filter tests

When running with ASAN enabled, runtime filters may take a lot longer to
be produced, triggering timeouts in the filter tests. This patch triples
the timeout time.

We still want the timeout to be reasonable as protection against
excessive regressions in filter production time, which is why I've not
set the timeout to a very large value, plus if the test fails and
filters aren't produced we don't want to hang the build for a large
timeout delay.

Change-Id: Ife1d36a78d6ad587462fe112afda573f6e480441
Reviewed-on: http://gerrit.cloudera.org:8080/2609
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: c06912ebb6e9f6e06fdb7b6f37b66f693a38c927
Parents: 077d0fa
Author: Henry Robinson <he...@cloudera.com>
Authored: Wed Mar 23 11:29:45 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 24 07:59:53 2016 +0000

----------------------------------------------------------------------
 .../queries/QueryTest/runtime_filters.test      | 32 ++++++++++----------
 .../queries/QueryTest/runtime_row_filters.test  | 26 ++++++++--------
 2 files changed, 29 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c06912eb/testdata/workloads/functional-query/queries/QueryTest/runtime_filters.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/runtime_filters.test b/testdata/workloads/functional-query/queries/QueryTest/runtime_filters.test
index 6fafbaa..3497867 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/runtime_filters.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/runtime_filters.test
@@ -18,7 +18,7 @@ row_regex: .*RowsRead: 2.43K .*
 ====
 ---- QUERY
 # Now turn on local filtering: we expect to see a reduction in scan volume.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes p join [BROADCAST] alltypestiny b
 on p.month = b.int_col and b.month = 1 and b.string_col = "1"
@@ -38,7 +38,7 @@ row_regex: .*Files rejected: 7 \(7\).*
 
 # Local mode. Filters won't be propagated to scan, so scans will read all rows.
 # Still give enough time for filters to show up (even if they won't)
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes p join [SHUFFLE] alltypestiny b
 on p.month = b.int_col and b.month = 1 and b.string_col = "1"
@@ -49,7 +49,7 @@ row_regex: .*RowsRead: 2.43K .*
 ====
 ---- QUERY
 # Shuffle join, global mode. Expect filters to be propagated.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes p join [SHUFFLE] alltypestiny b
 on p.month = b.int_col and b.month = 1 and b.string_col = "1"
@@ -71,7 +71,7 @@ row_regex: .*Files rejected: 7 \(7\).*
 
 # Local mode. Only the left-most scan will receive its filter, but since the scan of 'b'
 # will not, the lack of predicates means there is no filter effect.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypes b
@@ -85,7 +85,7 @@ row_regex: .*Files rejected: 0 .*
 ---- QUERY
 # Global mode. Scan of 'b' will receive highly effective filter, and will propagate that
 # to left-most scan.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypes b
@@ -124,7 +124,7 @@ row_regex: .*Files rejected: 8 .*
 ####################################################
 
 # Local mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -136,7 +136,7 @@ row_regex: .*FiltersReceived: 0 .*
 ====
 ---- QUERY
 # Global mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -155,7 +155,7 @@ row_regex: .*Files rejected: 8 .*
 ####################################################
 
 # Local mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [SHUFFLE] alltypestiny b
@@ -167,7 +167,7 @@ row_regex: .*FiltersReceived: 0 .*
 ====
 ---- QUERY
 # Global mode. Coordinator should report 1 filter update per backend.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [SHUFFLE] alltypestiny b
@@ -185,7 +185,7 @@ row_regex: .*FiltersReceived: 3 .*
 # Expect all but one partition to be filtered out by join expr.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -202,7 +202,7 @@ row_regex: .*Files rejected: 7 .*
 # Test case 8: filters do not pass through LOJ.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     LEFT OUTER join alltypestiny b
@@ -220,7 +220,7 @@ row_regex: .*RowsReturned: 2.43K .*
 # All partitions will be filtered out by the join condition.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     RIGHT OUTER join alltypestiny b
@@ -236,7 +236,7 @@ row_regex: .*Files rejected: 8 .*
 # Test case 10: filters do not pass through FOJ.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     FULL OUTER join alltypestiny b
@@ -255,7 +255,7 @@ row_regex: .*RowsReturned: 2.43K .*
 # filter will be disabled due to a high expected FP rate.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 set RUNTIME_BLOOM_FILTER_SIZE=4096;
 with build as (
@@ -281,7 +281,7 @@ row_regex: .*Rows rejected: 0 .*
 # per fragment instance, and three files rejected per scan.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypesagg a
     join alltypesagg b
@@ -301,7 +301,7 @@ row_regex: .*Files rejected: 3 .*
 # join in its root, which produces filters for the scan of t1.
 ####################################################
 
-set RUNTIME_FILTER_WAIT_TIME_MS=5000;
+set RUNTIME_FILTER_WAIT_TIME_MS=15000;
 set RUNTIME_FILTER_MODE=GLOBAL;
 with t1 as (select month x, bigint_col y from alltypes limit 7300),
      t2 as (select int_col x, bigint_col y from alltypestiny limit 2)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c06912eb/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test b/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
index ffba0ec..f47cc3b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
@@ -8,7 +8,7 @@
 # With filtering, expect 618 / 3 = 206 rows.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=OFF;
 select STRAIGHT_JOIN count(*) from alltypes p join [BROADCAST] alltypestiny b
 on p.id = b.id and b.month = 1 and b.string_col = "1"
@@ -19,7 +19,7 @@ row_regex: .*RowsRead: 2.43K .*
 ====
 ---- QUERY
 # Now turn on local filtering: we expect to see a reduction in scan volume.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes p join [BROADCAST] alltypestiny b
 on p.id = b.id and b.month = 1 and b.string_col = "1"
@@ -39,7 +39,7 @@ row_regex: .*Rows rejected: 2.43K .*
 
 # Local mode. Filters won't be propagated to scan, so scans will read all rows.
 # Still give enough time for filters to show up (even if they won't)
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes p join [SHUFFLE] alltypestiny b
 on p.id = b.int_col and b.month = 1 and b.string_col = "1"
@@ -71,7 +71,7 @@ row_regex: .*Rows rejected: 2.43K .*
 
 # Local mode. Only the left-most scan will receive its filter, but since the scan of 'b'
 # will not, the lack of predicates means there is no filter effect.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypes b
@@ -85,7 +85,7 @@ row_regex: .*Files rejected: 0 .*
 ---- QUERY
 # Global mode. Scan of 'b' will receive highly effective filter, and will propagate that
 # to left-most scan.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypes b
@@ -124,7 +124,7 @@ row_regex: .*Rows rejected: 2.43K .*
 ####################################################
 
 # Local mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -137,7 +137,7 @@ row_regex: .*Rows rejected: 2.43K .*
 ====
 ---- QUERY
 # Global mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -156,7 +156,7 @@ row_regex: .*Rows rejected: 2.43K .*
 ####################################################
 
 # Local mode. Coordinator should report 0 filter updates received.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [SHUFFLE] alltypestiny b
@@ -168,7 +168,7 @@ row_regex: .*FiltersReceived: 0 .*
 ====
 ---- QUERY
 # Global mode. Coordinator should report 1 filter updates per backend.
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [SHUFFLE] alltypestiny b
@@ -184,7 +184,7 @@ row_regex: .*FiltersReceived: 3 .*
 # Test case 7: filters with target exprs bound by > 1 slotref
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=LOCAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     join [BROADCAST] alltypestiny b
@@ -201,7 +201,7 @@ row_regex: .*Rows rejected: 2.43K .*
 # Test case 8: filters do not pass through LOJ.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     LEFT OUTER join alltypestiny b
@@ -218,7 +218,7 @@ row_regex: .*RowsReturned: 2.43K .*
 # Test case 9: filters do pass through ROJ.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     RIGHT OUTER join alltypestiny b
@@ -235,7 +235,7 @@ row_regex: .*Rows rejected: 2.43K .*
 # Test case 9: filters do not pass through FOJ.
 ####################################################
 
-SET RUNTIME_FILTER_WAIT_TIME_MS=5000;
+SET RUNTIME_FILTER_WAIT_TIME_MS=15000;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 select STRAIGHT_JOIN count(*) from alltypes a
     FULL OUTER join alltypestiny b


[04/50] incubator-impala git commit: IMPALA-3220: Skip logging empty ScannerContext's stream in parse error.

Posted by ta...@apache.org.
IMPALA-3220: Skip logging empty ScannerContext's stream in parse error.

It's possible for a scanner thread to run to completion and call
ScannerContext::ReleaseCompletedResources() only to pick up a cancellation
or a bad status from the the runtime state in CommitRows(), both of which
are scan node wide's states instead of per-thread state. In this case, the
streams in the scanner context may have been removed already so it's unsafe
to access them when logging parse error. Instead, check if the streams still
exist in the scanner context before using them in logging.

Change-Id: Ic8e7ab10e62fff755a0acb9c5649d6a062217045
Reviewed-on: http://gerrit.cloudera.org:8080/2605
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 077d0fab112ec16d416571898adc1c3b12165944
Parents: 10e7de7
Author: Michael Ho <kw...@cloudera.com>
Authored: Mon Mar 21 14:36:07 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 24 04:54:07 2016 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-scan-node.cc | 10 ++++++----
 be/src/exec/scanner-context.h |  4 ++++
 2 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/077d0fab/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 017c5f0..c43bdcb 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -1192,10 +1192,12 @@ Status HdfsScanNode::ProcessSplit(const vector<FilterContext>& filter_ctxs,
     ss << "Scan node (id=" << id() << ") ran into a parse error for scan range "
        << scan_range->file() << "(" << scan_range->offset() << ":"
        << scan_range->len() << ").";
-    if (partition->file_format() != THdfsFileFormat::PARQUET) {
-      // Parquet doesn't read the range end to end so the current offset isn't useful.
-      // TODO: make sure the parquet reader is outputting as much diagnostic
-      // information as possible.
+    // Parquet doesn't read the range end to end so the current offset isn't useful.
+    // TODO: make sure the parquet reader is outputting as much diagnostic
+    // information as possible.
+    // The error status may not necessarily be related to this scanner thread so this
+    // thread may have run to completion and closed all its streams already.
+    if (partition->file_format() != THdfsFileFormat::PARQUET && context.HasStream()) {
       ScannerContext::Stream* stream = context.GetStream();
       ss << " Processed " << stream->total_bytes_returned() << " bytes.";
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/077d0fab/be/src/exec/scanner-context.h
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.h b/be/src/exec/scanner-context.h
index b02d301..22e0928 100644
--- a/be/src/exec/scanner-context.h
+++ b/be/src/exec/scanner-context.h
@@ -244,6 +244,10 @@ class ScannerContext {
     Status ReportInvalidRead(int64_t length);
   };
 
+  bool HasStream() {
+    return !streams_.empty();
+  }
+
   Stream* GetStream(int idx = 0) {
     DCHECK_GE(idx, 0);
     DCHECK_LT(idx, streams_.size());


[50/50] incubator-impala git commit: Add explanation that Incubator repo is not buildable yet

Posted by ta...@apache.org.
Add explanation that Incubator repo is not buildable yet


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

Branch: refs/heads/master
Commit: f915d59aa4b028fafc3d12a028a2d937d2e19928
Parents: a65ffda
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Apr 12 14:15:55 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:15:55 2016 -0700

----------------------------------------------------------------------
 README.md | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f915d59a/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 63a70fe..d875a79 100644
--- a/README.md
+++ b/README.md
@@ -24,3 +24,10 @@ visit the [Impala homepage](http://impala.io/index.html).
 If you are interested in contributing to Impala as a developer, or learning more about
 Impala's internals and architecture, visit the
 [Impala wiki](https://github.com/cloudera/Impala/wiki).
+
+## Building Impala
+
+This Apache Incubator repository is currently not buildable but has the complete source
+code for Impala minus some third-party dependences. See https://github.com/cloudera/Impala
+for the buildable Impala source and https://issues.cloudera.org/browse/IMPALA-3223 to
+track progress on making this repository buildable.


[26/50] incubator-impala git commit: Regenerate complextypestbl files to include nested_struct.g field

Posted by ta...@apache.org.
Regenerate complextypestbl files to include nested_struct.g field

This field was included in the schema and data files, but the
checked-in generated parquet files didn't include it. It's not
referenced in any tests so we didn't catch it.

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

Branch: refs/heads/master
Commit: 2cbd327d4127fa895e312cf68f77427a401abed3
Parents: 5284130
Author: Skye Wanderman-Milne <sk...@cloudera.com>
Authored: Tue Mar 15 17:02:26 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Fri Apr 1 05:06:38 2016 +0000

----------------------------------------------------------------------
 testdata/ComplexTypesTbl/nonnullable.parq       | Bin 2659 -> 3190 bytes
 testdata/ComplexTypesTbl/nullable.parq          | Bin 3154 -> 3900 bytes
 .../functional/functional_schema_template.sql   |   2 +-
 .../QueryTest/nested-types-scanner-maps.test    |  57 +++++++++++++++++++
 4 files changed, 58 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2cbd327d/testdata/ComplexTypesTbl/nonnullable.parq
----------------------------------------------------------------------
diff --git a/testdata/ComplexTypesTbl/nonnullable.parq b/testdata/ComplexTypesTbl/nonnullable.parq
index 0d52af2..e13d9eb 100644
Binary files a/testdata/ComplexTypesTbl/nonnullable.parq and b/testdata/ComplexTypesTbl/nonnullable.parq differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2cbd327d/testdata/ComplexTypesTbl/nullable.parq
----------------------------------------------------------------------
diff --git a/testdata/ComplexTypesTbl/nullable.parq b/testdata/ComplexTypesTbl/nullable.parq
index bcd6d87..01955a6 100644
Binary files a/testdata/ComplexTypesTbl/nullable.parq and b/testdata/ComplexTypesTbl/nullable.parq differ

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2cbd327d/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 80487f1..840740f 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -562,7 +562,7 @@ int_array array<int>
 int_array_array array<array<int>>
 int_map map<string, int>
 int_map_array array<map<string, int>>
-nested_struct struct<a: int, b: array<int>, c: struct<d: array<array<struct<e: int, f: string>>>>, g: map<string, struct<h: struct<i: array<float>>>>>
+nested_struct struct<a: int, b: array<int>, c: struct<d: array<array<struct<e: int, f: string>>>>, g: map<string, struct<h: struct<i: array<double>>>>>
 ---- DEPENDENT_LOAD
 `hadoop fs -mkdir -p /test-warehouse/complextypestbl_parquet && \
 hadoop fs -put -f ${IMPALA_HOME}/testdata/ComplexTypesTbl/nullable.parq \

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2cbd327d/testdata/workloads/functional-query/queries/QueryTest/nested-types-scanner-maps.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/nested-types-scanner-maps.test b/testdata/workloads/functional-query/queries/QueryTest/nested-types-scanner-maps.test
index a2c6547..8daa573 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/nested-types-scanner-maps.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/nested-types-scanner-maps.test
@@ -58,3 +58,60 @@ select id, a.pos, m.key, m.value from complextypestbl t, t.int_map_array a, a.it
 ---- TYPES
 bigint,bigint,string,int
 ====
+---- QUERY
+select id, key from complextypestbl t, t.nested_struct.g
+---- RESULTS
+1,'foo'
+2,'g1'
+2,'g2'
+2,'g3'
+2,'g4'
+2,'g5'
+5,'foo'
+---- TYPES
+bigint,string
+====
+---- QUERY
+select id, key, item from complextypestbl t, t.nested_struct.g, g.value.h.i
+---- RESULTS
+1,'foo',1.1
+2,'g1',2.2
+2,'g1',NULL
+5,'foo',2.2
+5,'foo',3.3
+---- TYPES
+bigint,string,double
+====
+---- QUERY
+select key, item from complextypestbl.nested_struct.g, g.value.h.i
+---- RESULTS
+'foo',1.1
+'g1',2.2
+'g1',NULL
+'foo',2.2
+'foo',3.3
+---- TYPES
+string,double
+====
+---- QUERY
+select key, item, pos from complextypestbl.nested_struct.g, g.value.h.i
+---- RESULTS
+'foo',1.1,0
+'g1',2.2,0
+'g1',NULL,1
+'foo',2.2,0
+'foo',3.3,1
+---- TYPES
+string,double,bigint
+====
+---- QUERY
+select item from complextypestbl.nested_struct.g.value.h.i
+---- RESULTS
+1.1
+2.2
+NULL
+2.2
+3.3
+---- TYPES
+double
+====


[46/50] incubator-impala git commit: Fix typo in load-test-warehouse-snapshot.sh

Posted by ta...@apache.org.
Fix typo in load-test-warehouse-snapshot.sh

Change-Id: I2ef9b32cbc56819f80db864a6590a9a7b2732c9c
Reviewed-on: http://gerrit.cloudera.org:8080/2310
Reviewed-by: Lars Volker <lv...@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/ee8c3091
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/ee8c3091
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/ee8c3091

Branch: refs/heads/master
Commit: ee8c3091871ebf168000e81aee349689b6671792
Parents: 0d9028d
Author: Lars Volker <lv...@cloudera.com>
Authored: Wed Feb 24 16:09:10 2016 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 testdata/bin/load-test-warehouse-snapshot.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ee8c3091/testdata/bin/load-test-warehouse-snapshot.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/load-test-warehouse-snapshot.sh b/testdata/bin/load-test-warehouse-snapshot.sh
index 9545abf..07c785d 100755
--- a/testdata/bin/load-test-warehouse-snapshot.sh
+++ b/testdata/bin/load-test-warehouse-snapshot.sh
@@ -78,7 +78,7 @@ echo "Extracting tarball"
 tar -C ${SNAPSHOT_STAGING_DIR} -xzf ${SNAPSHOT_FILE}
 
 if [ ! -f ${SNAPSHOT_STAGING_DIR}/test-warehouse/githash.txt ]; then
-  echo "The test-warehouse snapshot does not containa githash, aborting load"
+  echo "The test-warehouse snapshot does not contain a githash.txt file, aborting load"
   exit 1
 fi
 


[48/50] incubator-impala git commit: IMPALA-3179: Fix alter table properties for Kudu tables

Posted by ta...@apache.org.
IMPALA-3179: Fix alter table properties for Kudu tables

This is one of the merge follow up tasks. It seems like there was just a
line missing to copy the metastore data into the Kudu table object. The
HDFS table class does the same thing as in this change.

Change-Id: I51c9942f2f398afb7dff2485da759a185ad7505f
Reviewed-on: http://gerrit.cloudera.org:8080/2728
Reviewed-by: Casey Ching <ca...@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/0d9028dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/0d9028dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/0d9028dd

Branch: refs/heads/master
Commit: 0d9028dd49af9b371d40aa91ea3c201f63454d1a
Parents: 4bdd0b9
Author: casey <ca...@cloudera.com>
Authored: Thu Mar 31 17:04:48 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java | 1 +
 tests/query_test/test_kudu.py                               | 7 +++----
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d9028dd/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java b/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
index 2b60766..541908a 100644
--- a/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
+++ b/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
@@ -179,6 +179,7 @@ public class KuduTable extends Table {
           "Cannot load Kudu table %s, table is corrupt.", name_));
     }
 
+    msTable_ = msTbl;
     kuduTableName_ = msTbl.getParameters().get(KEY_TABLE_NAME);
     kuduMasters_ = msTbl.getParameters().get(KEY_MASTER_ADDRESSES);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/0d9028dd/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index d5e4555..25e7d95 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -82,10 +82,9 @@ class TestKuduOperations(ImpalaTestSuite):
   def test_kudu_partition_ddl(self, vector):
     self.run_test_case('QueryTest/kudu_partition_ddl', vector, use_db="kududb_test")
 
-  # TODO(kudu-merge) IMPALA-3179 - Altering table properties is broken. When that is
-  # solved uncomment this.
-  #def test_kudu_alter_table(self, vector):
-  #  self.run_test_case('QueryTest/kudu_alter', vector, use_db="kududb_test")
+  @pytest.mark.execute_serially
+  def test_kudu_alter_table(self, vector):
+    self.run_test_case('QueryTest/kudu_alter', vector, use_db="kududb_test")
 
   def test_kudu_stats(self, vector):
     self.run_test_case('QueryTest/kudu_stats', vector, use_db="kududb_test")


[41/50] incubator-impala git commit: Use more efficient gutil implementation of Log2Ceiling

Posted by ta...@apache.org.
Use more efficient gutil implementation of Log2Ceiling

This was used in FreePool::Allocate, which is called a lot in some
workloads, so may have some perf impact. I was looking at using this
function for an allocator for the new buffer pool and it makes the
allocator code simpler if I can call this function freely without
perf concerns.

Modified gutil's implementation slightly so that gcc optimises it to
the following branch-free code:

.LHOTB8:
        .align 2
        .p2align 4,,15
        .globl  _ZN4Bits20Log2CeilingNonZero64Ey
        .type   _ZN4Bits20Log2CeilingNonZero64Ey, @function
_ZN4Bits20Log2CeilingNonZero64Ey:
.LFB1797:
        .loc 1 90 0
        .cfi_startproc
.LVL40:
.LBB33:
        .loc 1 92 0
        leaq    -1(%rdi), %rdx  #, D.40305
.LBB34:
.LBB35:
        .loc 2 155 0
        bsrq    %rdi, %rax      # n, tmp89
.LVL41:
.LBE35:
.LBE34:
        .loc 1 92 0
        andq    %rdx, %rdi      # D.40305, D.40305
.LVL42:
        .loc 1 95 0
        cmpq    $1, %rdi        #, D.40305
        sbbl    $-1, %eax       #, D.40304
.LVL43:
.LBE33:
        .loc 1 96 0
        ret
        .cfi_endproc

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

Branch: refs/heads/master
Commit: 92fafa1c0687141481e56ea6f09a67987d571d6e
Parents: 344fbc0
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Apr 8 10:23:03 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc |  3 ++-
 be/src/gutil/bits.cc                | 16 ---------------
 be/src/gutil/bits.h                 | 34 ++++++++++++++++++++++++++++++++
 be/src/runtime/disk-io-mgr.cc       | 12 ++++++-----
 be/src/runtime/free-pool.h          |  4 ++--
 be/src/runtime/runtime-filter.cc    |  6 +++---
 be/src/util/bit-util-test.cc        | 22 ++++++++++++---------
 be/src/util/bit-util.h              | 16 ---------------
 be/src/util/bloom-filter.h          |  4 ++--
 be/src/util/dict-encoding.h         |  3 ++-
 10 files changed, 65 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 9e18c66..ead3628 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -27,6 +27,7 @@
 #include "exec/scanner-context.inline.h"
 #include "exec/read-write-util.h"
 #include "exprs/expr.h"
+#include "gutil/bits.h"
 #include "runtime/collection-value-builder.h"
 #include "runtime/descriptors.h"
 #include "runtime/runtime-state.h"
@@ -1231,7 +1232,7 @@ Status HdfsParquetScanner::LevelDecoder::Init(const string& filename,
       if (num_bytes < 0) {
         return Status(TErrorCode::PARQUET_CORRUPT_VALUE, filename, num_bytes);
       }
-      int bit_width = BitUtil::Log2(max_level + 1);
+      int bit_width = Bits::Log2Ceiling64(max_level + 1);
       Reset(*data, num_bytes, bit_width);
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/gutil/bits.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/bits.cc b/be/src/gutil/bits.cc
index bc4e451..dec6f51 100644
--- a/be/src/gutil/bits.cc
+++ b/be/src/gutil/bits.cc
@@ -71,22 +71,6 @@ int Bits::Log2Floor_Portable(uint32 n) {
   return log;
 }
 
-int Bits::Log2Ceiling(uint32 n) {
-  int floor = Log2Floor(n);
-  if (n == (n &~ (n - 1)))              // zero or a power of two
-    return floor;
-  else
-    return floor + 1;
-}
-
-int Bits::Log2Ceiling64(uint64 n) {
-  int floor = Log2Floor64(n);
-  if (n == (n &~ (n - 1)))              // zero or a power of two
-    return floor;
-  else
-    return floor + 1;
-}
-
 int Bits::FindLSBSetNonZero_Portable(uint32 n) {
   int rc = 31;
   for (int i = 4, shift = 1 << 4; i >= 0; --i) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/gutil/bits.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/bits.h b/be/src/gutil/bits.h
index bacd655..579d954 100644
--- a/be/src/gutil/bits.h
+++ b/be/src/gutil/bits.h
@@ -79,6 +79,10 @@ class Bits {
   static int Log2Ceiling(uint32 n);
   static int Log2Ceiling64(uint64 n);
 
+  // Potentially faster version of Log2Ceiling() that returns an
+  // undefined value if n == 0
+  static int Log2CeilingNonZero64(uint64 n);
+
   // Return the first set least / most significant bit, 0-indexed.  Returns an
   // undefined value if n == 0.  FindLSBSetNonZero() is similar to ffs() except
   // that it's 0-indexed, while FindMSBSetNonZero() is the same as
@@ -139,6 +143,36 @@ inline int Bits::Log2FloorNonZero(uint32 n) {
   return 31 ^ __builtin_clz(n);
 }
 
+inline int Bits::Log2Ceiling(uint32 n) {
+  int floor = Log2Floor(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
+inline int Bits::Log2Ceiling64(uint64 n) {
+  int floor = Log2Floor64(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
+inline int Bits::Log2CeilingNonZero64(uint64 n) {
+  int floor = Log2FloorNonZero64(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
 inline int Bits::FindLSBSetNonZero(uint32 n) {
   return __builtin_ctz(n);
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr.cc b/be/src/runtime/disk-io-mgr.cc
index 80675d7..4788739 100644
--- a/be/src/runtime/disk-io-mgr.cc
+++ b/be/src/runtime/disk-io-mgr.cc
@@ -14,11 +14,13 @@
 
 #include "runtime/disk-io-mgr.h"
 #include "runtime/disk-io-mgr-internal.h"
-#include "util/hdfs-util.h"
 
-#include <gutil/strings/substitute.h>
 #include <boost/algorithm/string.hpp>
 
+#include "gutil/bits.h"
+#include "gutil/strings/substitute.h"
+#include "util/hdfs-util.h"
+
 DECLARE_bool(disable_mem_pools);
 
 #include "common/names.h"
@@ -270,7 +272,7 @@ DiskIoMgr::DiskIoMgr() :
         FileSystemUtil::MaxNumFileHandles()),
         &HdfsCachedFileHandle::Release) {
   int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2(max_buffer_size_scaled) + 1);
+  free_buffers_.resize(Bits::Log2Ceiling64(max_buffer_size_scaled) + 1);
   int num_local_disks = FLAGS_num_disks == 0 ? DiskInfo::num_disks() : FLAGS_num_disks;
   disk_queues_.resize(num_local_disks + REMOTE_NUM_DISKS);
   CheckSseSupport();
@@ -288,7 +290,7 @@ DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_disk, int min_buffer_s
     file_handle_cache_(min(FLAGS_max_cached_file_handles,
             FileSystemUtil::MaxNumFileHandles()), &HdfsCachedFileHandle::Release) {
   int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2(max_buffer_size_scaled) + 1);
+  free_buffers_.resize(Bits::Log2Ceiling64(max_buffer_size_scaled) + 1);
   if (num_local_disks == 0) num_local_disks = DiskInfo::num_disks();
   disk_queues_.resize(num_local_disks + REMOTE_NUM_DISKS);
   CheckSseSupport();
@@ -1129,7 +1131,7 @@ Status DiskIoMgr::WriteRangeHelper(FILE* file_handle, WriteRange* write_range) {
 
 int DiskIoMgr::free_buffers_idx(int64_t buffer_size) {
   int64_t buffer_size_scaled = BitUtil::Ceil(buffer_size, min_buffer_size_);
-  int idx = BitUtil::Log2(buffer_size_scaled);
+  int idx = Bits::Log2Ceiling64(buffer_size_scaled);
   DCHECK_GE(idx, 0);
   DCHECK_LT(idx, free_buffers_.size());
   return idx;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/free-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/free-pool.h b/be/src/runtime/free-pool.h
index 2985ecb..dfabaf0 100644
--- a/be/src/runtime/free-pool.h
+++ b/be/src/runtime/free-pool.h
@@ -20,6 +20,7 @@
 #include <string.h>
 #include <string>
 #include "common/logging.h"
+#include "gutil/bits.h"
 #include "runtime/mem-pool.h"
 #include "util/bit-util.h"
 
@@ -66,8 +67,7 @@ class FreePool {
     /// This is the typical malloc behavior. NULL is reserved for failures.
     if (size == 0) return reinterpret_cast<uint8_t*>(0x1);
 
-    /// Do ceil(log_2(size))
-    int free_list_idx = BitUtil::Log2(size);
+    int free_list_idx = Bits::Log2Ceiling64(size);
     DCHECK_LT(free_list_idx, NUM_LISTS);
 
     FreeListNode* allocation = lists_[free_list_idx].next;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/runtime-filter.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter.cc b/be/src/runtime/runtime-filter.cc
index 675e0f3..a68eb69 100644
--- a/be/src/runtime/runtime-filter.cc
+++ b/be/src/runtime/runtime-filter.cc
@@ -14,9 +14,9 @@
 
 #include "runtime/runtime-filter.inline.h"
 
-#include <gutil/strings/substitute.h>
-
 #include "common/names.h"
+#include "gutil/bits.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/client-cache.h"
 #include "runtime/exec-env.h"
 #include "service/impala-server.h"
@@ -43,7 +43,7 @@ RuntimeFilterBank::RuntimeFilterBank(const TQueryCtx& query_ctx, RuntimeState* s
   int32_t bloom_filter_size = query_ctx_.request.query_options.runtime_bloom_filter_size;
   bloom_filter_size = std::max(bloom_filter_size, MIN_BLOOM_FILTER_SIZE);
   bloom_filter_size = std::min(bloom_filter_size, MAX_BLOOM_FILTER_SIZE);
-  log_filter_size_ = BitUtil::Log2(bloom_filter_size);
+  log_filter_size_ = Bits::Log2Ceiling64(bloom_filter_size);
 }
 
 RuntimeFilter* RuntimeFilterBank::RegisterFilter(const TRuntimeFilterDesc& filter_desc,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util-test.cc b/be/src/util/bit-util-test.cc
index 9886429..f376737 100644
--- a/be/src/util/bit-util-test.cc
+++ b/be/src/util/bit-util-test.cc
@@ -19,6 +19,8 @@
 
 #include <boost/utility.hpp>
 #include <gtest/gtest.h>
+
+#include "gutil/bits.h"
 #include "util/bit-util.h"
 #include "util/cpu-info.h"
 
@@ -99,7 +101,7 @@ TEST(BitUtil, ByteSwap) {
   EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint64_t>(0)), 0);
   EXPECT_EQ(BitUtil::ByteSwap(
       static_cast<uint64_t>(0x1122334455667788)), 0x8877665544332211);
-  
+
   EXPECT_EQ(BitUtil::ByteSwap(static_cast<int64_t>(0)), 0);
   EXPECT_EQ(BitUtil::ByteSwap(
       static_cast<int64_t>(0x1122334455667788)), 0x8877665544332211);
@@ -112,14 +114,16 @@ TEST(BitUtil, ByteSwap) {
 }
 
 TEST(BitUtil, Log2) {
-  EXPECT_EQ(BitUtil::Log2(1), 0);
-  EXPECT_EQ(BitUtil::Log2(2), 1);
-  EXPECT_EQ(BitUtil::Log2(3), 2);
-  EXPECT_EQ(BitUtil::Log2(4), 2);
-  EXPECT_EQ(BitUtil::Log2(5), 3);
-  EXPECT_EQ(BitUtil::Log2(INT_MAX), 31);
-  EXPECT_EQ(BitUtil::Log2(UINT_MAX), 32);
-  EXPECT_EQ(BitUtil::Log2(ULLONG_MAX), 64);
+  // We use gutil's implementation in place of an older custom implementation in BitUtil.
+  // We leave this test here to ensure no test coverage is lost.
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(1), 0);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(2), 1);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(3), 2);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(4), 2);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(5), 3);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(INT_MAX), 31);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(UINT_MAX), 32);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(ULLONG_MAX), 64);
 }
 
 TEST(BitUtil, RoundUpToPowerOf2) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 0031558..e255f0c 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -144,22 +144,6 @@ class BitUtil {
     return (v << n) >> n;
   }
 
-  /// Returns ceil(log2(x)).
-  /// TODO: this could be faster if we use __builtin_clz.  Fix this if this ever shows up
-  /// in a hot path.
-  static inline int Log2(uint64_t x) {
-    DCHECK_GT(x, 0);
-    if (x == 1) return 0;
-    // Compute result = ceil(log2(x))
-    //                = floor(log2(x - 1)) + 1, for x > 1
-    // by finding the position of the most significant bit (1-indexed) of x - 1
-    // (floor(log2(n)) = MSB(n) (0-indexed))
-    --x;
-    int result = 1;
-    while (x >>= 1) ++result;
-    return result;
-  }
-
   /// Swaps the byte order (i.e. endianess)
   static inline int64_t ByteSwap(int64_t value) {
     return __builtin_bswap64(value);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bloom-filter.h
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index 32f6ee6..6fdd140 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -109,8 +109,8 @@ class BloomFilter {
   typedef uint64_t BucketWord;
 
   // log2(number of bits in a BucketWord)
-  // TODO: Use BitUtil::Log2(numeric_limits<BucketWord>::digits) once we enable C++14 for
-  // codegen.
+  // TODO: Use Bits::Log2Ceiling64(numeric_limits<BucketWord>::digits) once we enable
+  // C++14 for codegen.
   static const int LOG_BUCKET_WORD_BITS = 6;
   static const BucketWord BUCKET_WORD_MASK = 63; // 2^LOG_BUCKET_WORD_BITS - 1
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/dict-encoding.h
----------------------------------------------------------------------
diff --git a/be/src/util/dict-encoding.h b/be/src/util/dict-encoding.h
index e629f33..75e89ac 100644
--- a/be/src/util/dict-encoding.h
+++ b/be/src/util/dict-encoding.h
@@ -20,6 +20,7 @@
 #include <boost/foreach.hpp>
 #include <boost/unordered_map.hpp>
 
+#include "gutil/bits.h"
 #include "exec/parquet-common.h"
 #include "runtime/mem-pool.h"
 #include "runtime/string-value.h"
@@ -70,7 +71,7 @@ class DictEncoderBase {
   int bit_width() const {
     if (UNLIKELY(num_entries() == 0)) return 0;
     if (UNLIKELY(num_entries() == 1)) return 1;
-    return BitUtil::Log2(num_entries());
+    return Bits::Log2Ceiling64(num_entries());
   }
 
   /// Writes out any buffered indices to buffer preceded by the bit width of this data.


[20/50] incubator-impala git commit: Fix start-impala-cluster.py with -s 0.

Posted by ta...@apache.org.
Fix start-impala-cluster.py with -s 0.

Bug: Division by zero.

Starting only the statestored and catalogd can be
useful for debugging purposes. For example, it is
often convenient to start a single customized impalad
with start-impalad.sh, but that requires having
the statestored and catalogd already up.

Change-Id: I9abe40de6c6caea26b6faa03b7495f25cb07e0ac
Reviewed-on: http://gerrit.cloudera.org:8080/2666
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 51eef3a3ad09f97a4c319e635dfa068f39d2e92a
Parents: ce92844
Author: Alex Behm <al...@cloudera.com>
Authored: Wed Mar 30 11:43:26 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 31 03:18:44 2016 +0000

----------------------------------------------------------------------
 bin/start-impala-cluster.py | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/51eef3a3/bin/start-impala-cluster.py
----------------------------------------------------------------------
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index 1302a66..9c1f75f 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -212,6 +212,10 @@ def build_rm_args(instance_num):
   return RM_ARGS % (llama_address, cgroup_path, fs_cfg_path)
 
 def start_impalad_instances(cluster_size):
+  if cluster_size == 0:
+    # No impalad instances should be started.
+    return
+
   # The default memory limit for an impalad is 80% of the total system memory. On a
   # mini-cluster with 3 impalads that means 240%. Since having an impalad be OOM killed
   # is very annoying, the mem limit will be reduced. This can be overridden using the


[49/50] incubator-impala git commit: IMPALA-3302: Fix ASAN use-after-free failure in Kudu scanner

Posted by ta...@apache.org.
IMPALA-3302: Fix ASAN use-after-free failure in Kudu scanner

When fetching VARCHAR data from Kudu, Impala was using a pointer that
was owned and freed by the Kudu client. There was already a mechanism to
copy string data but it was only applied to STRING columns. The fix is
to apply the same mechanism to VARCHAR columns. That's done by adding
the VARCHAR column to the list of string type columns/slots that need to
be copied.

Change-Id: Ib513aae531086355e77ac9c35ed38728efaa0be9
Reviewed-on: http://gerrit.cloudera.org:8080/2749
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 62bb864704d5838726095bf90aeec1ba2c0ef881
Parents: 36c294b
Author: casey <ca...@cloudera.com>
Authored: Sun Apr 10 23:20:55 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 be/src/exec/kudu-scanner.cc | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/62bb8647/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index a89a7db..f0d0f57 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -88,7 +88,7 @@ Status KuduScanner::Open() {
   // Store columns that need relocation when materialized into the
   // destination row batch.
   for (int i = 0; i < scan_node_->tuple_desc_->slots().size(); ++i) {
-    if (scan_node_->tuple_desc_->slots()[i]->type().type == TYPE_STRING) {
+    if (scan_node_->tuple_desc_->slots()[i]->type().IsStringType()) {
       string_slots_.push_back(scan_node_->tuple_desc_->slots()[i]);
     }
   }
@@ -202,10 +202,12 @@ Status KuduScanner::DecodeRowsIntoRowBatch(RowBatch* row_batch,
     Tuple** tuple_mem, bool* batch_done) {
 
   // Short-circuit the count(*) case.
-  if (scan_node_->tuple_desc_->slots().empty()) return HandleEmptyProjection(row_batch, batch_done);
+  if (scan_node_->tuple_desc_->slots().empty()) {
+    return HandleEmptyProjection(row_batch, batch_done);
+  }
 
-  // TODO consider consolidating the tuple creation/initialization here with the version that
-  // happens inside the loop.
+  // TODO consider consolidating the tuple creation/initialization here with the version
+  // that happens inside the loop.
   int idx = row_batch->AddRow();
   TupleRow* row = row_batch->GetRow(idx);
   (*tuple_mem)->Init(scan_node_->tuple_desc()->num_null_bytes());
@@ -265,7 +267,7 @@ Status KuduScanner::RelocateValuesFromKudu(Tuple* tuple, MemPool* mem_pool) {
 
     // Extract the string value.
     void* slot_ptr = tuple->GetSlot(slot->tuple_offset());
-    DCHECK(slot->type().IsStringType());
+    DCHECK(slot->type().IsVarLenStringType());
 
     // The string value of the slot has a pointer to memory from the Kudu row.
     StringValue* val = reinterpret_cast<StringValue*>(slot_ptr);


[11/50] incubator-impala git commit: Print last 50 lines of log if data loading fails.

Posted by ta...@apache.org.
Print last 50 lines of log if data loading fails.

The 20 lines we dump currently are often not enough to
diagnose a failure quickly. Increasing to 50 lines.

Printing 50 lines is also consistent with our run-step
script which also prints 50 lines.

Change-Id: I353a2030be6fad1cd63879b4717e237344f85c73
Reviewed-on: http://gerrit.cloudera.org:8080/2632
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: b2ccb17c21b32856f8f23de1e0df1c5c800c9e0b
Parents: 5e51f1c
Author: Alex Behm <al...@cloudera.com>
Authored: Fri Mar 25 13:34:15 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Mon Mar 28 20:22:18 2016 +0000

----------------------------------------------------------------------
 testdata/bin/create-load-data.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b2ccb17c/testdata/bin/create-load-data.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index 6323e62..eace01e 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -146,7 +146,7 @@ function load-data {
   # Use unbuffered logging by executing with -u
   if ! impala-python -u ${IMPALA_HOME}/bin/load-data.py ${ARGS[@]} &> ${LOG_FILE}; then
     echo Error loading data. The end of the log file is:
-    tail -n 20 $LOG_FILE
+    tail -n 50 $LOG_FILE
     return 1
   fi
 }


[21/50] incubator-impala git commit: IMPALA-3274: Disable Kudu when not using the mini-cluster for testing

Posted by ta...@apache.org.
IMPALA-3274: Disable Kudu when not using the mini-cluster for testing

This is to unblock the builds.

Change-Id: I8281f0cf75facc1c04d1bf1e729fab2adf45a804
Reviewed-on: http://gerrit.cloudera.org:8080/2673
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 2809746b2c2f8621b6ed9c75410f41dd8253e255
Parents: 51eef3a
Author: Casey Ching <ca...@cloudera.com>
Authored: Wed Mar 30 17:07:20 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Thu Mar 31 11:30:44 2016 +0000

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


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2809746b/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 9831490..d8b1dde 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -180,6 +180,13 @@ elif [ "${TARGET_FILESYSTEM}" != "hdfs" ]; then
   return 1
 fi
 
+if [[ "$TARGET_FILESYSTEM" != "hdfs" ]]; then
+  # TODO: Kudu is disabled to unblock builds. More work is needed to re-enable it for
+  #       non-mini-cluster testing. The problem now is nothing starts the Kudu service
+  #       in the non-mini-cluster mode.
+  KUDU_IS_SUPPORTED=false
+fi
+
 # Directories where local cluster logs will go when running tests or loading data
 export IMPALA_LOGS_DIR=${IMPALA_HOME}/logs
 export IMPALA_CLUSTER_LOGS_DIR=${IMPALA_LOGS_DIR}/cluster


[16/50] incubator-impala git commit: IMPALA-2650: UDF EE tests: use unique databases in some tests

Posted by ta...@apache.org.
IMPALA-2650: UDF EE tests: use unique databases in some tests

Some of the end-to-end tests in query_test/test_udfs.py create UDFs in
the default database and leave them there. Other tests (e.g.,
test_functions_ddl) polling the default database and expecting to find
no UDFs will fail. It turns out this wouldn't happen in our Jenkins
builds (see IMPALA-2650 for more details as to why), but it manifests
itself with repeated impala-py.test runs in specific order.

The fix is to create the UDFs in databases unique to the test cases.
This leaves the default database pristine during these tests.

Testing:

Before, the following sequence of impala-py.test commands would cause
any subsequent runs of test_functions_ddl to fail:

$ # simulate a subset of serial tests that expect default DB not to have UDFs
$ impala-py.test -m "execute_serially" --workload_exploration_strategy \
    functional-query:exhaustive -k test_functions_ddl metadata/test_ddl.py
PASS
$ # simulate a subset of parallel tests that create UDFs in default DB
$ impala-py.test -n4 -m "not execute_serially" --workload_exploration_strategy \
    functional-query:exhaustive query_test/test_udfs.py
PASS
$ # rerun a subset of serial tests that passed before
$ impala-py.test -m "execute_serially" --workload_exploration_strategy \
    functional-query:exhaustive -k test_functions_ddl metadata/test_ddl.py
FAIL, because test_udfs left UDFs.

Now, I can run these over and over, and they pass.

Change-Id: Id4a8b4764fa310efaa4f6c6f06f64a4e18e44173
Reviewed-on: http://gerrit.cloudera.org:8080/2610
Reviewed-by: Michael Brown <mi...@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/b74e57a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/b74e57a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/b74e57a3

Branch: refs/heads/master
Commit: b74e57a31201c6ba3f2b1f62a8c16cdb3c1c9279
Parents: 943b7cc
Author: Michael Brown <mi...@cloudera.com>
Authored: Wed Mar 23 11:28:00 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 30 04:50:15 2016 +0000

----------------------------------------------------------------------
 tests/query_test/test_udfs.py | 69 +++++++++++++++++++++-----------------
 1 file changed, 38 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b74e57a3/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 5573ffc..a979886 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -88,16 +88,19 @@ class TestUdfs(ImpalaTestSuite):
       self.client.execute("drop database if exists udf_test cascade")
 
   @SkipIfLocal.multiple_impalad
-  def test_hive_udfs_missing_jar(self, vector):
+  def test_hive_udfs_missing_jar(self, vector, unique_database):
     """ IMPALA-2365: Impalad shouldn't crash if the udf jar isn't present
     on HDFS"""
     # Copy hive-exec.jar to a temporary file
     jar_path = get_fs_path("/test-warehouse/" + get_random_id(5) + ".jar")
     hive_jar = get_fs_path("/test-warehouse/hive-exec.jar")
     check_call(["hadoop", "fs", "-cp", hive_jar, jar_path])
-    drop_fn_stmt = "drop function if exists default.pi_missing_jar()"
-    create_fn_stmt = "create function default.pi_missing_jar() returns double \
-        location '%s' symbol='org.apache.hadoop.hive.ql.udf.UDFPI'" % jar_path
+    drop_fn_stmt = (
+        "drop function if exists "
+        "`{0}`.`pi_missing_jar`()".format(unique_database))
+    create_fn_stmt = (
+        "create function `{0}`.`pi_missing_jar`() returns double location '{1}' "
+        "symbol='org.apache.hadoop.hive.ql.udf.UDFPI'".format(unique_database, jar_path))
 
     cluster = ImpalaCluster()
     impalad = cluster.get_any_impalad()
@@ -118,8 +121,8 @@ class TestUdfs(ImpalaTestSuite):
     # we used to create the function. This is to bypass loading from
     # the cache
     try:
-      self.execute_query_using_client(client,
-          "select default.pi_missing_jar()", vector)
+      self.execute_query_using_client(
+          client, "select `{0}`.`pi_missing_jar`()".format(unique_database), vector)
       assert False, "Query expected to fail"
     except ImpalaBeeswaxException, e:
       assert "Failed to get file info" in str(e)
@@ -127,22 +130,24 @@ class TestUdfs(ImpalaTestSuite):
   def test_libs_with_same_filenames(self, vector):
     self.run_test_case('QueryTest/libs_with_same_filenames', vector)
 
-  def test_udf_update_via_drop(self, vector):
+  def test_udf_update_via_drop(self, vector, unique_database):
     """Test updating the UDF binary without restarting Impala. Dropping
     the function should remove the binary from the local cache."""
     # Run with sync_ddl to guarantee the drop is processed by all impalads.
     exec_options = vector.get_value('exec_option')
     exec_options['sync_ddl'] = 1
-    old_udf = os.path.join(os.environ['IMPALA_HOME'],
-        'testdata/udfs/impala-hive-udfs.jar')
-    new_udf = os.path.join(os.environ['IMPALA_HOME'],
-        'tests/test-hive-udfs/target/test-hive-udfs-1.0.jar')
+    old_udf = os.path.join(
+        os.environ['IMPALA_HOME'], 'testdata/udfs/impala-hive-udfs.jar')
+    new_udf = os.path.join(
+        os.environ['IMPALA_HOME'], 'tests/test-hive-udfs/target/test-hive-udfs-1.0.jar')
     udf_dst = get_fs_path('/test-warehouse/impala-hive-udfs2.jar')
 
-    drop_fn_stmt = 'drop function if exists default.udf_update_test_drop()'
-    create_fn_stmt = "create function default.udf_update_test_drop() returns string "\
-        "LOCATION '" + udf_dst + "' SYMBOL='com.cloudera.impala.TestUpdateUdf'"
-    query_stmt = "select default.udf_update_test_drop()"
+    drop_fn_stmt = (
+        'drop function if exists `{0}`.`udf_update_test_drop`()'.format(unique_database))
+    create_fn_stmt = (
+        "create function `{0}`.`udf_update_test_drop`() returns string LOCATION '{1}' "
+        "SYMBOL='com.cloudera.impala.TestUpdateUdf'".format(unique_database, udf_dst))
+    query_stmt = "select `{0}`.`udf_update_test_drop`()".format(unique_database)
 
     # Put the old UDF binary on HDFS, make the UDF in Impala and run it.
     check_call(["hadoop", "fs", "-put", "-f", old_udf, udf_dst])
@@ -157,48 +162,50 @@ class TestUdfs(ImpalaTestSuite):
     self.execute_query_expect_success(self.client, create_fn_stmt, exec_options)
     self.__run_query_all_impalads(exec_options, query_stmt, ["New UDF"])
 
-  def test_udf_update_via_create(self, vector):
+  def test_udf_update_via_create(self, vector, unique_database):
     """Test updating the UDF binary without restarting Impala. Creating a new function
     from the library should refresh the cache."""
     # Run with sync_ddl to guarantee the create is processed by all impalads.
     exec_options = vector.get_value('exec_option')
     exec_options['sync_ddl'] = 1
-    old_udf = os.path.join(os.environ['IMPALA_HOME'],
-        'testdata/udfs/impala-hive-udfs.jar')
-    new_udf = os.path.join(os.environ['IMPALA_HOME'],
-        'tests/test-hive-udfs/target/test-hive-udfs-1.0.jar')
+    old_udf = os.path.join(
+        os.environ['IMPALA_HOME'], 'testdata/udfs/impala-hive-udfs.jar')
+    new_udf = os.path.join(
+        os.environ['IMPALA_HOME'], 'tests/test-hive-udfs/target/test-hive-udfs-1.0.jar')
     udf_dst = get_fs_path('/test-warehouse/impala-hive-udfs3.jar')
     old_function_name = "udf_update_test_create1"
     new_function_name = "udf_update_test_create2"
 
-    drop_fn_template = 'drop function if exists default.%s()'
+    drop_fn_template = 'drop function if exists `{0}`.`{{0}}`()'.format(unique_database)
     self.execute_query_expect_success(
-      self.client, drop_fn_template % old_function_name, exec_options)
+        self.client, drop_fn_template.format(old_function_name), exec_options)
     self.execute_query_expect_success(
-      self.client, drop_fn_template % new_function_name, exec_options)
+        self.client, drop_fn_template.format(new_function_name), exec_options)
 
-    create_fn_template = "create function default.%s() returns string "\
-        "LOCATION '" + udf_dst + "' SYMBOL='com.cloudera.impala.TestUpdateUdf'"
-    query_template = "select default.%s()"
+    create_fn_template = (
+        "create function `{0}`.`{{0}}`() returns string LOCATION '{1}' "
+        "SYMBOL='com.cloudera.impala.TestUpdateUdf'".format(unique_database, udf_dst))
+
+    query_template = "select `{0}`.`{{0}}`()".format(unique_database)
 
     # Put the old UDF binary on HDFS, make the UDF in Impala and run it.
     check_call(["hadoop", "fs", "-put", "-f", old_udf, udf_dst])
     self.execute_query_expect_success(
-      self.client, create_fn_template % old_function_name, exec_options)
+        self.client, create_fn_template.format(old_function_name), exec_options)
     self.__run_query_all_impalads(
-      exec_options, query_template % old_function_name, ["Old UDF"])
+        exec_options, query_template.format(old_function_name), ["Old UDF"])
 
     # Update the binary, and create a new function using the binary. The new binary
     # should be running.
     check_call(["hadoop", "fs", "-put", "-f", new_udf, udf_dst])
     self.execute_query_expect_success(
-      self.client, create_fn_template % new_function_name, exec_options)
+        self.client, create_fn_template.format(new_function_name), exec_options)
     self.__run_query_all_impalads(
-      exec_options, query_template % new_function_name, ["New UDF"])
+        exec_options, query_template.format(new_function_name), ["New UDF"])
 
     # The old function should use the new library now
     self.__run_query_all_impalads(
-      exec_options, query_template % old_function_name, ["New UDF"])
+        exec_options, query_template.format(old_function_name), ["New UDF"])
 
   def test_drop_function_while_running(self, vector):
     self.client.execute("drop function if exists default.drop_while_running(BIGINT)")


[08/50] incubator-impala git commit: deploy: allow overriding of IMPALA_KUDU_PARCEL_URL

Posted by ta...@apache.org.
deploy: allow overriding of IMPALA_KUDU_PARCEL_URL

This is useful when testing an IMPALA_KUDU parcel in an on-demand
Jenkins repo.

Change-Id: I93f86f171e4cc6611d9714d7c58d943f296e43ff
Reviewed-on: http://gerrit.cloudera.org:8080/2634
Reviewed-by: Casey Ching <ca...@cloudera.com>
Tested-by: Casey Ching <ca...@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/35dfbafa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/35dfbafa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/35dfbafa

Branch: refs/heads/master
Commit: 35dfbafab6f83d7197248a9264fdcac805bdbc30
Parents: 76b6748
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Mar 25 15:33:12 2016 -0700
Committer: Casey Ching <ca...@cloudera.com>
Committed: Fri Mar 25 23:05:52 2016 +0000

----------------------------------------------------------------------
 infra/deploy/deploy.py | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/35dfbafa/infra/deploy/deploy.py
----------------------------------------------------------------------
diff --git a/infra/deploy/deploy.py b/infra/deploy/deploy.py
index 322ca52..bf36e07 100644
--- a/infra/deploy/deploy.py
+++ b/infra/deploy/deploy.py
@@ -30,12 +30,14 @@
 
 import argparse
 import hashlib
+import os
 import re
 import time
 
 from cm_api.api_client import ApiResource
 
-IMPALA_KUDU_PARCEL_URL = "http://archive.cloudera.com/beta/impala-kudu/parcels/latest"
+IMPALA_KUDU_PARCEL_URL = os.getenv("IMPALA_KUDU_PARCEL_URL",
+                                   "http://archive.cloudera.com/beta/impala-kudu/parcels/latest")
 IMPALA_KUDU_PARCEL_PRODUCT = "IMPALA_KUDU"
 MAX_PARCEL_REPO_WAIT_SECS = 60
 MAX_PARCEL_WAIT_SECS = 60 * 30


[17/50] incubator-impala git commit: IMPALA-3182: Reenable -Werror for kudu files

Posted by ta...@apache.org.
IMPALA-3182: Reenable -Werror for kudu files

Now the macros DISALLOW_COPY_AND_ASSIGN and COMPILE_ASSERT are potentially
defined by either the Kudu client or gutil. Either definition should be
fine. This change adds an "ifndef" around gutil's definitions to avoid
warning about redefining the macros. Kudu already has conditional
ifdefs.

Change-Id: I87c84b667c56dacc369bf9a2ff6550bb59b4fea9
Reviewed-on: http://gerrit.cloudera.org:8080/2660
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 2d0b944500f6a44fcf95ea5b519393bec0dc749c
Parents: b74e57a
Author: casey <ca...@cloudera.com>
Authored: Tue Mar 29 14:49:31 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Mar 30 06:18:23 2016 +0000

----------------------------------------------------------------------
 be/src/exec/CMakeLists.txt | 19 -------------------
 be/src/gutil/macros.h      |  4 ++++
 2 files changed, 4 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2d0b9445/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 002d38f..af86306 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -84,21 +84,6 @@ add_library(Exec
   unnest-node.cc
 )
 
-# Some of the macros defined in gutil (COMPILE_ASSERT, DISALLOW_COPY_AND_ASSIGN)
-# are redefined in Kudu's kudu/client/stubs.h and because "-Werror" is set, compilation
-# fails on macro redefinition warnings. The ideal workaround would be to just disable
-# that particular warning (which is possible in recent LLVM releases with
-# "-Wno-macro-redefined") but gcc doesn't have a flag to do it, so we simply disable
-# "-Werror" for these particular files.
-set_source_files_properties(kudu-scanner.cc PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")
-set_source_files_properties(kudu-scan-node.cc PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")
-set_source_files_properties(kudu-table-sink.cc PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")
-set_source_files_properties(kudu-util.cc PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")
-
 add_dependencies(Exec thrift-deps)
 
 ADD_BE_TEST(zigzag-test)
@@ -112,7 +97,3 @@ ADD_BE_TEST(row-batch-list-test)
 ADD_BE_TEST(incr-stats-util-test)
 ADD_BE_TEST(kudu-scan-node-test)
 ADD_BE_TEST(kudu-table-sink-test)
-SET_TARGET_PROPERTIES(kudu-scan-node-test PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")
-SET_TARGET_PROPERTIES(kudu-table-sink-test PROPERTIES COMPILE_FLAGS
-  "${CXX_COMMON_FLAGS} -Wno-error")

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/2d0b9445/be/src/gutil/macros.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/macros.h b/be/src/gutil/macros.h
index 9de3c2c..ca0daea 100644
--- a/be/src/gutil/macros.h
+++ b/be/src/gutil/macros.h
@@ -39,8 +39,10 @@ template <bool>
 struct CompileAssert {
 };
 
+#ifndef COMPILE_ASSERT
 #define COMPILE_ASSERT(expr, msg) \
   typedef CompileAssert<(bool(expr))> msg[bool(expr) ? 1 : -1]
+#endif
 
 // Implementation details of COMPILE_ASSERT:
 //
@@ -98,6 +100,7 @@ struct CompileAssert {
 // http://gcc.gnu.org/PR51213 in gcc-4.7 / Crosstool v16.
 // TODO(user): Remove "&& !defined(__clang_)" when =delete is
 // gcc-4.7 before =delete is allowed, go back to the C++98 definition.
+#ifndef DISALLOW_COPY_AND_ASSIGN
 #if LANG_CXX11 && !defined(__clang__)
 #define DISALLOW_COPY_AND_ASSIGN(TypeName) \
   TypeName(const TypeName&) = delete;      \
@@ -107,6 +110,7 @@ struct CompileAssert {
   TypeName(const TypeName&);               \
   void operator=(const TypeName&)
 #endif
+#endif
 
 // An older, politically incorrect name for the above.
 // Prefer DISALLOW_COPY_AND_ASSIGN for new code.


[13/50] incubator-impala git commit: Re-enable Kudu in build using client stubs when needed

Posted by ta...@apache.org.
Re-enable Kudu in build using client stubs when needed

The stubs in Impala broke during the merge commit. This commit removes
the stubs in hopes of improving robustness of the build. The original
problem (Kudu clients are only available for some OSs) is now addressed
by moving the stubbing into a dummy Kudu client. The dummy client only
allows linking to succeed, if any client method is called, Impala will
crash. Before calling any such method, Kudu availability must be
checked.

Change-Id: I4bf1c964faf21722137adc4f7ba7f78654f0f712
Reviewed-on: http://gerrit.cloudera.org:8080/2585
Reviewed-by: Casey Ching <ca...@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/39a28185
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/39a28185
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/39a28185

Branch: refs/heads/master
Commit: 39a28185e813e05a2b8a8afd8215754adb6ab327
Parents: 2b47c5a
Author: Casey Ching <ca...@cloudera.com>
Authored: Fri Mar 18 13:48:31 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Mar 29 23:57:54 2016 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  | 27 ++++---
 be/CMakeLists.txt                               |  4 +-
 be/src/common/global-flags.cc                   |  2 +
 be/src/common/init.cc                           |  2 +-
 be/src/exec/CMakeLists.txt                      | 14 ++--
 be/src/exec/data-sink.cc                        |  2 +
 be/src/exec/exec-node.cc                        |  2 +
 be/src/exec/kudu-scan-node-test.cc              |  5 +-
 be/src/exec/kudu-scan-node.cc                   |  5 +-
 be/src/exec/kudu-scan-node.h                    | 30 --------
 be/src/exec/kudu-scanner.cc                     |  4 -
 be/src/exec/kudu-table-sink-test.cc             |  5 +-
 be/src/exec/kudu-table-sink.cc                  |  5 +-
 be/src/exec/kudu-table-sink.h                   | 34 ---------
 be/src/exec/kudu-util.cc                        | 28 ++++++-
 be/src/exec/kudu-util.h                         | 22 +++---
 bin/bootstrap_toolchain.py                      | 28 ++-----
 bin/impala-config.sh                            |  5 +-
 common/thrift/generate_error_codes.py           |  5 +-
 testdata/cluster/admin                          |  1 +
 .../queries/QueryTest/kudu_partition_ddl.test   | 80 ++++++++++----------
 tests/common/impala_test_suite.py               |  5 +-
 tests/custom_cluster/test_kudu_not_available.py | 60 +++++++++++++++
 tests/query_test/test_kudu.py                   |  3 +-
 24 files changed, 187 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 71256c7..f1a8e01 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -307,18 +307,23 @@ message(STATUS "Avro include dir: " ${AVRO_INCLUDE_DIR})
 message(STATUS "Avro static library: " ${AVRO_STATIC_LIB})
 
 if($ENV{KUDU_IS_SUPPORTED} STREQUAL true)
-  add_definitions(-DUSE_KUDU)
-  # KuduClient can use GLOG
-  add_definitions(-DKUDU_HEADERS_USE_GLOG)
-  if(NOT $ENV{KUDU_CLIENT_DIR} EQUAL "")
-    set(kuduClient_DIR "$ENV{KUDU_CLIENT_DIR}/usr/local/share/kuduClient/cmake")
-  else()
-    set(kuduClient_DIR
-        "$ENV{IMPALA_TOOLCHAIN}/kudu-$ENV{IMPALA_KUDU_VERSION}/share/kuduClient/cmake")
-  endif()
-  find_package(kuduClient REQUIRED)
-  include_directories(${KUDU_CLIENT_INCLUDE_DIR})
+  # This can be removed after some toolchain work to improve the Kudu client stub
+  # library. That work involves overriding a method added to the 0.8 client. So
+  # we wait for that.
+  add_definitions(-DKUDU_CLIENT_SUPPORTED)
+endif()
+# KuduClient can use GLOG
+add_definitions(-DKUDU_HEADERS_USE_GLOG)
+if(NOT $ENV{KUDU_CLIENT_DIR} EQUAL "")
+  set(kuduClient_DIR "$ENV{KUDU_CLIENT_DIR}/usr/local/share/kuduClient/cmake")
+else()
+  set(kuduClient_DIR
+      "$ENV{IMPALA_TOOLCHAIN}/kudu-$ENV{IMPALA_KUDU_VERSION}/share/kuduClient/cmake")
 endif()
+# When KUDU_IS_SUPPORTED is false, the Kudu client is expected to be a non-functional
+# stub. It's still needed to link though.
+find_package(kuduClient REQUIRED)
+include_directories(SYSTEM ${KUDU_CLIENT_INCLUDE_DIR})
 
 # find jni headers and libs
 find_package(JNI REQUIRED)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 64328d0..c866f84 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -326,11 +326,9 @@ set (IMPALA_LINK_LIBS ${IMPALA_LINK_LIBS}
   ${LIBBZ2}
   ${AVRO_STATIC_LIB}
   ${JAVA_JVM_LIBRARY}
+  kudu_client
   -lrt
 )
-if($ENV{KUDU_IS_SUPPORTED} STREQUAL true)
-  list(APPEND IMPALA_LINK_LIBS kudu_client)
-endif()
 
 # The above link list does not include tcmalloc. This is because the Impala JVM support
 # libraries (libfesupport, libloggingsupport) cannot use tcmalloc in all cases. When they

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/common/global-flags.cc
----------------------------------------------------------------------
diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index b63abca..13f81fa 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -75,3 +75,5 @@ DEFINE_int32(stress_free_pool_alloc, 0, "A stress option which causes memory all
 DEFINE_int32(stress_datastream_recvr_delay_ms, 0, "A stress option that causes data "
     "stream receiver registration to be delayed. Effective in debug builds only.");
 #endif
+
+DEFINE_bool(disable_kudu, false, "If true, Kudu features will be disabled.");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index 0f0de47..645f8a2 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -221,7 +221,7 @@ void impala::InitCommonRuntime(int argc, char** argv, bool init_jvm,
     Expr::InitBuiltinsDummy();
   }
 
-  impala::InitKuduLogging();
+  if (impala::KuduIsAvailable()) impala::InitKuduLogging();
 
 #ifndef ADDRESS_SANITIZER
   // tcmalloc and address sanitizer can not be used together

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 110eecb..002d38f 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -110,11 +110,9 @@ ADD_BE_TEST(parquet-plain-test)
 ADD_BE_TEST(parquet-version-test)
 ADD_BE_TEST(row-batch-list-test)
 ADD_BE_TEST(incr-stats-util-test)
-if($ENV{KUDU_IS_SUPPORTED} STREQUAL true)
-  ADD_BE_TEST(kudu-scan-node-test)
-  ADD_BE_TEST(kudu-table-sink-test)
-  SET_TARGET_PROPERTIES(kudu-scan-node-test PROPERTIES COMPILE_FLAGS
-    "${CXX_COMMON_FLAGS} -Wno-error")
-  SET_TARGET_PROPERTIES(kudu-table-sink-test PROPERTIES COMPILE_FLAGS
-    "${CXX_COMMON_FLAGS} -Wno-error")
-endif()
+ADD_BE_TEST(kudu-scan-node-test)
+ADD_BE_TEST(kudu-table-sink-test)
+SET_TARGET_PROPERTIES(kudu-scan-node-test PROPERTIES COMPILE_FLAGS
+  "${CXX_COMMON_FLAGS} -Wno-error")
+SET_TARGET_PROPERTIES(kudu-table-sink-test PROPERTIES COMPILE_FLAGS
+  "${CXX_COMMON_FLAGS} -Wno-error")

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/data-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index 658e295..d4ad073 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -22,6 +22,7 @@
 #include "exec/hdfs-table-sink.h"
 #include "exec/hbase-table-sink.h"
 #include "exec/kudu-table-sink.h"
+#include "exec/kudu-util.h"
 #include "exprs/expr.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gen-cpp/ImpalaInternalService_constants.h"
@@ -62,6 +63,7 @@ Status DataSink::CreateDataSink(ObjectPool* pool,
           sink->reset(tmp_sink);
           break;
         case TTableSinkType::KUDU:
+          RETURN_IF_ERROR(CheckKuduAvailability());
           tmp_sink = new KuduTableSink(row_desc, output_exprs, thrift_sink);
           sink->reset(tmp_sink);
           break;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/exec-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 6b76ad9..cac57f8 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -33,6 +33,7 @@
 #include "exec/hbase-scan-node.h"
 #include "exec/hdfs-scan-node.h"
 #include "exec/kudu-scan-node.h"
+#include "exec/kudu-util.h"
 #include "exec/nested-loop-join-node.h"
 #include "exec/partitioned-aggregation-node.h"
 #include "exec/partitioned-hash-join-node.h"
@@ -292,6 +293,7 @@ Status ExecNode::CreateNode(ObjectPool* pool, const TPlanNode& tnode,
       *node = pool->Add(new DataSourceScanNode(pool, tnode, descs));
       break;
     case TPlanNodeType::KUDU_SCAN_NODE:
+      RETURN_IF_ERROR(CheckKuduAvailability());
       *node = pool->Add(new KuduScanNode(pool, tnode, descs));
       break;
     case TPlanNodeType::AGGREGATION_NODE:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-scan-node-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node-test.cc b/be/src/exec/kudu-scan-node-test.cc
index 0f1d6af..9e1cf4c 100644
--- a/be/src/exec/kudu-scan-node-test.cc
+++ b/be/src/exec/kudu-scan-node-test.cc
@@ -12,10 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifndef USE_KUDU
-#error Kudu is required
-#endif
-
 #include "exec/kudu-testutil.h"
 
 #include "common/init.h"
@@ -601,6 +597,7 @@ TEST_F(KuduScanNodeTest, BenchmarkScanNode) {
 } // namespace impala
 
 int main(int argc, char** argv) {
+  if (!impala::KuduClientIsSupported()) return 0;
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   impala::InitFeSupport();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node.cc b/be/src/exec/kudu-scan-node.cc
index 1c2f0e5..7b99081 100644
--- a/be/src/exec/kudu-scan-node.cc
+++ b/be/src/exec/kudu-scan-node.cc
@@ -12,8 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifdef USE_KUDU
-
 #include "exec/kudu-scan-node.h"
 
 #include <boost/algorithm/string.hpp>
@@ -77,6 +75,7 @@ KuduScanNode::KuduScanNode(ObjectPool* pool, const TPlanNode& tnode,
       done_(false),
       pushable_conjuncts_(tnode.kudu_scan_node.kudu_conjuncts),
       thread_avail_cb_id_(-1) {
+  DCHECK(KuduIsAvailable());
 }
 
 KuduScanNode::~KuduScanNode() {
@@ -466,5 +465,3 @@ done:
 }
 
 }  // namespace impala
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scan-node.h b/be/src/exec/kudu-scan-node.h
index 25bc58c..329f52a 100644
--- a/be/src/exec/kudu-scan-node.h
+++ b/be/src/exec/kudu-scan-node.h
@@ -17,9 +17,7 @@
 
 #include <boost/scoped_ptr.hpp>
 #include <gtest/gtest.h>
-#ifdef USE_KUDU
 #include <kudu/client/client.h>
-#endif
 
 #include "exec/scan-node.h"
 #include "runtime/descriptors.h"
@@ -32,8 +30,6 @@ namespace impala {
 class KuduScanner;
 class Tuple;
 
-#ifdef USE_KUDU
-
 /// A scan node that scans Kudu TabletServers.
 ///
 /// This scan node takes a set of ranges and uses a Kudu client to retrieve the data
@@ -191,32 +187,6 @@ class KuduScanNode : public ScanNode {
   RuntimeProfile::Counter* kudu_round_trips() const { return kudu_round_trips_; }
 };
 
-#else // No Kudu
-
-class KuduScanNode : public ScanNode {
- public:
-  KuduScanNode(ObjectPool *pool, const TPlanNode &tnode, const DescriptorTbl &descs)
-      : ScanNode(pool, tnode, descs) {}
-
-  ~KuduScanNode() {}
-
-  virtual Status Prepare(RuntimeState *state) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual Status Open(RuntimeState *state) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual Status GetNext(RuntimeState *state, RowBatch *row_batch, bool *eos) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual void Close(RuntimeState *state) {}
-};
-
-#endif
-
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index 24dc52a..a89a7db 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -12,8 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifdef USE_KUDU
-
 #include "exec/kudu-scanner.h"
 
 #include <boost/foreach.hpp>
@@ -370,5 +368,3 @@ Status KuduScanner::GetNextBlock() {
 }
 
 }  // namespace impala
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-table-sink-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink-test.cc b/be/src/exec/kudu-table-sink-test.cc
index 49a90f2..9e49d6d 100644
--- a/be/src/exec/kudu-table-sink-test.cc
+++ b/be/src/exec/kudu-table-sink-test.cc
@@ -12,10 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifndef USE_KUDU
-#error Kudu is required
-#endif
-
 #include "exec/kudu-testutil.h"
 
 #include "common/init.h"
@@ -304,6 +300,7 @@ TEST_F(KuduTableSinkTest, DeleteModThreeTwice) {
 } // namespace impala
 
 int main(int argc, char** argv) {
+  if (!impala::KuduClientIsSupported()) return 0;
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
   impala::InitFeSupport();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.cc b/be/src/exec/kudu-table-sink.cc
index 79f8ec8..b24c6ea 100644
--- a/be/src/exec/kudu-table-sink.cc
+++ b/be/src/exec/kudu-table-sink.cc
@@ -12,8 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifdef USE_KUDU
-
 #include "exec/kudu-table-sink.h"
 
 #include <sstream>
@@ -59,6 +57,7 @@ KuduTableSink::KuduTableSink(const RowDescriptor& row_desc,
       kudu_error_counter_(NULL),
       rows_written_(NULL),
       rows_written_rate_(NULL) {
+  DCHECK(KuduIsAvailable());
 }
 
 Status KuduTableSink::PrepareExprs(RuntimeState* state) {
@@ -295,5 +294,3 @@ void KuduTableSink::Close(RuntimeState* state) {
 }
 
 }  // namespace impala
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-table-sink.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.h b/be/src/exec/kudu-table-sink.h
index cc9ff02..486cb23 100644
--- a/be/src/exec/kudu-table-sink.h
+++ b/be/src/exec/kudu-table-sink.h
@@ -16,22 +16,17 @@
 #define IMPALA_EXEC_KUDU_TABLE_SINK_H
 
 #include <boost/scoped_ptr.hpp>
-#ifdef USE_KUDU
 #include <kudu/client/client.h>
-#endif
 
 #include "gen-cpp/ImpalaInternalService_constants.h"
 #include "common/status.h"
-#ifdef USE_KUDU
 #include "exec/kudu-util.h"
-#endif
 #include "exec/data-sink.h"
 #include "exprs/expr-context.h"
 #include "exprs/expr.h"
 
 namespace impala {
 
-#ifdef USE_KUDU
 /// Sink that takes RowBatches and writes them into Kudu.
 /// Currently the data is sent to Kudu on Send(), i.e. the data is batched on the
 /// KuduSession until all the rows in a RowBatch are applied and then the session
@@ -124,35 +119,6 @@ class KuduTableSink : public DataSink {
 
 };
 
-#else  // No Kudu
-
-class KuduTableSink : public DataSink {
- public:
-  KuduTableSink(const RowDescriptor& row_desc,
-      const std::vector<TExpr>& select_list_texprs, const TDataSink& tsink) {}
-
-  virtual Status Prepare(RuntimeState* state) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual Status Open(RuntimeState* state) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual Status Send(RuntimeState* state, RowBatch* batch, bool eos) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-
-  virtual void Close(RuntimeState* state) {}
-
-  virtual RuntimeProfile* profile() { return NULL; }
-
-  virtual Status FlushFinal(RuntimeState* state) {
-    return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
-  }
-};
-
-#endif
 }  // namespace impala
 
 #endif // IMPALA_EXEC_KUDU_TABLE_SINK_H

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-util.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.cc b/be/src/exec/kudu-util.cc
index 7332de9..5a5fb00 100644
--- a/be/src/exec/kudu-util.cc
+++ b/be/src/exec/kudu-util.cc
@@ -12,8 +12,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#ifdef USE_KUDU
-
 #include "exec/kudu-util.h"
 
 #include <algorithm>
@@ -32,8 +30,31 @@ using boost::algorithm::to_lower_copy;
 using kudu::client::KuduSchema;
 using kudu::client::KuduColumnSchema;
 
+DECLARE_bool(disable_kudu);
+
 namespace impala {
 
+bool KuduClientIsSupported() {
+#ifdef KUDU_CLIENT_SUPPORTED
+  return true;
+#else
+  return false;
+#endif
+}
+
+bool KuduIsAvailable() { return CheckKuduAvailability().ok(); }
+
+Status CheckKuduAvailability() {
+  if (KuduClientIsSupported()) {
+    if (FLAGS_disable_kudu) {
+      return Status(TErrorCode::KUDU_NOT_ENABLED);
+    } else{
+      return Status::OK();
+    }
+  }
+  return Status(TErrorCode::KUDU_NOT_SUPPORTED_ON_OS);
+}
+
 Status ImpalaToKuduType(const ColumnType& impala_type,
     kudu::client::KuduColumnSchema::DataType* kudu_type) {
   using kudu::client::KuduColumnSchema;
@@ -175,11 +196,10 @@ void LogKuduMessage(void* unused, kudu::client::KuduLogSeverity severity,
 }
 
 void InitKuduLogging() {
+  DCHECK(KuduIsAvailable());
   static kudu::client::KuduLoggingFunctionCallback<void*> log_cb(&LogKuduMessage, NULL);
   kudu::client::InstallLoggingCallback(&log_cb);
   kudu::client::SetVerboseLogLevel(FLAGS_v);
 }
 
 }  // namespace impala
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/be/src/exec/kudu-util.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.h b/be/src/exec/kudu-util.h
index f408825..47debdc 100644
--- a/be/src/exec/kudu-util.h
+++ b/be/src/exec/kudu-util.h
@@ -15,23 +15,31 @@
 #ifndef IMPALA_UTIL_KUDU_UTIL_H_
 #define IMPALA_UTIL_KUDU_UTIL_H_
 
-#ifdef USE_KUDU
 #include <kudu/client/callbacks.h>
 #include <kudu/client/client.h>
-#endif
 
 #include <boost/unordered_map.hpp>
 
 namespace impala {
 
-#ifdef USE_KUDU
-
 class TExpr;
 class KuduTableDescriptor;
 class Status;
 class ColumnType;
 class TupleDescriptor;
 
+/// Returns false when running on an operating system that Kudu doesn't support. If this
+/// check fails, there is no way Kudu should be expected to work. Exposed for testing.
+bool KuduClientIsSupported();
+
+/// Returns OK if Kudu is available or an error status containing the reason Kudu is not
+/// available. Kudu may not be available if no Kudu client is available for the platform
+/// or if Kudu was disabled by the startup flag --disable_kudu.
+Status CheckKuduAvailability();
+
+/// Convenience function for the bool equivalent of CheckKuduAvailability().
+bool KuduIsAvailable();
+
 Status ImpalaToKuduType(const ColumnType& impala_type,
     kudu::client::KuduColumnSchema::DataType* kudu_type);
 
@@ -71,11 +79,5 @@ void LogKuduMessage(kudu::client::KuduLogSeverity severity, const char* filename
     } \
   } while (0)
 
-#else  // No Kudu
-
-void InitKuduLogging() {}
-
-#endif
-
 } /// namespace impala
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/bin/bootstrap_toolchain.py
----------------------------------------------------------------------
diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py
index 1559d32..e12eef4 100755
--- a/bin/bootstrap_toolchain.py
+++ b/bin/bootstrap_toolchain.py
@@ -80,20 +80,6 @@ def download_package(destination, product, version, compiler):
   print "Extracting {0}".format(file_name)
   sh.tar(z=True, x=True, f=os.path.join(destination, file_name), directory=destination)
   sh.rm(os.path.join(destination, file_name))
-
-  if product == "kudu":
-    # The Kudu tarball is actually a renamed parcel. Rename the contents to match the
-    # naming convention.
-    kudu_dirs = glob.glob("{0}/KUDU*{1}*".format(destination, version))
-    if not kudu_dirs:
-      raise Exception("Could not find contents of Kudu tarball")
-    if len(kudu_dirs) > 1:
-      raise Exception("Found too many Kudu folders: %s" % (kudu_dirs, ))
-    new_dir = "{0}/{1}-{2}".format(destination, product, version)
-    if os.path.exists(new_dir):
-      shutil.rmtree(new_dir)
-    os.rename(kudu_dirs[0], new_dir)
-
   write_version_file(destination, product, version, compiler, label)
 
 def bootstrap(packages):
@@ -128,11 +114,9 @@ def bootstrap(packages):
 
   for p in packages:
     pkg_name, pkg_version = unpack_name_and_version(p)
-    if check_for_existing_package(destination, pkg_name, pkg_version,
-        "any" if pkg_name == "kudu" else compiler):
+    if check_for_existing_package(destination, pkg_name, pkg_version, compiler):
       continue
-    download_package(destination, pkg_name, pkg_version,
-        "any" if pkg_name == "kudu" else compiler)
+    download_package(destination, pkg_name, pkg_version, compiler)
 
 def package_directory(toolchain_root, pkg_name, pkg_version):
   dir_name = "{0}-{1}".format(pkg_name, pkg_version)
@@ -198,9 +182,7 @@ def unpack_name_and_version(package):
   return package[0], package[1]
 
 if __name__ == "__main__":
-  packages = ["avro", "boost", "bzip2", "gcc", "gflags", "glog",
-              "gperftools", "gtest", "llvm", ("llvm", "3.3-p1"), ("llvm", "3.7.0"),
-              "lz4", "openldap", "rapidjson", "re2", "snappy", "thrift", "zlib"]
-  if os.environ["KUDU_IS_SUPPORTED"] == "true":
-    packages.append("kudu")
+  packages = ["avro", "boost", "bzip2", "gcc", "gflags", "glog", "gperftools", "gtest",
+      "kudu", "llvm", ("llvm", "3.3-p1"), ("llvm", "3.7.0"), "lz4", "openldap",
+      "rapidjson", "re2", "snappy", "thrift", "zlib"]
   bootstrap(packages)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index bbcf2de..9831490 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -113,9 +113,6 @@ if [[ -z $KUDU_BUILD_DIR ]]; then
   fi
 fi
 
-# Always disable Kudu for now until some build problems are fixed.
-KUDU_IS_SUPPORTED=false
-
 export CDH_MAJOR_VERSION=5
 export HADOOP_LZO=${HADOOP_LZO-$IMPALA_HOME/../hadoop-lzo}
 export IMPALA_LZO=${IMPALA_LZO-$IMPALA_HOME/../Impala-lzo}
@@ -231,7 +228,7 @@ export IMPALA_GFLAGS_VERSION=2.0
 export IMPALA_GLOG_VERSION=0.3.2
 export IMPALA_GPERFTOOLS_VERSION=2.5
 export IMPALA_GTEST_VERSION=1.6.0
-export IMPALA_KUDU_VERSION=0.7.0.p0.425
+export IMPALA_KUDU_VERSION=0.7.0
 export IMPALA_LLVM_VERSION=3.3
 export IMPALA_LLVM_DEBUG_VERSION=3.3
 export IMPALA_LLVM_ASAN_VERSION=3.7.0

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/common/thrift/generate_error_codes.py
----------------------------------------------------------------------
diff --git a/common/thrift/generate_error_codes.py b/common/thrift/generate_error_codes.py
index b4f130d..5a8a033 100755
--- a/common/thrift/generate_error_codes.py
+++ b/common/thrift/generate_error_codes.py
@@ -233,7 +233,10 @@ error_codes = (
 
   ("IMPALA_KUDU_TYPE_MISSING", 74, "Impala type $0 is not available in Kudu."),
 
-  ("KUDU_NOT_SUPPORTED_ON_OS", 75, "Kudu is not supported on this operating system.")
+  ("KUDU_NOT_SUPPORTED_ON_OS", 75, "Kudu is not supported on this operating system."),
+
+  ("KUDU_NOT_ENABLED", 76, "Kudu features are disabled by the startup flag "
+   "--disable_kudu.")
 
 )
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/testdata/cluster/admin
----------------------------------------------------------------------
diff --git a/testdata/cluster/admin b/testdata/cluster/admin
index 286d526..8494502 100755
--- a/testdata/cluster/admin
+++ b/testdata/cluster/admin
@@ -413,6 +413,7 @@ function stop_cluster {
 
 function delete_data {
   rm -rf "$NODES_DIR/$NODE_PREFIX"*/data/dfs/{nn,dn}/*
+  rm -rf "$NODES_DIR/$NODE_PREFIX"*/var/lib/kudu/{master,tserver}/*
 }
 
 function delete_cluster {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
index 6824238..9651979 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
@@ -80,14 +80,14 @@ show table stats simple_hash
 ---- LABELS
 # Rows,Start Key,Stop Key,Leader Replica,# Replicas
 ---- RESULTS
--1,'','0000000000000001',regex:.*?:\d+,1
--1,'0000000000000001','00000001',regex:.*?:\d+,1
--1,'00000001','0000000100000001',regex:.*?:\d+,1
--1,'0000000100000001','00000002',regex:.*?:\d+,1
--1,'00000002','0000000200000001',regex:.*?:\d+,1
--1,'0000000200000001','00000003',regex:.*?:\d+,1
--1,'00000003','0000000300000001',regex:.*?:\d+,1
--1,'0000000300000001','',regex:.*?:\d+,1
+-1,'','0000000000000001',regex:.*?:\d+,3
+-1,'0000000000000001','00000001',regex:.*?:\d+,3
+-1,'00000001','0000000100000001',regex:.*?:\d+,3
+-1,'0000000100000001','00000002',regex:.*?:\d+,3
+-1,'00000002','0000000200000001',regex:.*?:\d+,3
+-1,'0000000200000001','00000003',regex:.*?:\d+,3
+-1,'00000003','0000000300000001',regex:.*?:\d+,3
+-1,'0000000300000001','',regex:.*?:\d+,3
 ---- TYPES
 INT,STRING,STRING,STRING,INT
 ====
@@ -108,18 +108,18 @@ show table stats simple_hash_range
 ---- LABELS
 # Rows,Start Key,Stop Key,Leader Replica,# Replicas
 ---- RESULTS
--1,'','000000008000000A6D617274696E',regex:.*?:\d+,1
--1,'000000008000000A6D617274696E','00000000800000145065746572',regex:.*?:\d+,1
--1,'00000000800000145065746572','00000001',regex:.*?:\d+,1
--1,'00000001','000000018000000A6D617274696E',regex:.*?:\d+,1
--1,'000000018000000A6D617274696E','00000001800000145065746572',regex:.*?:\d+,1
--1,'00000001800000145065746572','00000002',regex:.*?:\d+,1
--1,'00000002','000000028000000A6D617274696E',regex:.*?:\d+,1
--1,'000000028000000A6D617274696E','00000002800000145065746572',regex:.*?:\d+,1
--1,'00000002800000145065746572','00000003',regex:.*?:\d+,1
--1,'00000003','000000038000000A6D617274696E',regex:.*?:\d+,1
--1,'000000038000000A6D617274696E','00000003800000145065746572',regex:.*?:\d+,1
--1,'00000003800000145065746572','',regex:.*?:\d+,1
+-1,'','000000008000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000008000000A6D617274696E','00000000800000145065746572',regex:.*?:\d+,3
+-1,'00000000800000145065746572','00000001',regex:.*?:\d+,3
+-1,'00000001','000000018000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000018000000A6D617274696E','00000001800000145065746572',regex:.*?:\d+,3
+-1,'00000001800000145065746572','00000002',regex:.*?:\d+,3
+-1,'00000002','000000028000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000028000000A6D617274696E','00000002800000145065746572',regex:.*?:\d+,3
+-1,'00000002800000145065746572','00000003',regex:.*?:\d+,3
+-1,'00000003','000000038000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000038000000A6D617274696E','00000003800000145065746572',regex:.*?:\d+,3
+-1,'00000003800000145065746572','',regex:.*?:\d+,3
 ---- TYPES
 INT,STRING,STRING,STRING,INT
 ====
@@ -141,18 +141,18 @@ show table stats simple_hash_range_ctas
 ---- LABELS
 # Rows,Start Key,Stop Key,Leader Replica,# Replicas
 ---- RESULTS
--1,'','000000008000000A6D617274696E',regex:.*?:\d+,1
--1,'000000008000000A6D617274696E','00000000800000145065746572',regex:.*?:\d+,1
--1,'00000000800000145065746572','00000001',regex:.*?:\d+,1
--1,'00000001','000000018000000A6D617274696E',regex:.*?:\d+,1
--1,'000000018000000A6D617274696E','00000001800000145065746572',regex:.*?:\d+,1
--1,'00000001800000145065746572','00000002',regex:.*?:\d+,1
--1,'00000002','000000028000000A6D617274696E',regex:.*?:\d+,1
--1,'000000028000000A6D617274696E','00000002800000145065746572',regex:.*?:\d+,1
--1,'00000002800000145065746572','00000003',regex:.*?:\d+,1
--1,'00000003','000000038000000A6D617274696E',regex:.*?:\d+,1
--1,'000000038000000A6D617274696E','00000003800000145065746572',regex:.*?:\d+,1
--1,'00000003800000145065746572','',regex:.*?:\d+,1
+-1,'','000000008000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000008000000A6D617274696E','00000000800000145065746572',regex:.*?:\d+,3
+-1,'00000000800000145065746572','00000001',regex:.*?:\d+,3
+-1,'00000001','000000018000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000018000000A6D617274696E','00000001800000145065746572',regex:.*?:\d+,3
+-1,'00000001800000145065746572','00000002',regex:.*?:\d+,3
+-1,'00000002','000000028000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000028000000A6D617274696E','00000002800000145065746572',regex:.*?:\d+,3
+-1,'00000002800000145065746572','00000003',regex:.*?:\d+,3
+-1,'00000003','000000038000000A6D617274696E',regex:.*?:\d+,3
+-1,'000000038000000A6D617274696E','00000003800000145065746572',regex:.*?:\d+,3
+-1,'00000003800000145065746572','',regex:.*?:\d+,3
 ---- TYPES
 INT,STRING,STRING,STRING,INT
 ====
@@ -173,10 +173,10 @@ show table stats simple_hash_all_columns
 ---- LABELS
 # Rows,Start Key,Stop Key,Leader Replica,# Replicas
 ---- RESULTS
--1,'','00000001',regex:.*?:\d+,1
--1,'00000001','00000002',regex:.*?:\d+,1
--1,'00000002','00000003',regex:.*?:\d+,1
--1,'00000003','',regex:.*?:\d+,1
+-1,'','00000001',regex:.*?:\d+,3
+-1,'00000001','00000002',regex:.*?:\d+,3
+-1,'00000002','00000003',regex:.*?:\d+,3
+-1,'00000003','',regex:.*?:\d+,3
 ---- TYPES
 INT,STRING,STRING,STRING,INT
 ====
@@ -197,9 +197,9 @@ show table stats simple_range_all_columns
 ---- LABELS
 # Rows,Start Key,Stop Key,Leader Replica,# Replicas
 ---- RESULTS
--1,'','8000000161',regex:.*?:\d+,1
--1,'8000000161','8000000262',regex:.*?:\d+,1
--1,'8000000262','',regex:.*?:\d+,1
+-1,'','8000000161',regex:.*?:\d+,3
+-1,'8000000161','8000000262',regex:.*?:\d+,3
+-1,'8000000262','',regex:.*?:\d+,3
 ---- TYPES
 INT,STRING,STRING,STRING,INT
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/tests/common/impala_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index cf5f212..8f8bcce 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -22,6 +22,7 @@ import pytest
 import grp
 import re
 import string
+import time
 from getpass import getuser
 from functools import wraps
 from impala._thrift_gen.ImpalaService.ttypes import TImpalaQueryOptions
@@ -184,7 +185,7 @@ class ImpalaTestSuite(BaseTestSuite):
         LOG.info('Unexpected exception when executing ' + query_str + ' : ' + str(e))
 
   def run_test_case(self, test_file_name, vector, use_db=None, multiple_impalad=False,
-      encoding=None):
+      encoding=None, wait_secs_between_stmts=None):
     """
     Runs the queries in the specified test based on the vector values
 
@@ -254,6 +255,8 @@ class ImpalaTestSuite(BaseTestSuite):
           if set_pattern_match != None:
             query_options_changed.append(set_pattern_match.groups()[0])
           result = self.__execute_query(target_impalad_client, query, user=user)
+          if wait_secs_between_stmts:
+            time.sleep(wait_secs_between_stmts)
       except Exception as e:
         if 'CATCH' in test_section:
           # In error messages, some paths are always qualified and some are not.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/tests/custom_cluster/test_kudu_not_available.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_kudu_not_available.py b/tests/custom_cluster/test_kudu_not_available.py
new file mode 100644
index 0000000..9c8a455
--- /dev/null
+++ b/tests/custom_cluster/test_kudu_not_available.py
@@ -0,0 +1,60 @@
+# Copyright 2016 Cloudera Inc.
+#
+# Licensed 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.
+
+from impala.dbapi import connect
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+
+class TestKuduNotAvailable(CustomClusterTestSuite):
+  """Check that when Impala is started without Kudu support, statements that use Kudu
+     fail with the expected error message.
+  """
+
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @CustomClusterTestSuite.with_args(
+      impalad_args="--disable_kudu=true",
+      catalogd_args="--disable_kudu=true",
+      statestored_args="--disable_kudu=true")
+  def test_kudu_not_available(self):
+    conn = connect()
+    cursor = conn.cursor()
+    try:
+      cursor.execute("SHOW DATABASES")
+      if "functional_kudu" not in cursor:
+        # This should mean Kudu data couldn't be loaded because Kudu isn't supported on
+        # the OS.
+        return
+      cursor.execute("USE functional_kudu")
+
+      # CREATE TABLE succeeds, the execution is in the frontend only.
+      # https://issues.cloudera.org/browse/IMPALA-3233
+      self.assert_failure("SELECT * FROM tinytable", cursor)
+      self.assert_failure("INSERT INTO tinytable VALUES ('a', 'b')", cursor)
+      self.assert_failure("DELETE FROM tinytable", cursor)
+    finally:
+      cursor.close()
+      conn.close()
+
+  def assert_failure(self, stmt, cursor):
+    try:
+      cursor.execute(stmt)
+    except Exception as e:
+      error_msg = str(e)
+      assert "Kudu features are disabled" in error_msg \
+          or "Kudu is not supported" in error_msg
+      return
+    assert False, "Statement should have failed: %s" % stmt

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/39a28185/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index dc6a906..facd476 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -74,7 +74,8 @@ class TestKuduOperations(ImpalaTestSuite):
 
   @pytest.mark.execute_serially
   def test_insert_update_delete(self, vector):
-    self.run_test_case('QueryTest/kudu_crud', vector, use_db="kududb_test")
+    self.run_test_case('QueryTest/kudu_crud', vector, use_db="kududb_test",
+        wait_secs_between_stmts=1)
 
   @pytest.mark.execute_serially
   def test_kudu_partition_ddl(self, vector):


[32/50] incubator-impala git commit: IMPALA-3274: Always start Kudu for testing

Posted by ta...@apache.org.
IMPALA-3274: Always start Kudu for testing

Previously Kudu would only be started when the test configuration was
the standard mini-cluster. That led to failures during data loading when
testing without the mini-cluster (ex: local file system). Kudu doesn't
require any other services so now it'll be started for all test
environments.

Change-Id: I92643ca6ef1acdbf4d4cd2fa5faf9ac97a3f0865
Reviewed-on: http://gerrit.cloudera.org:8080/2690
Reviewed-by: Casey Ching <ca...@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/9d43aac6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/9d43aac6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/9d43aac6

Branch: refs/heads/master
Commit: 9d43aac6ce0c23ffee9378cbbd63cc3233670b84
Parents: 7050294
Author: Casey Ching <ca...@cloudera.com>
Authored: Wed Mar 30 14:11:22 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:02:35 2016 -0700

----------------------------------------------------------------------
 bin/impala-config.sh    |  7 -------
 testdata/bin/run-all.sh |  9 ++++-----
 testdata/cluster/admin  | 11 ++++++++---
 3 files changed, 12 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9d43aac6/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index d21b153..2ce5ad7 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -180,13 +180,6 @@ elif [ "${TARGET_FILESYSTEM}" != "hdfs" ]; then
   return 1
 fi
 
-if [[ "$TARGET_FILESYSTEM" != "hdfs" ]]; then
-  # TODO: Kudu is disabled to unblock builds. More work is needed to re-enable it for
-  #       non-mini-cluster testing. The problem now is nothing starts the Kudu service
-  #       in the non-mini-cluster mode.
-  KUDU_IS_SUPPORTED=false
-fi
-
 # Directories where local cluster logs will go when running tests or loading data
 export IMPALA_LOGS_DIR=${IMPALA_HOME}/logs
 export IMPALA_CLUSTER_LOGS_DIR=${IMPALA_LOGS_DIR}/cluster

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9d43aac6/testdata/bin/run-all.sh
----------------------------------------------------------------------
diff --git a/testdata/bin/run-all.sh b/testdata/bin/run-all.sh
index 2df6618..398088a 100755
--- a/testdata/bin/run-all.sh
+++ b/testdata/bin/run-all.sh
@@ -33,16 +33,15 @@ fi
 echo "Killing running services..."
 $IMPALA_HOME/testdata/bin/kill-all.sh &>${IMPALA_CLUSTER_LOGS_DIR}/kill-all.log
 
+echo "Starting cluster services..."
+$IMPALA_HOME/testdata/bin/run-mini-dfs.sh ${HDFS_FORMAT_CLUSTER-} 2>&1 | \
+    tee ${IMPALA_CLUSTER_LOGS_DIR}/run-mini-dfs.log
+
 # Starts up a mini-cluster which includes:
 # - HDFS with 3 DNs
 # - One Yarn ResourceManager
 # - Multiple Yarn NodeManagers, exactly one per HDFS DN
 if [[ ${DEFAULT_FS} == "hdfs://localhost:20500" ]]; then
-  echo "Starting all cluster services..."
-  echo " --> Starting mini-DFS cluster"
-  $IMPALA_HOME/testdata/bin/run-mini-dfs.sh ${HDFS_FORMAT_CLUSTER-} 2>&1 | \
-      tee ${IMPALA_CLUSTER_LOGS_DIR}/run-mini-dfs.log
-
   echo " --> Starting HBase"
   $IMPALA_HOME/testdata/bin/run-hbase.sh 2>&1 | \
       tee ${IMPALA_CLUSTER_LOGS_DIR}/run-hbase.log

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9d43aac6/testdata/cluster/admin
----------------------------------------------------------------------
diff --git a/testdata/cluster/admin b/testdata/cluster/admin
index 8494502..692f0e5 100755
--- a/testdata/cluster/admin
+++ b/testdata/cluster/admin
@@ -49,9 +49,14 @@ TEMPLATE_SUFFIX=".tmpl"
 # Each process should be marked with this so a "pkill -f" can be done to nuke everything.
 export KILL_CLUSTER_MARKER=IBelongToTheMiniCluster
 
-SUPPORTED_SERVICES=(hdfs yarn)
-if [[ "$CDH_MAJOR_VERSION" -ge 5 ]]; then
-  SUPPORTED_SERVICES+=(llama kms)
+if [[ "$TARGET_FILESYSTEM" == "hdfs" ]]; then
+  # The check above indicates that the regular mini-cluster is in use.
+  SUPPORTED_SERVICES=(hdfs yarn llama kms)
+else
+  # Either a remote distributed file system or a local non-distributed file system is
+  # in use. Currently the only service that is expected to work is Kudu, though in theory
+  # the other services could work after the proper configuration changes.
+  SUPPORTED_SERVICES=()
 fi
 if $KUDU_IS_SUPPORTED; then
   SUPPORTED_SERVICES+=(kudu)