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

[1/3] impala git commit: IMPALA-7408: add a debugging flag to disable reading fs data from catalogd

Repository: impala
Updated Branches:
  refs/heads/master 2a2c3daaa -> cddb35be9


IMPALA-7408: add a debugging flag to disable reading fs data from catalogd

Add the flag: --disable_catalog_data_ops_debug_only that skips loading
files from the file-system from catalogd. The flag is by default false
and its hidden. Its intent is to avoid time-consuming accesses to
the file-system when debugging metadata issues and the file-system
contents are not available. For example, a recent ~18 GB catalog
takes 10 hours to load without the flag set vs. 1 hour to load with
the flag. The extra time comes from accessing the file-system, failing,
and logging exceptions.

This flag specifically disables copying jars from the fs when loading
Java functions and it skips loading avro schema files. Additional cases
can be added under this flag if more are needed.

Testing:
- manually confirmed that jars and avro schema files are skipped.
- added a test to check the same behavior in a custom cluster test.
- ran core tests.

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


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

Branch: refs/heads/master
Commit: c692e5cc9ec2ab2d626c2510d300a03c27790a9b
Parents: 2a2c3da
Author: Vuk Ercegovac <ve...@cloudera.com>
Authored: Tue May 15 18:38:34 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Aug 15 01:58:18 2018 +0000

----------------------------------------------------------------------
 be/src/common/global-flags.cc                   |  5 +++
 be/src/util/backend-gflag-util.cc               |  3 ++
 common/thrift/BackendGflags.thrift              |  2 +
 .../impala/catalog/CatalogServiceCatalog.java   |  5 +++
 .../apache/impala/service/BackendConfig.java    |  4 ++
 .../org/apache/impala/util/AvroSchemaUtils.java |  6 ++-
 tests/common/custom_cluster_test_suite.py       | 35 ++++++++++++----
 .../test_disable_catalog_data_ops.py            | 44 ++++++++++++++++++++
 8 files changed, 94 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/be/src/common/global-flags.cc
----------------------------------------------------------------------
diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index c6df084..5c5dc03 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -199,6 +199,11 @@ DEFINE_string(reserved_words_version, "3.0.0", "Reserved words compatibility ver
     " version from which the reserved word list is taken. The value must be one of "
     "[\"2.11.0\", \"3.0.0\"].");
 
+DEFINE_bool_hidden(disable_catalog_data_ops_debug_only, false,
+    "Disable catalog operations that require access to file-system data blocks. "
+    "Examples are when catalog reads data blocks to load avro schemas and copy jars."
+    "Use only for testing/debugging, not in deployed clusters.");
+
 // ++========================++
 // || Startup flag graveyard ||
 // ++========================++

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/be/src/util/backend-gflag-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index d8d945a..c7982bf 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -53,6 +53,7 @@ DECLARE_string(reserved_words_version);
 DECLARE_string(sentry_config);
 DECLARE_double(max_filter_error_rate);
 DECLARE_int64(min_buffer_size);
+DECLARE_bool(disable_catalog_data_ops_debug_only);
 
 namespace impala {
 
@@ -95,6 +96,8 @@ Status GetThriftBackendGflags(JNIEnv* jni_env, jbyteArray* cfg_bytes) {
   cfg.__set_max_filter_error_rate(FLAGS_max_filter_error_rate);
   cfg.__set_min_buffer_size(FLAGS_min_buffer_size);
   cfg.__set_authorized_proxy_group_config(FLAGS_authorized_proxy_group_config);
+  cfg.__set_disable_catalog_data_ops_debug_only(
+      FLAGS_disable_catalog_data_ops_debug_only);
   RETURN_IF_ERROR(SerializeThriftMsg(jni_env, &cfg, cfg_bytes));
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/common/thrift/BackendGflags.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index c8c0dea..6852d27 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -79,4 +79,6 @@ struct TBackendGflags {
   26: required string authorized_proxy_group_config
 
   27: required bool use_local_catalog
+
+  28: required bool disable_catalog_data_ops_debug_only
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 7efff99..a3f5a1e 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -45,6 +45,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.JniUtil;
 import org.apache.impala.common.Pair;
 import org.apache.impala.common.Reference;
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TCatalog;
 import org.apache.impala.thrift.TCatalogObject;
@@ -793,6 +794,10 @@ public class CatalogServiceCatalog extends Catalog {
   private void loadJavaFunctions(Db db,
       List<org.apache.hadoop.hive.metastore.api.Function> functions) {
     Preconditions.checkNotNull(functions);
+    if (BackendConfig.INSTANCE.disableCatalogDataOpsDebugOnly()) {
+      LOG.info("Skip loading Java functions: catalog data ops disabled.");
+      return;
+    }
     LOG.info("Loading Java functions for database: " + db.getName());
     for (org.apache.hadoop.hive.metastore.api.Function function: functions) {
       try {

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/fe/src/main/java/org/apache/impala/service/BackendConfig.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 9dd3f94..9ff5f21 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -87,6 +87,10 @@ public class BackendConfig {
     return !Strings.isNullOrEmpty(backendCfg_.authorized_proxy_group_config);
   }
 
+  public boolean disableCatalogDataOpsDebugOnly() {
+    return backendCfg_.disable_catalog_data_ops_debug_only;
+  }
+
   // Inits the auth_to_local configuration in the static KerberosName class.
   private static void initAuthToLocal() {
     // If auth_to_local is enabled, we read the configuration hadoop.security.auth_to_local

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/fe/src/main/java/org/apache/impala/util/AvroSchemaUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/AvroSchemaUtils.java b/fe/src/main/java/org/apache/impala/util/AvroSchemaUtils.java
index 833204d..3c6bb51 100644
--- a/fe/src/main/java/org/apache/impala/util/AvroSchemaUtils.java
+++ b/fe/src/main/java/org/apache/impala/util/AvroSchemaUtils.java
@@ -37,6 +37,7 @@ import org.apache.impala.analysis.ColumnDef;
 import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FileSystemUtil;
+import org.apache.impala.service.BackendConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 /**
@@ -83,7 +84,7 @@ public abstract class AvroSchemaUtils {
       if (url.toLowerCase().startsWith("http://")) {
         urlStream = new URL(url).openStream();
         schema = IOUtils.toString(urlStream);
-      } else {
+      } else if (!BackendConfig.INSTANCE.disableCatalogDataOpsDebugOnly()) {
         Path path = new Path(url);
         FileSystem fs = null;
         fs = path.getFileSystem(FileSystemUtil.getConfiguration());
@@ -92,6 +93,9 @@ public abstract class AvroSchemaUtils {
               "Invalid avro.schema.url: %s. Path does not exist.", url));
         }
         schema = FileSystemUtil.readFile(path);
+      } else {
+        LOG.info(String.format(
+            "Avro schema, %s, not loaded from fs: catalog data ops disabled.", url));
       }
     } catch (AnalysisException e) {
       throw e;

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/tests/common/custom_cluster_test_suite.py
----------------------------------------------------------------------
diff --git a/tests/common/custom_cluster_test_suite.py b/tests/common/custom_cluster_test_suite.py
index d131252..4bb9ff8 100644
--- a/tests/common/custom_cluster_test_suite.py
+++ b/tests/common/custom_cluster_test_suite.py
@@ -170,21 +170,38 @@ class CustomClusterTestSuite(ImpalaTestSuite):
 
   def assert_impalad_log_contains(self, level, line_regex, expected_count=1):
     """
-    Assert that impalad log with specified level (e.g. ERROR, WARNING, INFO) contains
-    expected_count lines with a substring matching the regex. When using this method to
-    check log files of running processes, the caller should make sure that log buffering
-    has been disabled, for example by adding '-logbuflevel=-1' to the daemon startup
-    options.
+    Convenience wrapper around assert_log_contains for impalad logs.
+    """
+    self.assert_log_contains("impalad", level, line_regex, expected_count)
+
+  def assert_catalogd_log_contains(self, level, line_regex, expected_count=1):
+    """
+    Convenience wrapper around assert_log_contains for catalogd logs.
+    """
+    self.assert_log_contains("catalogd", level, line_regex, expected_count)
+
+  def assert_log_contains(self, daemon, level, line_regex, expected_count=1):
+    """
+    Assert that the daemon log with specified level (e.g. ERROR, WARNING, INFO) contains
+    expected_count lines with a substring matching the regex. When expected_count is -1,
+    at least one match is expected.
+    When using this method to check log files of running processes, the caller should
+    make sure that log buffering has been disabled, for example by adding
+    '-logbuflevel=-1' to the daemon startup options.
     """
     pattern = re.compile(line_regex)
     found = 0
-    log_file_path = os.path.join(self.impala_log_dir, "impalad." + level)
+    log_file_path = os.path.join(self.impala_log_dir, daemon + "." + level)
     # Resolve symlinks to make finding the file easier.
     log_file_path = os.path.realpath(log_file_path)
     with open(log_file_path) as log_file:
       for line in log_file:
         if pattern.search(line):
           found += 1
-    assert found == expected_count, ("Expected %d lines in file %s matching regex '%s'"
-        + ", but found %d lines. Last line was: \n%s") % (expected_count, log_file_path,
-                                                          line_regex, found, line)
+    if expected_count == -1:
+      assert found > 0, "Expected at least one line in file %s matching regex '%s'"\
+        ", but found none." % (log_file_path, line_regex)
+    else:
+      assert found == expected_count, "Expected %d lines in file %s matching regex '%s'"\
+        ", but found %d lines. Last line was: \n%s" %\
+        (expected_count, log_file_path, line_regex, found, line)

http://git-wip-us.apache.org/repos/asf/impala/blob/c692e5cc/tests/custom_cluster/test_disable_catalog_data_ops.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_disable_catalog_data_ops.py b/tests/custom_cluster/test_disable_catalog_data_ops.py
new file mode 100644
index 0000000..3e28a41
--- /dev/null
+++ b/tests/custom_cluster/test_disable_catalog_data_ops.py
@@ -0,0 +1,44 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+
+
+class TestDisableCatalogDataOps(CustomClusterTestSuite):
+  """Test Catalog behavior when --disable_catalog_data_ops_debug_only is set."""
+
+  # TODO(vercegovac): make the test more precise by starting from an empty database
+  # and adding specifically one java-udf and one avro table.
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      catalogd_args="--logbuflevel=-1 --disable_catalog_data_ops_debug_only=true")
+  def test_disable_catalog_data_ops(self):
+    # Expects that all Java UDF loading messages are for skip and that none of them load.
+    self.assert_catalogd_log_contains(
+      'INFO', "Skip loading Java functions: catalog data ops disabled.",
+      expected_count=-1)
+    self.assert_catalogd_log_contains(
+      'INFO', "Loading Java functions for database:", expected_count=0)
+
+    # Indirectly issues a load to a single avro table. Expects to skip the schema load.
+    self.client.execute("select count(*) from functional_avro.tinytable")
+    self.assert_catalogd_log_contains(
+      'INFO', "Avro schema, "
+      "hdfs://localhost:20500//test-warehouse/avro_schemas/functional/tinytable.json"
+      ", not loaded from fs: catalog data ops disabled.")


[2/3] impala git commit: IMPALA-7443: Fix intermittent GVO failures due to stale Maven cache

Posted by to...@apache.org.
IMPALA-7443: Fix intermittent GVO failures due to stale Maven cache

When switching to a new Maven repository, such as switching to a new
CDH_BUILD_NUMBER, Maven does not automatically pick up the new
repository. The patch forces Maven to run with -U on Jenkins to force
updating its local repository. The patch also puts some Jenkins jobs
into scripts in $IMPALA_HOME/bin/jenkins to make updating Jenkins jobs
easier.

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


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

Branch: refs/heads/master
Commit: 415300d0d82d2bdae1e07a6f56091716e40721ed
Parents: c692e5c
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Tue Aug 14 14:59:46 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Aug 15 02:14:10 2018 +0000

----------------------------------------------------------------------
 bin/jenkins/all-tests.sh                   | 29 +++++++++++++++++++++++++
 bin/jenkins/build-all-flag-combinations.sh |  2 ++
 bin/jenkins/build-only.sh                  | 28 ++++++++++++++++++++++++
 3 files changed, 59 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/415300d0/bin/jenkins/all-tests.sh
----------------------------------------------------------------------
diff --git a/bin/jenkins/all-tests.sh b/bin/jenkins/all-tests.sh
new file mode 100644
index 0000000..7917358
--- /dev/null
+++ b/bin/jenkins/all-tests.sh
@@ -0,0 +1,29 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# Run all Impala tests.
+
+set -euo pipefail
+trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
+
+cd "${IMPALA_HOME}"
+
+export IMPALA_MAVEN_OPTIONS="-U"
+
+source bin/bootstrap_development.sh
+bin/run-all-tests.sh

http://git-wip-us.apache.org/repos/asf/impala/blob/415300d0/bin/jenkins/build-all-flag-combinations.sh
----------------------------------------------------------------------
diff --git a/bin/jenkins/build-all-flag-combinations.sh b/bin/jenkins/build-all-flag-combinations.sh
index e6dfc1c..8dce06d 100755
--- a/bin/jenkins/build-all-flag-combinations.sh
+++ b/bin/jenkins/build-all-flag-combinations.sh
@@ -27,6 +27,8 @@
 set -euo pipefail
 trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
 
+export IMPALA_MAVEN_OPTIONS="-U"
+
 . bin/impala-config.sh
 
 # These are configurations for buildall.

http://git-wip-us.apache.org/repos/asf/impala/blob/415300d0/bin/jenkins/build-only.sh
----------------------------------------------------------------------
diff --git a/bin/jenkins/build-only.sh b/bin/jenkins/build-only.sh
new file mode 100644
index 0000000..d14ad6e
--- /dev/null
+++ b/bin/jenkins/build-only.sh
@@ -0,0 +1,28 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# Only run an Impala build.
+
+set -euo pipefail
+trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
+
+cd "${IMPALA_HOME}"
+
+export IMPALA_MAVEN_OPTIONS="-U"
+
+bin/bootstrap_build.sh


[3/3] impala git commit: IMPALA-7445: separate skippable tables from test_resource_limits

Posted by to...@apache.org.
IMPALA-7445: separate skippable tables from test_resource_limits

Separate the Hbase and Kudu tables used in test_resource_limits to their
own files so they are skippable in environments where those services are
not used.

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


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

Branch: refs/heads/master
Commit: cddb35be92b9b91e707aaf1781aed7999344a96a
Parents: 415300d
Author: Michael Brown <mi...@cloudera.com>
Authored: Tue Aug 14 14:50:02 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Aug 15 04:15:46 2018 +0000

----------------------------------------------------------------------
 .../QueryTest/query-resource-limits-hbase.test  | 13 ++++++++++++
 .../QueryTest/query-resource-limits-kudu.test   | 10 ++++++++++
 .../QueryTest/query-resource-limits.test        | 21 --------------------
 tests/query_test/test_resource_limits.py        | 13 +++++++++++-
 4 files changed, 35 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/cddb35be/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-hbase.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-hbase.test b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-hbase.test
new file mode 100644
index 0000000..e9ff0a3
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-hbase.test
@@ -0,0 +1,13 @@
+====
+---- QUERY
+# Bytes limit enforced for HBase.
+# Add a sleep to slow down query and avoid race with bytes check.
+set SCAN_BYTES_LIMIT="1k";
+select count(*)
+from (
+  select distinct *
+  from functional_hbase.alltypesagg
+  where sleep(100) = id) v
+---- CATCH
+row_regex:.*terminated due to scan bytes limit of 1.00 K.*
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/cddb35be/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-kudu.test b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-kudu.test
new file mode 100644
index 0000000..df6a9e0
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits-kudu.test
@@ -0,0 +1,10 @@
+====
+---- QUERY
+# Bytes limit not enforced for Kudu yet.
+set SCAN_BYTES_LIMIT="1k";
+select min(l_orderkey) from tpch_kudu.lineitem
+---- TYPES
+BIGINT
+---- RESULTS
+1
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/cddb35be/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits.test b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits.test
index e3cd401..daba679 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/query-resource-limits.test
@@ -52,24 +52,3 @@ select count(*) from tpch.lineitem l1,tpch.lineitem l2, tpch.lineitem l3 where
 ---- CATCH
 row_regex:.*expired due to execution time limit of 2s000ms.*
 ====
----- QUERY
-# Bytes limit not enforced for Kudu yet.
-set SCAN_BYTES_LIMIT="1k";
-select min(l_orderkey) from tpch_kudu.lineitem
----- TYPES
-BIGINT
----- RESULTS
-1
-====
----- QUERY
-# Bytes limit enforced for HBase.
-# Add a sleep to slow down query and avoid race with bytes check.
-set SCAN_BYTES_LIMIT="1k";
-select count(*)
-from (
-  select distinct *
-  from functional_hbase.alltypesagg
-  where sleep(100) = id) v
----- CATCH
-row_regex:.*terminated due to scan bytes limit of 1.00 K.*
-====

http://git-wip-us.apache.org/repos/asf/impala/blob/cddb35be/tests/query_test/test_resource_limits.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_resource_limits.py b/tests/query_test/test_resource_limits.py
index be0895b..58ae4a5 100644
--- a/tests/query_test/test_resource_limits.py
+++ b/tests/query_test/test_resource_limits.py
@@ -16,9 +16,10 @@
 # under the License.
 
 from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.skip import SkipIfEC, SkipIfLocal
+from tests.common.skip import SkipIfEC, SkipIfKudu, SkipIfLocal, SkipIfS3
 from tests.common.test_dimensions import create_parquet_dimension
 
+
 class TestResourceLimits(ImpalaTestSuite):
   """Test resource limit functionality."""
 
@@ -42,3 +43,13 @@ class TestResourceLimits(ImpalaTestSuite):
   @SkipIfLocal.multiple_impalad
   def test_resource_limits(self, vector):
     self.run_test_case('QueryTest/query-resource-limits', vector)
+
+  @SkipIfS3.hbase
+  @SkipIfLocal.multiple_impalad
+  def test_resource_limits_hbase(self, vector):
+    self.run_test_case('QueryTest/query-resource-limits-hbase', vector)
+
+  @SkipIfKudu.unsupported_env
+  @SkipIfLocal.multiple_impalad
+  def test_resource_limits_kudu(self, vector):
+    self.run_test_case('QueryTest/query-resource-limits-kudu', vector)